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 2022/11/09 01:01:13 UTC

[calcite] branch main updated: [CALCITE-5155] Custom time frames

This is an automated email from the ASF dual-hosted git repository.

jhyde pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git


The following commit(s) were added to refs/heads/main by this push:
     new a0e119ea42 [CALCITE-5155] Custom time frames
a0e119ea42 is described below

commit a0e119ea42def418957f214f539469f1aba76c18
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat Nov 5 16:34:26 2022 -0700

    [CALCITE-5155] Custom time frames
    
    Before this change, you can use the ISO SQL time units
    (SECOND, HOUR, DAY, MONTH, YEAR, etc.) to perform datetime
    arithmetic (FLOOR, CEIL, EXTRACT) and also when defining
    materialized views and using them in queries. But
    applications would like to be able to define their own time
    frames, such as "MINUTE15" (a 15 minute period aligned with
    the hour) or "MONTH4" (a 4 month period aligned with the
    year), or "WEEK(THURSDAY)" (a 7 day week that starts on a
    Thursday).
    
    After this change, applications can define their own time
    frames. We add a `class TimeFrameSet`, and in the
    `interface RelDataTypeSystem` we add a method `TimeFrameSet
    deriveTimeFrameSet(TimeFrameSet frameSet)`. This method is
    called during query preparation, and the application has
    the opportunity to define a set that contains custom and
    existing time frames.
    
    Time frames can be defined that are multiples of and multiply
    to built-in time frames (as, for example, MINUTE is a
    multiple of SECOND and MILLISECOND multiplies to SECOND).
    You can also define that a time frame is aligned with
    another (as, for example, DAY is aligned with MONTH even
    though the multiplier is not constant).
    
    The following functions allow time frame arguments:
     * DATEADD (Postgres, MSSql)
     * DATEDIFF (Postgres, MSSql)
     * DATEPART (MSSql)
     * DATE_PART (Postgres)
     * EXTRACT (Calcite built-in, also SQL standard)
     * CEIL (Calcite built-in)
     * FLOOR (Calcite built-in)
     * TIMESTAMPADD (Calcite built-in, also JDBC standard)
     * TIMESTAMPDIFF (Calcite-builtin, also JDBC standard)
     * TIMESTAMP_TRUNC (BigQuery)
     * TIME_TRUNC (BigQuery)
    
    Calls to the above functions with invalid time units would
    previously be a parse error and are now detected during
    validation.
    
    The SQL_TSI_xxx (e.g. SQL_TSI_HOUR) arguments are treated
    as time frames, and the parser passes them as identifiers.
    They are no longer reserved keywords.
    
    Previously, NANOSECOND and MILLISECOND were allowed in
    EXTRACT but no other functions. Now all functions that
    accept time frames accept the same time frames (built-in
    time intervals, identifiers for user-defined time frames,
    and SQL_TSI_xxx which are defined in the JDBC standard but
    are treated as identifiers until validation).
    
    The representation of calls to the above functions has
    changed. Previously the operand was a time unit, now it is
    an identifier.
    
    Deprecate SqlAbstractParserImpl.setTimeUnitCodes() and
    SqlParser.Config.timeUnitCodes(), because you can now create
    aliases for time units by creating custom time frames using
    TimeFrameSet.Builder.alias().
    
    Add commons-math3 as a dependency because TimeFrame uses
    BigFraction.
    
    Currently ISO_YEAR is not handled by DATEADD, TIMESTAMPADD,
    DATEDIFF, TIMESTAMPDIFF, etc. Adding or subtracting an
    ISO_YEAR will no-op. I don't know what the behavior should
    be.
    
    Close apache/calcite#2960
---
 babel/src/main/codegen/includes/parserImpls.ftl    |  22 +-
 .../org/apache/calcite/test/BabelParserTest.java   |  52 +-
 .../java/org/apache/calcite/test/BabelTest.java    |  61 ++
 bom/build.gradle.kts                               |   1 +
 core/build.gradle.kts                              |   1 +
 core/src/main/codegen/templates/Parser.jj          | 167 ++----
 .../main/java/org/apache/calcite/DataContext.java  |   6 +
 .../calcite/adapter/enumerable/RexImpTable.java    | 135 ++++-
 .../apache/calcite/jdbc/CalciteConnectionImpl.java |   6 +
 .../calcite/rel/type/DelegatingTypeSystem.java     |   4 +
 .../apache/calcite/rel/type/RelDataTypeSystem.java |  17 +
 .../org/apache/calcite/rel/type/TimeFrame.java     | 113 ++++
 .../org/apache/calcite/rel/type/TimeFrameSet.java  | 365 ++++++++++++
 .../org/apache/calcite/rel/type/TimeFrames.java    | 637 ++++++++++++++++++++
 .../java/org/apache/calcite/rex/RexBuilder.java    |   7 +-
 .../apache/calcite/runtime/CalciteResource.java    |   4 +-
 .../org/apache/calcite/runtime/SqlFunctions.java   | 104 ++++
 .../org/apache/calcite/sql/SqlCallBinding.java     |  20 +-
 .../java/org/apache/calcite/sql/SqlDialect.java    |   4 +
 .../apache/calcite/sql/SqlIntervalQualifier.java   |  51 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java    |  16 +-
 .../apache/calcite/sql/fun/SqlExtractFunction.java |  23 +-
 .../apache/calcite/sql/fun/SqlFloorFunction.java   |  25 +-
 .../calcite/sql/fun/SqlLibraryOperators.java       |  31 +-
 .../calcite/sql/fun/SqlTimestampAddFunction.java   |  44 +-
 .../calcite/sql/fun/SqlTimestampDiffFunction.java  |  21 +
 .../calcite/sql/parser/SqlAbstractParserImpl.java  |   5 +-
 .../org/apache/calcite/sql/parser/SqlParser.java   |  12 +-
 .../sql/type/IntervalOperandTypeChecker.java       |  58 ++
 .../org/apache/calcite/sql/type/OperandTypes.java  |  15 +
 .../apache/calcite/sql/validate/SqlValidator.java  |  17 +
 .../calcite/sql/validate/SqlValidatorImpl.java     |  28 +
 .../calcite/sql2rel/SqlNodeToRexConverterImpl.java |  27 +-
 .../calcite/sql2rel/StandardConvertletTable.java   |  70 ++-
 .../org/apache/calcite/util/BuiltInMethod.java     |  16 +
 .../apache/calcite/util/DateTimeStringUtils.java   |   2 +
 .../main/java/org/apache/calcite/util/NameMap.java |   2 +-
 .../org/apache/calcite/util/TimestampString.java   |   4 +
 .../calcite/runtime/CalciteResource.properties     |   2 +-
 .../org/apache/calcite/test/SqlValidatorTest.java  | 109 +++-
 .../org/apache/calcite/test/TimeFrameTest.java     | 649 +++++++++++++++++++++
 gradle.properties                                  |   1 +
 .../apache/calcite/sql/parser/SqlParserTest.java   | 209 +++----
 .../org/apache/calcite/test/SqlOperatorTest.java   | 595 +++++++++++++------
 44 files changed, 3183 insertions(+), 575 deletions(-)

diff --git a/babel/src/main/codegen/includes/parserImpls.ftl b/babel/src/main/codegen/includes/parserImpls.ftl
index 5be19d7a94..5a598fc29c 100644
--- a/babel/src/main/codegen/includes/parserImpls.ftl
+++ b/babel/src/main/codegen/includes/parserImpls.ftl
@@ -44,20 +44,21 @@ SqlNode DateFunctionCall() :
 
 SqlNode DateaddFunctionCall() :
 {
-    final SqlFunctionCategory funcType = SqlFunctionCategory.USER_DEFINED_FUNCTION;
     final Span s;
-    final SqlIdentifier qualifiedName;
-    final TimeUnit unit;
+    final SqlOperator op;
+    final SqlIntervalQualifier unit;
     final List<SqlNode> args;
     SqlNode e;
 }
 {
-    ( <DATEADD> | <DATEDIFF> | <DATE_PART> ) {
-        s = span();
-        qualifiedName = new SqlIdentifier(unquotedIdentifier(), getPos());
-    }
-    <LPAREN> unit = TimeUnit() {
-        args = startList(new SqlIntervalQualifier(unit, null, getPos()));
+    (   <DATE_PART>  { op = SqlLibraryOperators.DATE_PART; }
+    |   <DATEADD> { op = SqlLibraryOperators.DATEADD; }
+    |   <DATEDIFF> { op = SqlLibraryOperators.DATEDIFF; }
+    |   <DATEPART>  { op = SqlLibraryOperators.DATEPART; }
+    )
+    { s = span(); }
+    <LPAREN> unit = TimeUnitOrName() {
+        args = startList(unit);
     }
     (
         <COMMA> e = Expression(ExprContext.ACCEPT_SUB_QUERY) {
@@ -65,7 +66,7 @@ SqlNode DateaddFunctionCall() :
         }
     )*
     <RPAREN> {
-        return createCall(qualifiedName, s.end(this), funcType, null, args);
+        return op.createCall(s.end(this), args);
     }
 }
 
@@ -174,6 +175,7 @@ SqlCreate SqlCreateTable(Span s, boolean replace) :
     < DATE_PART: "DATE_PART" >
 |   < DATEADD: "DATEADD" >
 |   < DATEDIFF: "DATEDIFF" >
+|   < DATEPART: "DATEPART" >
 |   < NEGATE: "!" >
 |   < TILDE: "~" >
 }
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
index 3fafb4899b..ee5c4fb6e9 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.test;
 
-import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.dialect.MysqlSqlDialect;
 import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
@@ -35,7 +34,6 @@ import org.junit.jupiter.api.Test;
 
 import java.util.Arrays;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Objects;
 
 import static org.hamcrest.CoreMatchers.is;
@@ -179,8 +177,8 @@ class BabelParserTest extends SqlParserTest {
     final String sql = "SELECT DATEADD(day, 1, t),\n"
         + " DATEDIFF(week, 2, t),\n"
         + " DATE_PART(year, t) FROM mytable";
-    final String expected = "SELECT `DATEADD`(DAY, 1, `T`),"
-        + " `DATEDIFF`(WEEK, 2, `T`), `DATE_PART`(YEAR, `T`)\n"
+    final String expected = "SELECT DATEADD(DAY, 1, `T`),"
+        + " DATEDIFF(WEEK, 2, `T`), DATE_PART(YEAR, `T`)\n"
         + "FROM `MYTABLE`";
 
     sql(sql).ok(expected);
@@ -188,30 +186,28 @@ class BabelParserTest extends SqlParserTest {
 
   /** Overrides, adding tests for DATEADD, DATEDIFF, DATE_PART functions
    * in addition to EXTRACT. */
-  @Override protected void checkTimeUnitCodes(
-      Map<String, TimeUnit> timeUnitCodes) {
-    super.checkTimeUnitCodes(timeUnitCodes);
-
-    SqlParserFixture f = fixture()
-        .withConfig(config -> config.withTimeUnitCodes(timeUnitCodes));
-
-    timeUnitCodes.forEach((abbrev, timeUnit) -> {
-      String sql = "SELECT "
-          + "DATEADD(" + abbrev + ", 1, '2022-06-03 15:30:00.000'),"
-          + "DATEDIFF(" + abbrev + ", '2021-06-03 12:00:00.000', '2022-06-03 15:30:00.000'),"
-          + "DATE_PART(" + abbrev + ", '2022-06-03 15:30:00.000')";
-      String expected = "SELECT "
-          + "`DATEADD`(" + timeUnit + ", 1, '2022-06-03 15:30:00.000'), "
-          + "`DATEDIFF`(" + timeUnit + ", '2021-06-03 12:00:00.000', '2022-06-03 15:30:00.000'), "
-          + "`DATE_PART`(" + timeUnit + ", '2022-06-03 15:30:00.000')";
-      f.sql(sql).ok(expected);
-    });
-    f.sql("SELECT DATEADD(^A^, 1, NOW())")
-        .fails("'A' is not a valid datetime format");
-    if (timeUnitCodes.containsKey("S")) {
-      f.sql("SELECT DATEADD(S^.^A, 1, NOW())")
-          .fails("(?s).*Encountered \".\" at .*");
-    }
+  @Test protected void testTimeUnitCodes() {
+    super.testTimeUnitCodes();
+
+    // As for FLOOR in the base class, so for DATEADD, DATEDIFF, DATE_PART.
+    // Extensions such as 'y' remain as identifiers; they are resolved in the
+    // validator.
+    final String ts = "'2022-06-03 12:00:00.000'";
+    final String ts2 = "'2022-06-03 15:30:00.000'";
+    expr("DATEADD(year, 1, " + ts + ")")
+        .ok("DATEADD(YEAR, 1, " + ts + ")");
+    expr("DATEADD(y, 1, " + ts + ")")
+        .ok("DATEADD(`Y`, 1, " + ts + ")");
+    expr("DATEDIFF(year, 1, " + ts + ", " + ts2 + ")")
+        .ok("DATEDIFF(YEAR, 1, '2022-06-03 12:00:00.000', "
+            + "'2022-06-03 15:30:00.000')");
+    expr("DATEDIFF(y, 1, " + ts + ", " + ts2 + ")")
+        .ok("DATEDIFF(`Y`, 1, '2022-06-03 12:00:00.000', "
+            + "'2022-06-03 15:30:00.000')");
+    expr("DATE_PART(year, " + ts + ")")
+        .ok("DATE_PART(YEAR, '2022-06-03 12:00:00.000')");
+    expr("DATE_PART(y, " + ts + ")")
+        .ok("DATE_PART(`Y`, '2022-06-03 12:00:00.000')");
   }
 
   /** PostgreSQL and Redshift allow TIMESTAMP literals that contain only a
diff --git a/babel/src/test/java/org/apache/calcite/test/BabelTest.java b/babel/src/test/java/org/apache/calcite/test/BabelTest.java
index 64301cf6e5..afe05ed2ba 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelTest.java
@@ -17,6 +17,11 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.config.CalciteConnectionProperty;
+import org.apache.calcite.rel.type.DelegatingTypeSystem;
+import org.apache.calcite.rel.type.TimeFrameSet;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.fun.SqlLibrary;
+import org.apache.calcite.sql.fun.SqlLibraryOperatorTableFactory;
 import org.apache.calcite.sql.parser.SqlParserFixture;
 import org.apache.calcite.sql.parser.babel.SqlBabelParserImpl;
 
@@ -61,6 +66,11 @@ class BabelTest {
             Boolean.toString(lenient));
   }
 
+  static SqlOperatorTable operatorTableFor(SqlLibrary library) {
+    return SqlLibraryOperatorTableFactory.INSTANCE.getOperatorTable(
+        SqlLibrary.STANDARD, library);
+  }
+
   static Connection connect() throws SQLException {
     return connect(UnaryOperator.identity());
   }
@@ -130,6 +140,57 @@ class BabelTest {
         .fails("(?s).*Encountered \":\" at .*");
   }
 
+  /** Tests that DATEADD, DATEDIFF, DATEPART, DATE_PART allow custom time
+   * frames. */
+  @Test void testTimeFrames() {
+    final SqlValidatorFixture f = Fixtures.forValidator()
+        .withParserConfig(p -> p.withParserFactory(SqlBabelParserImpl.FACTORY))
+        .withOperatorTable(operatorTableFor(SqlLibrary.MSSQL))
+        .withFactory(tf ->
+            tf.withTypeSystem(typeSystem ->
+                new DelegatingTypeSystem(typeSystem) {
+                  @Override public TimeFrameSet deriveTimeFrameSet(
+                      TimeFrameSet frameSet) {
+                    return TimeFrameSet.builder()
+                        .addAll(frameSet)
+                        .addDivision("minute15", 4, "HOUR")
+                        .build();
+                  }
+                }));
+
+    final String ts = "timestamp '2020-06-27 12:34:56'";
+    final String ts2 = "timestamp '2020-06-27 13:45:56'";
+    f.withSql("SELECT DATEADD(YEAR, 3, " + ts + ")").ok();
+    f.withSql("SELECT DATEADD(HOUR^.^A, 3, " + ts + ")")
+        .fails("(?s).*Encountered \".\" at .*");
+    f.withSql("SELECT DATEADD(^A^, 3, " + ts + ")")
+        .fails("'A' is not a valid time frame");
+    f.withSql("SELECT DATEADD(minute15, 3, " + ts + ")")
+        .ok();
+    f.withSql("SELECT DATEDIFF(^A^, " + ts + ", " + ts2 + ")")
+        .fails("'A' is not a valid time frame");
+    f.withSql("SELECT DATEDIFF(minute15, " + ts + ", " + ts2 + ")")
+        .ok();
+    f.withSql("SELECT DATEPART(^A^, " + ts + ")")
+        .fails("'A' is not a valid time frame");
+    f.withSql("SELECT DATEPART(minute15, " + ts + ")")
+        .ok();
+
+    // Where DATEPART is MSSQL, DATE_PART is Postgres
+    f.withSql("SELECT ^DATE_PART(A, " + ts + ")^")
+        .fails("No match found for function signature "
+            + "DATE_PART\\(<INTERVAL_DAY_TIME>, <TIMESTAMP>\\)");
+    final SqlValidatorFixture f2 =
+        f.withOperatorTable(operatorTableFor(SqlLibrary.POSTGRESQL));
+    f2.withSql("SELECT ^DATEPART(A, " + ts + ")^")
+        .fails("No match found for function signature "
+            + "DATEPART\\(<INTERVAL_DAY_TIME>, <TIMESTAMP>\\)");
+    f2.withSql("SELECT DATE_PART(^A^, " + ts + ")")
+        .fails("'A' is not a valid time frame");
+    f2.withSql("SELECT DATE_PART(minute15, " + ts + ")")
+        .ok();
+  }
+
   @Test void testNullSafeEqual() {
     // x <=> y
     checkSqlResult("mysql", "SELECT 1 <=> NULL", "EXPR$0=false\n");
diff --git a/bom/build.gradle.kts b/bom/build.gradle.kts
index 019994894d..438b8b15a2 100644
--- a/bom/build.gradle.kts
+++ b/bom/build.gradle.kts
@@ -97,6 +97,7 @@ dependencies {
         apiv("org.apache.cassandra:cassandra-all")
         apiv("org.apache.commons:commons-dbcp2")
         apiv("org.apache.commons:commons-lang3")
+        apiv("org.apache.commons:commons-math3")
         apiv("org.apache.commons:commons-pool2")
         apiv("org.apache.geode:geode-core")
         apiv("org.apache.hadoop:hadoop-client", "hadoop")
diff --git a/core/build.gradle.kts b/core/build.gradle.kts
index dc671be395..3bafb99464 100644
--- a/core/build.gradle.kts
+++ b/core/build.gradle.kts
@@ -70,6 +70,7 @@ dependencies {
     implementation("net.hydromatic:aggdesigner-algorithm")
     implementation("org.apache.commons:commons-dbcp2")
     implementation("org.apache.commons:commons-lang3")
+    implementation("org.apache.commons:commons-math3")
     implementation("commons-io:commons-io")
     implementation("org.codehaus.janino:commons-compiler")
     implementation("org.codehaus.janino:janino")
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index f258e8235d..c00da30d18 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -161,7 +161,6 @@ public class ${parser.class} extends SqlAbstractParserImpl
     private Casing unquotedCasing;
     private Casing quotedCasing;
     private int identifierMaxLength;
-    private ImmutableMap<String, TimeUnit> timeUnitCodes;
     private SqlConformance conformance;
 
     /**
@@ -223,10 +222,6 @@ public class ${parser.class} extends SqlAbstractParserImpl
         this.identifierMaxLength = identifierMaxLength;
     }
 
-    public void setTimeUnitCodes(Map<String, TimeUnit> timeUnitCodes) {
-        this.timeUnitCodes = ImmutableMap.copyOf(timeUnitCodes);
-    }
-
     public void setConformance(SqlConformance conformance) {
         this.conformance = conformance;
     }
@@ -4949,113 +4944,56 @@ SqlIntervalQualifier IntervalQualifierStart() :
 
 /**
  * Parses time unit for CEIL and FLOOR functions.
+ *
+ * <p>Includes NANOSECOND, MILLISECOND, which were previously only allowed in
+ * the EXTRACT function.
+ *
+ * <p>Does not include SQL_TSI_DAY, SQL_TSI_FRAC_SECOND etc. These will be
+ * parsed as identifiers and can be resolved in the validator if they are
+ * registered as abbreviations in your time frame set.
  */
 TimeUnit TimeUnit() :
 {
-    final TimeUnit unit;
-}
-{
-    LOOKAHEAD(1)
-    (
-        <MILLISECOND> { return TimeUnit.MILLISECOND; }
-    |   <SECOND> { return TimeUnit.SECOND; }
-    |   <MINUTE> { return TimeUnit.MINUTE; }
-    |   <HOUR> { return TimeUnit.HOUR; }
-    |   <DAY> { return TimeUnit.DAY; }
-    |   <DOW> { return TimeUnit.DOW; }
-    |   <DOY> { return TimeUnit.DOY; }
-    |   <ISODOW> { return TimeUnit.ISODOW; }
-    |   <ISOYEAR> { return TimeUnit.ISOYEAR; }
-    |   <WEEK> { return TimeUnit.WEEK; }
-    |   <MONTH> { return TimeUnit.MONTH; }
-    |   <QUARTER> { return TimeUnit.QUARTER; }
-    |   <YEAR> { return TimeUnit.YEAR; }
-    |   <EPOCH> { return TimeUnit.EPOCH; }
-    |   <DECADE> { return TimeUnit.DECADE; }
-    |   <CENTURY> { return TimeUnit.CENTURY; }
-    |   <MILLENNIUM> { return TimeUnit.MILLENNIUM; }
-    )
-|
-    unit = TimeUnitIdentifier() { return unit; }
-}
-
-/**
- * Parses time unit for the EXTRACT function.
- * As for FLOOR and CEIL, but also includes NANOSECOND and MICROSECOND.
- */
-TimeUnit TimeUnitForExtract() :
-{
-    final TimeUnit unit;
-}
-{
-    LOOKAHEAD(1)
-    (
-        <NANOSECOND> { return TimeUnit.NANOSECOND; }
-    |   <MICROSECOND> { return TimeUnit.MICROSECOND; }
-    )
-|
-    unit = TimeUnit() { return unit; }
 }
-
-/**
- * Parses time unit for the TIME_TRUNC function.
- * This is a subset of time units appropriate for TIME data type.
- */
-TimeUnit TimeUnitForTimeTrunc() :
-{}
 {
-    <MILLISECOND> { return TimeUnit.MILLISECOND; }
-|   <SECOND> { return TimeUnit.SECOND; }
-|   <MINUTE> { return TimeUnit.MINUTE; }
-|   <HOUR> { return TimeUnit.HOUR; }
-}
-
-/**
- * Parses a simple identifier as a TimeUnit.
- */
-TimeUnit TimeUnitIdentifier() :
-{
-    final List<String> names = new ArrayList<String>();
-    final List<SqlParserPos> positions = new ArrayList<SqlParserPos>();
-}
-{
-    AddIdentifierSegment(names, positions) {
-        TimeUnit unit = timeUnitCodes.get(names.get(0));
-        if (unit != null) {
-          return unit;
-        }
-        throw SqlUtil.newContextException(positions.get(0),
-            RESOURCE.invalidDatetimeFormat(SqlIdentifier.getString(names)));
-    }
-}
-
-TimeUnit TimestampInterval() :
-{}
-{
-    <FRAC_SECOND> { return TimeUnit.MICROSECOND; }
+    <NANOSECOND> { return TimeUnit.NANOSECOND; }
 |   <MICROSECOND> { return TimeUnit.MICROSECOND; }
-|   <NANOSECOND> { return TimeUnit.NANOSECOND; }
-|   <SQL_TSI_FRAC_SECOND> { return TimeUnit.NANOSECOND; }
-|   <SQL_TSI_MICROSECOND> { return TimeUnit.MICROSECOND; }
+|   <MILLISECOND> { return TimeUnit.MILLISECOND; }
 |   <SECOND> { return TimeUnit.SECOND; }
-|   <SQL_TSI_SECOND> { return TimeUnit.SECOND; }
 |   <MINUTE> { return TimeUnit.MINUTE; }
-|   <SQL_TSI_MINUTE> { return TimeUnit.MINUTE; }
 |   <HOUR> { return TimeUnit.HOUR; }
-|   <SQL_TSI_HOUR> { return TimeUnit.HOUR; }
 |   <DAY> { return TimeUnit.DAY; }
-|   <SQL_TSI_DAY> { return TimeUnit.DAY; }
+|   <DOW> { return TimeUnit.DOW; }
+|   <DOY> { return TimeUnit.DOY; }
+|   <ISODOW> { return TimeUnit.ISODOW; }
+|   <ISOYEAR> { return TimeUnit.ISOYEAR; }
 |   <WEEK> { return TimeUnit.WEEK; }
-|   <SQL_TSI_WEEK> { return TimeUnit.WEEK; }
 |   <MONTH> { return TimeUnit.MONTH; }
-|   <SQL_TSI_MONTH> { return TimeUnit.MONTH; }
 |   <QUARTER> { return TimeUnit.QUARTER; }
-|   <SQL_TSI_QUARTER> { return TimeUnit.QUARTER; }
 |   <YEAR> { return TimeUnit.YEAR; }
-|   <SQL_TSI_YEAR> { return TimeUnit.YEAR; }
+|   <EPOCH> { return TimeUnit.EPOCH; }
+|   <DECADE> { return TimeUnit.DECADE; }
+|   <CENTURY> { return TimeUnit.CENTURY; }
+|   <MILLENNIUM> { return TimeUnit.MILLENNIUM; }
 }
 
-
+/** Parses a built-in time unit (e.g. "YEAR")
+ * or user-defined time frame (e.g. "MINUTE15")
+ * and in each case returns a {@link SqlIntervalQualifier}. */
+SqlIntervalQualifier TimeUnitOrName() : {
+    final TimeUnit unit;
+    final SqlIdentifier unitName;
+}
+{
+    LOOKAHEAD(1)
+    unit = TimeUnit() {
+        return new SqlIntervalQualifier(unit, null, getPos());
+    }
+|   unitName = SimpleIdentifier() {
+        return new SqlIntervalQualifier(unitName.getSimple(),
+            unitName.getParserPosition());
+    }
+}
 
 /**
  * Parses a dynamic parameter marker.
@@ -5885,7 +5823,7 @@ SqlNode BuiltinFunctionCall() :
     SqlNode e;
     final Span s;
     SqlDataTypeSpec dt;
-    final TimeUnit unit;
+    final SqlIntervalQualifier unit;
     final SqlNode node;
 }
 {
@@ -5904,8 +5842,8 @@ SqlNode BuiltinFunctionCall() :
         }
     |
         <EXTRACT> { s = span(); }
-        <LPAREN> unit = TimeUnitForExtract() {
-            args.add(new SqlIntervalQualifier(unit, null, getPos()));
+        <LPAREN> unit = TimeUnitOrName() {
+            args.add(unit);
         }
         <FROM>
         AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
@@ -6556,14 +6494,12 @@ SqlCall TimestampAddFunctionCall() :
 {
     final List<SqlNode> args = new ArrayList<SqlNode>();
     final Span s;
-    final TimeUnit interval;
+    final SqlIntervalQualifier unit;
 }
 {
     <TIMESTAMPADD> { s = span(); }
     <LPAREN>
-    interval = TimestampInterval() {
-        args.add(SqlLiteral.createSymbol(interval, getPos()));
-    }
+    unit = TimeUnitOrName() { args.add(unit); }
     <COMMA>
     AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
     <COMMA>
@@ -6581,14 +6517,12 @@ SqlCall TimestampDiffFunctionCall() :
 {
     final List<SqlNode> args = new ArrayList<SqlNode>();
     final Span s;
-    final TimeUnit interval;
+    final SqlIntervalQualifier unit;
 }
 {
     <TIMESTAMPDIFF> { s = span(); }
     <LPAREN>
-    interval = TimestampInterval() {
-        args.add(SqlLiteral.createSymbol(interval, getPos()));
-    }
+    unit = TimeUnitOrName() { args.add(unit); }
     <COMMA>
     AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
     <COMMA>
@@ -6606,16 +6540,14 @@ SqlCall TimestampTruncFunctionCall() :
 {
     final List<SqlNode> args = new ArrayList<SqlNode>();
     final Span s;
-    final TimeUnit unit;
+    final SqlIntervalQualifier unit;
 }
 {
     <TIMESTAMP_TRUNC> { s = span(); }
     <LPAREN>
     AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
     <COMMA>
-    unit = TimeUnit() {
-        args.add(new SqlIntervalQualifier(unit, null, getPos()));
-    }
+    unit = TimeUnitOrName() { args.add(unit); }
     <RPAREN> {
         return SqlLibraryOperators.TIMESTAMP_TRUNC.createCall(s.end(this), args);
     }
@@ -6628,16 +6560,14 @@ SqlCall TimeTruncFunctionCall() :
 {
     final List<SqlNode> args = new ArrayList<SqlNode>();
     final Span s;
-    final TimeUnit unit;
+    final SqlIntervalQualifier unit;
 }
 {
     <TIME_TRUNC> { s = span(); }
     <LPAREN>
     AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
     <COMMA>
-    unit = TimeUnitForTimeTrunc() {
-        args.add(new SqlIntervalQualifier(unit, null, getPos()));
-    }
+    unit = TimeUnitOrName() { args.add(unit); }
     <RPAREN> {
         return SqlLibraryOperators.TIME_TRUNC.createCall(s.end(this), args);
     }
@@ -6942,16 +6872,15 @@ SqlNode StandardFloorCeilOptions(Span s, boolean floorFlag) :
 {
     SqlNode e;
     final List<SqlNode> args = new ArrayList<SqlNode>();
-    TimeUnit unit;
+    final SqlIntervalQualifier unit;
     SqlCall function;
     final Span s1;
 }
 {
     <LPAREN> AddExpression(args, ExprContext.ACCEPT_SUB_QUERY)
     (
-        <TO>
-        unit = TimeUnit() {
-            args.add(new SqlIntervalQualifier(unit, null, getPos()));
+        <TO> unit = TimeUnitOrName() {
+            args.add(unit);
         }
     )?
     <RPAREN> {
diff --git a/core/src/main/java/org/apache/calcite/DataContext.java b/core/src/main/java/org/apache/calcite/DataContext.java
index a579b0abc3..9632b36173 100644
--- a/core/src/main/java/org/apache/calcite/DataContext.java
+++ b/core/src/main/java/org/apache/calcite/DataContext.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.advise.SqlAdvisor;
 
@@ -117,6 +118,11 @@ public interface DataContext {
      * time zone. */
     TIME_ZONE("timeZone", TimeZone.class),
 
+    /** Set of built-in and custom time frames for use in functions such as
+     * {@code FLOOR} and {@code EXTRACT}. Required; defaults to
+     * {@link org.apache.calcite.rel.type.TimeFrames#CORE}. */
+    TIME_FRAME_SET("timeFrameSet", TimeFrameSet.class),
+
     /** The query user.
      *
      * <p>Default value is "sa". */
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 11896896fe..396f640786 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
@@ -123,6 +123,7 @@ import static org.apache.calcite.sql.fun.SqlLibraryOperators.CONCAT2;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.CONCAT_FUNCTION;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.COSH;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATE;
+import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATEADD;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.DATE_FROM_UNIX_DATE;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.DAYNAME;
 import static org.apache.calcite.sql.fun.SqlLibraryOperators.DIFFERENCE;
@@ -315,6 +316,8 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUM;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SUM0;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.SYSTEM_USER;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TAN;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TIMESTAMP_ADD;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TIMESTAMP_DIFF;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRIM;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TRUNCATE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.TUMBLE;
@@ -483,21 +486,29 @@ public class RexImpTable {
       map.put(FLOOR,
           new FloorImplementor(BuiltInMethod.FLOOR.method.getName(),
               BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method,
-              BuiltInMethod.UNIX_DATE_FLOOR.method));
+            BuiltInMethod.UNIX_DATE_FLOOR.method,
+            BuiltInMethod.CUSTOM_TIMESTAMP_FLOOR.method,
+            BuiltInMethod.CUSTOM_DATE_FLOOR.method));
       map.put(CEIL,
           new FloorImplementor(BuiltInMethod.CEIL.method.getName(),
               BuiltInMethod.UNIX_TIMESTAMP_CEIL.method,
-              BuiltInMethod.UNIX_DATE_CEIL.method));
-
-      // TIMESTAMP_TRUNC and TIME_TRUNC methods are syntactic sugar for standard datetime FLOOR
-      map.put(TIMESTAMP_TRUNC,
-          new FloorImplementor(BuiltInMethod.FLOOR.method.getName(),
-              BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method,
-              BuiltInMethod.UNIX_DATE_FLOOR.method));
-      map.put(TIME_TRUNC,
-          new FloorImplementor(BuiltInMethod.FLOOR.method.getName(),
-              BuiltInMethod.UNIX_TIMESTAMP_FLOOR.method,
-              BuiltInMethod.UNIX_DATE_FLOOR.method));
+            BuiltInMethod.UNIX_DATE_CEIL.method,
+            BuiltInMethod.CUSTOM_TIMESTAMP_CEIL.method,
+            BuiltInMethod.CUSTOM_DATE_CEIL.method));
+      map.put(TIMESTAMP_ADD,
+          new TimestampAddImplementor("timestampAdd",
+              BuiltInMethod.CUSTOM_TIMESTAMP_ADD.method,
+              BuiltInMethod.CUSTOM_DATE_ADD.method));
+      map.put(DATEADD, map.get(TIMESTAMP_ADD));
+      map.put(TIMESTAMP_DIFF,
+          new TimestampDiffImplementor("timestampDiff",
+              BuiltInMethod.CUSTOM_TIMESTAMP_DIFF.method,
+              BuiltInMethod.CUSTOM_DATE_DIFF.method));
+
+      // TIMESTAMP_TRUNC and TIME_TRUNC methods are syntactic sugar for standard
+      // datetime FLOOR.
+      map.put(TIMESTAMP_TRUNC, map.get(FLOOR));
+      map.put(TIME_TRUNC, map.get(FLOOR));
 
 
       defineMethod(LAST_DAY, "lastDay", NullPolicy.STRICT);
@@ -2055,12 +2066,17 @@ public class RexImpTable {
   private static class FloorImplementor extends MethodNameImplementor {
     final Method timestampMethod;
     final Method dateMethod;
+    final Method customTimestampMethod;
+    final Method customDateMethod;
 
     FloorImplementor(String methodName, Method timestampMethod,
-        Method dateMethod) {
+        Method dateMethod, Method customTimestampMethod,
+        Method customDateMethod) {
       super(methodName, NullPolicy.STRICT, false);
       this.timestampMethod = timestampMethod;
       this.dateMethod = dateMethod;
+      this.customTimestampMethod = customTimestampMethod;
+      this.customDateMethod = customDateMethod;
     }
 
     @Override String getVariableName() {
@@ -2085,26 +2101,32 @@ public class RexImpTable {
         final Type type;
         final Method floorMethod;
         final boolean preFloor;
-        Expression operand = argValueList.get(0);
+        final Expression operand1 = argValueList.get(1);
+        final boolean custom = operand1.getType() == String.class;
+        Expression operand0 = argValueList.get(0);
         switch (call.getType().getSqlTypeName()) {
         case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-          operand = Expressions.call(
+          operand0 = Expressions.call(
               BuiltInMethod.TIMESTAMP_WITH_LOCAL_TIME_ZONE_TO_TIMESTAMP.method,
-              operand,
+              operand0,
               Expressions.call(BuiltInMethod.TIME_ZONE.method, translator.getRoot()));
           // fall through
         case TIMESTAMP:
           type = long.class;
-          floorMethod = timestampMethod;
+          floorMethod = custom ? customTimestampMethod : timestampMethod;
           preFloor = true;
           break;
         default:
           type = int.class;
-          floorMethod = dateMethod;
+          floorMethod = custom ? customDateMethod : dateMethod;
           preFloor = false;
         }
+        if (custom) {
+          return Expressions.call(floorMethod, translator.getRoot(),
+              operand1, operand0);
+        }
         final TimeUnitRange timeUnitRange =
-            (TimeUnitRange) requireNonNull(translator.getLiteralValue(argValueList.get(1)),
+            (TimeUnitRange) requireNonNull(translator.getLiteralValue(operand1),
             "timeUnitRange");
         switch (timeUnitRange) {
         case YEAR:
@@ -2112,13 +2134,12 @@ public class RexImpTable {
         case MONTH:
         case WEEK:
         case DAY:
-          final Expression operand1 =
-              preFloor ? call(operand, type, TimeUnit.DAY) : operand;
+          final Expression dayOperand0 =
+              preFloor ? call(operand0, type, TimeUnit.DAY) : operand0;
           return Expressions.call(floorMethod,
-              translator.getLiteral(argValueList.get(1)), operand1);
-        case NANOSECOND:
+              translator.getLiteral(operand1), dayOperand0);
         default:
-          return call(operand, type, timeUnitRange.startUnit);
+          return call(operand0, type, timeUnitRange.startUnit);
         }
 
       default:
@@ -2135,6 +2156,72 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code TIMESTAMPADD} function. */
+  private static class TimestampAddImplementor extends MethodNameImplementor {
+    final Method customTimestampMethod;
+    final Method customDateMethod;
+
+    TimestampAddImplementor(String methodName, Method customTimestampMethod,
+        Method customDateMethod) {
+      super(methodName, NullPolicy.STRICT, false);
+      this.customTimestampMethod = customTimestampMethod;
+      this.customDateMethod = customDateMethod;
+    }
+
+    @Override String getVariableName() {
+      return "timestampAdd";
+    }
+
+    @Override Expression implementSafe(final RexToLixTranslator translator,
+        final RexCall call, final List<Expression> argValueList) {
+      final Expression operand0 = argValueList.get(0);
+      final Expression operand1 = argValueList.get(1);
+      final Expression operand2 = argValueList.get(2);
+      switch (call.getType().getSqlTypeName()) {
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+      case TIMESTAMP:
+        return Expressions.call(customTimestampMethod, translator.getRoot(),
+            operand0, operand1, operand2);
+      default:
+        return Expressions.call(customDateMethod, translator.getRoot(),
+            operand0, operand1, operand2);
+      }
+    }
+  }
+
+  /** Implementor for the {@code TIMESTAMPDIFF} function. */
+  private static class TimestampDiffImplementor extends MethodNameImplementor {
+    final Method customTimestampMethod;
+    final Method customDateMethod;
+
+    TimestampDiffImplementor(String methodName, Method customTimestampMethod,
+        Method customDateMethod) {
+      super(methodName, NullPolicy.STRICT, false);
+      this.customTimestampMethod = customTimestampMethod;
+      this.customDateMethod = customDateMethod;
+    }
+
+    @Override String getVariableName() {
+      return "timestampDiff";
+    }
+
+    @Override Expression implementSafe(final RexToLixTranslator translator,
+        final RexCall call, final List<Expression> argValueList) {
+      return Expressions.call(getMethod(call), translator.getRoot(),
+          argValueList.get(0), argValueList.get(1), argValueList.get(2));
+    }
+
+    private Method getMethod(RexCall call) {
+      switch (call.operands.get(1).getType().getSqlTypeName()) {
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+      case TIMESTAMP:
+        return customTimestampMethod;
+      default:
+        return customDateMethod;
+      }
+    }
+  }
+
   /** Implementor for a function that generates calls to a given method. */
   private static class MethodImplementor extends AbstractRexCallImplementor {
     protected final Method method;
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
index 03deda30f4..cf01ec65e1 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -48,6 +48,8 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.rel.type.DelegatingTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.TimeFrameSet;
+import org.apache.calcite.rel.type.TimeFrames;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.SchemaVersion;
@@ -419,6 +421,9 @@ abstract class CalciteConnectionImpl
       Hook.CURRENT_TIME.run(timeHolder);
       final long time = timeHolder.get();
       final TimeZone timeZone = connection.getTimeZone();
+      final TimeFrameSet timeFrameSet =
+          connection.typeFactory.getTypeSystem()
+              .deriveTimeFrameSet(TimeFrames.CORE);
       final long localOffset = timeZone.getOffset(time);
       final long currentOffset = localOffset;
       final String user = "sa";
@@ -437,6 +442,7 @@ abstract class CalciteConnectionImpl
           .put(Variable.CURRENT_TIMESTAMP.camelName, time + currentOffset)
           .put(Variable.LOCAL_TIMESTAMP.camelName, time + localOffset)
           .put(Variable.TIME_ZONE.camelName, timeZone)
+          .put(Variable.TIME_FRAME_SET.camelName, timeFrameSet)
           .put(Variable.USER.camelName, user)
           .put(Variable.SYSTEM_USER.camelName, systemUser)
           .put(Variable.LOCALE.camelName, locale)
diff --git a/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java
index 76af9dfd76..96b8628a9b 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/DelegatingTypeSystem.java
@@ -96,4 +96,8 @@ public class DelegatingTypeSystem implements RelDataTypeSystem {
   @Override public boolean shouldConvertRaggedUnionTypesToVarying() {
     return typeSystem.shouldConvertRaggedUnionTypesToVarying();
   }
+
+  @Override public TimeFrameSet deriveTimeFrameSet(TimeFrameSet frameSet) {
+    return typeSystem.deriveTimeFrameSet(frameSet);
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
index 5f6095da76..1b4cc44632 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeSystem.java
@@ -407,4 +407,21 @@ public interface RelDataTypeSystem {
     return null;
   }
 
+  /** Returns a list of supported time frames.
+   *
+   * <p>The validator calls this method with {@link TimeFrames#CORE} as an
+   * argument, and the default implementation of this method returns its input,
+   * and therefore {@link TimeFrames#CORE CORE} is the default time frame set.
+   *
+   * <p>If you wish to use a custom time frame set, create an instance of
+   * {@code RelDataTypeSystem} that overrides this method. Your method should
+   * call {@link TimeFrameSet#builder()}, will probably add all or most of the
+   * time frames in the {@code frameSet} argument, and then call
+   * {@link TimeFrameSet.Builder#build()}.
+   *
+   * @param frameSet Set of built-in time frames
+   */
+  default TimeFrameSet deriveTimeFrameSet(TimeFrameSet frameSet) {
+    return frameSet;
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/rel/type/TimeFrame.java b/core/src/main/java/org/apache/calcite/rel/type/TimeFrame.java
new file mode 100644
index 0000000000..1ea43217d7
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/TimeFrame.java
@@ -0,0 +1,113 @@
+/*
+ * 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.rel.type;
+
+import org.apache.calcite.avatica.util.TimeUnit;
+
+import org.apache.commons.math3.fraction.BigFraction;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** Time frame.
+ *
+ * <p>Belongs to a {@link TimeFrameSet}.
+ * The default set is {@link TimeFrames#CORE};
+ * to create custom time frame sets, call {@link TimeFrameSet#builder()}. */
+public interface TimeFrame {
+  /** Returns the time frame set that this frame belongs to. */
+  TimeFrameSet frameSet();
+
+  /** Name of this time frame.
+   *
+   * <p>A time frame based on a built-in Avatica
+   * {@link org.apache.calcite.avatica.util.TimeUnit} will have the same
+   * name.
+   *
+   * @see TimeFrameSet#get(TimeUnit) */
+  String name();
+
+  /** If this time frame has units in common with another time frame, returns
+   * the number of this time frame in one of that time frame.
+   *
+   * <p>For example, {@code MONTH.per(YEAR)} returns 12;
+   * {@code YEAR.per(MONTH)} returns 1 / 12.
+   */
+  @Nullable BigFraction per(TimeFrame timeFrame);
+
+  /** Returns a date where this time frame is at the start of a cycle.
+   *
+   * <p>For example, the {@code WEEK} time frame starts on a Monday,
+   * and {@code 1970-01-05} was a Monday,
+   * and the date {@code 1970-01-05} is represented as integer 5,
+   * so for the {@code WEEK} time frame this method returns 5.
+   * But it would also be valid to return the date value of {@code 1900/01/01},
+   * which was also a Monday.  Because we know that a week is 7 days, we can
+   * compute every other point at which a week advances. */
+  default int dateEpoch() {
+    return 0;
+  }
+
+  /** Returns a timestamp where this time frame is at the start of a cycle.
+   *
+   * @see #dateEpoch() */
+  default long timestampEpoch() {
+    return 0L;
+  }
+
+  /** Returns a month number where this time frame is at the start of a cycle.
+   *
+   * @see #dateEpoch()
+   */
+  default int monthEpoch() {
+    return 0;
+  }
+
+  /** Whether this frame can roll up to {@code toFrame}.
+   *
+   * <p>Examples:
+   * <ul>
+   *   <li>{@code SECOND} can roll up to {@code MINUTE}, {@code HOUR},
+   *   {@code DAY}, {@code WEEK}, {@code MONTH}, {@code MILLENNIUM};
+   *   <li>{@code SECOND} cannot roll up to {@code MILLISECOND} (because it is
+   *   finer grained);
+   *   <li>{@code WEEK} cannot roll up to {@code MONTH}, {@code YEAR},
+   *   {@code MILLENNIUM} (because weeks cross month boundaries).
+   * </ul>
+   *
+   * <p>If two time frames have the same core, and one is an integer simple
+   * multiple of another, and they have the same offset, then they can roll up.
+   * For example, suppose that {@code MINUTE15} and {@code HOUR3} are both based
+   * on {@code SECOND};
+   * {@code MINUTE15} is 15 * 60 seconds and
+   * {@code HOUR3} is 3 * 60 * 60 seconds;
+   * therefore one {@code HOUR3} interval equals twelve {@code MINUTE15}
+   * intervals.
+   * They have the same offset (both start at {@code 1970-01-01 00:00:00}) and
+   * therefore {@code MINUTE15} can roll up to {@code HOUR3}.
+   *
+   * <p>Even if two frames are not multiples, if they are aligned then they can
+   * roll up. {@code MONTH} and {@code DAY} are an example. For more about
+   * alignment, see {@link TimeFrameSet.Builder#addRollup(String, String)}.
+   */
+  boolean canRollUpTo(TimeFrame toFrame);
+
+  /** Returns the built-in unit of this frame, or null if it does not correspond
+   * to a built-in unit. */
+  default @Nullable TimeUnit unit() {
+    return frameSet().getUnit(this);
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java b/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java
new file mode 100644
index 0000000000..52713dc122
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java
@@ -0,0 +1,365 @@
+/*
+ * 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.rel.type;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.runtime.SqlFunctions;
+import org.apache.calcite.util.NameMap;
+import org.apache.calcite.util.TimestampString;
+import org.apache.calcite.util.Util;
+
+import org.apache.commons.math3.fraction.BigFraction;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Iterables;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.math.BigInteger;
+import java.util.NavigableMap;
+import java.util.Objects;
+
+import static org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY;
+
+import static java.lang.Math.floorDiv;
+import static java.lang.Math.floorMod;
+import static java.util.Objects.requireNonNull;
+
+/** Set of {@link TimeFrame} definitions.
+ *
+ * <p>Every SQL statement has a time frame set, and is accessed via
+ * {@link RelDataTypeSystem#deriveTimeFrameSet(TimeFrameSet)}. If you want to
+ * use a custom set of time frames, you should override that method. */
+public class TimeFrameSet {
+  final ImmutableMap<String, TimeFrames.TimeFrameImpl> map;
+  final ImmutableMultimap<TimeFrames.TimeFrameImpl, TimeFrames.TimeFrameImpl> rollupMap;
+  private final NameMap<TimeFrames.TimeFrameImpl> nameMap;
+
+  TimeFrameSet(ImmutableMap<String, TimeFrames.TimeFrameImpl> map,
+      ImmutableMultimap<TimeFrames.TimeFrameImpl, TimeFrames.TimeFrameImpl> rollupMap) {
+    this.map = requireNonNull(map, "map");
+    this.nameMap = NameMap.immutableCopyOf(map);
+    this.rollupMap = requireNonNull(rollupMap, "rollupMap");
+  }
+
+  /** Creates a Builder. */
+  public static Builder builder() {
+    return new TimeFrames.BuilderImpl();
+  }
+
+  /** Returns the time frame with the given name (case-insensitive),
+   * or returns null. */
+  public @Nullable TimeFrame getOpt(String name) {
+    final NavigableMap<String, TimeFrames.TimeFrameImpl> range =
+        nameMap.range(name, false);
+    @Nullable TimeFrame timeFrame =
+        Iterables.getFirst(range.values(), null);
+    while (timeFrame instanceof TimeFrames.AliasFrame) {
+      timeFrame = ((TimeFrames.AliasFrame) timeFrame).frame;
+    }
+    return timeFrame;
+  }
+
+  /** Returns the time frame with the given name,
+   * or throws {@link IllegalArgumentException} if not found.
+   * If {@code name} is an alias, resolves to the underlying frame. */
+  public TimeFrame get(String name) {
+    TimeFrame timeFrame = getOpt(name);
+    if (timeFrame == null) {
+      throw new IllegalArgumentException("unknown frame: " + name);
+    }
+    return timeFrame;
+  }
+
+  /** Returns the time frame with the given name,
+   * or throws {@link IllegalArgumentException}. */
+  public TimeFrame get(TimeUnit timeUnit) {
+    return get(timeUnit.name());
+  }
+
+  /** Computes "FLOOR(date TO frame)", where {@code date} is the number of
+   * days since UNIX Epoch. */
+  public int floorDate(int date, TimeFrame frame) {
+    return floorCeilDate(date, frame, false);
+  }
+
+  /** Computes "FLOOR(date TO frame)", where {@code date} is the number of
+   * days since UNIX Epoch. */
+  public int ceilDate(int date, TimeFrame frame) {
+    return floorCeilDate(date, frame, true);
+  }
+
+  /** Computes "FLOOR(timestamp TO frame)" or "FLOOR(timestamp TO frame)",
+   * where {@code date} is the number of days since UNIX Epoch. */
+  private int floorCeilDate(int date, TimeFrame frame, boolean ceil) {
+    final TimeFrame dayFrame = get(TimeUnit.DAY);
+    final BigFraction perDay = frame.per(dayFrame);
+    if (perDay != null
+        && perDay.getNumerator().equals(BigInteger.ONE)) {
+      final int m = perDay.getDenominator().intValueExact(); // 7 for WEEK
+      final int mod = floorMod(date - frame.dateEpoch(), m);
+      return date - mod + (ceil ? m : 0);
+    }
+    final TimeFrame monthFrame = get(TimeUnit.MONTH);
+    final BigFraction perMonth = frame.per(monthFrame);
+    if (perMonth != null
+        && perMonth.getNumerator().equals(BigInteger.ONE)) {
+      final int y2 =
+          (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
+      final int m2 =
+          (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date);
+      final int fullMonth = TimeFrames.fullMonth(y2, m2);
+
+      final int m = perMonth.getDenominator().intValueExact(); // e.g. 12 for YEAR
+      final int mod = floorMod(fullMonth - frame.monthEpoch(), m);
+      return TimeFrames.mdToUnixDate(fullMonth - mod + (ceil ? m : 0), 1);
+    }
+    final TimeFrame isoYearFrame = get(TimeUnit.ISOYEAR);
+    final BigFraction perIsoYear = frame.per(isoYearFrame);
+    if (perIsoYear != null
+        && perIsoYear.getNumerator().equals(BigInteger.ONE)) {
+      return TimeFrames.floorCeilIsoYear(date, ceil);
+    }
+    return date;
+  }
+
+  /** Computes "FLOOR(timestamp TO frame)", where {@code ts} is the number of
+   * milliseconds since UNIX Epoch. */
+  public long floorTimestamp(long ts, TimeFrame frame) {
+    return floorCeilTimestamp(ts, frame, false);
+  }
+
+  /** Computes "CEIL(timestamp TO frame)", where {@code ts} is the number of
+   * milliseconds since UNIX Epoch. */
+  public long ceilTimestamp(long ts, TimeFrame frame) {
+    return floorCeilTimestamp(ts, frame, true);
+  }
+
+  /** Computes "FLOOR(ts TO frame)" or "CEIL(ts TO frame)",
+   * where {@code ts} is the number of milliseconds since UNIX Epoch. */
+  private long floorCeilTimestamp(long ts, TimeFrame frame, boolean ceil) {
+    final TimeFrame millisecondFrame = get(TimeUnit.MILLISECOND);
+    final BigFraction perMillisecond = frame.per(millisecondFrame);
+    if (perMillisecond != null
+        && perMillisecond.getNumerator().equals(BigInteger.ONE)) {
+      final long m = perMillisecond.getDenominator().longValue(); // e.g. 60,000 for MINUTE
+      final long mod = floorMod(ts - frame.timestampEpoch(), m);
+      return ts - mod + (ceil ? m : 0);
+    }
+    final TimeFrame monthFrame = get(TimeUnit.MONTH);
+    final BigFraction perMonth = frame.per(monthFrame);
+    if (perMonth != null
+        && perMonth.getNumerator().equals(BigInteger.ONE)) {
+      final long ts2 = floorTimestamp(ts, get(TimeUnit.DAY));
+      final int d2 = (int) (ts2 / DateTimeUtils.MILLIS_PER_DAY);
+      final int y2 =
+          (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, d2);
+      final int m2 =
+          (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, d2);
+      final int fullMonth = TimeFrames.fullMonth(y2, m2);
+
+      final int m = perMonth.getDenominator().intValueExact(); // e.g. 12 for YEAR
+      final int mod = floorMod(fullMonth - frame.monthEpoch(), m);
+      return TimeFrames.unixTimestamp(fullMonth - mod + (ceil ? m : 0), 1, 0, 0, 0);
+    }
+    final TimeFrame isoYearFrame = get(TimeUnit.ISOYEAR);
+    final BigFraction perIsoYear = frame.per(isoYearFrame);
+    if (perIsoYear != null
+        && perIsoYear.getNumerator().equals(BigInteger.ONE)) {
+      final long ts2 = floorTimestamp(ts, get(TimeUnit.DAY));
+      final int d2 = (int) (ts2 / DateTimeUtils.MILLIS_PER_DAY);
+      return (long) TimeFrames.floorCeilIsoYear(d2, ceil) * MILLIS_PER_DAY;
+    }
+    return ts;
+  }
+
+  /** Returns the time unit that this time frame is based upon, or null. */
+  public @Nullable TimeUnit getUnit(TimeFrame timeFrame) {
+    final TimeUnit timeUnit = Util.enumVal(TimeUnit.class, timeFrame.name());
+    if (timeUnit == null) {
+      return null;
+    }
+    TimeFrame timeFrame1 = getOpt(timeUnit.name());
+    return Objects.equals(timeFrame1, timeFrame) ? timeUnit : null;
+  }
+
+  public int addDate(int date, int interval, TimeFrame frame) {
+    final TimeFrame dayFrame = get(TimeUnit.DAY);
+    final BigFraction perDay = frame.per(dayFrame);
+    if (perDay != null
+        && perDay.getNumerator().equals(BigInteger.ONE)) {
+      final int m = perDay.getDenominator().intValueExact(); // 7 for WEEK
+      return date + interval * m;
+    }
+
+    final TimeFrame monthFrame = get(TimeUnit.MONTH);
+    final BigFraction perMonth = frame.per(monthFrame);
+    if (perMonth != null
+        && perMonth.getNumerator().equals(BigInteger.ONE)) {
+      final int m = perMonth.getDenominator().intValueExact(); // e.g. 12 for YEAR
+      return SqlFunctions.addMonths(date, interval * m);
+    }
+
+    // Unknown time frame. Return the original value unchanged.
+    return date;
+  }
+
+  public long addTimestamp(long timestamp, long interval, TimeFrame frame) {
+    final TimeFrame msFrame = get(TimeUnit.MILLISECOND);
+    final BigFraction perMilli = frame.per(msFrame);
+    if (perMilli != null
+        && perMilli.getNumerator().equals(BigInteger.ONE)) {
+      // 1,000 for SECOND, 86,400,000 for DAY
+      final long m = perMilli.getDenominator().longValueExact();
+      return timestamp + interval * m;
+    }
+    final TimeFrame monthFrame = get(TimeUnit.MONTH);
+    final BigFraction perMonth = frame.per(monthFrame);
+    if (perMonth != null
+        && perMonth.getNumerator().equals(BigInteger.ONE)) {
+      final long m = perMonth.getDenominator().longValueExact(); // e.g. 12 for YEAR
+      return SqlFunctions.addMonths(timestamp, (int) (interval * m));
+    }
+
+    // Unknown time frame. Return the original value unchanged.
+    return timestamp;
+  }
+
+  public int diffDate(int date, int date2, TimeFrame frame) {
+    final TimeFrame dayFrame = get(TimeUnit.DAY);
+    final BigFraction perDay = frame.per(dayFrame);
+    if (perDay != null
+        && perDay.getNumerator().equals(BigInteger.ONE)) {
+      final int m = perDay.getDenominator().intValueExact(); // 7 for WEEK
+      final int delta = date2 - date;
+      return floorDiv(delta, m);
+    }
+
+    final TimeFrame monthFrame = get(TimeUnit.MONTH);
+    final BigFraction perMonth = frame.per(monthFrame);
+    if (perMonth != null
+        && perMonth.getNumerator().equals(BigInteger.ONE)) {
+      final int m = perMonth.getDenominator().intValueExact(); // e.g. 12 for YEAR
+      final int delta = DateTimeUtils.subtractMonths(date2, date);
+      return floorDiv(delta, m);
+    }
+
+    // Unknown time frame. Return the original value unchanged.
+    return date;
+  }
+
+  public long diffTimestamp(long timestamp, long timestamp2, TimeFrame frame) {
+    final TimeFrame msFrame = get(TimeUnit.MILLISECOND);
+    final BigFraction perMilli = frame.per(msFrame);
+    if (perMilli != null
+        && perMilli.getNumerator().equals(BigInteger.ONE)) {
+      // 1,000 for SECOND, 86,400,000 for DAY
+      final long m = perMilli.getDenominator().longValueExact();
+      final long delta = timestamp2 - timestamp;
+      return floorDiv(delta, m);
+    }
+    final TimeFrame monthFrame = get(TimeUnit.MONTH);
+    final BigFraction perMonth = frame.per(monthFrame);
+    if (perMonth != null
+        && perMonth.getNumerator().equals(BigInteger.ONE)) {
+      final long m = perMonth.getDenominator().longValueExact(); // e.g. 12 for YEAR
+      final long delta = DateTimeUtils.subtractMonths(timestamp2, timestamp);
+      return floorDiv(delta, m);
+    }
+
+    // Unknown time frame. Return the original value unchanged.
+    return timestamp;
+  }
+
+  /** Builds a collection of time frames. */
+  public interface Builder {
+    /** Creates a {@code TimeFrameSet}. */
+    TimeFrameSet build();
+
+    /** Defines a core time frame. */
+    Builder addCore(String name);
+
+    /** Defines a time frame that is the number of a minor unit within a major
+     * frame. For example, the "DOY" frame has minor "DAY" and major "YEAR". */
+    Builder addQuotient(String name, String minorName,
+        String majorName);
+
+    /** Defines a time frame that consists of {@code count} instances of
+     * a base frame. */
+    Builder addMultiple(String name, Number count, String baseName);
+
+    /** Defines a time frame such that each base frame consists of {@code count}
+     * instances of the new frame. */
+    Builder addDivision(String name, Number count, String baseName);
+
+    /** Defines a rollup from one frame to another.
+     *
+     * <p>An explicit rollup is not necessary for frames where one is a multiple
+     * of another (such as {@code MILLISECOND} to {@code HOUR}). Only use this
+     * method for frames that are not multiples (such as {@code DAY} to
+     * {@code MONTH}).
+     *
+     * <p>How do we automatically roll up from say, "minute15" to "hour7"?
+     * Because we know the following:
+     * <ul>
+     *   <li>"minute15" and "hour7" are based on the same core frame (seconds);
+     *   <li>"minute15" is 15 * 60 seconds, "hour7" is 7 * 60 * 60 seconds,
+     *     and the one divides the other;
+     *   <li>They have the same offset, 1970-01-01 00:00:00. (Different offsets
+     *     would be OK too, as they are a whole multiple apart.)
+     * </ul>
+     *
+     * <p>A month is not a fixed multiple of days, but a rollup is still
+     * possible, because the start of a month is always aligned with the start
+     * of a day. This means that you can compute a month total by adding up the
+     * day totals for all days in that month. This is useful if you have an
+     * aggregate table on DAY and you want to answer a query on {@code MONTH}.
+     *
+     * <p>There is no rollup from {@code WEEK} to {@code YEAR}, because of a
+     * lack of alignment: a year does not start on the first day of a week, and
+     * so you cannot compute the total for, say, the year 2022 by adding the
+     * totals for all weeks that fall in 2022.
+     *
+     * <p>Incidentally, {@code ISOWEEK} and {@code ISOYEAR} are designed so that
+     * {@code ISOWEEK} can roll up to {@code ISOYEAR}. Every {@code ISOYEAR} and
+     * {@code ISOWEEK} start on a Monday, so they are aligned. An
+     * {@code ISOYEAR} consists of either 52 or 53 {@code ISOWEEK} instances,
+     * but the variable multiple is not a problem; the alignment ensures that
+     * rollup is valid. */
+    Builder addRollup(String fromName, String toName);
+
+    /** Adds all time frames in {@code timeFrameSet} to this {@code Builder}. */
+    Builder addAll(TimeFrameSet timeFrameSet);
+
+    /** Replaces the epoch of the most recently added frame. */
+    Builder withEpoch(TimestampString epoch);
+
+    /** Defines an alias for an existing frame.
+     *
+     * <p>For example, {@code add("Y", "YEAR")} adds "Y" as an alias for the
+     * built-in frame {@code YEAR}.
+     *
+     * @param name The alias
+     * @param originalName Name of the existing frame
+     */
+    Builder addAlias(String name, String originalName);
+  }
+
+}
diff --git a/core/src/main/java/org/apache/calcite/rel/type/TimeFrames.java b/core/src/main/java/org/apache/calcite/rel/type/TimeFrames.java
new file mode 100644
index 0000000000..8eb4bd13a0
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/type/TimeFrames.java
@@ -0,0 +1,637 @@
+/*
+ * 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.rel.type;
+
+import org.apache.calcite.avatica.util.DateTimeUtils;
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.util.MonotonicSupplier;
+import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.TimestampString;
+
+import org.apache.commons.math3.fraction.BigFraction;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableMultimap;
+import com.google.common.collect.Iterables;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.math.BigInteger;
+import java.util.Calendar;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Supplier;
+
+import static org.apache.calcite.avatica.util.DateTimeUtils.EPOCH_JULIAN;
+
+import static java.lang.Math.floorDiv;
+import static java.lang.Math.floorMod;
+import static java.util.Objects.requireNonNull;
+
+/** Utilities for {@link TimeFrame}. */
+public class TimeFrames {
+  private TimeFrames() {
+  }
+
+  /** The core time frame set. Includes the time frames for all Avatica time
+   * units plus ISOWEEK:
+   *
+   * <ul>
+   *   <li>SECOND, and multiples MINUTE, HOUR, DAY, WEEK (starts on a Sunday),
+   *   sub-multiples MILLISECOND, MICROSECOND, NANOSECOND,
+   *   quotients DOY, DOW;
+   *   <li>MONTH, and multiples QUARTER, YEAR, DECADE, CENTURY, MILLENNIUM;
+   *   <li>ISOYEAR, and sub-unit ISOWEEK (starts on a Monday), quotient ISODOW;
+   * </ul>
+   *
+   * <p>Does not include EPOCH.
+   */
+  public static final TimeFrameSet CORE =
+      addTsi(addCore(new BuilderImpl())).build();
+
+  private static BuilderImpl addCore(BuilderImpl b) {
+    b.addCore(TimeUnit.SECOND);
+    b.addSub(TimeUnit.MINUTE, false, 60, TimeUnit.SECOND);
+    b.addSub(TimeUnit.HOUR, false, 60, TimeUnit.MINUTE);
+    b.addSub(TimeUnit.DAY, false, 24, TimeUnit.HOUR);
+    b.addSub(TimeUnit.WEEK, false, 7, TimeUnit.DAY,
+        new TimestampString(1970, 1, 4, 0, 0, 0)); // a sunday
+    b.addSub(TimeUnit.MILLISECOND, true, 1_000, TimeUnit.SECOND);
+    b.addSub(TimeUnit.MICROSECOND, true, 1_000, TimeUnit.MILLISECOND);
+    b.addSub(TimeUnit.NANOSECOND, true, 1_000, TimeUnit.MICROSECOND);
+
+    b.addSub(TimeUnit.EPOCH, false, 1, TimeUnit.SECOND,
+        new TimestampString(1970, 1, 1, 0, 0, 0));
+
+    b.addCore(TimeUnit.MONTH);
+    b.addSub(TimeUnit.QUARTER, false, 3, TimeUnit.MONTH);
+    b.addSub(TimeUnit.YEAR, false, 12, TimeUnit.MONTH);
+    b.addSub(TimeUnit.DECADE, false, 10, TimeUnit.YEAR);
+    b.addSub(TimeUnit.CENTURY, false, 100, TimeUnit.YEAR,
+        new TimestampString(2001, 1, 1, 0, 0, 0));
+    b.addSub(TimeUnit.MILLENNIUM, false, 1_000, TimeUnit.YEAR,
+        new TimestampString(2001, 1, 1, 0, 0, 0));
+
+    b.addCore(TimeUnit.ISOYEAR);
+    b.addSub("ISOWEEK", false, 7, TimeUnit.DAY.name(),
+        new TimestampString(1970, 1, 5, 0, 0, 0)); // a monday
+
+    b.addQuotient(TimeUnit.DOY, TimeUnit.DAY, TimeUnit.YEAR);
+    b.addQuotient(TimeUnit.DOW, TimeUnit.DAY, TimeUnit.WEEK);
+    b.addQuotient(TimeUnit.ISODOW.name(), TimeUnit.DAY.name(), "ISOWEEK");
+
+    b.addRollup(TimeUnit.DAY, TimeUnit.MONTH);
+    b.addRollup("ISOWEEK", TimeUnit.ISOYEAR.name());
+    return b;
+  }
+
+  /** Adds abbreviations used by {@code TIMESTAMPADD}, {@code TIMESTAMPDIFF}
+   * functions. */
+  private static BuilderImpl addTsi(BuilderImpl b) {
+    b.addAlias("FRAC_SECOND", TimeUnit.MICROSECOND.name());
+    b.addAlias("SQL_TSI_FRAC_SECOND", TimeUnit.NANOSECOND.name());
+    b.addAlias("SQL_TSI_MICROSECOND", TimeUnit.MICROSECOND.name());
+    b.addAlias("SQL_TSI_SECOND", TimeUnit.SECOND.name());
+    b.addAlias("SQL_TSI_MINUTE", TimeUnit.MINUTE.name());
+    b.addAlias("SQL_TSI_HOUR", TimeUnit.HOUR.name());
+    b.addAlias("SQL_TSI_DAY", TimeUnit.DAY.name());
+    b.addAlias("SQL_TSI_WEEK", TimeUnit.WEEK.name());
+    b.addAlias("SQL_TSI_MONTH", TimeUnit.MONTH.name());
+    b.addAlias("SQL_TSI_QUARTER", TimeUnit.QUARTER.name());
+    b.addAlias("SQL_TSI_YEAR", TimeUnit.YEAR.name());
+    return b;
+  }
+
+  /** Given a date, returns the date of the first day of its ISO Year.
+   * Usually occurs in the same calendar year, but may be as early as Dec 29
+   * of the previous calendar year.
+   *
+   * <p>After
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5369">[CALCITE-5369]
+   * In Avatica DateTimeUtils, add support for FLOOR and CEIL to ISOYEAR</a> is
+   * fixed, we can use {@link DateTimeUtils#unixDateFloor} instead of this
+   * method. */
+  static int floorCeilIsoYear(int date, boolean ceil) {
+    final int year =
+        (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
+    return (int) firstMondayOfFirstWeek(year + (ceil ? 1 : 0)) - EPOCH_JULIAN;
+  }
+
+  /** Returns the first day of the first week of a year.
+   * Per ISO-8601 it is the Monday of the week that contains Jan 4,
+   * or equivalently, it is a Monday between Dec 29 and Jan 4.
+   * Sometimes it is in the year before the given year. */
+  // Note: copied from DateTimeUtils
+  static long firstMondayOfFirstWeek(int year) {
+    final long janFirst = DateTimeUtils.ymdToJulian(year, 1, 1);
+    final long janFirstDow = floorMod(janFirst + 1, (long) 7); // sun=0, sat=6
+    return janFirst + (11 - janFirstDow) % 7 - 3;
+  }
+
+  /** Returns the number of months since 1 BCE.
+   *
+   * <p>Parameters mean the same as in
+   * {@link DateTimeUtils#ymdToJulian(int, int, int)}.
+   *
+   * @param year Year (e.g. 2020 means 2020 CE, 0 means 1 BCE)
+   * @param month Month (e.g. 1 means January)
+   */
+  static int fullMonth(int year, int month) {
+    return year * 12 + (month - 1);
+  }
+
+  /** Given a {@link #fullMonth(int, int)} value, returns the month
+   * (1 means January). */
+  static int fullMonthToMonth(int fullMonth) {
+    return floorMod(fullMonth, 12) + 1;
+  }
+
+  /** Given a {@link #fullMonth(int, int)} value, returns the year
+   * (2020 means 2020 CE). */
+  static int fullMonthToYear(int fullMonth) {
+    return floorDiv(fullMonth, 12);
+  }
+
+  /** As {@link DateTimeUtils#unixTimestamp(int, int, int, int, int, int)}
+   * but based on a fullMonth value (per {@link #fullMonth(int, int)}). */
+  static long unixTimestamp(int fullMonth, int day, int hour,
+      int minute, int second) {
+    final int year = fullMonthToYear(fullMonth);
+    final int month = fullMonthToMonth(fullMonth);
+    return DateTimeUtils.unixTimestamp(year, month, day, hour, minute, second);
+  }
+
+  static int mdToUnixDate(int fullMonth, int day) {
+    final int year = fullMonthToYear(fullMonth);
+    final int month = fullMonthToMonth(fullMonth);
+    return DateTimeUtils.ymdToUnixDate(year, month, day);
+  }
+
+  private static boolean canDirectlyRollUp(TimeFrameImpl from,
+      TimeFrameImpl to) {
+    if (from.core().equals(to.core())) {
+      if (divisible(from.coreMultiplier(), to.coreMultiplier())) {
+        BigFraction diff = new BigFraction(from.core().epochDiff(from, to));
+        return divisible(from.coreMultiplier(), diff);
+      }
+      return false;
+    }
+    return false;
+  }
+
+  /** Returns whether {@code numerator} is divisible by {@code denominator}.
+   *
+   * <p>For example, {@code divisible(6, 2)} returns {@code true};
+   * {@code divisible(0, 2)} also returns {@code true};
+   * {@code divisible(2, 6)} returns {@code false}. */
+  private static boolean divisible(BigFraction numerator,
+      BigFraction denominator) {
+    return denominator.equals(BigFraction.ZERO)
+        || numerator
+        .divide(denominator)
+        .getNumerator()
+        .abs()
+        .equals(BigInteger.ONE);
+  }
+
+  /** Implementation of {@link TimeFrameSet.Builder}. */
+  static class BuilderImpl implements TimeFrameSet.Builder {
+    BuilderImpl() {
+    }
+
+    final MonotonicSupplier<TimeFrameSet> frameSetSupplier =
+        new MonotonicSupplier<>();
+    final Map<String, TimeFrameImpl> map = new LinkedHashMap<>();
+    final ImmutableMultimap.Builder<TimeFrameImpl, TimeFrameImpl> rollupList =
+        ImmutableMultimap.builder();
+
+    @Override public TimeFrameSet build() {
+      final TimeFrameSet frameSet =
+          new TimeFrameSet(ImmutableMap.copyOf(map), rollupList.build());
+      frameSetSupplier.accept(frameSet);
+      return frameSet;
+    }
+
+    /** Converts a number to an exactly equivalent {@code BigInteger}.
+     * May silently lose precision if n is a {@code Float} or {@code Double}. */
+    static BigInteger toBigInteger(Number number) {
+      return number instanceof BigInteger ? (BigInteger) number
+          : BigInteger.valueOf(number.longValue());
+    }
+
+    /** Returns the time frame with the given name,
+     * or throws {@link IllegalArgumentException}. */
+    TimeFrameImpl getFrame(String name) {
+      final TimeFrameImpl timeFrame = map.get(name);
+      if (timeFrame == null) {
+        throw new IllegalArgumentException("unknown frame: " + name);
+      }
+      return timeFrame;
+    }
+
+    /** Adds a frame.
+     *
+     * <p>If a frame with this name already exists, throws
+     * {@link IllegalArgumentException} and leaves the builder in the same
+     * state.
+     *
+     * <p>It is very important that we don't allow replacement of frames.
+     * If replacement were allowed, people would be able to create a DAG
+     * (e.g. two routes from DAY to MONTH with different multipliers)
+     * or a cycle (e.g. one SECOND equals 1,000 MILLISECOND
+     * and one MILLISECOND equals 20 SECOND). Those scenarios give rise to
+     * inconsistent multipliers. */
+    private BuilderImpl addFrame(String name, TimeFrameImpl frame) {
+      final TimeFrameImpl previousFrame =
+          map.put(name, requireNonNull(frame, "frame"));
+      if (previousFrame != null) {
+        // There was already a frame with that name. Replace the old frame
+        // (so that that builder is still valid usable) and throw.
+        map.put(name, previousFrame);
+        throw new IllegalArgumentException("duplicate frame: " + name);
+      }
+      return this;
+    }
+
+    @Override public BuilderImpl addCore(String name) {
+      return addFrame(name, new CoreFrame(frameSetSupplier, name));
+    }
+
+    /** Defines a time unit that consists of {@code count} instances of
+     * {@code baseUnit}. */
+    BuilderImpl addSub(String name, boolean divide, Number count,
+        String baseName, TimestampString epoch) {
+      final TimeFrameImpl baseFrame = getFrame(baseName);
+      final BigInteger factor = toBigInteger(count);
+
+      final CoreFrame coreFrame = baseFrame.core();
+      final BigFraction coreFactor = divide
+          ? baseFrame.coreMultiplier().divide(factor)
+          : baseFrame.coreMultiplier().multiply(factor);
+
+      return addFrame(name,
+          new SubFrame(name, baseFrame, divide, factor, coreFrame, coreFactor,
+              epoch));
+    }
+
+    @Override public BuilderImpl addQuotient(String name,
+        String minorName, String majorName) {
+      final TimeFrameImpl minorFrame = getFrame(minorName);
+      final TimeFrameImpl majorFrame = getFrame(majorName);
+      return addFrame(name, new QuotientFrame(name, minorFrame, majorFrame));
+    }
+
+    @Override public BuilderImpl addMultiple(String name, Number count,
+        String baseName) {
+      return addSub(name, false, count, baseName, TimestampString.EPOCH);
+    }
+
+    @Override public BuilderImpl addDivision(String name, Number count,
+        String baseName) {
+      return addSub(name, true, count, baseName, TimestampString.EPOCH);
+    }
+
+    @Override public BuilderImpl addRollup(String fromName, String toName) {
+      final TimeFrameImpl fromFrame = getFrame(fromName);
+      final TimeFrameImpl toFrame = getFrame(toName);
+      rollupList.put(fromFrame, toFrame);
+      return this;
+    }
+
+    @Override public BuilderImpl addAll(TimeFrameSet timeFrameSet) {
+      timeFrameSet.map.values().forEach(frame -> frame.replicate(this));
+      return this;
+    }
+
+    @Override public BuilderImpl withEpoch(TimestampString epoch) {
+      final Map.Entry<String, TimeFrameImpl> entry =
+          Iterables.getLast(map.entrySet());
+      final String name = entry.getKey();
+      final SubFrame value =
+          requireNonNull((SubFrame) map.remove(name));
+      value.replicateWithEpoch(this, epoch);
+      return this;
+    }
+
+    @Override public BuilderImpl addAlias(String name, String originalName) {
+      final TimeFrameImpl frame = getFrame(originalName);
+      return addFrame(name, new AliasFrame(name, frame));
+    }
+
+    // Extra methods for Avatica's built-in time frames.
+
+    void addCore(TimeUnit unit) {
+      addCore(unit.name());
+    }
+
+    void addSub(TimeUnit unit, boolean divide, Number count,
+        TimeUnit baseUnit) {
+      addSub(unit, divide, count, baseUnit, TimestampString.EPOCH);
+    }
+
+    void addSub(TimeUnit unit, boolean divide, Number count,
+        TimeUnit baseUnit, TimestampString epoch) {
+      addSub(unit.name(), divide, count, baseUnit.name(), epoch);
+    }
+
+    void addRollup(TimeUnit fromUnit, TimeUnit toUnit) {
+      addRollup(fromUnit.name(), toUnit.name());
+    }
+
+    void addQuotient(TimeUnit unit, TimeUnit minor, TimeUnit major) {
+      addQuotient(unit.name(), minor.name(), major.name());
+    }
+  }
+
+  /** Implementation of {@link TimeFrame}. */
+  abstract static class TimeFrameImpl implements TimeFrame {
+    final String name;
+    final Supplier<TimeFrameSet> frameSetSupplier;
+
+    TimeFrameImpl(Supplier<TimeFrameSet> frameSetSupplier, String name) {
+      this.frameSetSupplier =
+          requireNonNull(frameSetSupplier, "frameSetSupplier");
+      this.name = requireNonNull(name, "name");
+    }
+
+    @Override public String toString() {
+      return name;
+    }
+
+    @Override public TimeFrameSet frameSet() {
+      return frameSetSupplier.get();
+    }
+
+    @Override public String name() {
+      return name;
+    }
+
+    @Override public @Nullable BigFraction per(TimeFrame timeFrame) {
+      // Note: The following algorithm is not very efficient. It becomes less
+      // efficient as the number of time frames increases. A more efficient
+      // algorithm would be for TimeFrameSet.Builder.build() to call this method
+      // for each pair of time frames and cache the results in a list:
+      //
+      //   (coreFrame,
+      //   [(subFrame0, multiplier0),
+      //    ...
+      //    (subFrameN, multiplierN)])
+
+      final Map<TimeFrame, BigFraction> map = new HashMap<>();
+      final Map<TimeFrame, BigFraction> map2 = new HashMap<>();
+      expand(map, BigFraction.ONE);
+      ((TimeFrameImpl) timeFrame).expand(map2, BigFraction.ONE);
+      final Set<BigFraction> fractions = new HashSet<>();
+      for (Map.Entry<TimeFrame, BigFraction> entry : map.entrySet()) {
+        final BigFraction value2 = map2.get(entry.getKey());
+        if (value2 != null) {
+          fractions.add(value2.divide(entry.getValue()));
+        }
+      }
+
+      switch (fractions.size()) {
+      case 0:
+        // There is no path from this TimeFrame to that.
+        return null;
+      case 1:
+        return Iterables.getOnlyElement(fractions);
+      default:
+        // If there are multiple units in common, the multipliers must be the
+        // same for all. If they are not, the must have somehow created a
+        // TimeFrameSet that has multiple paths between units (i.e. a DAG),
+        // or has a cycle. TimeFrameSet.Builder is supposed to prevent all of
+        // these, and so we throw an AssertionError.
+        throw new AssertionError("inconsistent multipliers for " + this
+            + ".per(" + timeFrame + "): " + fractions);
+      }
+    }
+
+    protected void expand(Map<TimeFrame, BigFraction> map, BigFraction f) {
+      map.put(this, f);
+    }
+
+    /** Adds a time frame like this to a builder. */
+    abstract void replicate(BuilderImpl b);
+
+    protected abstract CoreFrame core();
+
+    protected abstract BigFraction coreMultiplier();
+
+    @Override public boolean canRollUpTo(TimeFrame toFrame) {
+      if (toFrame == this) {
+        return true;
+      }
+      if (toFrame instanceof TimeFrameImpl) {
+        final TimeFrameImpl toFrame1 = (TimeFrameImpl) toFrame;
+        if (canDirectlyRollUp(this, toFrame1)) {
+          return true;
+        }
+        final TimeFrameSet frameSet = frameSet();
+        if (frameSet.rollupMap.entries().contains(Pair.of(this, toFrame1))) {
+          return true;
+        }
+        // Hard-code roll-up via DAY-to-MONTH bridge, for now.
+        final TimeFrameImpl day =
+            requireNonNull(frameSet.map.get(TimeUnit.DAY.name()));
+        final TimeFrameImpl month =
+            requireNonNull(frameSet.map.get(TimeUnit.MONTH.name()));
+        if (canDirectlyRollUp(this, day)
+            && canDirectlyRollUp(month, toFrame1)) {
+          return true;
+        }
+        // Hard-code roll-up via ISOWEEK-to-ISOYEAR bridge, for now.
+        final TimeFrameImpl isoYear =
+            requireNonNull(frameSet.map.get(TimeUnit.ISOYEAR.name()));
+        final TimeFrameImpl isoWeek =
+            requireNonNull(frameSet.map.get("ISOWEEK"));
+        if (canDirectlyRollUp(this, isoWeek)
+            && canDirectlyRollUp(isoYear, toFrame1)) {
+          return true;
+        }
+      }
+      return false;
+    }
+  }
+
+  /** Core time frame (such as SECOND, MONTH, ISOYEAR). */
+  static class CoreFrame extends TimeFrameImpl {
+    CoreFrame(Supplier<TimeFrameSet> frameSetSupplier, String name) {
+      super(frameSetSupplier, name);
+    }
+
+    @Override void replicate(BuilderImpl b) {
+      b.addCore(name);
+    }
+
+    @Override protected CoreFrame core() {
+      return this;
+    }
+
+    @Override protected BigFraction coreMultiplier() {
+      return BigFraction.ONE;
+    }
+
+    /** Returns the difference between the epochs of two frames, in the
+     * units of this core frame. */
+    BigInteger epochDiff(TimeFrameImpl from, TimeFrameImpl to) {
+      assert from.core() == this;
+      assert to.core() == this;
+      switch (name) {
+      case "MONTH":
+        return BigInteger.valueOf(from.monthEpoch())
+            .subtract(BigInteger.valueOf(to.monthEpoch()));
+      default:
+        return BigInteger.valueOf(from.timestampEpoch())
+            .subtract(BigInteger.valueOf(to.timestampEpoch()));
+      }
+    }
+  }
+
+  /** A time frame that is composed of another time frame.
+   *
+   * <p>For example, {@code MINUTE} is composed of 60 {@code SECOND};
+   * (factor = 60, divide = false);
+   * {@code MILLISECOND} is composed of 1 / 1000 {@code SECOND}
+   * (factor = 1000, divide = true).
+   *
+   * <p>A sub-time frame S is aligned with its parent frame P;
+   * that is, every instance of S belongs to one instance of P.
+   * Every {@code MINUTE} belongs to one {@code HOUR};
+   * not every {@code WEEK} belongs to precisely one {@code MONTH} or
+   * {@code MILLENNIUM}.
+   */
+  static class SubFrame extends TimeFrameImpl {
+    private final TimeFrameImpl base;
+    private final boolean divide;
+    private final BigInteger multiplier;
+    private final CoreFrame coreFrame;
+
+    /** The number of core frames that are equivalent to one of these. For
+     * example, MINUTE, HOUR, MILLISECOND all have core = SECOND, and have
+     * multipliers 60, 3,600, 1 / 1,000 respectively. */
+    private final BigFraction coreMultiplier;
+    private final TimestampString epoch;
+
+    SubFrame(String name, TimeFrameImpl base, boolean divide,
+        BigInteger multiplier, CoreFrame coreFrame,
+        BigFraction coreMultiplier, TimestampString epoch) {
+      super(base.frameSetSupplier, name);
+      this.base = requireNonNull(base, "base");
+      this.divide = divide;
+      this.multiplier = requireNonNull(multiplier, "multiplier");
+      this.coreFrame = requireNonNull(coreFrame, "coreFrame");
+      this.coreMultiplier = requireNonNull(coreMultiplier, "coreMultiplier");
+      this.epoch = requireNonNull(epoch, "epoch");
+    }
+
+    @Override public String toString() {
+      return name + ", composedOf " + multiplier + " " + base.name;
+    }
+
+    @Override public int dateEpoch() {
+      return (int) floorDiv(epoch.getMillisSinceEpoch(),
+          DateTimeUtils.MILLIS_PER_DAY);
+    }
+
+    @Override public int monthEpoch() {
+      final Calendar calendar = epoch.toCalendar();
+      int y = calendar.get(Calendar.YEAR); // 2020 CE is represented by 2020
+      int m = calendar.get(Calendar.MONTH) + 1; // January is represented by 1
+      return fullMonth(y, m);
+    }
+
+    @Override public long timestampEpoch() {
+      return epoch.getMillisSinceEpoch();
+    }
+
+    @Override void replicate(BuilderImpl b) {
+      b.addSub(name, divide, multiplier, base.name, epoch);
+    }
+
+    /** Returns a copy of this TimeFrameImpl with a given epoch. */
+    void replicateWithEpoch(BuilderImpl b, TimestampString epoch) {
+      b.addSub(name, divide, multiplier, base.name, epoch);
+    }
+
+    @Override protected void expand(Map<TimeFrame, BigFraction> map,
+        BigFraction f) {
+      super.expand(map, f);
+      base.expand(map, divide ? f.divide(multiplier) : f.multiply(multiplier));
+    }
+
+    @Override protected CoreFrame core() {
+      return coreFrame;
+    }
+
+    @Override protected BigFraction coreMultiplier() {
+      return coreMultiplier;
+    }
+  }
+
+  /** Frame that defines is based on a minor frame and resets whenever the major
+   * frame resets. For example, "DOY" (day of year) is based on DAY and resets
+   * every YEAR. */
+  static class QuotientFrame extends TimeFrameImpl {
+    private final TimeFrameImpl minorFrame;
+    private final TimeFrameImpl majorFrame;
+
+    QuotientFrame(String name, TimeFrameImpl minorFrame,
+        TimeFrameImpl majorFrame) {
+      super(minorFrame.frameSetSupplier, name);
+      this.minorFrame = requireNonNull(minorFrame, "minorFrame");
+      this.majorFrame = requireNonNull(majorFrame, "majorFrame");
+    }
+
+    @Override void replicate(BuilderImpl b) {
+      b.addQuotient(name, minorFrame.name, majorFrame.name);
+    }
+
+    @Override protected CoreFrame core() {
+      return minorFrame.core();
+    }
+
+    @Override protected BigFraction coreMultiplier() {
+      return minorFrame.coreMultiplier();
+    }
+  }
+
+  /** Frame that defines an alias. */
+  static class AliasFrame extends TimeFrameImpl {
+    final TimeFrameImpl frame;
+
+    AliasFrame(String name, TimeFrameImpl frame) {
+      super(frame.frameSetSupplier, name);
+      this.frame = requireNonNull(frame, "frame");
+    }
+
+    @Override void replicate(BuilderImpl b) {
+      b.addAlias(name, frame.name);
+    }
+
+    @Override protected CoreFrame core() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override protected BigFraction coreMultiplier() {
+      throw new UnsupportedOperationException();
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 4188a512d6..b8a961d9ba 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -75,6 +75,8 @@ import java.util.Objects;
 import java.util.function.IntPredicate;
 import java.util.stream.Collectors;
 
+import static com.google.common.base.Verify.verifyNotNull;
+
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 
 /**
@@ -1284,7 +1286,10 @@ public class RexBuilder {
    */
   public RexLiteral makeIntervalLiteral(
       SqlIntervalQualifier intervalQualifier) {
-    assert intervalQualifier != null;
+    verifyNotNull(intervalQualifier);
+    if (intervalQualifier.timeFrameName != null) {
+      return makePreciseStringLiteral(intervalQualifier.timeFrameName);
+    }
     return makeFlag(intervalQualifier.timeUnitRange);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 4e6ee08241..de7c8fa423 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -584,8 +584,8 @@ public interface CalciteResource {
   @BaseMessage("Cannot call table function here: ''{0}''")
   ExInst<CalciteException> cannotCallTableFunctionHere(String a0);
 
-  @BaseMessage("''{0}'' is not a valid datetime format")
-  ExInst<CalciteException> invalidDatetimeFormat(String a0);
+  @BaseMessage("''{0}'' is not a valid time frame")
+  ExInst<SqlValidatorException> invalidTimeFrame(String a0);
 
   @BaseMessage("Cannot INSERT into generated column ''{0}''")
   ExInst<SqlValidatorException> insertIntoAlwaysGenerated(String a0);
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 01fc85d963..aa1ed8a926 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -32,6 +32,8 @@ import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.function.NonDeterministic;
 import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.rel.type.TimeFrame;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.FlatLists.ComparableList;
 import org.apache.calcite.sql.fun.SqlLibraryOperators;
 import org.apache.calcite.util.NumberUtil;
@@ -2553,6 +2555,108 @@ public class SqlFunctions {
     return (Locale) DataContext.Variable.LOCALE.get(root);
   }
 
+  /** SQL {@code DATEADD} function applied to a custom time frame.
+   *
+   * <p>Custom time frames are created as part of a {@link TimeFrameSet}.
+   * This method retrieves the session's time frame set from the
+   * {@link DataContext.Variable#TIME_FRAME_SET} variable, then looks up the
+   * time frame by name. */
+  public static int customDateAdd(DataContext root,
+      String timeFrameName, int interval, int date) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.addDate(date, interval, timeFrame);
+  }
+
+  /** SQL {@code TIMESTAMPADD} function applied to a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static long customTimestampAdd(DataContext root,
+      String timeFrameName, long interval, long timestamp) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.addTimestamp(timestamp, interval, timeFrame);
+  }
+
+  /** SQL {@code DATEDIFF} function applied to a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static int customDateDiff(DataContext root,
+      String timeFrameName, int date, int date2) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.diffDate(date, date2, timeFrame);
+  }
+
+  /** SQL {@code TIMESTAMPDIFF} function applied to a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static long customTimestampDiff(DataContext root,
+      String timeFrameName, long timestamp, long timestamp2) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.diffTimestamp(timestamp, timestamp2, timeFrame);
+  }
+
+  /** SQL {@code FLOOR} function applied to a {@code DATE} value
+   * and a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static int customDateFloor(DataContext root,
+      String timeFrameName, int date) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.floorDate(date, timeFrame);
+  }
+
+  /** SQL {@code CEIL} function applied to a {@code DATE} value
+   * and a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static int customDateCeil(DataContext root,
+      String timeFrameName, int date) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.ceilDate(date, timeFrame);
+  }
+
+  /** SQL {@code FLOOR} function applied to a {@code TIMESTAMP} value
+   * and a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static long customTimestampFloor(DataContext root,
+      String timeFrameName, long timestamp) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.floorTimestamp(timestamp, timeFrame);
+  }
+
+  /** SQL {@code CEIL} function applied to a {@code TIMESTAMP} value
+   * and a custom time frame.
+   *
+   * <p>Custom time frames are created and accessed as described in
+   * {@link #customDateAdd}. */
+  public static long customTimestampCeil(DataContext root,
+      String timeFrameName, long timestamp) {
+    final TimeFrameSet timeFrameSet =
+        requireNonNull(DataContext.Variable.TIME_FRAME_SET.get(root));
+    final TimeFrame timeFrame = timeFrameSet.get(timeFrameName);
+    return timeFrameSet.ceilTimestamp(timestamp, timeFrame);
+  }
+
   /** SQL {@code TRANSLATE(string, search_chars, replacement_chars)}
    * function. */
   public static String translate3(String s, String search, String replacement) {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
index 8852a0df02..3504228944 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -16,7 +16,10 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.TimeFrame;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.sql.fun.SqlLiteralChainOperator;
@@ -270,7 +273,7 @@ public class SqlCallBinding extends SqlOperatorBinding {
     return valueAs(node, clazz);
   }
 
-  private static <T extends Object> @Nullable T valueAs(SqlNode node, Class<T> clazz) {
+  private <T extends Object> @Nullable T valueAs(SqlNode node, Class<T> clazz) {
     final SqlLiteral literal;
     switch (node.getKind()) {
     case ARRAY_VALUE_CONSTRUCTOR:
@@ -308,6 +311,21 @@ public class SqlCallBinding extends SqlOperatorBinding {
 
     case INTERVAL_QUALIFIER:
       final SqlIntervalQualifier q = (SqlIntervalQualifier) node;
+      if (q.timeFrameName != null) {
+        // Custom time frames can only be cast to String. You can do more with
+        // them when validator has resolved to a TimeFrame.
+        final TimeFrameSet timeFrameSet = validator.getTimeFrameSet();
+        final TimeFrame timeFrame = timeFrameSet.getOpt(q.timeFrameName);
+        if (clazz == String.class) {
+          return clazz.cast(q.timeFrameName);
+        }
+        if (clazz == TimeUnit.class
+            && timeFrame != null) {
+          TimeUnit timeUnit = timeFrameSet.getUnit(timeFrame);
+          return clazz.cast(timeUnit);
+        }
+        return null;
+      }
       final SqlIntervalLiteral.IntervalValue intervalValue =
           new SqlIntervalLiteral.IntervalValue(q, 1, q.toString());
       literal = new SqlLiteral(intervalValue, q.typeName(), q.pos);
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index 1e48cabe50..4654f8fea1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -489,6 +489,10 @@ public class SqlDialect {
    * <code>INTERVAL '1 2:3:4' DAY(4) TO SECOND(4)</code>. */
   public void unparseSqlIntervalQualifier(SqlWriter writer,
       SqlIntervalQualifier qualifier, RelDataTypeSystem typeSystem) {
+    if (qualifier.timeFrameName != null) {
+      SqlIntervalQualifier.asIdentifier(qualifier).unparse(writer, 0, 0);
+      return;
+    }
     final String start = qualifier.timeUnitRange.startUnit.name();
     final int fractionalSecondPrecision =
         qualifier.getFractionalSecondPrecision(typeSystem);
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index 4c468f3bb6..1bf7980b95 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -32,13 +32,14 @@ import org.apache.calcite.util.Util;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.math.BigDecimal;
-import java.util.Objects;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 import static org.apache.calcite.util.Static.RESOURCE;
 
+import static java.util.Objects.requireNonNull;
+
 /**
  * Represents an INTERVAL qualifier.
  *
@@ -95,25 +96,32 @@ public class SqlIntervalQualifier extends SqlNode {
   //~ Instance fields --------------------------------------------------------
 
   private final int startPrecision;
+  public final @Nullable String timeFrameName;
   public final TimeUnitRange timeUnitRange;
   private final int fractionalSecondPrecision;
 
   //~ Constructors -----------------------------------------------------------
 
+  private SqlIntervalQualifier(SqlParserPos pos, @Nullable String timeFrameName,
+      TimeUnitRange timeUnitRange, int startPrecision,
+      int fractionalSecondPrecision) {
+    super(pos);
+    this.timeFrameName = timeFrameName;
+    this.timeUnitRange = requireNonNull(timeUnitRange, "timeUnitRange");
+    this.startPrecision = startPrecision;
+    this.fractionalSecondPrecision = fractionalSecondPrecision;
+  }
+
   public SqlIntervalQualifier(
       TimeUnit startUnit,
       int startPrecision,
       @Nullable TimeUnit endUnit,
       int fractionalSecondPrecision,
       SqlParserPos pos) {
-    super(pos);
-    if (endUnit == startUnit) {
-      endUnit = null;
-    }
-    this.timeUnitRange =
-        TimeUnitRange.of(Objects.requireNonNull(startUnit, "startUnit"), endUnit);
-    this.startPrecision = startPrecision;
-    this.fractionalSecondPrecision = fractionalSecondPrecision;
+    this(pos, null,
+        TimeUnitRange.of(requireNonNull(startUnit, "startUnit"),
+            endUnit == startUnit ? null : endUnit),
+        startPrecision, fractionalSecondPrecision);
   }
 
   public SqlIntervalQualifier(
@@ -128,6 +136,15 @@ public class SqlIntervalQualifier extends SqlNode {
         pos);
   }
 
+  /** Creates a qualifier based on a time frame name. */
+  public SqlIntervalQualifier(String timeFrameName,
+      SqlParserPos pos) {
+    this(pos, requireNonNull(timeFrameName, "timeFrameName"),
+        // EPOCH is a placeholder because code expects a non-null TimeUnitRange.
+        TimeUnitRange.EPOCH, RelDataType.PRECISION_NOT_SPECIFIED,
+        RelDataType.PRECISION_NOT_SPECIFIED);
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   @Override public SqlKind getKind() {
@@ -1187,4 +1204,20 @@ public class SqlIntervalQualifier extends SqlNode {
         RESOURCE.intervalFieldExceedsPrecision(
             value, type.name() + "(" + precision + ")"));
   }
+
+  /** Converts a {@link SqlIntervalQualifier} to a
+   * {@link org.apache.calcite.sql.SqlIdentifier} if it is a time frame
+   * reference.
+   *
+   * <p>Helps with unparsing of EXTRACT, FLOOR, CEIL functions. */
+  public static SqlNode asIdentifier(SqlNode node) {
+    if (node instanceof SqlIntervalQualifier) {
+      SqlIntervalQualifier intervalQualifier = (SqlIntervalQualifier) node;
+      if (intervalQualifier.timeFrameName != null) {
+        return new SqlIdentifier(intervalQualifier.timeFrameName,
+            node.getParserPosition());
+      }
+    }
+    return node;
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 9da1b1ee08..ffcce09d17 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.metadata.NullSentinel;
 import org.apache.calcite.rel.type.RelDataType;
@@ -288,6 +289,7 @@ public class SqlLiteral extends SqlNode {
       return clazz.cast(NullSentinel.INSTANCE);
     }
     requireNonNull(value, "value");
+    final SqlIntervalQualifier qualifier;
     switch (typeName) {
     case CHAR:
       if (clazz == String.class) {
@@ -345,15 +347,18 @@ public class SqlLiteral extends SqlNode {
     case INTERVAL_MONTH:
       final SqlIntervalLiteral.IntervalValue valMonth =
           (SqlIntervalLiteral.IntervalValue) value;
+      qualifier = valMonth.getIntervalQualifier();
       if (clazz == Long.class) {
         return clazz.cast(valMonth.getSign()
             * SqlParserUtil.intervalToMonths(valMonth));
       } else if (clazz == BigDecimal.class) {
         return clazz.cast(BigDecimal.valueOf(getValueAs(Long.class)));
       } else if (clazz == TimeUnitRange.class) {
-        return clazz.cast(valMonth.getIntervalQualifier().timeUnitRange);
+        return clazz.cast(qualifier.timeUnitRange);
+      } else if (clazz == TimeUnit.class) {
+        return clazz.cast(qualifier.timeUnitRange.startUnit);
       } else if (clazz == SqlIntervalQualifier.class) {
-        return clazz.cast(valMonth.getIntervalQualifier());
+        return clazz.cast(qualifier);
       }
       break;
     case INTERVAL_DAY:
@@ -368,15 +373,18 @@ public class SqlLiteral extends SqlNode {
     case INTERVAL_SECOND:
       final SqlIntervalLiteral.IntervalValue valTime =
           (SqlIntervalLiteral.IntervalValue) value;
+      qualifier = valTime.getIntervalQualifier();
       if (clazz == Long.class) {
         return clazz.cast(valTime.getSign()
             * SqlParserUtil.intervalToMillis(valTime));
       } else if (clazz == BigDecimal.class) {
         return clazz.cast(BigDecimal.valueOf(getValueAs(Long.class)));
       } else if (clazz == TimeUnitRange.class) {
-        return clazz.cast(valTime.getIntervalQualifier().timeUnitRange);
+        return clazz.cast(qualifier.timeUnitRange);
+      } else if (clazz == TimeUnit.class) {
+        return clazz.cast(qualifier.timeUnitRange.startUnit);
       } else if (clazz == SqlIntervalQualifier.class) {
-        return clazz.cast(valTime.getIntervalQualifier());
+        return clazz.cast(qualifier);
       }
       break;
     default:
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
index a77792a16e..07789aa7cc 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlExtractFunction.java
@@ -20,12 +20,15 @@ import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.sql.SqlCall;
 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.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Util;
 
 import static org.apache.calcite.sql.validate.SqlNonNullableAccessors.getOperandLiteralValueOrThrow;
@@ -60,12 +63,30 @@ public class SqlExtractFunction extends SqlFunction {
       int leftPrec,
       int rightPrec) {
     final SqlWriter.Frame frame = writer.startFunCall(getName());
-    call.operand(0).unparse(writer, 0, 0);
+    SqlIntervalQualifier.asIdentifier(call.operand(0))
+        .unparse(writer, 0, 0);
     writer.sep("FROM");
     call.operand(1).unparse(writer, 0, 0);
     writer.endFunCall(frame);
   }
 
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    super.validateCall(call, validator, scope, operandScope);
+
+    // This is either a time unit or a time frame:
+    //
+    //  * In "EXTRACT(YEAR FROM x)" operand 0 is a SqlIntervalQualifier with
+    //    startUnit = YEAR and timeFrameName = null.
+    //
+    //  * In "EXTRACT(MINUTE15 FROM x)" operand 0 is a SqlIntervalQualifier with
+    //    startUnit = EPOCH and timeFrameName = 'MINUTE15'.
+    //
+    // If the latter, check that timeFrameName is valid.
+    validator.validateTimeFrame(
+        (SqlIntervalQualifier) call.getOperandList().get(0));
+  }
+
   @Override public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
     TimeUnitRange value = getOperandLiteralValueOrThrow(call, 0, TimeUnitRange.class);
     switch (value) {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
index b98bfaadab..30e7bda900 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlFloorFunction.java
@@ -19,6 +19,7 @@ package org.apache.calcite.sql.fun;
 import org.apache.calcite.sql.SqlCall;
 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.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
@@ -29,6 +30,8 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import com.google.common.base.Preconditions;
 
@@ -63,13 +66,33 @@ public class SqlFloorFunction extends SqlMonotonicUnaryFunction {
     if (call.operandCount() == 2) {
       call.operand(0).unparse(writer, 0, 100);
       writer.sep("TO");
-      call.operand(1).unparse(writer, 100, 0);
+      SqlIntervalQualifier.asIdentifier(call.operand(1))
+          .unparse(writer, 100, 0);
     } else {
       call.operand(0).unparse(writer, 0, 0);
     }
     writer.endFunCall(frame);
   }
 
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    super.validateCall(call, validator, scope, operandScope);
+
+    if (call.operandCount() > 1) {
+      // This is either a time unit or a time frame:
+      //
+      //  * In "FLOOR(x TO YEAR)" operand 1 is a SqlIntervalQualifier with
+      //    startUnit = YEAR and timeFrameName = null.
+      //
+      //  * In "FLOOR(x TO MINUTE15)" operand 1 is a SqlIntervalQualifier with
+      //    startUnit = EPOCH and timeFrameName = 'MINUTE15'.
+      //
+      // If the latter, check that timeFrameName is valid.
+      validator.validateTimeFrame(
+          (SqlIntervalQualifier) call.getOperandList().get(1));
+    }
+  }
+
   /**
    * Copies a {@link SqlCall}, replacing the time unit operand with the given
    * literal.
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index 87d5aae262..306a33b39f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlAggFunction;
@@ -41,10 +42,13 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Optionality;
 
+import com.google.common.collect.ImmutableSet;
+
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 import static org.apache.calcite.sql.fun.SqlLibrary.BIG_QUERY;
 import static org.apache.calcite.sql.fun.SqlLibrary.HIVE;
@@ -635,6 +639,26 @@ public abstract class SqlLibraryOperators {
           OperandTypes.STRING_STRING,
           SqlFunctionCategory.TIMEDATE);
 
+  private static final Set<TimeUnitRange> TIME_UNITS =
+      ImmutableSet.of(TimeUnitRange.HOUR,
+          TimeUnitRange.MINUTE,
+          TimeUnitRange.SECOND);
+
+  private static final Set<TimeUnitRange> MONTH_UNITS =
+      ImmutableSet.of(TimeUnitRange.MILLENNIUM,
+          TimeUnitRange.CENTURY,
+          TimeUnitRange.DECADE,
+          TimeUnitRange.YEAR,
+          TimeUnitRange.MONTH);
+
+  private static final Set<TimeUnitRange> DATE_UNITS =
+      ImmutableSet.of(TimeUnitRange.WEEK,
+          TimeUnitRange.DAY);
+
+  private static final Set<TimeUnitRange> TIMESTAMP_UNITS =
+      ImmutableSet.<TimeUnitRange>builder()
+          .addAll(MONTH_UNITS).addAll(DATE_UNITS).addAll(TIME_UNITS).build();
+
   /** The "TIME_TRUNC(time_expression, time_part)" function (BigQuery);
    * truncates a TIME value to the granularity of time_part. The TIME value is
    * always rounded to the beginning of time_part. */
@@ -644,7 +668,8 @@ public abstract class SqlLibraryOperators {
           SqlKind.OTHER_FUNCTION,
           ReturnTypes.TIME_NULLABLE,
           null,
-          OperandTypes.TIME_INTERVAL,
+          OperandTypes.sequence("'TIME_TRUNC(<TIME>, <DATETIME_INTERVAL>)'",
+              OperandTypes.TIME, OperandTypes.interval(TIME_UNITS)),
           SqlFunctionCategory.TIMEDATE);
 
   /** The "TIMESTAMP_TRUNC(timestamp_expression, date_time_part[, time_zone])"
@@ -657,7 +682,9 @@ public abstract class SqlLibraryOperators {
           SqlKind.OTHER_FUNCTION,
           ReturnTypes.TIMESTAMP_NULLABLE,
           null,
-          OperandTypes.TIMESTAMP_INTERVAL,
+          OperandTypes.sequence(
+              "'TIMESTAMP_TRUNC(<TIMESTAMP>, <DATETIME_INTERVAL>)'",
+              OperandTypes.TIMESTAMP, OperandTypes.interval(TIMESTAMP_UNITS)),
           SqlFunctionCategory.TIMEDATE);
 
   /** The "TIMESTAMP_SECONDS(bigint)" function; returns a TIMESTAMP value
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
index 6870b5bdb2..7b96f6e05c 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
@@ -19,15 +19,21 @@ 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.SqlCall;
 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.type.OperandTypes;
 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.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
-import static org.apache.calcite.sql.validate.SqlNonNullableAccessors.getOperandLiteralValueOrThrow;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import static org.apache.calcite.util.Util.first;
 
 /**
  * The <code>TIMESTAMPADD</code> function, which adds an interval to a
@@ -61,23 +67,23 @@ public class SqlTimestampAddFunction extends SqlFunction {
   private static final int MICROSECOND_PRECISION = 6;
 
   private static final SqlReturnTypeInference RETURN_TYPE_INFERENCE =
-      opBinding -> {
-        final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
-        return deduceType(typeFactory,
-            getOperandLiteralValueOrThrow(opBinding, 0, TimeUnit.class),
-            opBinding.getOperandType(1), opBinding.getOperandType(2));
-      };
+      opBinding ->
+          deduceType(opBinding.getTypeFactory(),
+              opBinding.getOperandLiteralValue(0, TimeUnit.class),
+              opBinding.getOperandType(1), opBinding.getOperandType(2));
 
   public static RelDataType deduceType(RelDataTypeFactory typeFactory,
-      TimeUnit timeUnit, RelDataType operandType1, RelDataType operandType2) {
+      @Nullable TimeUnit timeUnit, RelDataType operandType1,
+      RelDataType operandType2) {
     final RelDataType type;
-    switch (timeUnit) {
+    TimeUnit timeUnit2 = first(timeUnit, TimeUnit.EPOCH);
+    switch (timeUnit2) {
     case HOUR:
     case MINUTE:
     case SECOND:
     case MILLISECOND:
     case MICROSECOND:
-      switch (timeUnit) {
+      switch (timeUnit2) {
       case MILLISECOND:
         type = typeFactory.createSqlType(SqlTypeName.TIMESTAMP,
             MILLISECOND_PRECISION);
@@ -95,6 +101,7 @@ public class SqlTimestampAddFunction extends SqlFunction {
       }
       break;
     default:
+    case EPOCH:
       type = operandType2;
     }
     return typeFactory.createTypeWithNullability(type,
@@ -102,6 +109,23 @@ public class SqlTimestampAddFunction extends SqlFunction {
             || operandType2.isNullable());
   }
 
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    super.validateCall(call, validator, scope, operandScope);
+
+    // This is either a time unit or a time frame:
+    //
+    //  * In "TIMESTAMPADD(YEAR, 2, x)" operand 0 is a SqlIntervalQualifier
+    //    with startUnit = YEAR and timeFrameName = null.
+    //
+    //  * In "TIMESTAMPADD(MINUTE15, 2, x) operand 0 is a SqlIntervalQualifier
+    //    with startUnit = EPOCH and timeFrameName = 'MINUTE15'.
+    //
+    // If the latter, check that timeFrameName is valid.
+    validator.validateTimeFrame(
+        (SqlIntervalQualifier) call.getOperandList().get(0));
+  }
+
   /** Creates a SqlTimestampAddFunction. */
   SqlTimestampAddFunction(String name) {
     super(name, SqlKind.TIMESTAMP_ADD, RETURN_TYPE_INFERENCE, null,
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
index c7bfdcdb6a..eb77639582 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
@@ -19,13 +19,17 @@ 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.SqlCall;
 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.SqlOperatorBinding;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 /**
  * The <code>TIMESTAMPDIFF</code> function, which calculates the difference
@@ -76,4 +80,21 @@ class SqlTimestampDiffFunction extends SqlFunction {
             SqlTypeFamily.DATETIME),
         SqlFunctionCategory.TIMEDATE);
   }
+
+  @Override public void validateCall(SqlCall call, SqlValidator validator,
+      SqlValidatorScope scope, SqlValidatorScope operandScope) {
+    super.validateCall(call, validator, scope, operandScope);
+
+    // This is either a time unit or a time frame:
+    //
+    //  * In "TIMESTAMPADD(YEAR, 2, x)" operand 0 is a SqlIntervalQualifier
+    //    with startUnit = YEAR and timeFrameName = null.
+    //
+    //  * In "TIMESTAMPADD(MINUTE15, 2, x) operand 0 is a SqlIntervalQualifier
+    //    with startUnit = EPOCH and timeFrameName = 'MINUTE15'.
+    //
+    // If the latter, check that timeFrameName is valid.
+    validator.validateTimeFrame(
+        (SqlIntervalQualifier) call.getOperandList().get(0));
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
index 0fa5602bff..9c46a38215 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
@@ -19,6 +19,7 @@ package org.apache.calcite.sql.parser;
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.config.CharLiteralStyle;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
@@ -547,7 +548,9 @@ public abstract class SqlAbstractParserImpl {
   /**
    * Sets the map from identifier to time unit.
    */
-  public abstract void setTimeUnitCodes(Map<String, TimeUnit> timeUnitCodes);
+  @Deprecated // to be removed before 2.0
+  public void setTimeUnitCodes(Map<String, TimeUnit> timeUnitCodes) {
+  }
 
   /**
    * Sets the SQL language conformance level.
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
index a0c514774d..237a366606 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
@@ -21,6 +21,8 @@ import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.config.CharLiteralStyle;
 import org.apache.calcite.config.Lex;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
@@ -52,6 +54,7 @@ public class SqlParser {
   /** Default value of {@link Config#timeUnitCodes()}.
    * The map is empty, which means that there are no abbreviations other than
    * the time unit names ("YEAR", "SECOND", etc.) */
+  @Deprecated // to be removed before 2.0
   public static final ImmutableMap<String, TimeUnit> DEFAULT_IDENTIFIER_TIMEUNIT_MAP =
       ImmutableMap.of();
 
@@ -324,7 +327,14 @@ public class SqlParser {
      *
      * <p>For example, if the map contains the entry
      * ("Y", {@link TimeUnit#YEAR}) then you can write
-     * "{@code EXTRACT(S FROM orderDate)}". */
+     * "{@code EXTRACT(S FROM orderDate)}".
+     *
+     * @deprecated This property is deprecated, and has no effect. All
+     * non-standard time units are now parsed as identifiers, and resolved in
+     * the validator. You can define custom time frames using
+     * {@link RelDataTypeSystem#deriveTimeFrameSet(TimeFrameSet)}. To alias a
+     * time frame, use {@link TimeFrameSet.Builder#addAlias(String, String)}. */
+    @Deprecated // to be removed before 2.0
     @Value.Default default Map<String, TimeUnit> timeUnitCodes() {
       return DEFAULT_IDENTIFIER_TIMEUNIT_MAP;
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/type/IntervalOperandTypeChecker.java b/core/src/main/java/org/apache/calcite/sql/type/IntervalOperandTypeChecker.java
new file mode 100644
index 0000000000..145f812803
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/type/IntervalOperandTypeChecker.java
@@ -0,0 +1,58 @@
+/*
+ * 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.type;
+
+import org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.calcite.sql.SqlCallBinding;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.util.Static;
+
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Parameter type-checking strategy whether the operand must be an interval.
+ */
+public class IntervalOperandTypeChecker implements SqlSingleOperandTypeChecker {
+
+  private final ImmutableSet<TimeUnitRange> unitSet;
+
+  IntervalOperandTypeChecker(ImmutableSet<TimeUnitRange> unitSet) {
+    this.unitSet = unitSet;
+  }
+
+  @Override public boolean checkSingleOperandType(SqlCallBinding callBinding,
+      SqlNode node, int iFormalOperand, boolean throwOnFailure) {
+    final SqlNode operand = callBinding.operand(iFormalOperand);
+    if (operand instanceof SqlIntervalQualifier) {
+      final SqlIntervalQualifier interval = (SqlIntervalQualifier) operand;
+      if (unitSet.contains(interval.timeUnitRange)) {
+        return true;
+      }
+      if (throwOnFailure) {
+        throw callBinding.getValidator().newValidationError(operand,
+            Static.RESOURCE.invalidTimeFrame(interval.timeUnitRange.name()));
+      }
+    }
+    return false;
+  }
+
+  @Override public String getAllowedSignatures(SqlOperator op, String opName) {
+    return "<INTERVAL>";
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index fb3552bf76..d0ffc19e5e 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.type;
 
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeComparability;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -29,6 +30,7 @@ import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -92,6 +94,16 @@ public abstract class OperandTypes {
     return family(families, i -> false);
   }
 
+  /**
+   * Creates a checker that passes if the operand is an interval appropriate for
+   * a given date/time type. For example, the time frame HOUR is appropriate for
+   * type TIMESTAMP or DATE but not TIME.
+   */
+  public static SqlSingleOperandTypeChecker interval(
+      Iterable<TimeUnitRange> ranges) {
+    return new IntervalOperandTypeChecker(ImmutableSet.copyOf(ranges));
+  }
+
   /**
    * Creates a checker for user-defined functions (including user-defined
    * aggregate functions, table functions, and table macros).
@@ -346,6 +358,9 @@ public abstract class OperandTypes {
   public static final SqlSingleOperandTypeChecker DATE =
       family(SqlTypeFamily.DATE);
 
+  public static final SqlSingleOperandTypeChecker TIME =
+      family(SqlTypeFamily.TIME);
+
   public static final SqlSingleOperandTypeChecker TIMESTAMP =
       family(SqlTypeFamily.TIMESTAMP);
 
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index 1143e0afc4..c052411870 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -20,6 +20,8 @@ import org.apache.calcite.config.NullCollation;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.TimeFrame;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Resources;
@@ -782,6 +784,21 @@ public interface SqlValidator {
   @API(status = API.Status.INTERNAL, since = "1.23")
   SqlValidator transform(UnaryOperator<SqlValidator.Config> transform);
 
+  /** Returns the set of allowed time frames. */
+  TimeFrameSet getTimeFrameSet();
+
+  /** Validates a time frame.
+   *
+   * <p>A time frame is either a built-in time frame based on a time unit such
+   * as {@link org.apache.calcite.avatica.util.TimeUnitRange#HOUR},
+   * or is a custom time frame represented by a name in
+   * {@link SqlIntervalQualifier#timeFrameName}. A custom time frame is
+   * validated against {@link #getTimeFrameSet()}.
+   *
+   * <p>Returns a time frame, or throws.
+   */
+  TimeFrame validateTimeFrame(SqlIntervalQualifier intervalQualifier);
+
   //~ Inner Class ------------------------------------------------------------
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index d49db759c5..b42c88582b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -28,6 +28,9 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelRecordType;
+import org.apache.calcite.rel.type.TimeFrame;
+import org.apache.calcite.rel.type.TimeFrameSet;
+import org.apache.calcite.rel.type.TimeFrames;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexPatternFieldRef;
 import org.apache.calcite.rex.RexVisitor;
@@ -239,6 +242,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   protected final RelDataType unknownType;
   private final RelDataType booleanType;
 
+  protected final TimeFrameSet timeFrameSet;
+
   /**
    * Map of derived RelDataType for each node. This is an IdentityHashMap
    * since in some cases (such as null literals) we need to discriminate by
@@ -293,6 +298,10 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     this.opTab = requireNonNull(opTab, "opTab");
     this.catalogReader = requireNonNull(catalogReader, "catalogReader");
     this.typeFactory = requireNonNull(typeFactory, "typeFactory");
+    final RelDataTypeSystem typeSystem = typeFactory.getTypeSystem();
+    this.timeFrameSet =
+        requireNonNull(typeSystem.deriveTimeFrameSet(TimeFrames.CORE),
+            "timeFrameSet");
     this.config = requireNonNull(config, "config");
 
     // It is assumed that unknown type is nullable by default
@@ -341,6 +350,10 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return unknownType;
   }
 
+  @Override public TimeFrameSet getTimeFrameSet() {
+    return timeFrameSet;
+  }
+
   @Override public SqlNodeList expandStar(
       SqlNodeList selectList,
       SqlSelect select,
@@ -3328,6 +3341,21 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
+  @Override public TimeFrame validateTimeFrame(SqlIntervalQualifier qualifier) {
+    if (qualifier.timeFrameName == null) {
+      final TimeFrame timeFrame = timeFrameSet.get(qualifier.getUnit());
+      return requireNonNull(timeFrame,
+          () -> "time frame for " + qualifier.getUnit());
+    }
+    final @Nullable TimeFrame timeFrame =
+        timeFrameSet.getOpt(qualifier.timeFrameName);
+    if (timeFrame != null) {
+      return timeFrame;
+    }
+    throw newValidationError(qualifier,
+        RESOURCE.invalidTimeFrame(qualifier.timeFrameName));
+  }
+
   /**
    * Validates the FROM clause of a query, or (recursively) a child node of
    * the FROM clause: AS, OVER, JOIN, VALUES, or sub-query.
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
index 0c32974161..987f777a6b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
@@ -18,7 +18,6 @@ package org.apache.calcite.sql2rel;
 
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
@@ -27,8 +26,6 @@ import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlTimeLiteral;
 import org.apache.calcite.sql.SqlTimestampLiteral;
-import org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.util.BitString;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.NlsString;
@@ -78,30 +75,18 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
   @Override public RexNode convertLiteral(
       SqlRexContext cx,
       SqlLiteral literal) {
-    RexBuilder rexBuilder = cx.getRexBuilder();
-    RelDataTypeFactory typeFactory = cx.getTypeFactory();
-    SqlValidator validator = cx.getValidator();
+    final RexBuilder rexBuilder = cx.getRexBuilder();
     if (literal.getValue() == null) {
-      // Since there is no eq. RexLiteral of SqlLiteral.Unknown we
-      // treat it as a cast(null as boolean)
-      RelDataType type;
-      if (literal.getTypeName() == SqlTypeName.BOOLEAN) {
-        type = typeFactory.createSqlType(SqlTypeName.BOOLEAN);
-        type = typeFactory.createTypeWithNullability(type, true);
-      } else {
-        type = validator.getValidatedNodeType(literal);
-      }
+      RelDataType type = cx.getValidator().getValidatedNodeType(literal);
       return rexBuilder.makeNullLiteral(type);
     }
 
-    final BitString bitString;
     switch (literal.getTypeName()) {
     case DECIMAL:
       // exact number
       BigDecimal bd = literal.getValueAs(BigDecimal.class);
-      return rexBuilder.makeExactLiteral(
-          bd,
-          literal.createSqlType(typeFactory));
+      RelDataType type = literal.createSqlType(cx.getTypeFactory());
+      return rexBuilder.makeExactLiteral(bd, type);
 
     case DOUBLE:
       // approximate type
@@ -113,13 +98,13 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
     case BOOLEAN:
       return rexBuilder.makeLiteral(literal.getValueAs(Boolean.class));
     case BINARY:
-      bitString = literal.getValueAs(BitString.class);
+      final BitString bitString = literal.getValueAs(BitString.class);
       Preconditions.checkArgument((bitString.getBitCount() % 8) == 0,
           "incomplete octet");
 
       // An even number of hexits (e.g. X'ABCD') makes whole number
       // of bytes.
-      ByteString byteString = new ByteString(bitString.getAsByteArray());
+      final ByteString byteString = new ByteString(bitString.getAsByteArray());
       return rexBuilder.makeBinaryLiteral(byteString);
     case SYMBOL:
       return rexBuilder.makeFlag(literal.getValueAs(Enum.class));
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 82d5f1ced8..3b5eda2869 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rel.type.TimeFrame;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexCallBinding;
@@ -94,6 +95,7 @@ import java.util.Objects;
 import java.util.stream.Collectors;
 
 import static org.apache.calcite.sql.type.NonNullableAccessors.getComponentTypeOrThrow;
+import static org.apache.calcite.util.Util.first;
 
 import static java.util.Objects.requireNonNull;
 
@@ -1826,12 +1828,33 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       // TIMESTAMPADD(unit, count, timestamp)
       //  => timestamp + count * INTERVAL '1' UNIT
       final RexBuilder rexBuilder = cx.getRexBuilder();
-      final SqlLiteral unitLiteral = call.operand(0);
-      final TimeUnit unit = unitLiteral.getValueAs(TimeUnit.class);
+      SqlIntervalQualifier qualifier = call.operand(0);
+      final TimeFrame timeFrame = cx.getValidator().validateTimeFrame(qualifier);
+      final TimeUnit unit = first(timeFrame.unit(), TimeUnit.EPOCH);
+      final RelDataType type = cx.getValidator().getValidatedNodeType(call);
+      final RexNode op1 = cx.convertExpression(call.operand(1));
+      final RexNode op2 = cx.convertExpression(call.operand(2));
+      if (unit == TimeUnit.EPOCH && qualifier.timeFrameName != null) {
+        // Custom time frames have a different path. They are kept as names,
+        // and then handled by Java functions such as
+        // SqlFunctions.customTimestampAdd.
+        final RexLiteral timeFrameName =
+            rexBuilder.makeLiteral(qualifier.timeFrameName);
+        // If the TIMESTAMPADD call has type TIMESTAMP and op2 has type DATE
+        // (which can happen for sub-day time frames such as HOUR), cast op2 to
+        // TIMESTAMP.
+        final RexNode op2b = rexBuilder.makeCast(type, op2, false);
+        return rexBuilder.makeCall(type, SqlStdOperatorTable.TIMESTAMP_ADD,
+            ImmutableList.of(timeFrameName, op1, op2b));
+      }
+
+      if (qualifier.getUnit() != unit) {
+        qualifier =
+            new SqlIntervalQualifier(unit, null,
+                qualifier.getParserPosition());
+      }
+
       RexNode interval2Add;
-      SqlIntervalQualifier qualifier =
-          new SqlIntervalQualifier(unit, null, unitLiteral.getParserPosition());
-      RexNode op1 = cx.convertExpression(call.operand(1));
       switch (unit) {
       case MICROSECOND:
       case NANOSECOND:
@@ -1848,7 +1871,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       }
 
       return rexBuilder.makeCall(SqlStdOperatorTable.DATETIME_PLUS,
-          cx.convertExpression(call.operand(2)), interval2Add);
+          op2, interval2Add);
     }
   }
 
@@ -1858,8 +1881,22 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       // TIMESTAMPDIFF(unit, t1, t2)
       //    => (t2 - t1) UNIT
       final RexBuilder rexBuilder = cx.getRexBuilder();
-      final SqlLiteral unitLiteral = call.operand(0);
-      TimeUnit unit = unitLiteral.getValueAs(TimeUnit.class);
+      SqlIntervalQualifier qualifier = call.operand(0);
+      final TimeFrame timeFrame = cx.getValidator().validateTimeFrame(qualifier);
+      final TimeUnit unit = first(timeFrame.unit(), TimeUnit.EPOCH);
+
+      final RexNode op1 = cx.convertExpression(call.operand(1));
+      final RexNode op2 = cx.convertExpression(call.operand(2));
+      if (unit == TimeUnit.EPOCH && qualifier.timeFrameName != null) {
+        // Custom time frames have a different path. They are kept as names, and
+        // then handled by Java functions.
+        final RexLiteral timeFrameName =
+            rexBuilder.makeLiteral(qualifier.timeFrameName);
+        return rexBuilder.makeCall(cx.getValidator().getValidatedNodeType(call),
+            SqlStdOperatorTable.TIMESTAMP_DIFF,
+            ImmutableList.of(timeFrameName, op1, op2));
+      }
+
       BigDecimal multiplier = BigDecimal.ONE;
       BigDecimal divider = BigDecimal.ONE;
       SqlTypeName sqlTypeName = unit == TimeUnit.NANOSECOND
@@ -1872,19 +1909,24 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       case WEEK:
         multiplier = BigDecimal.valueOf(DateTimeUtils.MILLIS_PER_SECOND);
         divider = unit.multiplier;
-        unit = TimeUnit.SECOND;
+        qualifier =
+            new SqlIntervalQualifier(TimeUnit.SECOND, null,
+                qualifier.getParserPosition());
         break;
       case QUARTER:
+      case CENTURY:
+      case MILLENNIUM:
         divider = unit.multiplier;
-        unit = TimeUnit.MONTH;
+        qualifier =
+            new SqlIntervalQualifier(TimeUnit.MONTH, null,
+                qualifier.getParserPosition());
         break;
       default:
+        qualifier =
+            new SqlIntervalQualifier(unit, null,
+                qualifier.getParserPosition());
         break;
       }
-      final SqlIntervalQualifier qualifier =
-          new SqlIntervalQualifier(unit, null, SqlParserPos.ZERO);
-      final RexNode op2 = cx.convertExpression(call.operand(2));
-      final RexNode op1 = cx.convertExpression(call.operand(1));
       final RelDataType intervalType =
           cx.getTypeFactory().createTypeWithNullability(
               cx.getTypeFactory().createSqlIntervalType(qualifier),
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 6c1b56d15d..55d2f07b94 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -505,6 +505,22 @@ public enum BuiltInMethod {
       TimeUnitRange.class, long.class),
   UNIX_TIMESTAMP_CEIL(DateTimeUtils.class, "unixTimestampCeil",
       TimeUnitRange.class, long.class),
+  CUSTOM_DATE_ADD(SqlFunctions.class, "customDateAdd",
+      DataContext.class, String.class, int.class, int.class),
+  CUSTOM_DATE_DIFF(SqlFunctions.class, "customDateDiff",
+      DataContext.class, String.class, int.class, int.class),
+  CUSTOM_DATE_FLOOR(SqlFunctions.class, "customDateFloor",
+      DataContext.class, String.class, int.class),
+  CUSTOM_DATE_CEIL(SqlFunctions.class, "customDateCeil",
+      DataContext.class, String.class, int.class),
+  CUSTOM_TIMESTAMP_ADD(SqlFunctions.class, "customTimestampAdd",
+      DataContext.class, String.class, long.class, long.class),
+  CUSTOM_TIMESTAMP_DIFF(SqlFunctions.class, "customTimestampDiff",
+      DataContext.class, String.class, long.class, long.class),
+  CUSTOM_TIMESTAMP_FLOOR(SqlFunctions.class, "customTimestampFloor",
+      DataContext.class, String.class, long.class),
+  CUSTOM_TIMESTAMP_CEIL(SqlFunctions.class, "customTimestampCeil",
+      DataContext.class, String.class, long.class),
   LAST_DAY(SqlFunctions.class, "lastDay", int.class),
   DAYNAME_WITH_TIMESTAMP(SqlFunctions.class,
       "dayNameWithTimestamp", long.class, Locale.class),
diff --git a/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java b/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java
index aff1229ca7..c2da847b76 100644
--- a/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java
+++ b/core/src/main/java/org/apache/calcite/util/DateTimeStringUtils.java
@@ -24,6 +24,8 @@ import java.util.TimeZone;
 
 /**
  * Utility methods to manipulate String representation of DateTime values.
+ *
+ * @see DateTimeUtils
  */
 public class DateTimeStringUtils {
 
diff --git a/core/src/main/java/org/apache/calcite/util/NameMap.java b/core/src/main/java/org/apache/calcite/util/NameMap.java
index 0a63332e81..a41e3cdd3d 100644
--- a/core/src/main/java/org/apache/calcite/util/NameMap.java
+++ b/core/src/main/java/org/apache/calcite/util/NameMap.java
@@ -62,7 +62,7 @@ public class NameMap<V> {
   }
 
   /** Creates a NameMap that is an immutable copy of a given map. */
-  public static <V> NameMap immutableCopyOf(Map<String, V> names) {
+  public static <V> NameMap<V> immutableCopyOf(Map<String, V> names) {
     return new NameMap<>(ImmutableSortedMap.copyOf(names, COMPARATOR));
   }
 
diff --git a/core/src/main/java/org/apache/calcite/util/TimestampString.java b/core/src/main/java/org/apache/calcite/util/TimestampString.java
index 48c058d3ba..b6248351be 100644
--- a/core/src/main/java/org/apache/calcite/util/TimestampString.java
+++ b/core/src/main/java/org/apache/calcite/util/TimestampString.java
@@ -40,6 +40,10 @@ public class TimestampString implements Comparable<TimestampString> {
           + " "
           + "[0-9][0-9]:[0-9][0-9]:[0-9][0-9](\\.[0-9]*[1-9])?");
 
+  /** The Unix epoch. */
+  public static final TimestampString EPOCH =
+      new TimestampString(1970, 1, 1, 0, 0, 0);
+
   final String v;
 
   /** Creates a TimeString. */
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 04d4d6844e..c904535c98 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -194,7 +194,7 @@ InvalidBoolean=''{0}'' is not a valid boolean value
 ArgumentMustBeValidPrecision=Argument to function ''{0}'' must be a valid precision between ''{1,number,#}'' and ''{2,number,#}''
 IllegalArgumentForTableFunctionCall=Wrong arguments for table function ''{0}'' call. Expected ''{1}'', actual ''{2}''
 CannotCallTableFunctionHere=Cannot call table function here: ''{0}''
-InvalidDatetimeFormat=''{0}'' is not a valid datetime format
+InvalidTimeFrame=''{0}'' is not a valid time frame
 InsertIntoAlwaysGenerated=Cannot INSERT into generated column ''{0}''
 ArgumentMustHaveScaleZero=Argument to function ''{0}'' must have a scale of 0
 PreparationAborted=Statement preparation aborted
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 81ab2ccc3b..d972c830bb 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -20,9 +20,11 @@ import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.config.Lex;
+import org.apache.calcite.rel.type.DelegatingTypeSystem;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -60,6 +62,7 @@ import org.apache.calcite.util.ImmutableBitSet;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Ordering;
 
 import org.junit.jupiter.api.Disabled;
@@ -77,6 +80,7 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Objects;
+import java.util.function.BiConsumer;
 import java.util.function.Consumer;
 
 import static org.apache.calcite.test.Matchers.isCharset;
@@ -4079,9 +4083,9 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .columnType("INTEGER");
 
     expr("timestampadd(^incorrect^, 1, current_timestamp)")
-        .fails("(?s).*Was expecting one of.*");
+        .fails("'INCORRECT' is not a valid time frame");
     expr("timestampdiff(^incorrect^, current_timestamp, current_timestamp)")
-        .fails("(?s).*Was expecting one of.*");
+        .fails("'INCORRECT' is not a valid time frame");
   }
 
   @Test void testTimestampAddNullInterval() {
@@ -4297,6 +4301,107 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .columnType("INTERVAL SECOND NOT NULL");
   }
 
+  /** Tests that EXTRACT, FLOOR, CEIL functions accept abbreviations for
+   * time units (such as "Y" for "YEAR").
+   *
+   * <p>This used to be accomplished via the now deprecated
+   * {@code timeUnitCodes} method in {@link SqlParser.Config}, and is now
+   * accomplished via
+   * {@link RelDataTypeSystem#deriveTimeFrameSet(TimeFrameSet)}. */
+  @Test void testTimeUnitCodes() {
+    final Map<String, TimeUnit> simpleCodes =
+        ImmutableMap.<String, TimeUnit>builder()
+            .put("Y", TimeUnit.YEAR)
+            .put("M", TimeUnit.MONTH)
+            .put("D", TimeUnit.DAY)
+            .put("H", TimeUnit.HOUR)
+            .put("N", TimeUnit.MINUTE)
+            .put("S", TimeUnit.SECOND)
+            .build();
+
+    // Time unit abbreviations for Microsoft SQL Server
+    final Map<String, TimeUnit> mssqlCodes =
+        ImmutableMap.<String, TimeUnit>builder()
+            .put("Y", TimeUnit.YEAR)
+            .put("YY", TimeUnit.YEAR)
+            .put("YYYY", TimeUnit.YEAR)
+            .put("Q", TimeUnit.QUARTER)
+            .put("QQ", TimeUnit.QUARTER)
+            .put("M", TimeUnit.MONTH)
+            .put("MM", TimeUnit.MONTH)
+            .put("W", TimeUnit.WEEK)
+            .put("WK", TimeUnit.WEEK)
+            .put("WW", TimeUnit.WEEK)
+            .put("DY", TimeUnit.DOY)
+            .put("DW", TimeUnit.DOW)
+            .put("D", TimeUnit.DAY)
+            .put("DD", TimeUnit.DAY)
+            .put("H", TimeUnit.HOUR)
+            .put("HH", TimeUnit.HOUR)
+            .put("N", TimeUnit.MINUTE)
+            .put("MI", TimeUnit.MINUTE)
+            .put("S", TimeUnit.SECOND)
+            .put("SS", TimeUnit.SECOND)
+            .put("MS", TimeUnit.MILLISECOND)
+            .build();
+
+    checkTimeUnitCodes(ImmutableMap.of());
+    checkTimeUnitCodes(simpleCodes);
+    checkTimeUnitCodes(mssqlCodes);
+  }
+
+  /** Checks parsing of built-in functions that accept time unit
+   * abbreviations.
+   *
+   * <p>For example, {@code EXTRACT(Y FROM orderDate)} is using
+   * "Y" as an abbreviation for "YEAR".
+   *
+   * <p>Override if your parser supports more such functions. */
+  protected void checkTimeUnitCodes(Map<String, TimeUnit> timeUnitCodes) {
+    SqlValidatorFixture f = fixture()
+        .withFactory(tf ->
+            tf.withTypeSystem(typeSystem ->
+                new DelegatingTypeSystem(typeSystem) {
+                  @Override public TimeFrameSet deriveTimeFrameSet(
+                      TimeFrameSet frameSet) {
+                    TimeFrameSet.Builder b = TimeFrameSet.builder();
+                    b.addAll(frameSet);
+                    timeUnitCodes.forEach((name, unit) ->
+                        b.addAlias(name, unit.name()));
+                    return b.build();
+                  }
+                }));
+    final String ts = "TIMESTAMP '2020-08-27 18:16:43'";
+    BiConsumer<String, TimeUnit> validConsumer = (abbrev, timeUnit) -> {
+      f.withSql("select extract(" + abbrev + " from " + ts + ")").ok();
+      f.withSql("select floor(" + ts + " to " + abbrev + ")").ok();
+      f.withSql("select ceil(" + ts + " to " + abbrev + ")").ok();
+    };
+    BiConsumer<String, TimeUnit> invalidConsumer = (abbrev, timeUnit) -> {
+      final String upAbbrev = abbrev.toUpperCase(Locale.ROOT);
+      String message = "'" + upAbbrev + "' is not a valid time frame";
+      f.withSql("select extract(^" + abbrev + "^ from " + ts + ")")
+          .fails(message);
+      f.withSql("SELECT FLOOR(" + ts + " to ^" + abbrev + "^)")
+          .fails(message);
+      f.withSql("SELECT CEIL(" + ts + " to ^" + abbrev + "^)")
+          .fails(message);
+    };
+
+    // Check that each valid code passes each query that it should.
+    timeUnitCodes.forEach(validConsumer);
+
+    // If "M" is a valid code then "m" should be also.
+    timeUnitCodes.forEach((abbrev, timeUnit) ->
+        validConsumer.accept(abbrev.toLowerCase(Locale.ROOT), timeUnit));
+
+    // Check that invalid codes generate the right error messages.
+    final Map<String, TimeUnit> invalidCodes =
+        ImmutableMap.of("A", TimeUnit.YEAR,
+            "a", TimeUnit.YEAR);
+    invalidCodes.forEach(invalidConsumer);
+  }
+
   public void checkWinFuncExpWithWinClause(
       String sql,
       String expectedMsgPattern) {
diff --git a/core/src/test/java/org/apache/calcite/test/TimeFrameTest.java b/core/src/test/java/org/apache/calcite/test/TimeFrameTest.java
new file mode 100644
index 0000000000..046b94ffd6
--- /dev/null
+++ b/core/src/test/java/org/apache/calcite/test/TimeFrameTest.java
@@ -0,0 +1,649 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.calcite.rel.type.TimeFrame;
+import org.apache.calcite.rel.type.TimeFrameSet;
+import org.apache.calcite.rel.type.TimeFrames;
+import org.apache.calcite.util.Pair;
+
+import org.apache.commons.math3.fraction.BigFraction;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.hamcrest.Matcher;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.calcite.avatica.util.DateTimeUtils.dateStringToUnixDate;
+import static org.apache.calcite.avatica.util.DateTimeUtils.timestampStringToUnixDate;
+import static org.apache.calcite.avatica.util.DateTimeUtils.unixDateToString;
+import static org.apache.calcite.avatica.util.DateTimeUtils.unixTimestampToString;
+import static org.apache.calcite.avatica.util.TimeUnit.CENTURY;
+import static org.apache.calcite.avatica.util.TimeUnit.DAY;
+import static org.apache.calcite.avatica.util.TimeUnit.DECADE;
+import static org.apache.calcite.avatica.util.TimeUnit.HOUR;
+import static org.apache.calcite.avatica.util.TimeUnit.ISODOW;
+import static org.apache.calcite.avatica.util.TimeUnit.ISOYEAR;
+import static org.apache.calcite.avatica.util.TimeUnit.MICROSECOND;
+import static org.apache.calcite.avatica.util.TimeUnit.MILLENNIUM;
+import static org.apache.calcite.avatica.util.TimeUnit.MILLISECOND;
+import static org.apache.calcite.avatica.util.TimeUnit.MINUTE;
+import static org.apache.calcite.avatica.util.TimeUnit.MONTH;
+import static org.apache.calcite.avatica.util.TimeUnit.NANOSECOND;
+import static org.apache.calcite.avatica.util.TimeUnit.QUARTER;
+import static org.apache.calcite.avatica.util.TimeUnit.SECOND;
+import static org.apache.calcite.avatica.util.TimeUnit.WEEK;
+import static org.apache.calcite.avatica.util.TimeUnit.YEAR;
+
+import static org.hamcrest.CoreMatchers.notNullValue;
+import static org.hamcrest.CoreMatchers.nullValue;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.Is.is;
+import static org.junit.jupiter.api.Assertions.fail;
+
+import static java.util.Objects.requireNonNull;
+
+/** Unit test for {@link org.apache.calcite.rel.type.TimeFrame}. */
+public class TimeFrameTest {
+  /** Unit test for {@link org.apache.calcite.rel.type.TimeFrames#CORE}. */
+  @Test void testAvaticaTimeFrame() {
+    final TimeFrameSet timeFrameSet = TimeFrames.CORE;
+    final TimeFrame year = timeFrameSet.get(TimeUnit.YEAR);
+    assertThat(year, notNullValue());
+    assertThat(year.name(), is("YEAR"));
+    assertThat(timeFrameSet.getUnit(year), is(YEAR));
+
+    final TimeFrame month = timeFrameSet.get(MONTH);
+    assertThat(month, notNullValue());
+    assertThat(month.name(), is("MONTH"));
+    assertThat(timeFrameSet.getUnit(month), is(MONTH));
+
+    final Number monthPerYear = month.per(year);
+    assertThat(monthPerYear, notNullValue());
+    assertThat(monthPerYear, is(new BigFraction(12)));
+    final Number yearPerMonth = year.per(month);
+    assertThat(yearPerMonth, notNullValue());
+    assertThat(yearPerMonth, is(BigFraction.ONE.divide(12)));
+    final Number monthPerMonth = month.per(month);
+    assertThat(monthPerMonth, notNullValue());
+    assertThat(monthPerMonth, is(BigFraction.ONE));
+
+    final TimeFrame second = timeFrameSet.get(TimeUnit.SECOND);
+    assertThat(second, notNullValue());
+    assertThat(second.name(), is("SECOND"));
+
+    final TimeFrame minute = timeFrameSet.get(TimeUnit.MINUTE);
+    assertThat(minute, notNullValue());
+    assertThat(minute.name(), is("MINUTE"));
+
+    final TimeFrame nano = timeFrameSet.get(TimeUnit.NANOSECOND);
+    assertThat(nano, notNullValue());
+    assertThat(nano.name(), is("NANOSECOND"));
+
+    final Number secondPerMonth = second.per(month);
+    assertThat(secondPerMonth, nullValue());
+    final Number nanoPerMinute = nano.per(minute);
+    assertThat(nanoPerMinute, notNullValue());
+    assertThat(nanoPerMinute,
+        is(BigFraction.ONE.multiply(1_000).multiply(1_000).multiply(1_000)
+            .multiply(60)));
+
+    // ISOWEEK is the only core time frame without a corresponding time unit.
+    // There is no TimeUnit.ISOWEEK.
+    final TimeFrame isoWeek = timeFrameSet.get("ISOWEEK");
+    assertThat(isoWeek, notNullValue());
+    assertThat(isoWeek.name(), is("ISOWEEK"));
+    assertThat(timeFrameSet.getUnit(isoWeek), nullValue());
+
+    // FRAC_SECOND is an alias.
+    final TimeFrame fracSecond = timeFrameSet.get("FRAC_SECOND");
+    assertThat(fracSecond, notNullValue());
+    assertThat(fracSecond.name(), is("MICROSECOND"));
+    assertThat(timeFrameSet.getUnit(fracSecond), is(MICROSECOND));
+
+    // SQL_TSI_QUARTER is an alias.
+    final TimeFrame sqlTsiQuarter = timeFrameSet.get("SQL_TSI_QUARTER");
+    assertThat(sqlTsiQuarter, notNullValue());
+    assertThat(sqlTsiQuarter.name(), is("QUARTER"));
+    assertThat(timeFrameSet.getUnit(sqlTsiQuarter), is(QUARTER));
+  }
+
+  @Test void testConflict() {
+    TimeFrameSet.Builder b = TimeFrameSet.builder();
+    b.addCore("SECOND");
+    b.addMultiple("MINUTE", 60, "SECOND");
+    b.addMultiple("HOUR", 60, "MINUTE");
+    b.addMultiple("DAY", 24, "SECOND");
+    b.addDivision("MILLISECOND", 1_000, "SECOND");
+
+    // It's important that TimeFrame.Builder throws when you attempt to add
+    // a frame with the same name. It prevents DAGs and cycles.
+    try {
+      b.addDivision("MILLISECOND", 10_000, "MINUTE");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("duplicate frame: MILLISECOND"));
+    }
+
+    try {
+      b.addCore("SECOND");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("duplicate frame: SECOND"));
+    }
+
+    try {
+      b.addQuotient("SECOND", "MINUTE", "HOUR");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("duplicate frame: SECOND"));
+    }
+
+    try {
+      b.addQuotient("MINUTE_OF_WEEK", "MINUTE", "WEEK");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("unknown frame: WEEK"));
+    }
+
+    try {
+      b.addQuotient("DAY_OF_WEEK", "DAY", "YEAR");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("unknown frame: YEAR"));
+    }
+
+    try {
+      b.addAlias("SECOND", "DAY");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("duplicate frame: SECOND"));
+    }
+
+    try {
+      b.addAlias("FOO", "BAZ");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("unknown frame: BAZ"));
+    }
+
+    // Can't define NANOSECOND in terms of a frame that has not been defined
+    // yet.
+    try {
+      b.addDivision("NANOSECOND", 1_000, "MICROSECOND");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("unknown frame: MICROSECOND"));
+    }
+
+    // We can define NANOSECOND and MICROSECOND as long as we define each frame
+    // in terms of previous frames.
+    b.addDivision("NANOSECOND", 1_000_000, "MILLISECOND");
+    b.addMultiple("MICROSECOND", 1_000, "NANOSECOND");
+
+    // Can't define a frame in terms of itself.
+    // (I guess you should use a core frame.)
+    try {
+      b.addDivision("PICOSECOND", 1, "PICOSECOND");
+      fail("expected error");
+    } catch (IllegalArgumentException e) {
+      assertThat(e.getMessage(), is("unknown frame: PICOSECOND"));
+    }
+
+    final TimeFrameSet timeFrameSet = b.build();
+
+    final TimeFrame second = timeFrameSet.get("SECOND");
+    final TimeFrame hour = timeFrameSet.get("HOUR");
+    assertThat(hour.per(second), is(BigFraction.ONE.divide(3_600)));
+    final TimeFrame millisecond = timeFrameSet.get("MILLISECOND");
+    assertThat(hour.per(millisecond), is(BigFraction.ONE.divide(3_600_000)));
+    final TimeFrame nanosecond = timeFrameSet.get("NANOSECOND");
+    assertThat(nanosecond.per(second),
+        is(BigFraction.ONE.multiply(1_000_000_000)));
+  }
+
+  @Test void testEvalFloor() {
+    final Fixture f = new Fixture();
+    f.checkDateFloor("1970-08-01", WEEK, is("1970-07-26")); // saturday
+    f.checkDateFloor("1970-08-02", WEEK, is("1970-08-02")); // sunday
+    f.checkDateFloor("1970-08-03", WEEK, is("1970-08-02")); // monday
+    f.checkDateFloor("1970-08-04", WEEK, is("1970-08-02")); // tuesday
+
+    f.checkDateFloor("1970-08-01", f.isoWeek, is("1970-07-27")); // saturday
+    f.checkDateFloor("1970-08-02", f.isoWeek, is("1970-07-27")); // sunday
+    f.checkDateFloor("1970-08-03", f.isoWeek, is("1970-08-03")); // monday
+    f.checkDateFloor("1970-08-04", f.isoWeek, is("1970-08-03")); // tuesday
+
+    f.checkTimestampFloor("1970-01-01 01:23:45", HOUR,
+        0, is("1970-01-01 01:00:00"));
+    f.checkTimestampFloor("1970-01-01 01:23:45", MINUTE,
+        0, is("1970-01-01 01:23:00"));
+    f.checkTimestampFloor("1970-01-01 01:23:45.67", SECOND,
+        0, is("1970-01-01 01:23:45"));
+    f.checkTimestampFloor("1970-01-01 01:23:45.6789012345", MILLISECOND,
+        4, is("1970-01-01 01:23:45.6790"));
+    // Time frames can represent unlimited precision, but out representation of
+    // timestamp can't represent more than millisecond precision.
+    f.checkTimestampFloor("1970-01-01 01:23:45.6789012345", MICROSECOND,
+        7, is("1970-01-01 01:23:45.6790000"));
+
+    f.checkTimestampFloor("1971-12-25 01:23:45", DAY,
+        0, is("1971-12-25 00:00:00"));
+    f.checkTimestampFloor("1971-12-25 01:23:45", WEEK,
+        0, is("1971-12-19 00:00:00"));
+  }
+
+  @Test void testCanRollUp() {
+    final Fixture f = new Fixture();
+
+    // The rollup from DAY to MONTH is special. It provides the bridge between
+    // the frames in the SECOND family and those in the MONTH family.
+    f.checkCanRollUp(DAY, MONTH, true);
+    f.checkCanRollUp(MONTH, DAY, false);
+
+    // Note 0: when we pass TimeUnit.ISODOW to tests, we mean f.ISOWEEK.
+
+    f.checkCanRollUp(NANOSECOND, NANOSECOND, true);
+    f.checkCanRollUp(NANOSECOND, MICROSECOND, true);
+    f.checkCanRollUp(NANOSECOND, MILLISECOND, true);
+    f.checkCanRollUp(NANOSECOND, SECOND, true);
+    f.checkCanRollUp(NANOSECOND, MINUTE, true);
+    f.checkCanRollUp(NANOSECOND, HOUR, true);
+    f.checkCanRollUp(NANOSECOND, DAY, true);
+    f.checkCanRollUp(NANOSECOND, WEEK, true);
+    f.checkCanRollUp(NANOSECOND, f.isoWeek, true); // see note 0
+    f.checkCanRollUp(NANOSECOND, MONTH, true);
+    f.checkCanRollUp(NANOSECOND, QUARTER, true);
+    f.checkCanRollUp(NANOSECOND, YEAR, true);
+    f.checkCanRollUp(NANOSECOND, ISOYEAR, true);
+    f.checkCanRollUp(NANOSECOND, CENTURY, true);
+    f.checkCanRollUp(NANOSECOND, DECADE, true);
+    f.checkCanRollUp(NANOSECOND, MILLENNIUM, true);
+
+    f.checkCanRollUp(MICROSECOND, NANOSECOND, false);
+    f.checkCanRollUp(MICROSECOND, MICROSECOND, true);
+    f.checkCanRollUp(MICROSECOND, MILLISECOND, true);
+    f.checkCanRollUp(MICROSECOND, SECOND, true);
+    f.checkCanRollUp(MICROSECOND, MINUTE, true);
+    f.checkCanRollUp(MICROSECOND, HOUR, true);
+    f.checkCanRollUp(MICROSECOND, DAY, true);
+    f.checkCanRollUp(MICROSECOND, WEEK, true);
+    f.checkCanRollUp(MICROSECOND, f.isoWeek, true);
+    f.checkCanRollUp(MICROSECOND, MONTH, true);
+    f.checkCanRollUp(MICROSECOND, QUARTER, true);
+    f.checkCanRollUp(MICROSECOND, YEAR, true);
+    f.checkCanRollUp(MICROSECOND, ISOYEAR, true);
+    f.checkCanRollUp(MICROSECOND, CENTURY, true);
+    f.checkCanRollUp(MICROSECOND, DECADE, true);
+    f.checkCanRollUp(MICROSECOND, MILLENNIUM, true);
+
+    f.checkCanRollUp(MILLISECOND, NANOSECOND, false);
+    f.checkCanRollUp(MILLISECOND, MICROSECOND, false);
+    f.checkCanRollUp(MILLISECOND, MILLISECOND, true);
+    f.checkCanRollUp(MILLISECOND, SECOND, true);
+    f.checkCanRollUp(MILLISECOND, MINUTE, true);
+    f.checkCanRollUp(MILLISECOND, HOUR, true);
+    f.checkCanRollUp(MILLISECOND, DAY, true);
+    f.checkCanRollUp(MILLISECOND, WEEK, true);
+    f.checkCanRollUp(MILLISECOND, f.isoWeek, true);
+    f.checkCanRollUp(MILLISECOND, MONTH, true);
+    f.checkCanRollUp(MILLISECOND, QUARTER, true);
+    f.checkCanRollUp(MILLISECOND, YEAR, true);
+    f.checkCanRollUp(MILLISECOND, ISOYEAR, true);
+    f.checkCanRollUp(MILLISECOND, CENTURY, true);
+    f.checkCanRollUp(MILLISECOND, DECADE, true);
+    f.checkCanRollUp(MILLISECOND, MILLENNIUM, true);
+
+    f.checkCanRollUp(SECOND, NANOSECOND, false);
+    f.checkCanRollUp(SECOND, MICROSECOND, false);
+    f.checkCanRollUp(SECOND, MILLISECOND, false);
+    f.checkCanRollUp(SECOND, SECOND, true);
+    f.checkCanRollUp(SECOND, MINUTE, true);
+    f.checkCanRollUp(SECOND, HOUR, true);
+    f.checkCanRollUp(SECOND, DAY, true);
+    f.checkCanRollUp(SECOND, WEEK, true);
+    f.checkCanRollUp(SECOND, f.isoWeek, true);
+    f.checkCanRollUp(SECOND, MONTH, true);
+    f.checkCanRollUp(SECOND, QUARTER, true);
+    f.checkCanRollUp(SECOND, YEAR, true);
+    f.checkCanRollUp(SECOND, ISOYEAR, true);
+    f.checkCanRollUp(SECOND, CENTURY, true);
+    f.checkCanRollUp(SECOND, DECADE, true);
+    f.checkCanRollUp(SECOND, MILLENNIUM, true);
+
+    f.checkCanRollUp(MINUTE, NANOSECOND, false);
+    f.checkCanRollUp(MINUTE, MICROSECOND, false);
+    f.checkCanRollUp(MINUTE, MILLISECOND, false);
+    f.checkCanRollUp(MINUTE, SECOND, false);
+    f.checkCanRollUp(MINUTE, MINUTE, true);
+    f.checkCanRollUp(MINUTE, HOUR, true);
+    f.checkCanRollUp(MINUTE, DAY, true);
+    f.checkCanRollUp(MINUTE, WEEK, true);
+    f.checkCanRollUp(MINUTE, f.isoWeek, true);
+    f.checkCanRollUp(MINUTE, MONTH, true);
+    f.checkCanRollUp(MINUTE, QUARTER, true);
+    f.checkCanRollUp(MINUTE, YEAR, true);
+    f.checkCanRollUp(MINUTE, ISOYEAR, true);
+    f.checkCanRollUp(MINUTE, CENTURY, true);
+    f.checkCanRollUp(MINUTE, DECADE, true);
+    f.checkCanRollUp(MINUTE, MILLENNIUM, true);
+
+    f.checkCanRollUp(HOUR, NANOSECOND, false);
+    f.checkCanRollUp(HOUR, MICROSECOND, false);
+    f.checkCanRollUp(HOUR, MILLISECOND, false);
+    f.checkCanRollUp(HOUR, SECOND, false);
+    f.checkCanRollUp(HOUR, MINUTE, false);
+    f.checkCanRollUp(HOUR, HOUR, true);
+    f.checkCanRollUp(HOUR, DAY, true);
+    f.checkCanRollUp(HOUR, WEEK, true);
+    f.checkCanRollUp(HOUR, f.isoWeek, true);
+    f.checkCanRollUp(HOUR, MONTH, true);
+    f.checkCanRollUp(HOUR, QUARTER, true);
+    f.checkCanRollUp(HOUR, YEAR, true);
+    f.checkCanRollUp(HOUR, ISOYEAR, true);
+    f.checkCanRollUp(HOUR, DECADE, true);
+    f.checkCanRollUp(HOUR, CENTURY, true);
+    f.checkCanRollUp(HOUR, MILLENNIUM, true);
+
+    f.checkCanRollUp(DAY, NANOSECOND, false);
+    f.checkCanRollUp(DAY, MICROSECOND, false);
+    f.checkCanRollUp(DAY, MILLISECOND, false);
+    f.checkCanRollUp(DAY, SECOND, false);
+    f.checkCanRollUp(DAY, MINUTE, false);
+    f.checkCanRollUp(DAY, HOUR, false);
+    f.checkCanRollUp(DAY, DAY, true);
+    f.checkCanRollUp(DAY, WEEK, true);
+    f.checkCanRollUp(DAY, f.isoWeek, true);
+    f.checkCanRollUp(DAY, MONTH, true);
+    f.checkCanRollUp(DAY, QUARTER, true);
+    f.checkCanRollUp(DAY, YEAR, true);
+    f.checkCanRollUp(DAY, ISOYEAR, true);
+    f.checkCanRollUp(DAY, DECADE, true);
+    f.checkCanRollUp(DAY, CENTURY, true);
+    f.checkCanRollUp(DAY, MILLENNIUM, true);
+
+    // Note 1. WEEK cannot roll up to MONTH, YEAR or higher.
+    // Some weeks cross month, year, decade, century and millennium boundaries.
+
+    // Note 2. WEEK, MONTH, QUARTER, YEAR, DECADE, CENTURY, MILLENNIUM cannot
+    // roll up to ISOYEAR. Only f.ISOWEEK can roll up to ISOYEAR.
+
+    f.checkCanRollUp(WEEK, NANOSECOND, false);
+    f.checkCanRollUp(WEEK, MICROSECOND, false);
+    f.checkCanRollUp(WEEK, MILLISECOND, false);
+    f.checkCanRollUp(WEEK, SECOND, false);
+    f.checkCanRollUp(WEEK, MINUTE, false);
+    f.checkCanRollUp(WEEK, HOUR, false);
+    f.checkCanRollUp(WEEK, DAY, false);
+    f.checkCanRollUp(WEEK, WEEK, true);
+    f.checkCanRollUp(WEEK, f.isoWeek, false);
+    f.checkCanRollUp(WEEK, MONTH, false); // see note 1
+    f.checkCanRollUp(WEEK, QUARTER, false); // see note 1
+    f.checkCanRollUp(WEEK, YEAR, false); // see note 1
+    f.checkCanRollUp(WEEK, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(WEEK, DECADE, false); // see note 1
+    f.checkCanRollUp(WEEK, CENTURY, false); // see note 1
+    f.checkCanRollUp(WEEK, MILLENNIUM, false); // see note 1
+
+    f.checkCanRollUp(f.isoWeek, NANOSECOND, false);
+    f.checkCanRollUp(f.isoWeek, MICROSECOND, false);
+    f.checkCanRollUp(f.isoWeek, MILLISECOND, false);
+    f.checkCanRollUp(f.isoWeek, SECOND, false);
+    f.checkCanRollUp(f.isoWeek, MINUTE, false);
+    f.checkCanRollUp(f.isoWeek, HOUR, false);
+    f.checkCanRollUp(f.isoWeek, DAY, false);
+    f.checkCanRollUp(f.isoWeek, WEEK, false);
+    f.checkCanRollUp(f.isoWeek, f.isoWeek, true);
+    f.checkCanRollUp(f.isoWeek, MONTH, false); // see note 1
+    f.checkCanRollUp(f.isoWeek, QUARTER, false); // see note 1
+    f.checkCanRollUp(f.isoWeek, YEAR, false); // see note 1
+    f.checkCanRollUp(f.isoWeek, ISOYEAR, true); // see note 2
+    f.checkCanRollUp(f.isoWeek, DECADE, false); // see note 1
+    f.checkCanRollUp(f.isoWeek, CENTURY, false); // see note 1
+    f.checkCanRollUp(f.isoWeek, MILLENNIUM, false); // see note 1
+
+    f.checkCanRollUp(MONTH, NANOSECOND, false);
+    f.checkCanRollUp(MONTH, MICROSECOND, false);
+    f.checkCanRollUp(MONTH, MILLISECOND, false);
+    f.checkCanRollUp(MONTH, SECOND, false);
+    f.checkCanRollUp(MONTH, MINUTE, false);
+    f.checkCanRollUp(MONTH, HOUR, false);
+    f.checkCanRollUp(MONTH, DAY, false);
+    f.checkCanRollUp(MONTH, WEEK, false);
+    f.checkCanRollUp(MONTH, f.isoWeek, false);
+    f.checkCanRollUp(MONTH, MONTH, true);
+    f.checkCanRollUp(MONTH, QUARTER, true);
+    f.checkCanRollUp(MONTH, YEAR, true);
+    f.checkCanRollUp(MONTH, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(MONTH, DECADE, true);
+    f.checkCanRollUp(MONTH, CENTURY, true);
+    f.checkCanRollUp(MONTH, MILLENNIUM, true);
+
+    f.checkCanRollUp(QUARTER, NANOSECOND, false);
+    f.checkCanRollUp(QUARTER, MICROSECOND, false);
+    f.checkCanRollUp(QUARTER, MILLISECOND, false);
+    f.checkCanRollUp(QUARTER, SECOND, false);
+    f.checkCanRollUp(QUARTER, MINUTE, false);
+    f.checkCanRollUp(QUARTER, HOUR, false);
+    f.checkCanRollUp(QUARTER, DAY, false);
+    f.checkCanRollUp(QUARTER, WEEK, false);
+    f.checkCanRollUp(QUARTER, f.isoWeek, false);
+    f.checkCanRollUp(QUARTER, MONTH, false);
+    f.checkCanRollUp(QUARTER, QUARTER, true);
+    f.checkCanRollUp(QUARTER, YEAR, true);
+    f.checkCanRollUp(QUARTER, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(QUARTER, DECADE, true);
+    f.checkCanRollUp(QUARTER, CENTURY, true);
+    f.checkCanRollUp(QUARTER, MILLENNIUM, true);
+
+    f.checkCanRollUp(YEAR, NANOSECOND, false);
+    f.checkCanRollUp(YEAR, MICROSECOND, false);
+    f.checkCanRollUp(YEAR, MILLISECOND, false);
+    f.checkCanRollUp(YEAR, SECOND, false);
+    f.checkCanRollUp(YEAR, MINUTE, false);
+    f.checkCanRollUp(YEAR, HOUR, false);
+    f.checkCanRollUp(YEAR, DAY, false);
+    f.checkCanRollUp(YEAR, WEEK, false);
+    f.checkCanRollUp(YEAR, f.isoWeek, false);
+    f.checkCanRollUp(YEAR, MONTH, false);
+    f.checkCanRollUp(YEAR, QUARTER, false);
+    f.checkCanRollUp(YEAR, YEAR, true);
+    f.checkCanRollUp(YEAR, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(YEAR, DECADE, true);
+    f.checkCanRollUp(YEAR, CENTURY, true);
+    f.checkCanRollUp(YEAR, MILLENNIUM, true);
+
+    // Note 3. DECADE cannot roll up to CENTURY or MILLENNIUM
+    // because decade starts on year 0, the others start on year 1.
+    // For example, 2000 is start of a decade, but 2001 is start of a century
+    // and millennium.
+    f.checkCanRollUp(DECADE, NANOSECOND, false);
+    f.checkCanRollUp(DECADE, MICROSECOND, false);
+    f.checkCanRollUp(DECADE, MILLISECOND, false);
+    f.checkCanRollUp(DECADE, SECOND, false);
+    f.checkCanRollUp(DECADE, MINUTE, false);
+    f.checkCanRollUp(DECADE, HOUR, false);
+    f.checkCanRollUp(DECADE, DAY, false);
+    f.checkCanRollUp(DECADE, WEEK, false);
+    f.checkCanRollUp(DECADE, f.isoWeek, false);
+    f.checkCanRollUp(DECADE, MONTH, false);
+    f.checkCanRollUp(DECADE, QUARTER, false);
+    f.checkCanRollUp(DECADE, YEAR, false);
+    f.checkCanRollUp(DECADE, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(DECADE, DECADE, true);
+    f.checkCanRollUp(DECADE, CENTURY, false); // see note 3
+    f.checkCanRollUp(DECADE, MILLENNIUM, false); // see note 3
+
+    f.checkCanRollUp(CENTURY, NANOSECOND, false);
+    f.checkCanRollUp(CENTURY, MICROSECOND, false);
+    f.checkCanRollUp(CENTURY, MILLISECOND, false);
+    f.checkCanRollUp(CENTURY, SECOND, false);
+    f.checkCanRollUp(CENTURY, MINUTE, false);
+    f.checkCanRollUp(CENTURY, HOUR, false);
+    f.checkCanRollUp(CENTURY, DAY, false);
+    f.checkCanRollUp(CENTURY, WEEK, false);
+    f.checkCanRollUp(CENTURY, f.isoWeek, false);
+    f.checkCanRollUp(CENTURY, MONTH, false);
+    f.checkCanRollUp(CENTURY, QUARTER, false);
+    f.checkCanRollUp(CENTURY, YEAR, false);
+    f.checkCanRollUp(CENTURY, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(CENTURY, DECADE, false);
+    f.checkCanRollUp(CENTURY, CENTURY, true);
+    f.checkCanRollUp(CENTURY, MILLENNIUM, true);
+
+    f.checkCanRollUp(MILLENNIUM, NANOSECOND, false);
+    f.checkCanRollUp(MILLENNIUM, MICROSECOND, false);
+    f.checkCanRollUp(MILLENNIUM, MILLISECOND, false);
+    f.checkCanRollUp(MILLENNIUM, SECOND, false);
+    f.checkCanRollUp(MILLENNIUM, MINUTE, false);
+    f.checkCanRollUp(MILLENNIUM, HOUR, false);
+    f.checkCanRollUp(MILLENNIUM, DAY, false);
+    f.checkCanRollUp(MILLENNIUM, WEEK, false);
+    f.checkCanRollUp(MILLENNIUM, f.isoWeek, false);
+    f.checkCanRollUp(MILLENNIUM, MONTH, false);
+    f.checkCanRollUp(MILLENNIUM, QUARTER, false);
+    f.checkCanRollUp(MILLENNIUM, YEAR, false);
+    f.checkCanRollUp(MILLENNIUM, ISOYEAR, false); // see note 2
+    f.checkCanRollUp(MILLENNIUM, DECADE, false);
+    f.checkCanRollUp(MILLENNIUM, CENTURY, false);
+    f.checkCanRollUp(MILLENNIUM, MILLENNIUM, true);
+  }
+
+  /** Test fixture. Contains everything you need to write fluent tests. */
+  static class Fixture {
+    final TimeUnit isoWeek = TimeUnit.ISODOW;
+
+    final TimeFrameSet timeFrameSet = TimeFrames.CORE;
+
+    private static final ImmutableMap<String, Pair<String, String>> MAP;
+
+    static {
+      String[] values = {
+          "NANOSECOND", "2022-06-25 12:34:56.123234456",
+          "2022-06-25 12:34:56.123234456",
+          "MICROSECOND", "2022-06-25 12:34:56.123234",
+          "2022-06-25 12:34:56.123234",
+          "MILLISECOND", "2022-06-25 12:34:56.123", "2022-06-25 12:34:56.124",
+          "SECOND", "2022-06-25 12:34:56", "2022-06-25 12:34:57",
+          "MINUTE", "2022-06-25 12:34:00", "2022-06-25 12:35:00",
+          "HOUR", "2022-06-25 12:00:00", "2022-06-25 13:00:00",
+          "DAY", "2022-06-25 00:00:00", "2022-06-26 00:00:00",
+          "WEEK", "2022-06-19 00:00:00", "2022-06-26 00:00:00",
+          "ISOWEEK", "2022-06-20 00:00:00", "2022-06-27 00:00:00",
+          "MONTH", "2022-06-01 00:00:00", "2022-07-01 00:00:00",
+          "QUARTER", "2022-04-01 00:00:00", "2022-07-01 00:00:00",
+          "YEAR", "2022-01-01 00:00:00", "2023-01-01 00:00:00",
+          "ISOYEAR", "2022-01-03 00:00:00", "2023-01-02 00:00:00",
+          "DECADE", "2020-01-01 00:00:00", "2030-01-01 00:00:00",
+          "CENTURY", "2001-01-01 00:00:00", "2101-01-01 00:00:00",
+          "MILLENNIUM", "2001-01-01 00:00:00", "3001-01-01 00:00:00",
+      };
+      ImmutableMap.Builder<String, Pair<String, String>> b =
+          ImmutableMap.builder();
+      for (int i = 0; i < values.length;) {
+        b.put(values[i++], Pair.of(values[i++], values[i++]));
+      }
+      MAP = b.build();
+    }
+
+    private TimeFrame frame(TimeUnit unit) {
+      if (unit == ISODOW) {
+        // Just for testing. We want to test f.ISOWEEK but there is no TimeUnit
+        // for it, so we use ISODOW as a stand-in.
+        return timeFrameSet.get("ISOWEEK");
+      }
+      return timeFrameSet.get(unit);
+    }
+
+    void checkDateFloor(String in, TimeUnit unit, Matcher<String> matcher) {
+      int inDate = dateStringToUnixDate(in);
+      int outDate = timeFrameSet.floorDate(inDate, frame(unit));
+      assertThat("floor(" + in + " to " + unit + ")",
+          unixDateToString(outDate), matcher);
+    }
+
+    void checkTimestampFloor(String in, TimeUnit unit, int precision,
+        Matcher<String> matcher) {
+      long inTs = timestampStringToUnixDate(in);
+      long outTs = timeFrameSet.floorTimestamp(inTs, frame(unit));
+      assertThat("floor(" + in + " to " + unit + ")",
+          unixTimestampToString(outTs, precision), matcher);
+    }
+
+    void checkTimestampCeil(String in, TimeUnit unit, int precision,
+        Matcher<String> matcher) {
+      long inTs = timestampStringToUnixDate(in);
+      long outTs = timeFrameSet.ceilTimestamp(inTs, frame(unit));
+      assertThat("ceil(" + in + " to " + unit + ")",
+          unixTimestampToString(outTs, precision), matcher);
+    }
+
+    void checkCanRollUp(TimeUnit fromUnit, TimeUnit toUnit, boolean can) {
+      TimeFrame fromFrame = frame(fromUnit);
+      TimeFrame toFrame = frame(toUnit);
+      if (can) {
+        assertThat("can roll up " + fromUnit + " to " + toUnit,
+            fromFrame.canRollUpTo(toFrame),
+            is(true));
+
+        final int precision;
+        switch (toUnit) {
+        case NANOSECOND:
+          precision = 9;
+          break;
+        case MICROSECOND:
+          precision = 6;
+          break;
+        case MILLISECOND:
+          precision = 3;
+          break;
+        default:
+          precision = 0;
+        }
+        if (precision <= 3) {
+          // Cannot test conversion to NANOSECOND or MICROSECOND because the
+          // representation is milliseconds.
+          final Pair<String, String> fromPair =
+              requireNonNull(MAP.get(fromFrame.name()));
+          final String timestampString = requireNonNull(fromPair.left);
+          final Pair<String, String> toPair =
+              requireNonNull(MAP.get(toFrame.name()));
+          final String floorTimestampString = requireNonNull(toPair.left);
+          final String ceilTimestampString = requireNonNull(toPair.right);
+          checkTimestampFloor(timestampString, toUnit, precision,
+              is(floorTimestampString));
+          checkTimestampCeil(timestampString, toUnit, precision,
+              is(ceilTimestampString));
+
+          final String dateString = timestampString.substring(0, 10);
+          final String floorDateString = floorTimestampString.substring(0, 10);
+          checkDateFloor(dateString, toUnit, is(floorDateString));
+        }
+
+        // The 'canRollUpTo' method should be a partial order.
+        // A partial order is reflexive (for all x, x = x)
+        // and antisymmetric (for all x, y, if x <= y and x != y, then !(y <= x))
+        assertThat(toFrame.canRollUpTo(fromFrame), is(fromUnit == toUnit));
+      } else {
+        assertThat("can roll up " + fromUnit + " to " + toUnit,
+            fromFrame.canRollUpTo(toFrame),
+            is(false));
+      }
+    }
+  }
+}
diff --git a/gradle.properties b/gradle.properties
index b5ff589585..a9b751740c 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -89,6 +89,7 @@ commons-codec.version=1.13
 commons-dbcp2.version=2.6.0
 commons-io.version=2.11.0
 commons-lang3.version=3.8
+commons-math3.version=3.6.1
 commons-pool2.version=2.6.2
 dropwizard-metrics.version=4.0.5
 
diff --git a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 9ab85b88b4..e95dc05631 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -17,7 +17,6 @@
 package org.apache.calcite.sql.parser;
 
 import org.apache.calcite.avatica.util.Quoting;
-import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlExplain;
@@ -46,7 +45,6 @@ import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSortedSet;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
@@ -60,14 +58,11 @@ import org.junit.jupiter.api.Test;
 import java.io.Reader;
 import java.io.StringReader;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 import java.util.Locale;
-import java.util.Map;
 import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
-import java.util.function.BiConsumer;
 import java.util.function.Consumer;
 import java.util.function.UnaryOperator;
 import java.util.stream.Collectors;
@@ -8083,88 +8078,37 @@ public class SqlParserTest {
   /** Tests that EXTRACT, FLOOR, CEIL functions accept abbreviations for
    * time units (such as "Y" for "YEAR") when configured via
    * {@link Config#timeUnitCodes()}. */
-  @Test void testTimeUnitCodes() {
-    final Map<String, TimeUnit> simpleCodes =
-        ImmutableMap.<String, TimeUnit>builder()
-            .put("Y", TimeUnit.YEAR)
-            .put("M", TimeUnit.MONTH)
-            .put("D", TimeUnit.DAY)
-            .put("H", TimeUnit.HOUR)
-            .put("N", TimeUnit.MINUTE)
-            .put("S", TimeUnit.SECOND)
-            .build();
-
-    // Time unit abbreviations for Microsoft SQL Server
-    final Map<String, TimeUnit> mssqlCodes =
-        ImmutableMap.<String, TimeUnit>builder()
-            .put("Y", TimeUnit.YEAR)
-            .put("YY", TimeUnit.YEAR)
-            .put("YYYY", TimeUnit.YEAR)
-            .put("Q", TimeUnit.QUARTER)
-            .put("QQ", TimeUnit.QUARTER)
-            .put("M", TimeUnit.MONTH)
-            .put("MM", TimeUnit.MONTH)
-            .put("W", TimeUnit.WEEK)
-            .put("WK", TimeUnit.WEEK)
-            .put("WW", TimeUnit.WEEK)
-            .put("DY", TimeUnit.DOY)
-            .put("DW", TimeUnit.DOW)
-            .put("D", TimeUnit.DAY)
-            .put("DD", TimeUnit.DAY)
-            .put("H", TimeUnit.HOUR)
-            .put("HH", TimeUnit.HOUR)
-            .put("N", TimeUnit.MINUTE)
-            .put("MI", TimeUnit.MINUTE)
-            .put("S", TimeUnit.SECOND)
-            .put("SS", TimeUnit.SECOND)
-            .put("MS", TimeUnit.MILLISECOND)
-            .build();
-
-    checkTimeUnitCodes(Config.DEFAULT.timeUnitCodes());
-    checkTimeUnitCodes(simpleCodes);
-    checkTimeUnitCodes(mssqlCodes);
-  }
-
-  /** Checks parsing of built-in functions that accept time unit
-   * abbreviations.
-   *
-   * <p>For example, {@code EXTRACT(Y FROM orderDate)} is using
-   * "Y" as an abbreviation for "YEAR".
-   *
-   * <p>Override if your parser supports more such functions. */
-  protected void checkTimeUnitCodes(Map<String, TimeUnit> timeUnitCodes) {
-    SqlParserFixture f = fixture()
-        .withConfig(config -> config.withTimeUnitCodes(timeUnitCodes));
-    BiConsumer<String, TimeUnit> validConsumer = (abbrev, timeUnit) -> {
-      f.sql("select extract(" + abbrev + " from x)")
-          .ok("SELECT EXTRACT(" + timeUnit + " FROM `X`)");
-      f.sql("select floor(x to " + abbrev + ")")
-          .ok("SELECT FLOOR(`X` TO " + timeUnit + ")");
-      f.sql("select ceil(x to " + abbrev + ")")
-          .ok("SELECT CEIL(`X` TO " + timeUnit + ")");
-    };
-    BiConsumer<String, TimeUnit> invalidConsumer = (abbrev, timeUnit) -> {
-      final String upAbbrev = abbrev.toUpperCase(Locale.ROOT);
-      f.sql("select extract(^" + abbrev + "^ from x)")
-          .fails("'" + upAbbrev + "' is not a valid datetime format");
-      f.sql("SELECT FLOOR(x to ^" + abbrev + "^)")
-          .fails("'" + upAbbrev + "' is not a valid datetime format");
-      f.sql("SELECT CEIL(x to ^" + abbrev + "^)")
-          .fails("'" + upAbbrev + "' is not a valid datetime format");
-    };
-
-    // Check that each valid code passes each query that it should.
-    timeUnitCodes.forEach(validConsumer);
-
-    // If "M" is a valid code then "m" should be also.
-    timeUnitCodes.forEach((abbrev, timeUnit) ->
-        validConsumer.accept(abbrev.toLowerCase(Locale.ROOT), timeUnit));
-
-    // Check that invalid codes generate the right error messages.
-    final Map<String, TimeUnit> invalidCodes =
-        ImmutableMap.of("A", TimeUnit.YEAR,
-            "a", TimeUnit.YEAR);
-    invalidCodes.forEach(invalidConsumer);
+  @Test protected void testTimeUnitCodes() {
+    // YEAR is a built-in time frame. When unparsed, it looks like a keyword.
+    // (Note no backticks around YEAR.)
+    expr("floor(d to year)")
+        .ok("FLOOR(`D` TO YEAR)");
+    // Y is an extension time frame. (Or rather, it could be, if you configure
+    // it.) When unparsed, it looks like an identifier (backticks around Y.)
+    expr("floor(d to y)")
+        .ok("FLOOR(`D` TO `Y`)");
+
+    // As for FLOOR, so for CEIL.
+    expr("ceil(d to year)").ok("CEIL(`D` TO YEAR)");
+    expr("ceil(d to y)").ok("CEIL(`D` TO `Y`)");
+
+    // CEILING is a synonym for CEIL.
+    expr("ceiling(d to year)").ok("CEIL(`D` TO YEAR)");
+    expr("ceiling(d to y)").ok("CEIL(`D` TO `Y`)");
+
+    // As for FLOOR, so for EXTRACT.
+    expr("extract(year from d)").ok("EXTRACT(YEAR FROM `D`)");
+    expr("extract(y from d)").ok("EXTRACT(`Y` FROM `D`)");
+
+    // MICROSECOND, NANOSECOND used to be native for EXTRACT but not for FLOOR
+    // or CEIL. Now they are native, and so appear as keywords, without
+    // backticks.
+    expr("floor(d to nanosecond)").ok("FLOOR(`D` TO NANOSECOND)");
+    expr("floor(d to microsecond)").ok("FLOOR(`D` TO MICROSECOND)");
+    expr("ceil(d to nanosecond)").ok("CEIL(`D` TO NANOSECOND)");
+    expr("ceiling(d to microsecond)").ok("CEIL(`D` TO MICROSECOND)");
+    expr("extract(nanosecond from d)").ok("EXTRACT(NANOSECOND FROM `D`)");
+    expr("extract(microsecond from d)").ok("EXTRACT(MICROSECOND FROM `D`)");
   }
 
   @Test void testGeometry() {
@@ -8258,58 +8202,52 @@ public class SqlParserTest {
         .ok("CAST(`X` AS VARBINARY)");
   }
 
-  @Test void testTimestampAddAndDiff() {
-    Map<String, List<String>> tsi = ImmutableMap.<String, List<String>>builder()
-        .put("MICROSECOND",
-            Arrays.asList("FRAC_SECOND", "MICROSECOND", "SQL_TSI_MICROSECOND"))
-        .put("NANOSECOND", Arrays.asList("NANOSECOND", "SQL_TSI_FRAC_SECOND"))
-        .put("SECOND", Arrays.asList("SECOND", "SQL_TSI_SECOND"))
-        .put("MINUTE", Arrays.asList("MINUTE", "SQL_TSI_MINUTE"))
-        .put("HOUR", Arrays.asList("HOUR", "SQL_TSI_HOUR"))
-        .put("DAY", Arrays.asList("DAY", "SQL_TSI_DAY"))
-        .put("WEEK", Arrays.asList("WEEK", "SQL_TSI_WEEK"))
-        .put("MONTH", Arrays.asList("MONTH", "SQL_TSI_MONTH"))
-        .put("QUARTER", Arrays.asList("QUARTER", "SQL_TSI_QUARTER"))
-        .put("YEAR", Arrays.asList("YEAR", "SQL_TSI_YEAR"))
-        .build();
-
-    List<String> functions = ImmutableList.<String>builder()
-        .add("timestampadd(%1$s, 12, current_timestamp)")
-        .add("timestampdiff(%1$s, current_timestamp, current_timestamp)")
-        .build();
-
-    for (Map.Entry<String, List<String>> intervalGroup : tsi.entrySet()) {
-      for (String function : functions) {
-        for (String interval : intervalGroup.getValue()) {
-          expr(String.format(Locale.ROOT, function, interval, ""))
-              .ok(String.format(Locale.ROOT, function, intervalGroup.getKey(), "`")
-                  .toUpperCase(Locale.ROOT));
-        }
-      }
-    }
-
-    expr("timestampadd(^incorrect^, 1, current_timestamp)")
-        .fails("(?s).*Was expecting one of.*");
-    expr("timestampdiff(^incorrect^, current_timestamp, current_timestamp)")
-        .fails("(?s).*Was expecting one of.*");
-  }
-
   @Test void testTimestampAdd() {
     final String sql = "select * from t\n"
-        + "where timestampadd(sql_tsi_month, 5, hiredate) < curdate";
+        + "where timestampadd(month, 5, hiredate) < curdate";
     final String expected = "SELECT *\n"
         + "FROM `T`\n"
         + "WHERE (TIMESTAMPADD(MONTH, 5, `HIREDATE`) < `CURDATE`)";
     sql(sql).ok(expected);
+
+    // SQL_TSI_MONTH is treated as a user-defined time frame, hence appears as
+    // an identifier (with backticks) when unparsed. It will be resolved to
+    // MICROSECOND during validation.
+    final String sql2 = "select * from t\n"
+        + "where timestampadd(sql_tsi_month, 5, hiredate) < curdate";
+    final String expected2 = "SELECT *\n"
+        + "FROM `T`\n"
+        + "WHERE (TIMESTAMPADD(`SQL_TSI_MONTH`, 5, `HIREDATE`) < `CURDATE`)";
+    sql(sql2).ok(expected2);
+
+    // Previously a parse error, now an identifier that the validator will find
+    // to be invalid.
+    expr("timestampadd(incorrect, 1, current_timestamp)")
+        .ok("TIMESTAMPADD(`INCORRECT`, 1, CURRENT_TIMESTAMP)");
   }
 
   @Test void testTimestampDiff() {
     final String sql = "select * from t\n"
-        + "where timestampdiff(frac_second, 5, hiredate) < curdate";
+        + "where timestampdiff(microsecond, 5, hiredate) < curdate";
     final String expected = "SELECT *\n"
         + "FROM `T`\n"
         + "WHERE (TIMESTAMPDIFF(MICROSECOND, 5, `HIREDATE`) < `CURDATE`)";
     sql(sql).ok(expected);
+
+    // FRAC_SECOND is treated as a user-defined time frame, hence appears as
+    // an identifier (with backticks) when unparsed. It will be resolved to
+    // MICROSECOND during validation.
+    final String sql2 = "select * from t\n"
+        + "where timestampdiff(frac_second, 5, hiredate) < curdate";
+    final String expected2 = "SELECT *\n"
+        + "FROM `T`\n"
+        + "WHERE (TIMESTAMPDIFF(`FRAC_SECOND`, 5, `HIREDATE`) < `CURDATE`)";
+    sql(sql2).ok(expected2);
+
+    // Previously a parse error, now an identifier that the validator will find
+    // to be invalid.
+    expr("timestampdiff(incorrect, current_timestamp, current_timestamp)")
+        .ok("TIMESTAMPDIFF(`INCORRECT`, CURRENT_TIMESTAMP, CURRENT_TIMESTAMP)");
   }
 
   @Test void testTimeTrunc() {
@@ -8318,9 +8256,18 @@ public class SqlParserTest {
         + "FROM `T`";
     sql(sql).ok(expected);
 
-    // should fail for time unit not appropriate for TIME type.
-    final String weekSql = "select time_trunc(time '15:30:00', ^week^) from t";
-    sql(weekSql).fails("(?s).*Was expecting one of.*");
+    // Syntactically valid; validator will complain because WEEK is for DATE or
+    // TIMESTAMP but not for TIME.
+    final String sql2 = "select time_trunc(time '15:30:00', week) from t";
+    final String expected2 = "SELECT TIME_TRUNC(TIME '15:30:00', WEEK)\n"
+        + "FROM `T`";
+    sql(sql2).ok(expected2);
+
+    // note backticks
+    final String sql3 = "select time_trunc(time '15:30:00', incorrect) from t";
+    final String expected3 = "SELECT TIME_TRUNC(TIME '15:30:00', `INCORRECT`)\n"
+        + "FROM `T`";
+    sql(sql3).ok(expected3);
   }
 
   @Test void testTimestampTrunc() {
@@ -8328,6 +8275,12 @@ public class SqlParserTest {
     final String expected = "SELECT TIMESTAMP_TRUNC(TIMESTAMP '2008-12-25 15:30:00', WEEK)\n"
         + "FROM `T`";
     sql(sql).ok(expected);
+
+    // note backticks
+    final String sql3 = "select timestamp_trunc(time '15:30:00', incorrect) from t";
+    String expected3 = "SELECT TIMESTAMP_TRUNC(TIME '15:30:00', `INCORRECT`)\n"
+        + "FROM `T`";
+    sql(sql3).ok(expected3);
   }
 
   @Test void testUnnest() {
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
index b281989593..9d011d7d50 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -20,8 +20,11 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.config.CalciteConnectionProperty;
 import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.plan.Strong;
+import org.apache.calcite.rel.type.DelegatingTypeSystem;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.rel.type.TimeFrameSet;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Hook;
@@ -64,6 +67,7 @@ import org.apache.calcite.util.Holder;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.TimestampString;
+import org.apache.calcite.util.TryThreadLocal;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -82,6 +86,7 @@ import java.sql.SQLException;
 import java.sql.Statement;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Collections;
 import java.util.HashSet;
@@ -173,6 +178,7 @@ import static org.junit.jupiter.api.Assumptions.assumeTrue;
  * null arguments or null results.</li>
  * </ul>
  */
+@SuppressWarnings("MethodCanBeStatic")
 public class SqlOperatorTest {
   //~ Static fields/initializers ---------------------------------------------
 
@@ -205,6 +211,27 @@ public class SqlOperatorTest {
       Pattern.compile(
           "[0-9][0-9][0-9][0-9]-[0-9][0-9]-[0-9][0-9]");
 
+  public static final List<String> MICROSECOND_VARIANTS =
+      Arrays.asList("FRAC_SECOND", "MICROSECOND", "SQL_TSI_MICROSECOND");
+  public static final List<String> NANOSECOND_VARIANTS =
+      Arrays.asList("NANOSECOND", "SQL_TSI_FRAC_SECOND");
+  public static final List<String> SECOND_VARIANTS =
+      Arrays.asList("SECOND", "SQL_TSI_SECOND");
+  public static final List<String> MINUTE_VARIANTS =
+      Arrays.asList("MINUTE", "SQL_TSI_MINUTE");
+  public static final List<String> HOUR_VARIANTS =
+      Arrays.asList("HOUR", "SQL_TSI_HOUR");
+  public static final List<String> DAY_VARIANTS =
+      Arrays.asList("DAY", "SQL_TSI_DAY");
+  public static final List<String> WEEK_VARIANTS =
+      Arrays.asList("WEEK", "SQL_TSI_WEEK");
+  public static final List<String> MONTH_VARIANTS =
+      Arrays.asList("MONTH", "SQL_TSI_MONTH");
+  public static final List<String> QUARTER_VARIANTS =
+      Arrays.asList("QUARTER", "SQL_TSI_QUARTER");
+  public static final List<String> YEAR_VARIANTS =
+      Arrays.asList("YEAR", "SQL_TSI_YEAR");
+
   /** Minimum and maximum values for each exact and approximate numeric
    * type. */
   enum Numeric {
@@ -3098,7 +3125,7 @@ public class SqlOperatorTest {
     checkRlikeFails(f.withLibrary(SqlLibrary.ORACLE));
   }
 
-  void checkRlike(SqlOperatorFixture f) {
+  static void checkRlike(SqlOperatorFixture f) {
     f.checkBoolean("'Merrisa@gmail.com' rlike '.+@*\\.com'", true);
     f.checkBoolean("'Merrisa@gmail.com' rlike '.com$'", true);
     f.checkBoolean("'acbd' rlike '^ac+'", true);
@@ -3115,7 +3142,7 @@ public class SqlOperatorTest {
     f.checkBoolean("'Merrisa@gmail.com' not rlike 'Merrisa_'", true);
   }
 
-  void checkRlikeFails(SqlOperatorFixture f) {
+  static void checkRlikeFails(SqlOperatorFixture f) {
     final String noRlike = "(?s).*No match found for function signature RLIKE";
     f.checkFails("^'Merrisa@gmail.com' rlike '.+@*\\.com'^", noRlike, false);
     f.checkFails("^'acb' rlike 'acb|efg'^", noRlike, false);
@@ -3814,7 +3841,7 @@ public class SqlOperatorTest {
     checkIf(f.withLibrary(SqlLibrary.SPARK));
   }
 
-  private void checkIf(SqlOperatorFixture f) {
+  private static void checkIf(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.IF);
     f.checkString("if(1 = 2, 1, 2)", "2", "INTEGER NOT NULL");
     f.checkString("if('abc'='xyz', 'abc', 'xyz')", "xyz",
@@ -5783,7 +5810,7 @@ public class SqlOperatorTest {
     checkSubstringFunction(f.withConformance(SqlConformanceEnum.BIG_QUERY));
   }
 
-  void checkSubstringFunction(SqlOperatorFixture f) {
+  private static void checkSubstringFunction(SqlOperatorFixture f) {
     f.setFor(SqlStdOperatorTable.SUBSTRING);
     f.checkString("substring('abc' from 1 for 2)",
         "ab", "VARCHAR(3) NOT NULL");
@@ -6178,7 +6205,7 @@ public class SqlOperatorTest {
     checkDecodeFunc(fixture().withLibrary(SqlLibrary.ORACLE));
   }
 
-  void checkDecodeFunc(SqlOperatorFixture f) {
+  private static void checkDecodeFunc(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.DECODE, VmName.EXPAND);
     f.checkScalar("decode(0, 0, 'a', 1, 'b', 2, 'c')", "a", "CHAR(1)");
     f.checkScalar("decode(1, 0, 'a', 1, 'b', 2, 'c')", "b", "CHAR(1)");
@@ -6418,7 +6445,7 @@ public class SqlOperatorTest {
     checkStringAggFuncFails(f.withLibrary(SqlLibrary.MYSQL));
   }
 
-  private void checkStringAggFunc(SqlOperatorFixture f) {
+  private static void checkStringAggFunc(SqlOperatorFixture f) {
     final String[] values = {"'x'", "null", "'yz'"};
     f.checkAgg("string_agg(x)", values, isSingle("x,yz"));
     f.checkAgg("string_agg(x,':')", values, isSingle("x:yz"));
@@ -6432,7 +6459,7 @@ public class SqlOperatorTest {
         false);
   }
 
-  private void checkStringAggFuncFails(SqlOperatorFixture f) {
+  private static void checkStringAggFuncFails(SqlOperatorFixture f) {
     final String[] values = {"'x'", "'y'"};
     f.checkAggFails("^string_agg(x)^", values,
         "No match found for function signature STRING_AGG\\(<CHARACTER>\\)",
@@ -6454,7 +6481,7 @@ public class SqlOperatorTest {
     checkGroupConcatFuncFails(f.withLibrary(SqlLibrary.POSTGRESQL));
   }
 
-  private void checkGroupConcatFunc(SqlOperatorFixture f) {
+  private static void checkGroupConcatFunc(SqlOperatorFixture f) {
     final String[] values = {"'x'", "null", "'yz'"};
     f.checkAgg("group_concat(x)", values, isSingle("x,yz"));
     f.checkAgg("group_concat(x,':')", values, isSingle("x:yz"));
@@ -6471,7 +6498,7 @@ public class SqlOperatorTest {
         false);
   }
 
-  private void checkGroupConcatFuncFails(SqlOperatorFixture t) {
+  private static void checkGroupConcatFuncFails(SqlOperatorFixture t) {
     final String[] values = {"'x'", "'y'"};
     t.checkAggFails("^group_concat(x)^", values,
         "No match found for function signature GROUP_CONCAT\\(<CHARACTER>\\)",
@@ -6493,7 +6520,7 @@ public class SqlOperatorTest {
     checkArrayAggFuncFails(f.withLibrary(SqlLibrary.MYSQL));
   }
 
-  private void checkArrayAggFunc(SqlOperatorFixture f) {
+  private static void checkArrayAggFunc(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.ARRAY_CONCAT_AGG, VM_FENNEL, VM_JAVA);
     final String[] values = {"'x'", "null", "'yz'"};
     f.checkAgg("array_agg(x)", values, isSingle("[x, yz]"));
@@ -6508,7 +6535,7 @@ public class SqlOperatorTest {
         isSingle("[yz, x]"));
   }
 
-  private void checkArrayAggFuncFails(SqlOperatorFixture t) {
+  private static void checkArrayAggFuncFails(SqlOperatorFixture t) {
     t.setFor(SqlLibraryOperators.ARRAY_CONCAT_AGG, VM_FENNEL, VM_JAVA);
     final String[] values = {"'x'", "'y'"};
     final String expectedError = "No match found for function signature "
@@ -6528,7 +6555,7 @@ public class SqlOperatorTest {
     checkArrayConcatAggFuncFails(f.withLibrary(SqlLibrary.MYSQL));
   }
 
-  void checkArrayConcatAggFunc(SqlOperatorFixture t) {
+  private static void checkArrayConcatAggFunc(SqlOperatorFixture t) {
     t.setFor(SqlLibraryOperators.ARRAY_CONCAT_AGG, VM_FENNEL, VM_JAVA);
     t.checkFails("array_concat_agg(^*^)",
         "(?s)Encountered \"\\*\" at .*", false);
@@ -6552,7 +6579,7 @@ public class SqlOperatorTest {
     t.checkAgg("array_concat_agg(x)", values2, isSingle("[0, 1, 1, 2]"));
   }
 
-  void checkArrayConcatAggFuncFails(SqlOperatorFixture t) {
+  private static void checkArrayConcatAggFuncFails(SqlOperatorFixture t) {
     t.setFor(SqlLibraryOperators.ARRAY_CONCAT_AGG, VM_FENNEL, VM_JAVA);
     final String[] values = {"'x'", "'y'"};
     final String expectedError = "No match found for function signature "
@@ -6905,6 +6932,8 @@ public class SqlOperatorTest {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.EXTRACT, VM_FENNEL, VM_JAVA);
 
+    f.checkFails("extract(^a^ from date '2008-2-23')",
+        "'A' is not a valid time frame", false);
     f.checkScalar("extract(epoch from date '2008-2-23')",
         "1203724800", // number of seconds elapsed since timestamp
         // '1970-01-01 00:00:00' for given date
@@ -6983,6 +7012,8 @@ public class SqlOperatorTest {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.EXTRACT, VM_FENNEL, VM_JAVA);
 
+    f.checkFails("extract(^a^ from timestamp '2008-2-23 12:34:56')",
+        "'A' is not a valid time frame", false);
     f.checkScalar("extract(epoch from timestamp '2008-2-23 12:34:56')",
         "1203770096", // number of seconds elapsed since timestamp
         // '1970-01-01 00:00:00' for given date
@@ -7039,9 +7070,12 @@ public class SqlOperatorTest {
         "2", "BIGINT NOT NULL");
   }
 
-  @Test void testExtractFunc() {
+  @Test void testExtractInterval() {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.EXTRACT, VM_FENNEL, VM_JAVA);
+
+    f.checkFails("extract(^a^ from interval '2 3:4:5.678' day to second)",
+        "'A' is not a valid time frame", false);
     f.checkScalar("extract(day from interval '2 3:4:5.678' day to second)",
         "2", "BIGINT NOT NULL");
     f.checkScalar("extract(day from interval '23456 3:4:5.678' day(5) to second)",
@@ -7300,10 +7334,6 @@ public class SqlOperatorTest {
         "(?s)Cannot apply 'FLOOR' to arguments .*", false);
     f.checkFails("^floor('abcde' to minute)^",
         "(?s)Cannot apply 'FLOOR' to arguments .*", false);
-    f.checkFails("floor(timestamp '2015-02-19 12:34:56.78' to ^microsecond^)",
-        "'MICROSECOND' is not a valid datetime format", false);
-    f.checkFails("floor(timestamp '2015-02-19 12:34:56.78' to ^nanosecond^)",
-        "'NANOSECOND' is not a valid datetime format", false);
     f.checkScalar("floor(time '12:34:56' to minute)",
         "12:34:00", "TIME(0) NOT NULL");
     f.checkScalar("floor(timestamp '2015-02-19 12:34:56.78' to second)",
@@ -7341,10 +7371,6 @@ public class SqlOperatorTest {
         "(?s)Cannot apply 'CEIL' to arguments .*", false);
     f.checkFails("^ceil('abcde' to minute)^",
         "(?s)Cannot apply 'CEIL' to arguments .*", false);
-    f.checkFails("ceil(timestamp '2015-02-19 12:34:56.78' to ^microsecond^)",
-        "'MICROSECOND' is not a valid datetime format", false);
-    f.checkFails("ceil(timestamp '2015-02-19 12:34:56.78' to ^nanosecond^)",
-        "'NANOSECOND' is not a valid datetime format", false);
     f.checkScalar("ceil(time '12:34:56' to minute)",
         "12:35:00", "TIME(0) NOT NULL");
     f.checkScalar("ceil(time '12:59:56' to minute)",
@@ -7374,6 +7400,76 @@ public class SqlOperatorTest {
     f.checkNull("ceiling(cast(null as timestamp) to month)");
   }
 
+  /** Tests {@code FLOOR}, {@code CEIL}, {@code TIMESTAMPADD},
+   * {@code TIMESTAMPDIFF} functions with custom time frames. */
+  @Test void testCustomTimeFrame() {
+    final SqlOperatorFixture f = fixture()
+        .withFactory(tf ->
+            tf.withTypeSystem(typeSystem ->
+                new DelegatingTypeSystem(typeSystem) {
+                  @Override public TimeFrameSet deriveTimeFrameSet(
+                      TimeFrameSet frameSet) {
+                    return TimeFrameSet.builder()
+                        .addAll(frameSet)
+                        .addDivision("minute15", 4, "HOUR")
+                        .addMultiple("month4", 4, "MONTH")
+                        .build();
+                  }
+                }));
+    f.checkScalar("floor(timestamp '2020-06-27 12:34:56' to \"minute15\")",
+        "2020-06-27 12:30:00", "TIMESTAMP(0) NOT NULL");
+    f.checkScalar("floor(timestamp '2020-06-27 12:34:56' to \"month4\")",
+        "2020-05-01 00:00:00", "TIMESTAMP(0) NOT NULL");
+    f.checkScalar("floor(date '2020-06-27' to \"month4\")",
+        "2020-05-01", "DATE NOT NULL");
+
+    f.checkScalar("ceil(timestamp '2020-06-27 12:34:56' to \"minute15\")",
+        "2020-06-27 12:45:00", "TIMESTAMP(0) NOT NULL");
+    f.checkFails("ceil(timestamp '2020-06-27 12:34:56' to ^\"minute25\"^)",
+        "'minute25' is not a valid time frame", false);
+
+    f.checkScalar(
+        "timestampadd(\"minute15\", 7, timestamp '2016-02-24 12:42:25')",
+        "2016-02-24 14:27:25", "TIMESTAMP(0) NOT NULL");
+    f.checkScalar(
+        "timestampadd(\"month4\", 7, timestamp '2016-02-24 12:42:25')",
+        "2018-06-24 12:42:25", "TIMESTAMP(0) NOT NULL");
+    f.checkScalar("timestampadd(\"month4\", 7, date '2016-02-24')",
+        "2018-06-24", "DATE NOT NULL");
+
+    f.checkScalar("timestampdiff(\"minute15\", "
+            + "timestamp '2016-02-24 12:42:25', "
+            + "timestamp '2016-02-24 15:42:25')",
+        "12", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", "
+            + "timestamp '2016-02-24 12:42:25', "
+            + "timestamp '2016-02-24 15:42:25')",
+        "0", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", "
+            + "timestamp '2016-02-24 12:42:25', "
+            + "timestamp '2018-02-24 15:42:25')",
+        "6", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", "
+            + "timestamp '2016-02-24 12:42:25', "
+            + "timestamp '2018-02-23 15:42:25')",
+        "5", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", date '2016-02-24', "
+            + "date '2020-03-24')",
+        "12", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", date '2016-02-24', "
+            + "date '2016-06-23')",
+        "0", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", date '2016-02-24', "
+            + "date '2016-06-24')",
+        "1", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", date '2016-02-24', "
+            + "date '2015-10-24')",
+        "-1", "INTEGER NOT NULL");
+    f.checkScalar("timestampdiff(\"month4\", date '2016-02-24', "
+            + "date '2016-02-23')",
+        "0", "INTEGER NOT NULL");
+  }
+
   @Test void testFloorFuncInterval() {
     final SqlOperatorFixture f = fixture();
     if (!f.brokenTestsEnabled()) {
@@ -7410,99 +7506,139 @@ public class SqlOperatorTest {
   @Test void testTimestampAdd() {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.TIMESTAMP_ADD, VmName.EXPAND);
-    f.checkScalar(
-        "timestampadd(MICROSECOND, 2000000, timestamp '2016-02-24 12:42:25')",
-        "2016-02-24 12:42:27",
-        "TIMESTAMP(3) NOT NULL");
-    f.checkScalar(
-        "timestampadd(SQL_TSI_SECOND, 2, timestamp '2016-02-24 12:42:25')",
-        "2016-02-24 12:42:27",
-        "TIMESTAMP(0) NOT NULL");
-    f.checkScalar(
-        "timestampadd(NANOSECOND, 3000000000, timestamp '2016-02-24 12:42:25')",
-        "2016-02-24 12:42:28",
-        "TIMESTAMP(0) NOT NULL");
-    f.checkScalar(
-        "timestampadd(SQL_TSI_FRAC_SECOND, 2000000000, timestamp '2016-02-24 12:42:25')",
-        "2016-02-24 12:42:27",
-        "TIMESTAMP(0) NOT NULL");
-    f.checkScalar(
-        "timestampadd(MINUTE, 2, timestamp '2016-02-24 12:42:25')",
-        "2016-02-24 12:44:25",
-        "TIMESTAMP(0) NOT NULL");
-    f.checkScalar("timestampadd(HOUR, -2000, timestamp '2016-02-24 12:42:25')",
-        "2015-12-03 04:42:25",
-        "TIMESTAMP(0) NOT NULL");
-    f.checkNull("timestampadd(HOUR, CAST(NULL AS INTEGER),"
-        + " timestamp '2016-02-24 12:42:25')");
-    f.checkNull("timestampadd(HOUR, -200, CAST(NULL AS TIMESTAMP))");
-    f.checkScalar("timestampadd(MONTH, 3, timestamp '2016-02-24 12:42:25')",
-        "2016-05-24 12:42:25", "TIMESTAMP(0) NOT NULL");
-    f.checkScalar("timestampadd(MONTH, 3, cast(null as timestamp))",
-        isNullValue(), "TIMESTAMP(0)");
+    MICROSECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", 2000000, timestamp '2016-02-24 12:42:25')",
+            "2016-02-24 12:42:27",
+            "TIMESTAMP(3) NOT NULL"));
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", 2, timestamp '2016-02-24 12:42:25')",
+            "2016-02-24 12:42:27",
+            "TIMESTAMP(0) NOT NULL"));
+    NANOSECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", 3000000000, timestamp '2016-02-24 12:42:25')",
+            "2016-02-24 12:42:28",
+            "TIMESTAMP(0) NOT NULL"));
+    NANOSECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", 2000000000, timestamp '2016-02-24 12:42:25')",
+            "2016-02-24 12:42:27",
+            "TIMESTAMP(0) NOT NULL"));
+    MINUTE_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", 2, timestamp '2016-02-24 12:42:25')",
+            "2016-02-24 12:44:25",
+            "TIMESTAMP(0) NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", -2000, timestamp '2016-02-24 12:42:25')",
+            "2015-12-03 04:42:25",
+            "TIMESTAMP(0) NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkNull("timestampadd(" + s + ", CAST(NULL AS INTEGER),"
+            + " timestamp '2016-02-24 12:42:25')"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkNull("timestampadd(" + s + ", -200, CAST(NULL AS TIMESTAMP))"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s
+                + ", 3, timestamp '2016-02-24 12:42:25')",
+            "2016-05-24 12:42:25", "TIMESTAMP(0) NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 3, cast(null as timestamp))",
+            isNullValue(), "TIMESTAMP(0)"));
 
     // TIMESTAMPADD with DATE; returns a TIMESTAMP value for sub-day intervals.
-    f.checkScalar("timestampadd(MONTH, 1, date '2016-06-15')",
-        "2016-07-15", "DATE NOT NULL");
-    f.checkScalar("timestampadd(DAY, 1, date '2016-06-15')",
-        "2016-06-16", "DATE NOT NULL");
-    f.checkScalar("timestampadd(HOUR, -1, date '2016-06-15')",
-        "2016-06-14 23:00:00", "TIMESTAMP(0) NOT NULL");
-    f.checkScalar("timestampadd(MINUTE, 1, date '2016-06-15')",
-        "2016-06-15 00:01:00", "TIMESTAMP(0) NOT NULL");
-    f.checkScalar("timestampadd(SQL_TSI_SECOND, -1, date '2016-06-15')",
-        "2016-06-14 23:59:59", "TIMESTAMP(0) NOT NULL");
-    f.checkScalar("timestampadd(SECOND, 1, date '2016-06-15')",
-        "2016-06-15 00:00:01", "TIMESTAMP(0) NOT NULL");
-    f.checkScalar("timestampadd(SECOND, 1, cast(null as date))",
-        isNullValue(), "TIMESTAMP(0)");
-    f.checkScalar("timestampadd(DAY, 1, cast(null as date))",
-        isNullValue(), "DATE");
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, date '2016-06-15')",
+            "2016-07-15", "DATE NOT NULL"));
+    DAY_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, date '2016-06-15')",
+            "2016-06-16", "DATE NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, date '2016-06-15')",
+            "2016-06-14 23:00:00", "TIMESTAMP(0) NOT NULL"));
+    MINUTE_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, date '2016-06-15')",
+            "2016-06-15 00:01:00", "TIMESTAMP(0) NOT NULL"));
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, date '2016-06-15')",
+            "2016-06-14 23:59:59", "TIMESTAMP(0) NOT NULL"));
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, date '2016-06-15')",
+            "2016-06-15 00:00:01", "TIMESTAMP(0) NOT NULL"));
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, cast(null as date))",
+            isNullValue(), "TIMESTAMP(0)"));
+    DAY_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, cast(null as date))",
+            isNullValue(), "DATE"));
 
     // Round to the last day of previous month
-    f.checkScalar("timestampadd(MONTH, 1, date '2016-05-31')",
-        "2016-06-30", "DATE NOT NULL");
-    f.checkScalar("timestampadd(MONTH, 5, date '2016-01-31')",
-        "2016-06-30", "DATE NOT NULL");
-    f.checkScalar("timestampadd(MONTH, -1, date '2016-03-31')",
-        "2016-02-29", "DATE NOT NULL");
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, date '2016-05-31')",
+            "2016-06-30", "DATE NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 5, date '2016-01-31')",
+            "2016-06-30", "DATE NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, date '2016-03-31')",
+            "2016-02-29", "DATE NOT NULL"));
 
     // TIMESTAMPADD with time; returns a time value.The interval is positive.
-    f.checkScalar("timestampadd(SECOND, 1, time '23:59:59')",
-        "00:00:00", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(MINUTE, 1, time '00:00:00')",
-        "00:01:00", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(MINUTE, 1, time '23:59:59')",
-        "00:00:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(HOUR, 1, time '23:59:59')",
-        "00:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(DAY, 15, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(WEEK, 3, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(MONTH, 6, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(QUARTER, 1, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(YEAR, 10, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, time '23:59:59')",
+            "00:00:00", "TIME(0) NOT NULL"));
+    MINUTE_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, time '00:00:00')",
+            "00:01:00", "TIME(0) NOT NULL"));
+    MINUTE_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, time '23:59:59')",
+            "00:00:59", "TIME(0) NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, time '23:59:59')",
+            "00:59:59", "TIME(0) NOT NULL"));
+    DAY_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 15, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    WEEK_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 3, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 6, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    QUARTER_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 1, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    YEAR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", 10, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
     // TIMESTAMPADD with time; returns a time value .The interval is negative.
-    f.checkScalar("timestampadd(SECOND, -1, time '00:00:00')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(MINUTE, -1, time '00:00:00')",
-        "23:59:00", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(HOUR, -1, time '00:00:00')",
-        "23:00:00", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(DAY, -1, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(WEEK, -1, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(MONTH, -1, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(QUARTER, -1, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
-    f.checkScalar("timestampadd(YEAR, -1, time '23:59:59')",
-        "23:59:59", "TIME(0) NOT NULL");
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '00:00:00')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    MINUTE_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '00:00:00')",
+            "23:59:00", "TIME(0) NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '00:00:00')",
+            "23:00:00", "TIME(0) NOT NULL"));
+    DAY_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    WEEK_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    QUARTER_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
+    YEAR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampadd(" + s + ", -1, time '23:59:59')",
+            "23:59:59", "TIME(0) NOT NULL"));
   }
 
   @Test void testTimestampAddFractionalSeconds() {
@@ -7526,80 +7662,119 @@ public class SqlOperatorTest {
   @Test void testTimestampDiff() {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.TIMESTAMP_DIFF, VmName.EXPAND);
-    f.checkScalar("timestampdiff(HOUR, "
-        + "timestamp '2016-02-24 12:42:25', "
-        + "timestamp '2016-02-24 15:42:25')",
-        "3", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MICROSECOND, "
-        + "timestamp '2016-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:20')",
-        "-5000000", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(SQL_TSI_FRAC_SECOND, "
-        + "timestamp '2016-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:20')",
-        "-5000000000", "BIGINT NOT NULL");
-    f.checkScalar("timestampdiff(NANOSECOND, "
-        + "timestamp '2016-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:20')",
-        "-5000000000", "BIGINT NOT NULL");
-    f.checkScalar("timestampdiff(YEAR, "
-        + "timestamp '2014-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:25')",
-        "2", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(WEEK, "
-        + "timestamp '2014-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:25')",
-        "104", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(WEEK, "
-        + "timestamp '2014-02-19 12:42:25', "
-        + "timestamp '2016-02-24 12:42:25')",
-        "105", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MONTH, "
-        + "timestamp '2014-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:25')",
-        "24", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MONTH, "
-        + "timestamp '2019-09-01 00:00:00', "
-        + "timestamp '2020-03-01 00:00:00')",
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2016-02-24 12:42:25', "
+                + "timestamp '2016-02-24 15:42:25')",
+            "3", "INTEGER NOT NULL"));
+    MICROSECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2016-02-24 12:42:25', "
+                + "timestamp '2016-02-24 12:42:20')",
+            "-5000000", "INTEGER NOT NULL"));
+    NANOSECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2016-02-24 12:42:25', "
+                + "timestamp '2016-02-24 12:42:20')",
+            "-5000000000", "BIGINT NOT NULL"));
+    YEAR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2014-02-24 12:42:25', "
+                + "timestamp '2016-02-24 12:42:25')",
+            "2", "INTEGER NOT NULL"));
+    WEEK_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2014-02-24 12:42:25', "
+                + "timestamp '2016-02-24 12:42:25')",
+            "104", "INTEGER NOT NULL"));
+    WEEK_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2014-02-19 12:42:25', "
+                + "timestamp '2016-02-24 12:42:25')",
+            "105", "INTEGER NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2014-02-24 12:42:25', "
+                + "timestamp '2016-02-24 12:42:25')",
+            "24", "INTEGER NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2019-09-01 00:00:00', "
+                + "timestamp '2020-03-01 00:00:00')",
+            "6", "INTEGER NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2019-09-01 00:00:00', "
+                + "timestamp '2016-08-01 00:00:00')",
+            "-37", "INTEGER NOT NULL"));
+    QUARTER_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2014-02-24 12:42:25', "
+                + "timestamp '2016-02-24 12:42:25')",
+            "8", "INTEGER NOT NULL"));
+    // Until 1.33, CENTURY was an invalid time frame for TIMESTAMPDIFF
+    f.checkScalar("timestampdiff(CENTURY, "
+            + "timestamp '2014-02-24 12:42:25', "
+            + "timestamp '2614-02-24 12:42:25')",
         "6", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MONTH, "
-        + "timestamp '2019-09-01 00:00:00', "
-        + "timestamp '2016-08-01 00:00:00')",
-        "-37", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(QUARTER, "
-        + "timestamp '2014-02-24 12:42:25', "
-        + "timestamp '2016-02-24 12:42:25')",
-        "8", "INTEGER NOT NULL");
-    f.checkFails("timestampdiff(^CENTURY^, "
-        + "timestamp '2014-02-24 12:42:25', "
-        + "timestamp '2614-02-24 12:42:25')",
-        "(?s)Encountered \"CENTURY\" at .*", false);
-    f.checkScalar("timestampdiff(QUARTER, "
-        + "timestamp '2014-02-24 12:42:25', "
-        + "cast(null as timestamp))",
-        isNullValue(), "INTEGER");
-    f.checkScalar("timestampdiff(QUARTER, "
-        + "cast(null as timestamp), "
-        + "timestamp '2014-02-24 12:42:25')",
-        isNullValue(), "INTEGER");
+    QUARTER_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "timestamp '2014-02-24 12:42:25', "
+                + "cast(null as timestamp))",
+            isNullValue(), "INTEGER"));
+    QUARTER_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "cast(null as timestamp), "
+                + "timestamp '2014-02-24 12:42:25')",
+            isNullValue(), "INTEGER"));
 
     // timestampdiff with date
-    f.checkScalar("timestampdiff(MONTH, date '2016-03-15', date '2016-06-14')",
-        "2", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MONTH, date '2019-09-01', date '2020-03-01')",
-        "6", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MONTH, date '2019-09-01', date '2016-08-01')",
-        "-37", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(DAY, date '2016-06-15', date '2016-06-14')",
-        "-1", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(HOUR, date '2016-06-15', date '2016-06-14')",
-        "-24", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(MINUTE, date '2016-06-15',  date '2016-06-15')",
-        "0", "INTEGER NOT NULL");
-    f.checkScalar("timestampdiff(SECOND, cast(null as date), date '2016-06-15')",
-        isNullValue(), "INTEGER");
-    f.checkScalar("timestampdiff(DAY, date '2016-06-15', cast(null as date))",
-        isNullValue(), "INTEGER");
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2016-03-15', date '2016-06-14')",
+            "2", "INTEGER NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2019-09-01', date '2020-03-01')",
+            "6", "INTEGER NOT NULL"));
+    MONTH_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2019-09-01', date '2016-08-01')",
+            "-37", "INTEGER NOT NULL"));
+    DAY_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2016-06-15', date '2016-06-14')",
+            "-1", "INTEGER NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2016-06-15', date '2016-06-14')",
+            "-24", "INTEGER NOT NULL"));
+    HOUR_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2016-06-15',  date '2016-06-15')",
+            "0", "INTEGER NOT NULL"));
+    MINUTE_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2016-06-15', date '2016-06-14')",
+            "-1440", "INTEGER NOT NULL"));
+    SECOND_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "cast(null as date), date '2016-06-15')",
+            isNullValue(), "INTEGER"));
+    DAY_VARIANTS.forEach(s ->
+        f.checkScalar("timestampdiff(" + s + ", "
+                + "date '2016-06-15', cast(null as date))",
+            isNullValue(), "INTEGER"));
+  }
+
+  /** The {@code DATEDIFF} function is implemented in the Babel parser but not
+   * the Core parser, and therefore gives validation errors. */
+  @Test void testDateDiff() {
+    final SqlOperatorFixture f = fixture()
+        .setFor(SqlLibraryOperators.DATEDIFF);
+    f.checkFails("datediff(^\"MONTH\"^, '2019-09-14',  '2019-09-15')",
+        "(?s)Column 'MONTH' not found in any table",
+        false);
   }
 
   @Test void testTimeTrunc() {
@@ -7614,13 +7789,7 @@ public class SqlOperatorTest {
         .withLibrary(SqlLibrary.BIG_QUERY)
         .setFor(SqlLibraryOperators.TIME_TRUNC);
     f.checkFails("time_trunc(time '12:34:56', ^year^)",
-        "Encountered \"year\" at line 1, column 37\\.\n"
-            + "Was expecting one of:\n"
-            + "    \"HOUR\" \\.\\.\\.\n"
-            + "    \"MILLISECOND\" \\.\\.\\.\n"
-            + "    \"MINUTE\" \\.\\.\\.\n"
-            + "    \"SECOND\" \\.\\.\\.\n"
-            + "    ", false);
+        "'YEAR' is not a valid time frame", false);
     f.checkFails("^time_trunc(123.45, minute)^",
         "Cannot apply 'TIME_TRUNC' to arguments of type "
             + "'TIME_TRUNC\\(<DECIMAL\\(5, 2\\)>, <INTERVAL MINUTE>\\)'\\. "
@@ -7650,10 +7819,18 @@ public class SqlOperatorTest {
             + "'TIMESTAMP_TRUNC\\(<INTEGER>, <INTERVAL HOUR>\\)'\\. "
             + "Supported form\\(s\\): 'TIMESTAMP_TRUNC\\(<TIMESTAMP>, <DATETIME_INTERVAL>\\)'",
         false);
+    f.checkFails("^timestamp_trunc(100, foo)^",
+        "Cannot apply 'TIMESTAMP_TRUNC' to arguments of type "
+            + "'TIMESTAMP_TRUNC\\(<INTEGER>, <INTERVAL `FOO`>\\)'\\. "
+            + "Supported form\\(s\\): 'TIMESTAMP_TRUNC\\(<TIMESTAMP>, <DATETIME_INTERVAL>\\)'",
+        false);
+
     f.checkFails("timestamp_trunc(timestamp '2015-02-19 12:34:56.78', ^microsecond^)",
-        "'MICROSECOND' is not a valid datetime format", false);
+        "'MICROSECOND' is not a valid time frame", false);
     f.checkFails("timestamp_trunc(timestamp '2015-02-19 12:34:56.78', ^nanosecond^)",
-        "'NANOSECOND' is not a valid datetime format", false);
+        "'NANOSECOND' is not a valid time frame", false);
+    f.checkFails("timestamp_trunc(timestamp '2015-02-19 12:34:56.78', ^millisecond^)",
+        "'MILLISECOND' is not a valid time frame", false);
     f.checkScalar("timestamp_trunc(timestamp '2015-02-19 12:34:56.78', second)",
         "2015-02-19 12:34:56", "TIMESTAMP(0) NOT NULL");
     f.checkScalar("timestamp_trunc(timestamp '2015-02-19 12:34:56', minute)",
@@ -8298,7 +8475,7 @@ public class SqlOperatorTest {
     checkBoolAndFunc(f.withLibrary(SqlLibrary.POSTGRESQL));
   }
 
-  void checkBoolAndFunc(SqlOperatorFixture f) {
+  private static void checkBoolAndFunc(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.BOOL_AND, VM_EXPAND);
 
     f.checkFails("bool_and(^*^)", "Unknown identifier '\\*'", false);
@@ -8334,7 +8511,7 @@ public class SqlOperatorTest {
     checkBoolOrFunc(f.withLibrary(SqlLibrary.POSTGRESQL));
   }
 
-  void checkBoolOrFunc(SqlOperatorFixture f) {
+  private static void checkBoolOrFunc(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.BOOL_OR, VM_EXPAND);
 
     f.checkFails("bool_or(^*^)", "Unknown identifier '\\*'", false);
@@ -8370,7 +8547,7 @@ public class SqlOperatorTest {
     checkLogicalAndFunc(f.withLibrary(SqlLibrary.BIG_QUERY));
   }
 
-  void checkLogicalAndFunc(SqlOperatorFixture f) {
+  private static void checkLogicalAndFunc(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.LOGICAL_AND, VM_EXPAND);
 
     f.checkFails("logical_and(^*^)", "Unknown identifier '\\*'", false);
@@ -8406,7 +8583,7 @@ public class SqlOperatorTest {
     checkLogicalOrFunc(f.withLibrary(SqlLibrary.BIG_QUERY));
   }
 
-  void checkLogicalOrFunc(SqlOperatorFixture f) {
+  private static void checkLogicalOrFunc(SqlOperatorFixture f) {
     f.setFor(SqlLibraryOperators.LOGICAL_OR, VM_EXPAND);
 
     f.checkFails("logical_or(^*^)", "Unknown identifier '\\*'", false);
@@ -8864,9 +9041,15 @@ public class SqlOperatorTest {
         SqlTester.ParameterChecker parameterChecker,
         SqlTester.ResultChecker resultChecker) {
       super.check(factory, query, typeChecker, parameterChecker, resultChecker);
+      final RelDataTypeSystem typeSystem =
+          factory.typeSystemTransform.apply(RelDataTypeSystem.DEFAULT);
       final ConnectionFactory connectionFactory =
-          factory.connectionFactory;
-      try (Connection connection = connectionFactory.createConnection();
+          factory.connectionFactory
+              .with(CalciteConnectionProperty.TYPE_SYSTEM,
+                  CustomTimeFrameTypeSystem.class.getName());
+      try (TryThreadLocal.Memo ignore =
+               CustomTimeFrameTypeSystem.DELEGATE.push(typeSystem);
+           Connection connection = connectionFactory.createConnection();
            Statement statement = connection.createStatement()) {
         final ResultSet resultSet =
             statement.executeQuery(query);
@@ -8970,4 +9153,28 @@ public class SqlOperatorTest {
           .replace("$3", values[3]);
     }
   }
+
+  /** Type system whose constructor reads from a thread-local. You must invoke
+   * the constructor in the same thread, but once constructed you can use from
+   * other threads.
+   *
+   * <p>It's a bit strange, but the best we can do to pass objects via Avatica's
+   * plugin system until
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5295">[CALCITE-5295]
+   * Read the values of plugins (such as connect string properties) from
+   * ThreadLocal fields</a> is fixed.
+   */
+  public static class CustomTimeFrameTypeSystem extends DelegatingTypeSystem {
+    /** Assign to this thread-local before you instantiate a FooTypeSystem
+     * (in the same thread) and your FooTypeSystem will behave in the same
+     * way. */
+    public static final TryThreadLocal<RelDataTypeSystem> DELEGATE =
+        TryThreadLocal.of(DEFAULT);
+
+    /** Creates a CustomTimeFrameTypeSystem, taking a snapshot of
+     * {@link #DELEGATE}. */
+    public CustomTimeFrameTypeSystem() {
+      super(DELEGATE.get());
+    }
+  }
 }