You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by da...@apache.org on 2019/11/25 09:57:55 UTC

[calcite] branch master updated: [CALCITE-3534] Support parse(unparse) alien system non-standard data type

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 07c1efb  [CALCITE-3534] Support parse(unparse) alien system non-standard data type
07c1efb is described below

commit 07c1efbb2b42a0e27f9d01968d357d9c1e17cd51
Author: yuzhao.cyz <yu...@gmail.com>
AuthorDate: Mon Nov 25 14:17:04 2019 +0800

    [CALCITE-3534] Support parse(unparse) alien system non-standard data type
    
    * Add a new type name specification class SqlAlienSystemTypeNameSpec
    * Refactor SqlUserDefinedTypeNameSpec, before this patch, we use this
    class for cases where the SqlAlienSystemTypeNameSpec should be used
    * Add more doc to SqlDataTypeSpec
---
 .../calcite/sql/SqlAlienSystemTypeNameSpec.java    | 93 ++++++++++++++++++++++
 .../org/apache/calcite/sql/SqlDataTypeSpec.java    | 85 +++++++++-----------
 .../calcite/sql/SqlUserDefinedTypeNameSpec.java    | 13 +--
 .../calcite/sql/dialect/BigQuerySqlDialect.java    | 37 ++++-----
 .../apache/calcite/sql/dialect/HiveSqlDialect.java |  7 +-
 .../calcite/sql/dialect/MysqlSqlDialect.java       |  7 +-
 .../calcite/sql/dialect/OracleSqlDialect.java      | 12 +--
 .../calcite/sql/dialect/PostgresqlSqlDialect.java  |  8 +-
 8 files changed, 168 insertions(+), 94 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/sql/SqlAlienSystemTypeNameSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlAlienSystemTypeNameSpec.java
new file mode 100644
index 0000000..e6fab0f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/SqlAlienSystemTypeNameSpec.java
@@ -0,0 +1,93 @@
+/*
+ * 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.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
+
+import java.util.Objects;
+
+/**
+ * Represents a type name for an alien system. For example,
+ * UNSIGNED is a built-in type in MySQL which is synonym of INTEGER.
+ *
+ * <p>You can use this class to define a customized type name with specific alias,
+ * for example, in some systems, STRING is synonym of VARCHAR
+ * and BYTES is synonym of VARBINARY.
+ *
+ * <p>Internally we may use the {@link SqlAlienSystemTypeNameSpec} to unparse
+ * as the builtin data type name for some alien systems during rel-to-sql conversion.
+ */
+public class SqlAlienSystemTypeNameSpec extends SqlBasicTypeNameSpec {
+  //~ Instance fields --------------------------------------------------------
+
+  // Type alias used for unparsing.
+  private final String typeAlias;
+
+  //~ Constructors -----------------------------------------------------------
+
+  /**
+   * Creates a {@code SqlAlienSystemTypeNameSpec} instance.
+   *
+   * @param typeAlias Type alias of the alien system
+   * @param typeName  Type name the {@code typeAlias} implies as the (standard) basic type name
+   * @param pos       The parser position
+   */
+  public SqlAlienSystemTypeNameSpec(
+      String typeAlias,
+      SqlTypeName typeName,
+      SqlParserPos pos) {
+    this(typeAlias, typeName, -1, pos);
+  }
+
+  /**
+   * Creates a {@code SqlAlienSystemTypeNameSpec} instance.
+   *
+   * @param typeAlias Type alias of the alien system
+   * @param typeName  Type name the {@code typeAlias} implies as the (standard) basic type name
+   * @param precision Type Precision
+   * @param pos       The parser position
+   */
+  public SqlAlienSystemTypeNameSpec(
+      String typeAlias,
+      SqlTypeName typeName,
+      int precision,
+      SqlParserPos pos) {
+    super(typeName, precision, pos);
+    this.typeAlias = typeAlias;
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
+    writer.keyword(typeAlias);
+  }
+
+  @Override public boolean equalsDeep(SqlTypeNameSpec node, Litmus litmus) {
+    if (!(node instanceof SqlAlienSystemTypeNameSpec)) {
+      return litmus.fail("{} != {}", this, node);
+    }
+    SqlAlienSystemTypeNameSpec that = (SqlAlienSystemTypeNameSpec) node;
+    if (!Objects.equals(this.typeAlias, that.typeAlias)) {
+      return litmus.fail("{} != {}", this, node);
+    }
+    return super.equalsDeep(node, litmus);
+  }
+}
+
+// End SqlAlienSystemTypeNameSpec.java
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
index 6c2d7c7..6f53e61 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
@@ -34,33 +34,41 @@ import java.util.TimeZone;
  * <p>A <code>SqlDataTypeSpec</code> is immutable; once created, you cannot
  * change any of the fields.</p>
  *
- * <p>todo: This should really be a subtype of {@link SqlCall}.</p>
+ * <p>We support the following data type expressions:
  *
- * <p>we support complex type expressions
- * like:</p>
- *
- * <blockquote><code>ROW(<br>
- *   foo NUMBER(5, 2) NOT NULL,<br>
- *   rec ROW(b BOOLEAN, i MyUDT NOT NULL))</code></blockquote>
- *
- * <p>Internally we use {@link SqlRowTypeNameSpec} to specify row data type name.
- *
- * <p>We support simple data types like CHAR, VARCHAR and DOUBLE,
- * with optional precision and scale.</p>
- *
- * <p>Internally we use {@link SqlBasicTypeNameSpec} to specify basic sql data type name.
+ * <ul>
+ *   <li>Complex data type expression like:
+ *   <blockquote><code>ROW(<br>
+ *     foo NUMBER(5, 2) NOT NULL,<br>
+ *       rec ROW(b BOOLEAN, i MyUDT NOT NULL))</code></blockquote>
+ *   Internally we use {@link SqlRowTypeNameSpec} to specify row data type name.
+ *   </li>
+ *   <li>Simple data type expression like CHAR, VARCHAR and DOUBLE
+ *   with optional precision and scale;
+ *   Internally we use {@link SqlBasicTypeNameSpec} to specify basic sql data type name.
+ *   </li>
+ *   <li>Collection data type expression like:
+ *   <blockquote><code>
+ *     INT ARRAY;
+ *     VARCHAR(20) MULTISET;
+ *     INT ARRAY MULTISET;</code></blockquote>
+ *   Internally we use {@link SqlCollectionTypeNameSpec} to specify collection data type name.
+ *   </li>
+ *   <li>User defined data type expression like `My_UDT`;
+ *   Internally we use {@link SqlUserDefinedTypeNameSpec} to specify user defined data type name.
+ *   </li>
+ * </ul>
  */
 public class SqlDataTypeSpec extends SqlNode {
   //~ Instance fields --------------------------------------------------------
 
   private final SqlTypeNameSpec typeNameSpec;
-  private final SqlTypeNameSpec baseTypeName;
   private final TimeZone timeZone;
 
-  /** Whether data type is allows nulls.
+  /** Whether data type allows nulls.
    *
    * <p>Nullable is nullable! Null means "not specified". E.g.
-   * {@code CAST(x AS INTEGER)} preserves has the same nullability as {@code x}.
+   * {@code CAST(x AS INTEGER)} preserves the same nullability as {@code x}.
    */
   private Boolean nullable;
 
@@ -70,7 +78,7 @@ public class SqlDataTypeSpec extends SqlNode {
    * Creates a type specification representing a type.
    *
    * @param typeNameSpec The type name can be basic sql type, row type,
-   *                     collections type and user defined type.
+   *                     collections type and user defined type
    */
   public SqlDataTypeSpec(
       final SqlTypeNameSpec typeNameSpec,
@@ -82,8 +90,8 @@ public class SqlDataTypeSpec extends SqlNode {
    * Creates a type specification representing a type, with time zone specified.
    *
    * @param typeNameSpec The type name can be basic sql type, row type,
-   *                     collections type and user defined type.
-   * @param timeZone     Specified time zone.
+   *                     collections type and user defined type
+   * @param timeZone     Specified time zone
    */
   public SqlDataTypeSpec(
       final SqlTypeNameSpec typeNameSpec,
@@ -93,41 +101,21 @@ public class SqlDataTypeSpec extends SqlNode {
   }
 
   /**
-   * Creates a type specification representing a type, with time zone
-   * and nullability specified.
-   *
-   * @param typeNameSpec The type name can be basic sql type, row type,
-   *                     collections type and user defined type.
-   * @param timeZone     Specified time zone.
-   * @param nullable     The nullability.
-   */
-  public SqlDataTypeSpec(
-      SqlTypeNameSpec typeNameSpec,
-      TimeZone timeZone,
-      Boolean nullable,
-      SqlParserPos pos) {
-    this(typeNameSpec, typeNameSpec, timeZone, nullable, pos);
-  }
-
-  /**
    * Creates a type specification representing a type, with time zone,
    * nullability and base type name specified.
    *
    * @param typeNameSpec The type name can be basic sql type, row type,
-   *                     collections type and user defined type.
-   * @param baseTypeName The base type name.
-   * @param timeZone     Specified time zone.
-   * @param nullable     The nullability.
+   *                     collections type and user defined type
+   * @param timeZone     Specified time zone
+   * @param nullable     The nullability
    */
   public SqlDataTypeSpec(
       SqlTypeNameSpec typeNameSpec,
-      SqlTypeNameSpec baseTypeName,
       TimeZone timeZone,
       Boolean nullable,
       SqlParserPos pos) {
     super(pos);
     this.typeNameSpec = typeNameSpec;
-    this.baseTypeName = baseTypeName;
     this.timeZone = timeZone;
     this.nullable = nullable;
   }
@@ -227,7 +215,7 @@ public class SqlDataTypeSpec extends SqlNode {
    * <p>Throws an error if the type is not found.
    *
    * @param nullable Whether the type is nullable if the type specification
-   *                 does not explicitly state.
+   *                 does not explicitly state
    */
   public RelDataType deriveType(SqlValidator validator, boolean nullable) {
     RelDataType type;
@@ -243,11 +231,12 @@ public class SqlDataTypeSpec extends SqlNode {
 
   /**
    * Fix up the nullability of the {@code type}.
-   * @param typeFactory Type factory.
-   * @param type        The type to coerce nullability.
+   *
+   * @param typeFactory Type factory
+   * @param type        The type to coerce nullability
    * @param nullable    Default nullability to use if this type specification does not
-   *                    specify nullability.
-   * @return type with specified nullability or the default.
+   *                    specify nullability
+   * @return Type with specified nullability or the default(false)
    */
   private RelDataType fixUpNullability(RelDataTypeFactory typeFactory,
       RelDataType type, boolean nullable) {
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUserDefinedTypeNameSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlUserDefinedTypeNameSpec.java
index c82ff20..d507eaf 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUserDefinedTypeNameSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUserDefinedTypeNameSpec.java
@@ -26,9 +26,6 @@ import org.apache.calcite.util.Litmus;
  *
  * <p>Usually you should register the UDT into the {@link org.apache.calcite.jdbc.CalciteSchema}
  * first before referencing it in the sql statement.
- *
- * <p>Internally we may new the {@code SqlUserDefinedTypeNameSpec} instance directly
- * for some sql dialects during rel-to-sql conversion.
  */
 public class SqlUserDefinedTypeNameSpec extends SqlTypeNameSpec {
 
@@ -53,15 +50,7 @@ public class SqlUserDefinedTypeNameSpec extends SqlTypeNameSpec {
   }
 
   @Override public void unparse(SqlWriter writer, int leftPrec, int rightPrec) {
-    final String name = getTypeName().names.get(0);
-    if (name.startsWith("_")) {
-      // We're generating a type for an alien system. For example,
-      // UNSIGNED is a built-in type in MySQL.
-      // (Need a more elegant way than '_' of flagging this.)
-      writer.keyword(name.substring(1));
-    } else {
-      getTypeName().unparse(writer, leftPrec, rightPrec);
-    }
+    getTypeName().unparse(writer, leftPrec, rightPrec);
   }
 
   @Override public boolean equalsDeep(SqlTypeNameSpec spec, Litmus litmus) {
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
index fc035ae..412a5dc 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/BigQuerySqlDialect.java
@@ -21,19 +21,19 @@ import org.apache.calcite.config.NullCollation;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexCall;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.sql.SqlAlienSystemTypeNameSpec;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSetOperator;
 import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.sql.SqlUserDefinedTypeNameSpec;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 
 import com.google.common.collect.ImmutableList;
@@ -160,41 +160,42 @@ public class BigQuerySqlDialect extends SqlDialect {
    */
   @Override public SqlNode getCastSpec(final RelDataType type) {
     if (type instanceof BasicSqlType) {
-      switch (type.getSqlTypeName()) {
+      final SqlTypeName typeName = type.getSqlTypeName();
+      switch (typeName) {
       // BigQuery only supports INT64 for integer types.
-      case BIGINT:
-      case INTEGER:
       case TINYINT:
       case SMALLINT:
-        return createSqlDataTypeSpecByName("INT64");
+      case INTEGER:
+      case BIGINT:
+        return createSqlDataTypeSpecByName("INT64", typeName);
       // BigQuery only supports FLOAT64(aka. Double) for floating point types.
       case FLOAT:
       case DOUBLE:
-        return createSqlDataTypeSpecByName("FLOAT64");
+        return createSqlDataTypeSpecByName("FLOAT64", typeName);
       case DECIMAL:
-        return createSqlDataTypeSpecByName("NUMERIC");
+        return createSqlDataTypeSpecByName("NUMERIC", typeName);
       case BOOLEAN:
-        return createSqlDataTypeSpecByName("BOOL");
+        return createSqlDataTypeSpecByName("BOOL", typeName);
       case CHAR:
       case VARCHAR:
-        return createSqlDataTypeSpecByName("STRING");
-      case VARBINARY:
+        return createSqlDataTypeSpecByName("STRING", typeName);
       case BINARY:
-        return createSqlDataTypeSpecByName("BYTES");
+      case VARBINARY:
+        return createSqlDataTypeSpecByName("BYTES", typeName);
       case DATE:
-        return createSqlDataTypeSpecByName("DATE");
+        return createSqlDataTypeSpecByName("DATE", typeName);
       case TIME:
-        return createSqlDataTypeSpecByName("TIME");
+        return createSqlDataTypeSpecByName("TIME", typeName);
       case TIMESTAMP:
-        return createSqlDataTypeSpecByName("TIMESTAMP");
+        return createSqlDataTypeSpecByName("TIMESTAMP", typeName);
       }
     }
     return super.getCastSpec(type);
   }
 
-  private SqlDataTypeSpec createSqlDataTypeSpecByName(String identifierName) {
-    SqlUserDefinedTypeNameSpec typeNameSpec = new SqlUserDefinedTypeNameSpec(
-            new SqlIdentifier(identifierName, SqlParserPos.ZERO), SqlParserPos.ZERO);
+  private SqlDataTypeSpec createSqlDataTypeSpecByName(String typeAlias, SqlTypeName typeName) {
+    SqlAlienSystemTypeNameSpec typeNameSpec = new SqlAlienSystemTypeNameSpec(
+        typeAlias, typeName, SqlParserPos.ZERO);
     return new SqlDataTypeSpec(typeNameSpec, SqlParserPos.ZERO);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java
index 31bf0b5..c009133 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/HiveSqlDialect.java
@@ -18,15 +18,14 @@ package org.apache.calcite.sql.dialect;
 
 import org.apache.calcite.config.NullCollation;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlAlienSystemTypeNameSpec;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSyntax;
-import org.apache.calcite.sql.SqlUserDefinedTypeNameSpec;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlSubstringFunction;
@@ -148,8 +147,8 @@ public class HiveSqlDialect extends SqlDialect {
     if (type instanceof BasicSqlType) {
       switch (type.getSqlTypeName()) {
       case INTEGER:
-        SqlUserDefinedTypeNameSpec typeNameSpec = new SqlUserDefinedTypeNameSpec(
-            new SqlIdentifier("INT", SqlParserPos.ZERO), SqlParserPos.ZERO);
+        SqlAlienSystemTypeNameSpec typeNameSpec = new SqlAlienSystemTypeNameSpec(
+            "INT", type.getSqlTypeName(), SqlParserPos.ZERO);
         return new SqlDataTypeSpec(typeNameSpec, SqlParserPos.ZERO);
       }
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java
index 7c2654c..6086479 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MysqlSqlDialect.java
@@ -22,6 +22,7 @@ import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.config.NullCollation;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.calcite.sql.SqlAlienSystemTypeNameSpec;
 import org.apache.calcite.sql.SqlBasicCall;
 import org.apache.calcite.sql.SqlBasicTypeNameSpec;
 import org.apache.calcite.sql.SqlCall;
@@ -35,7 +36,6 @@ import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
-import org.apache.calcite.sql.SqlUserDefinedTypeNameSpec;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.fun.SqlCase;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -133,7 +133,10 @@ public class MysqlSqlDialect extends SqlDialect {
     case INTEGER:
     case BIGINT:
       return new SqlDataTypeSpec(
-          new SqlUserDefinedTypeNameSpec("_SIGNED", SqlParserPos.ZERO),
+          new SqlAlienSystemTypeNameSpec(
+              "SIGNED",
+              type.getSqlTypeName(),
+              SqlParserPos.ZERO),
           SqlParserPos.ZERO);
     }
     return super.getCastSpec(type);
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java
index 6c6640c..0454b6b 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/OracleSqlDialect.java
@@ -21,6 +21,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
 import org.apache.calcite.sql.SqlAbstractDateTimeLiteral;
+import org.apache.calcite.sql.SqlAlienSystemTypeNameSpec;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDateLiteral;
@@ -29,7 +30,6 @@ import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlTimeLiteral;
 import org.apache.calcite.sql.SqlTimestampLiteral;
-import org.apache.calcite.sql.SqlUserDefinedTypeNameSpec;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.fun.SqlFloorFunction;
@@ -89,23 +89,23 @@ public class OracleSqlDialect extends SqlDialect {
     String castSpec;
     switch (type.getSqlTypeName()) {
     case SMALLINT:
-      castSpec = "_NUMBER(5)";
+      castSpec = "NUMBER(5)";
       break;
     case INTEGER:
-      castSpec = "_NUMBER(10)";
+      castSpec = "NUMBER(10)";
       break;
     case BIGINT:
-      castSpec = "_NUMBER(19)";
+      castSpec = "NUMBER(19)";
       break;
     case DOUBLE:
-      castSpec = "_DOUBLE PRECISION";
+      castSpec = "DOUBLE PRECISION";
       break;
     default:
       return super.getCastSpec(type);
     }
 
     return new SqlDataTypeSpec(
-        new SqlUserDefinedTypeNameSpec(castSpec, SqlParserPos.ZERO),
+        new SqlAlienSystemTypeNameSpec(castSpec, type.getSqlTypeName(), SqlParserPos.ZERO),
         SqlParserPos.ZERO);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
index 0e40231..4a0bbef 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
@@ -21,12 +21,12 @@ import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+import org.apache.calcite.sql.SqlAlienSystemTypeNameSpec;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlUserDefinedTypeNameSpec;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.fun.SqlFloorFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -77,18 +77,18 @@ public class PostgresqlSqlDialect extends SqlDialect {
     switch (type.getSqlTypeName()) {
     case TINYINT:
       // Postgres has no tinyint (1 byte), so instead cast to smallint (2 bytes)
-      castSpec = "_smallint";
+      castSpec = "smallint";
       break;
     case DOUBLE:
       // Postgres has a double type but it is named differently
-      castSpec = "_double precision";
+      castSpec = "double precision";
       break;
     default:
       return super.getCastSpec(type);
     }
 
     return new SqlDataTypeSpec(
-        new SqlUserDefinedTypeNameSpec(castSpec, SqlParserPos.ZERO),
+        new SqlAlienSystemTypeNameSpec(castSpec, type.getSqlTypeName(), SqlParserPos.ZERO),
         SqlParserPos.ZERO);
   }