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 2023/01/25 07:39:21 UTC

[calcite] 04/04: [CALCITE-5424] Customize handling of literals based on type system

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

commit a326bd2d0e0b4b6b3336f10217b0ecbb79522239
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Fri Dec 9 01:54:54 2022 -0800

    [CALCITE-5424] Customize handling of literals based on type system
    
    Literals introduced by the keyword DATE, TIME, DATETIME,
    TIMESTAMP, TIMESTAMP WITH LOCAL TIME ZONE are represented by
    the parser by new class SqlUnknownLiteral. Determining the
    actual type is deferred until validation time; the validator
    determines the actual type based on the type system's type
    alias map, and then validates the character string.
    
    Close apache/calcite#3036
    
    Co-authored-by: Julian Hyde <jh...@apache.org>
    Co-authored-by: Oliver Lee <ol...@google.com>
---
 .../org/apache/calcite/test/BabelParserTest.java   | 11 ++--
 core/src/main/codegen/templates/Parser.jj          | 16 +++++-
 .../apache/calcite/rel/rel2sql/SqlImplementor.java |  2 +-
 .../calcite/rex/RexToSqlNodeConverterImpl.java     |  2 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java    | 41 +++++++++++++-
 .../apache/calcite/sql/SqlTimestampLiteral.java    | 10 ++--
 .../org/apache/calcite/sql/SqlUnknownLiteral.java  | 64 ++++++++++++++++++++++
 .../apache/calcite/sql/parser/SqlParserUtil.java   | 16 +++++-
 .../org/apache/calcite/sql/type/SqlTypeName.java   | 18 +++++-
 .../apache/calcite/sql/validate/SqlValidator.java  |  7 +++
 .../calcite/sql/validate/SqlValidatorImpl.java     | 27 ++++++++-
 .../calcite/sql2rel/SqlNodeToRexConverterImpl.java |  4 ++
 .../apache/calcite/sql/test/SqlAdvisorTest.java    |  1 +
 .../org/apache/calcite/test/SqlValidatorTest.java  | 31 +++++++++++
 core/src/test/resources/sql/misc.iq                | 14 +++++
 .../org/apache/calcite/test/ServerParserTest.java  |  2 +-
 site/_docs/reference.md                            |  1 +
 .../apache/calcite/sql/parser/SqlParserTest.java   | 36 +++++++-----
 .../org/apache/calcite/test/SqlOperatorTest.java   |  4 +-
 19 files changed, 268 insertions(+), 39 deletions(-)

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 04e4bb13cf..9cccc5e448 100644
--- a/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
+++ b/babel/src/test/java/org/apache/calcite/test/BabelParserTest.java
@@ -213,15 +213,16 @@ class BabelParserTest extends SqlParserTest {
   /** PostgreSQL and Redshift allow TIMESTAMP literals that contain only a
    * date part. */
   @Test void testShortTimestampLiteral() {
+    // Parser doesn't actually check the contents of the string. The validator
+    // will convert it to '1969-07-20 00:00:00', when it has decided that
+    // TIMESTAMP maps to the TIMESTAMP type.
     sql("select timestamp '1969-07-20'")
-        .ok("SELECT TIMESTAMP '1969-07-20 00:00:00'");
+        .ok("SELECT TIMESTAMP '1969-07-20'");
     // PostgreSQL allows the following. We should too.
     sql("select ^timestamp '1969-07-20 1:2'^")
-        .fails("Illegal TIMESTAMP literal '1969-07-20 1:2': not in format "
-            + "'yyyy-MM-dd HH:mm:ss'"); // PostgreSQL gives 1969-07-20 01:02:00
+        .ok("SELECT TIMESTAMP '1969-07-20 1:2'");
     sql("select ^timestamp '1969-07-20:23:'^")
-        .fails("Illegal TIMESTAMP literal '1969-07-20:23:': not in format "
-            + "'yyyy-MM-dd HH:mm:ss'"); // PostgreSQL gives 1969-07-20 23:00:00
+        .ok("SELECT TIMESTAMP '1969-07-20:23:'");
   }
 
   /** Tests parsing PostgreSQL-style "::" cast operator. */
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index 0514b815e2..4db4d15083 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -4609,15 +4609,24 @@ SqlLiteral DateTimeLiteral() :
     }
 |
     <DATE> { s = span(); } p = SimpleStringLiteral() {
-      return SqlParserUtil.parseDateLiteral(p, s.end(this));
+      return SqlLiteral.createUnknown("DATE", p, s.end(this));
+    }
+|
+    <DATETIME> { s = span(); } p = SimpleStringLiteral() {
+        return SqlLiteral.createUnknown("DATETIME", p, s.end(this));
     }
 |
     <TIME> { s = span(); } p = SimpleStringLiteral() {
-        return SqlParserUtil.parseTimeLiteral(p, s.end(this));
+      return SqlLiteral.createUnknown("TIME", p, s.end(this));
     }
 |
+    LOOKAHEAD(2)
     <TIMESTAMP> { s = span(); } p = SimpleStringLiteral() {
-        return SqlParserUtil.parseTimestampLiteral(p, s.end(this));
+        return SqlLiteral.createUnknown("TIMESTAMP", p, s.end(this));
+    }
+|
+    <TIMESTAMP> { s = span(); } <WITH> <LOCAL> <TIME> <ZONE> p = SimpleStringLiteral() {
+        return SqlLiteral.createUnknown("TIMESTAMP WITH LOCAL TIME ZONE", p, s.end(this));
     }
 }
 
@@ -7644,6 +7653,7 @@ SqlPostfixOperator PostfixRowOperator() :
 |   < DATABASE: "DATABASE" >
 |   < DATE: "DATE" >
 |   < DATE_TRUNC: "DATE_TRUNC" >
+|   < DATETIME: "DATETIME" >
 |   < DATETIME_INTERVAL_CODE: "DATETIME_INTERVAL_CODE" >
 |   < DATETIME_INTERVAL_PRECISION: "DATETIME_INTERVAL_PRECISION" >
 |   < DAY: "DAY" >
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
index 39a93bc9f8..36d28435a5 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/SqlImplementor.java
@@ -1409,7 +1409,7 @@ public abstract class SqlImplementor {
       return SqlLiteral.createTime(castNonNull(literal.getValueAs(TimeString.class)),
           literal.getType().getPrecision(), POS);
     case TIMESTAMP:
-      return SqlLiteral.createTimestamp(
+      return SqlLiteral.createTimestamp(typeName,
           castNonNull(literal.getValueAs(TimestampString.class)),
           literal.getType().getPrecision(), POS);
     case ANY:
diff --git a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
index 426ff6d8f9..d18d7745ae 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexToSqlNodeConverterImpl.java
@@ -85,7 +85,7 @@ public class RexToSqlNodeConverterImpl implements RexToSqlNodeConverter {
     // Timestamp
     if (SqlTypeFamily.TIMESTAMP.getTypeNames().contains(
         literal.getTypeName())) {
-      return SqlLiteral.createTimestamp(
+      return SqlLiteral.createTimestamp(literal.getTypeName(),
           requireNonNull(literal.getValueAs(TimestampString.class),
               "literal.getValueAs(TimestampString.class)"),
           0,
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 ffcce09d17..e60c000196 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -205,6 +205,7 @@ public class SqlLiteral extends SqlNode {
     case TIME:
       return value instanceof TimeString;
     case TIMESTAMP:
+    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
       return value instanceof TimestampString;
     case INTERVAL_YEAR:
     case INTERVAL_YEAR_MONTH:
@@ -229,6 +230,8 @@ public class SqlLiteral extends SqlNode {
           || (value instanceof SqlSampleSpec);
     case MULTISET:
       return true;
+    case UNKNOWN:
+      return value instanceof String;
     case INTEGER: // not allowed -- use Decimal
     case VARCHAR: // not allowed -- use Char
     case VARBINARY: // not allowed -- use Binary
@@ -826,6 +829,28 @@ public class SqlLiteral extends SqlNode {
     }
   }
 
+  /** Creates a literal whose type is unknown until validation time.
+   * The literal has a tag that looks like a type name, but the tag cannot be
+   * resolved until validation time, when we have the mapping from type aliases
+   * to types.
+   *
+   * <p>For example,
+   * <blockquote>{@code
+   * TIMESTAMP '1969-07-20 22:56:00'
+   * }</blockquote>
+   * calls {@code createUnknown("TIMESTAMP", "1969-07-20 22:56:00")}; at
+   * validate time, we may discover that "TIMESTAMP" maps to the type
+   * "TIMESTAMP WITH LOCAL TIME ZONE".
+   *
+   * @param tag Type name, e.g. "TIMESTAMP", "TIMESTAMP WITH LOCAL TIME ZONE"
+   * @param value String encoding of the value
+   * @param pos Parser position
+   */
+  public static SqlLiteral createUnknown(String tag, String value,
+      SqlParserPos pos) {
+    return new SqlUnknownLiteral(tag, value, pos);
+  }
+
   @Deprecated // to be removed before 2.0
   public static SqlDateLiteral createDate(
       Calendar calendar,
@@ -844,15 +869,25 @@ public class SqlLiteral extends SqlNode {
       Calendar calendar,
       int precision,
       SqlParserPos pos) {
-    return createTimestamp(TimestampString.fromCalendarFields(calendar),
-        precision, pos);
+    return createTimestamp(SqlTypeName.TIMESTAMP,
+        TimestampString.fromCalendarFields(calendar), precision, pos);
   }
 
+  @Deprecated // to be removed before 2.0
   public static SqlTimestampLiteral createTimestamp(
       TimestampString ts,
       int precision,
       SqlParserPos pos) {
-    return new SqlTimestampLiteral(ts, precision, false, pos);
+    return createTimestamp(SqlTypeName.TIMESTAMP, ts, precision, pos);
+  }
+
+  /** Creates a TIMESTAMP or TIMESTAMP WITH TIME ZONE literal. */
+  public static SqlTimestampLiteral createTimestamp(
+      SqlTypeName typeName,
+      TimestampString ts,
+      int precision,
+      SqlParserPos pos) {
+    return new SqlTimestampLiteral(ts, precision, typeName, pos);
   }
 
   @Deprecated // to be removed before 2.0
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
index 62869df839..ac169cad30 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlTimestampLiteral.java
@@ -34,9 +34,11 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
   //~ Constructors -----------------------------------------------------------
 
   SqlTimestampLiteral(TimestampString ts, int precision,
-      boolean hasTimeZone, SqlParserPos pos) {
-    super(ts, hasTimeZone, SqlTypeName.TIMESTAMP, precision, pos);
+      SqlTypeName typeName, SqlParserPos pos) {
+    super(ts, false, typeName, precision, pos);
     Preconditions.checkArgument(this.precision >= 0);
+    Preconditions.checkArgument(typeName == SqlTypeName.TIMESTAMP
+        || typeName == SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -45,11 +47,11 @@ public class SqlTimestampLiteral extends SqlAbstractDateTimeLiteral {
     return new SqlTimestampLiteral(
         (TimestampString) Objects.requireNonNull(value, "value"),
         precision,
-        hasTimeZone, pos);
+        getTypeName(), pos);
   }
 
   @Override public String toString() {
-    return "TIMESTAMP '" + toFormattedString() + "'";
+    return getTypeName() + " '" + toFormattedString() + "'";
   }
 
   /**
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUnknownLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlUnknownLiteral.java
new file mode 100644
index 0000000000..56fc375d1d
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUnknownLiteral.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.parser.SqlParserUtil;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.NlsString;
+import org.apache.calcite.util.Util;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Literal whose type is not yet known.
+ */
+public class SqlUnknownLiteral extends SqlLiteral {
+  public final String tag;
+
+  SqlUnknownLiteral(String tag, String value, SqlParserPos pos) {
+    super(requireNonNull(value, "value"), SqlTypeName.UNKNOWN, pos);
+    this.tag = requireNonNull(tag, "tag");
+  }
+
+  @Override public String getValue() {
+    return (String) requireNonNull(super.getValue(), "value");
+  }
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    final NlsString nlsString = new NlsString(getValue(), null, null);
+    writer.keyword(tag);
+    writer.literal(nlsString.asSql(true, true, writer.getDialect()));
+  }
+
+
+  /** Converts this unknown literal to a literal of known type. */
+  public SqlLiteral resolve(SqlTypeName typeName) {
+    switch (typeName) {
+    case DATE:
+      return SqlParserUtil.parseDateLiteral(getValue(), pos);
+    case TIME:
+      return SqlParserUtil.parseTimeLiteral(getValue(), pos);
+    case TIMESTAMP:
+      return SqlParserUtil.parseTimestampLiteral(getValue(), pos);
+    case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+      return SqlParserUtil.parseTimestampWithLocalTimeZoneLiteral(getValue(), pos);
+    default:
+      throw Util.unexpected(typeName);
+    }
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
index c3f69220bb..270f897808 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
@@ -40,6 +40,7 @@ import org.apache.calcite.sql.SqlTimestampLiteral;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.DateString;
 import org.apache.calcite.util.PrecedenceClimbingParser;
 import org.apache.calcite.util.TimeString;
@@ -338,6 +339,17 @@ public final class SqlParserUtil {
 
   public static SqlTimestampLiteral parseTimestampLiteral(String s,
       SqlParserPos pos) {
+    return parseTimestampLiteral(SqlTypeName.TIMESTAMP, s, pos);
+  }
+
+  public static SqlTimestampLiteral parseTimestampWithLocalTimeZoneLiteral(
+      String s, SqlParserPos pos) {
+    return parseTimestampLiteral(SqlTypeName.TIMESTAMP_WITH_LOCAL_TIME_ZONE, s,
+        pos);
+  }
+
+  private static SqlTimestampLiteral parseTimestampLiteral(SqlTypeName typeName,
+      String s, SqlParserPos pos) {
     final Format format = Format.get();
     DateTimeUtils.PrecisionTime pt = null;
     // Allow timestamp literals with and without time fields (as does
@@ -352,13 +364,13 @@ public final class SqlParserUtil {
     }
     if (pt == null) {
       throw SqlUtil.newContextException(pos,
-          RESOURCE.illegalLiteral("TIMESTAMP", s,
+          RESOURCE.illegalLiteral(typeName.getName().replace('_', ' '), s,
               RESOURCE.badFormat(DateTimeUtils.TIMESTAMP_FORMAT_STRING).str()));
     }
     final TimestampString ts =
         TimestampString.fromCalendarFields(pt.getCalendar())
             .withFraction(pt.getFraction());
-    return SqlLiteral.createTimestamp(ts, pt.getPrecision(), pos);
+    return SqlLiteral.createTimestamp(typeName, ts, pt.getPrecision(), pos);
   }
 
   public static SqlIntervalLiteral parseIntervalLiteral(SqlParserPos pos,
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
index 4849fa1e9f..526cad95de 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeName.java
@@ -298,6 +298,14 @@ public enum SqlTypeName {
     return VALUES_MAP.get(name);
   }
 
+  /** Returns the SqlTypeName value whose name or {@link #getSpaceName()}
+   * matches the given name, or throws {@link IllegalArgumentException}; never
+   * returns null. */
+  public static SqlTypeName lookup(String tag) {
+    String tag2 = tag.replace(' ', '_');
+    return valueOf(tag2);
+  }
+
   public boolean allowsNoPrecNoScale() {
     return (signatures & PrecScale.NO_NO) != 0;
   }
@@ -945,7 +953,7 @@ public enum SqlTypeName {
           ? TimeString.fromCalendarFields((Calendar) o)
           : (TimeString) o, 0 /* todo */, pos);
     case TIMESTAMP:
-      return SqlLiteral.createTimestamp(o instanceof Calendar
+      return SqlLiteral.createTimestamp(this, o instanceof Calendar
           ? TimestampString.fromCalendarFields((Calendar) o)
           : (TimestampString) o, 0 /* todo */, pos);
     default:
@@ -955,7 +963,13 @@ public enum SqlTypeName {
 
   /** Returns the name of this type. */
   public String getName() {
-    return toString();
+    return name();
+  }
+
+  /** Returns the name of this type, with underscores converted to spaces,
+   * for example "TIMESTAMP WITH LOCAL TIME ZONE", "DATE". */
+  public String getSpaceName() {
+    return name().replace('_', ' ');
   }
 
   /**
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 50e2151f1e..4c05824ce7 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
@@ -705,6 +705,13 @@ public interface SqlValidator {
    */
   SqlNode expand(SqlNode expr, SqlValidatorScope scope);
 
+  /** Resolves a literal.
+   *
+   * <p>Usually returns the literal unchanged, but if the literal is of type
+   * {@link org.apache.calcite.sql.type.SqlTypeName#UNKNOWN} looks up its type
+   * and converts to the appropriate literal subclass. */
+  SqlLiteral resolveLiteral(SqlLiteral literal);
+
   /**
    * Returns whether a field is a system field. Such fields may have
    * particular properties such as sortedness and nullability.
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 bcc1dc9776..99e52c5e4e 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
@@ -76,6 +76,7 @@ import org.apache.calcite.sql.SqlSelectKeyword;
 import org.apache.calcite.sql.SqlSnapshot;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.SqlTableFunction;
+import org.apache.calcite.sql.SqlUnknownLiteral;
 import org.apache.calcite.sql.SqlUnpivot;
 import org.apache.calcite.sql.SqlUnresolvedFunction;
 import org.apache.calcite.sql.SqlUpdate;
@@ -6145,6 +6146,26 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     assert feature.getProperties().get("FeatureDefinition") != null;
   }
 
+  @Override public SqlLiteral resolveLiteral(SqlLiteral literal) {
+    switch (literal.getTypeName()) {
+    case UNKNOWN:
+      final SqlUnknownLiteral unknownLiteral = (SqlUnknownLiteral) literal;
+      final SqlIdentifier identifier =
+          new SqlIdentifier(unknownLiteral.tag, SqlParserPos.ZERO);
+      final @Nullable RelDataType type = catalogReader.getNamedType(identifier);
+      final SqlTypeName typeName;
+      if (type != null) {
+        typeName = type.getSqlTypeName();
+      } else {
+        typeName = SqlTypeName.lookup(unknownLiteral.tag);
+      }
+      return unknownLiteral.resolve(typeName);
+
+    default:
+      return literal;
+    }
+  }
+
   public SqlNode expandSelectExpr(SqlNode expr,
       SelectScope scope, SqlSelect select) {
     final Expander expander = new SelectExpander(this, scope, select);
@@ -6443,7 +6464,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
 
     @Override public RelDataType visit(SqlLiteral literal) {
-      return literal.createSqlType(typeFactory);
+      return resolveLiteral(literal).createSqlType(typeFactory);
     }
 
     @Override public RelDataType visit(SqlCall call) {
@@ -6592,6 +6613,10 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       return expandedExpr;
     }
 
+    @Override public @Nullable SqlNode visit(SqlLiteral literal) {
+      return validator.resolveLiteral(literal);
+    }
+
     @Override protected SqlNode visitScoped(SqlCall call) {
       switch (call.getKind()) {
       case SCALAR_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 987f777a6b..9a8bc7e04b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlNodeToRexConverterImpl.java
@@ -137,6 +137,10 @@ public class SqlNodeToRexConverterImpl implements SqlNodeToRexConverter {
       return rexBuilder.makeIntervalLiteral(
           literal.getValueAs(BigDecimal.class),
           sqlIntervalQualifier);
+
+    case UNKNOWN:
+      return convertLiteral(cx, cx.getValidator().resolveLiteral(literal));
+
     default:
       throw Util.unexpected(literal.getTypeName());
     }
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 9f47e6b28c..7cc1f882e0 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -162,6 +162,7 @@ class SqlAdvisorTest extends SqlValidatorTestCase {
           "KEYWORD(CURRENT_USER)",
           "KEYWORD(CURSOR)",
           "KEYWORD(DATE)",
+          "KEYWORD(DATETIME)",
           "KEYWORD(DENSE_RANK)",
           "KEYWORD(ELEMENT)",
           "KEYWORD(EVERY)",
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 7fcf6cf5a1..26d5b3cfe1 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -251,6 +251,37 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .columnType("BOOLEAN");
   }
 
+  /** Tests that date-time literals with invalid strings are considered invalid.
+   * Originally the parser did that checking, but now the parser creates a
+   * {@link org.apache.calcite.sql.SqlUnknownLiteral} and the checking is
+   * deferred to the validator. */
+  @Test void testLiteral() {
+    expr("^DATE '12/21/99'^")
+        .fails("(?s).*Illegal DATE literal.*");
+    expr("^TIME '1230:33'^")
+        .fails("(?s).*Illegal TIME literal.*");
+    expr("^TIME '12:00:00 PM'^")
+        .fails("(?s).*Illegal TIME literal.*");
+    expr("^TIMESTAMP '12-21-99, 12:30:00'^")
+        .fails("(?s).*Illegal TIMESTAMP literal.*");
+    expr("^TIMESTAMP WITH LOCAL TIME ZONE '12-21-99, 12:30:00'^")
+        .fails("(?s).*Illegal TIMESTAMP WITH LOCAL TIME ZONE literal.*");
+  }
+
+  /** PostgreSQL and Redshift allow TIMESTAMP literals that contain only a
+   * date part. */
+  @Test void testShortTimestampLiteral() {
+    sql("select timestamp '1969-07-20'")
+        .ok();
+    // PostgreSQL allows the following. We should too.
+    sql("select ^timestamp '1969-07-20 1:2'^")
+        .fails("Illegal TIMESTAMP literal '1969-07-20 1:2': not in format "
+            + "'yyyy-MM-dd HH:mm:ss'"); // PostgreSQL gives 1969-07-20 01:02:00
+    sql("select ^timestamp '1969-07-20:23:'^")
+        .fails("Illegal TIMESTAMP literal '1969-07-20:23:': not in format "
+            + "'yyyy-MM-dd HH:mm:ss'"); // PostgreSQL gives 1969-07-20 23:00:00
+  }
+
   @Test void testBooleans() {
     sql("select TRUE OR unknowN from (values(true))").ok();
     sql("select false AND unknown from (values(true))").ok();
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 8761d0c632..8ce073905f 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -2359,4 +2359,18 @@ null
 14
 !ok
 
+# TIMESTAMP literals without a time part are OK.
+SELECT TIMESTAMP '1969-07-20' AS ts;
+TS
+1969-07-20 00:00:00
+!ok
+
+# Short TIMESTAMP literals are equivalent to long TIMESTAMP literals
+SELECT TIMESTAMP '1969-07-20' + i AS ts
+FROM (VALUES (INTERVAL '1' DAY)) AS t (i)
+GROUP BY TIMESTAMP '1969-07-20 00:00:00' + i;
+TS
+1969-07-21 00:00:00
+!ok
+
 # End misc.iq
diff --git a/server/src/test/java/org/apache/calcite/test/ServerParserTest.java b/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
index de03ef7317..64ab59ef9c 100644
--- a/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
+++ b/server/src/test/java/org/apache/calcite/test/ServerParserTest.java
@@ -74,7 +74,7 @@ class ServerParserTest extends SqlParserTest {
         + "OPTIONS (`ABOOLEAN` TRUE,"
         + " `ANINTEGER` -45,"
         + " `ADATE` DATE '1970-03-21',"
-        + " `quoted.id` TIMESTAMP '1970-03-21 12:04:56.78',"
+        + " `quoted.id` TIMESTAMP '1970-03-21 12:4:56.78',"
         + " `ASTRING` 'foo''bar')";
     sql(sql).ok(expected);
   }
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 65c4a0eaa7..4de83ccc24 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -542,6 +542,7 @@ CURSOR_NAME,
 DATA,
 DATABASE,
 **DATE**,
+**DATETIME**,
 DATETIME_INTERVAL_CODE,
 DATETIME_INTERVAL_PRECISION,
 DATE_TRUNC,
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 9b5fc4aca2..4481485d40 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
@@ -27,6 +27,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlSetOption;
+import org.apache.calcite.sql.SqlUnknownLiteral;
 import org.apache.calcite.sql.SqlWriterConfig;
 import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.dialect.SparkSqlDialect;
@@ -34,6 +35,7 @@ import org.apache.calcite.sql.parser.SqlParser.Config;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.test.SqlTestFactory;
 import org.apache.calcite.sql.test.SqlTests;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.test.DiffTestCase;
@@ -207,6 +209,7 @@ public class SqlParserTest {
       "CYCLE",                               "99", "2003", "2011", "2014", "c",
       "DATA",                                "99",
       "DATE",                          "92", "99", "2003", "2011", "2014", "c",
+      "DATETIME",                                                          "c",
       "DAY",                           "92", "99", "2003", "2011", "2014", "c",
       "DAYS",                                              "2011",
       "DEALLOCATE",                    "92", "99", "2003", "2011", "2014", "c",
@@ -5259,8 +5262,7 @@ public class SqlParserTest {
 
     // Time literals
     expr("TIME '12:01:01'").same();
-    expr("TIME '12:01:01.'")
-        .ok("TIME '12:01:01'");
+    expr("TIME '12:01:01.'").same();
     expr("TIME '12:01:01.000'").same();
     expr("TIME '12:01:01.001'").same();
     expr("TIME '12:01:01.01023456789'").same();
@@ -5268,21 +5270,23 @@ public class SqlParserTest {
     // Timestamp literals
     expr("TIMESTAMP '2004-12-01 12:01:01'").same();
     expr("TIMESTAMP '2004-12-01 12:01:01.1'").same();
-    expr("TIMESTAMP '2004-12-01 12:01:01.'")
-        .ok("TIMESTAMP '2004-12-01 12:01:01'");
+    expr("TIMESTAMP '2004-12-01 12:01:01.'").same();
     expr("TIMESTAMP  '2004-12-01 12:01:01.010234567890'")
         .ok("TIMESTAMP '2004-12-01 12:01:01.010234567890'");
     expr("TIMESTAMP '2004-12-01 12:01:01.01023456789'").same();
 
-    // Failures.
-    sql("^DATE '12/21/99'^")
-        .fails("(?s).*Illegal DATE literal.*");
-    sql("^TIME '1230:33'^")
-        .fails("(?s).*Illegal TIME literal.*");
-    sql("^TIME '12:00:00 PM'^")
-        .fails("(?s).*Illegal TIME literal.*");
-    sql("^TIMESTAMP '12-21-99, 12:30:00'^")
-        .fails("(?s).*Illegal TIMESTAMP literal.*");
+    // Datetime, Timestamp with local time zone literals.
+    expr("DATETIME '2004-12-01 12:01:01'")
+        .same();
+
+    // Value strings that are illegal for their type are considered valid at
+    // parse time, invalid at validate time. See SqlValidatorTest.testLiteral.
+    expr("^DATE '12/21/99'^").same();
+    expr("^TIME '1230:33'^").same();
+    expr("^TIME '12:00:00 PM'^").same();
+    expr("TIMESTAMP '12-21-99, 12:30:00'").same();
+    expr("TIMESTAMP WITH LOCAL TIME ZONE '12-21-99, 12:30:00'").same();
+    expr("DATETIME '12-21-99, 12:30:00'").same();
   }
 
   /**
@@ -8850,8 +8854,12 @@ public class SqlParserTest {
   protected static String varToStr(Hoist.Variable v) {
     if (v.node instanceof SqlLiteral) {
       SqlLiteral literal = (SqlLiteral) v.node;
+      SqlTypeName typeName = literal.getTypeName();
       return "[" + v.ordinal
-          + ":" + literal.getTypeName()
+          + ":"
+          + (typeName == SqlTypeName.UNKNOWN
+              ? ((SqlUnknownLiteral) literal).tag
+              : typeName.getName())
           + ":" + literal.toValue()
           + "]";
     } else {
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 3d03cd2cb3..b56c566567 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -9814,8 +9814,8 @@ public class SqlOperatorTest {
         return SqlLiteral.createCharString(value.toString(), SqlParserPos.ZERO);
       case TIMESTAMP:
         TimestampString ts = TimestampString.fromMillisSinceEpoch((Long) value);
-        return SqlLiteral.createTimestamp(ts, type.getPrecision(),
-            SqlParserPos.ZERO);
+        return SqlLiteral.createTimestamp(type.getSqlTypeName(), ts,
+            type.getPrecision(), SqlParserPos.ZERO);
       default:
         throw new AssertionError(type);
       }