You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jc...@apache.org on 2018/12/04 00:19:17 UTC

calcite git commit: [CALCITE-2713] JDBC adapter may generate casts on PostgreSQL for VARCHAR type exceeding max length

Repository: calcite
Updated Branches:
  refs/heads/master be5404713 -> 6c7a7edd1


[CALCITE-2713] JDBC adapter may generate casts on PostgreSQL for VARCHAR type exceeding max length

Close apache/calcite#944


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

Branch: refs/heads/master
Commit: 6c7a7edd1b9b11f2c0826db04d60cc2b62bea636
Parents: be54047
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Mon Dec 3 14:15:52 2018 -0800
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Mon Dec 3 16:19:00 2018 -0800

----------------------------------------------------------------------
 .../java/org/apache/calcite/sql/SqlDialect.java | 52 ++++++++++++++++----
 .../sql/dialect/PostgresqlSqlDialect.java       | 26 +++++++++-
 .../rel/rel2sql/RelToSqlConverterTest.java      | 47 ++++++++++++++++++
 3 files changed, 114 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/6c7a7edd/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
----------------------------------------------------------------------
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 c70aeeb..819c9a9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -23,6 +23,7 @@ import org.apache.calcite.linq4j.function.Experimental;
 import org.apache.calcite.rel.RelFieldCollation;
 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.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.dialect.JethroDataSqlDialect;
@@ -133,6 +134,7 @@ public class SqlDialect {
   private final String identifierEscapedQuote;
   private final DatabaseProduct databaseProduct;
   protected final NullCollation nullCollation;
+  private final RelDataTypeSystem dataTypeSystem;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -190,6 +192,7 @@ public class SqlDialect {
    */
   public SqlDialect(Context context) {
     this.nullCollation = Objects.requireNonNull(context.nullCollation());
+    this.dataTypeSystem = Objects.requireNonNull(context.dataTypeSystem());
     this.databaseProduct =
         Objects.requireNonNull(context.databaseProduct());
     String identifierQuoteString = context.identifierQuoteString();
@@ -214,7 +217,7 @@ public class SqlDialect {
   /** Creates an empty context. Use {@link #EMPTY_CONTEXT} if possible. */
   protected static Context emptyContext() {
     return new ContextImpl(DatabaseProduct.UNKNOWN, null, null, -1, -1, null,
-        NullCollation.HIGH, JethroDataSqlDialect.JethroInfo.EMPTY);
+        NullCollation.HIGH, RelDataTypeSystemImpl.DEFAULT, JethroDataSqlDialect.JethroInfo.EMPTY);
   }
 
   /**
@@ -291,6 +294,11 @@ public class SqlDialect {
     }
   }
 
+  /** Returns the type system implementation for this dialect. */
+  public RelDataTypeSystem getTypeSystem() {
+    return dataTypeSystem;
+  }
+
   /**
    * Encloses an identifier in quotation marks appropriate for the current SQL
    * dialect.
@@ -684,9 +692,18 @@ public class SqlDialect {
 
   public SqlNode getCastSpec(RelDataType type) {
     if (type instanceof BasicSqlType) {
+      int precision = type.getPrecision();
+      switch (type.getSqlTypeName()) {
+      case VARCHAR:
+        // if needed, adjust varchar length to max length supported by the system
+        int maxPrecision = getTypeSystem().getMaxPrecision(type.getSqlTypeName());
+        if (type.getPrecision() > maxPrecision) {
+          precision = maxPrecision;
+        }
+      }
       return new SqlDataTypeSpec(
           new SqlIdentifier(type.getSqlTypeName().name(), SqlParserPos.ZERO),
-              type.getPrecision(),
+              precision,
               type.getScale(),
               type.getCharset() != null
                   && supportsCharSet()
@@ -1037,6 +1054,8 @@ public class SqlDialect {
     Context withIdentifierQuoteString(String identifierQuoteString);
     @Nonnull NullCollation nullCollation();
     Context withNullCollation(@Nonnull NullCollation nullCollation);
+    @Nonnull RelDataTypeSystem dataTypeSystem();
+    Context withDataTypeSystem(@Nonnull RelDataTypeSystem dataTypeSystem);
     JethroDataSqlDialect.JethroInfo jethroInfo();
     Context withJethroInfo(JethroDataSqlDialect.JethroInfo jethroInfo);
   }
@@ -1050,12 +1069,14 @@ public class SqlDialect {
     private final int databaseMinorVersion;
     private final String identifierQuoteString;
     private final NullCollation nullCollation;
+    private final RelDataTypeSystem dataTypeSystem;
     private final JethroDataSqlDialect.JethroInfo jethroInfo;
 
     private ContextImpl(DatabaseProduct databaseProduct,
         String databaseProductName, String databaseVersion,
         int databaseMajorVersion, int databaseMinorVersion,
         String identifierQuoteString, NullCollation nullCollation,
+        RelDataTypeSystem dataTypeSystem,
         JethroDataSqlDialect.JethroInfo jethroInfo) {
       this.databaseProduct = Objects.requireNonNull(databaseProduct);
       this.databaseProductName = databaseProductName;
@@ -1064,6 +1085,7 @@ public class SqlDialect {
       this.databaseMinorVersion = databaseMinorVersion;
       this.identifierQuoteString = identifierQuoteString;
       this.nullCollation = Objects.requireNonNull(nullCollation);
+      this.dataTypeSystem = Objects.requireNonNull(dataTypeSystem);
       this.jethroInfo = Objects.requireNonNull(jethroInfo);
     }
 
@@ -1075,7 +1097,7 @@ public class SqlDialect {
         @Nonnull DatabaseProduct databaseProduct) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     public String databaseProductName() {
@@ -1085,7 +1107,7 @@ public class SqlDialect {
     public Context withDatabaseProductName(String databaseProductName) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     public String databaseVersion() {
@@ -1095,7 +1117,7 @@ public class SqlDialect {
     public Context withDatabaseVersion(String databaseVersion) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     public int databaseMajorVersion() {
@@ -1105,7 +1127,7 @@ public class SqlDialect {
     public Context withDatabaseMajorVersion(int databaseMajorVersion) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     public int databaseMinorVersion() {
@@ -1115,7 +1137,7 @@ public class SqlDialect {
     public Context withDatabaseMinorVersion(int databaseMinorVersion) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     public String identifierQuoteString() {
@@ -1125,7 +1147,7 @@ public class SqlDialect {
     public Context withIdentifierQuoteString(String identifierQuoteString) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     @Nonnull public NullCollation nullCollation() {
@@ -1135,7 +1157,17 @@ public class SqlDialect {
     public Context withNullCollation(@Nonnull NullCollation nullCollation) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
+    }
+
+    @Nonnull public RelDataTypeSystem dataTypeSystem() {
+      return dataTypeSystem;
+    }
+
+    public Context withDataTypeSystem(@Nonnull RelDataTypeSystem dataTypeSystem) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, databaseMajorVersion, databaseMinorVersion,
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
 
     @Nonnull public JethroDataSqlDialect.JethroInfo jethroInfo() {
@@ -1145,7 +1177,7 @@ public class SqlDialect {
     public Context withJethroInfo(JethroDataSqlDialect.JethroInfo jethroInfo) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation, jethroInfo);
+          identifierQuoteString, nullCollation, dataTypeSystem, jethroInfo);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/6c7a7edd/core/src/main/java/org/apache/calcite/sql/dialect/PostgresqlSqlDialect.java
----------------------------------------------------------------------
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 61028a9..5f1b8d0 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
@@ -18,6 +18,8 @@ package org.apache.calcite.sql.dialect;
 
 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.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDialect;
@@ -27,15 +29,37 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.fun.SqlFloorFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * A <code>SqlDialect</code> implementation for the PostgreSQL database.
  */
 public class PostgresqlSqlDialect extends SqlDialect {
+
+  /** PostgreSQL type system. */
+  private static final RelDataTypeSystem POSTGRESQL_TYPE_SYSTEM =
+      new RelDataTypeSystemImpl() {
+        @Override public int getMaxPrecision(SqlTypeName typeName) {
+          switch (typeName) {
+          case VARCHAR:
+            // From htup_details.h in postgresql:
+            // MaxAttrSize is a somewhat arbitrary upper limit on the declared size of
+            // data fields of char(n) and similar types.  It need not have anything
+            // directly to do with the *actual* upper limit of varlena values, which
+            // is currently 1Gb (see TOAST structures in postgres.h).  I've set it
+            // at 10Mb which seems like a reasonable number --- tgl 8/6/00. */
+            return 10 * 1024 * 1024;
+          default:
+            return super.getMaxPrecision(typeName);
+          }
+        }
+      };
+
   public static final SqlDialect DEFAULT =
       new PostgresqlSqlDialect(EMPTY_CONTEXT
           .withDatabaseProduct(DatabaseProduct.POSTGRESQL)
-          .withIdentifierQuoteString("\""));
+          .withIdentifierQuoteString("\"")
+          .withDataTypeSystem(POSTGRESQL_TYPE_SYSTEM));
 
   /** Creates a PostgresqlSqlDialect. */
   public PostgresqlSqlDialect(Context context) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/6c7a7edd/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index 93676c1..ca27782 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -24,12 +24,14 @@ import org.apache.calcite.plan.hep.HepProgram;
 import org.apache.calcite.plan.hep.HepProgramBuilder;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.rules.UnionMergeRule;
+import org.apache.calcite.rel.type.RelDataTypeSystemImpl;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlDialect.Context;
+import org.apache.calcite.sql.SqlDialect.DatabaseProduct;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlWriter;
@@ -37,8 +39,10 @@ import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.dialect.HiveSqlDialect;
 import org.apache.calcite.sql.dialect.JethroDataSqlDialect;
 import org.apache.calcite.sql.dialect.MysqlSqlDialect;
+import org.apache.calcite.sql.dialect.PostgresqlSqlDialect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
+import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.test.CalciteAssert;
 import org.apache.calcite.test.RelBuilderTest;
@@ -268,6 +272,30 @@ public class RelToSqlConverterTest {
   }
 
   /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2713">[CALCITE-2713]
+   * JDBC adapter may generate casts on PostgreSQL for VARCHAR type exceeding
+   * max length</a>. */
+  @Test public void testCastLongVarchar1() {
+    final String query = "select cast(\"store_id\" as VARCHAR(10485761))\n"
+        + " from \"expense_fact\"";
+    final String expected = "SELECT CAST(\"store_id\" AS VARCHAR(256))\n"
+        + "FROM \"foodmart\".\"expense_fact\"";
+    sql(query).withPostgresqlModifiedTypeSystem().ok(expected);
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2713">[CALCITE-2713]
+   * JDBC adapter may generate casts on PostgreSQL for VARCHAR type exceeding
+   * max length</a>. */
+  @Test public void testCastLongVarchar2() {
+    final String query = "select cast(\"store_id\" as VARCHAR(175))\n"
+        + " from \"expense_fact\"";
+    final String expected = "SELECT CAST(\"store_id\" AS VARCHAR(175))\n"
+        + "FROM \"foodmart\".\"expense_fact\"";
+    sql(query).withPostgresqlModifiedTypeSystem().ok(expected);
+  }
+
+  /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-1174">[CALCITE-1174]
    * When generating SQL, translate SUM0(x) to COALESCE(SUM(x), 0)</a>. */
   @Test public void testSum0BecomesCoalesce() {
@@ -2980,6 +3008,25 @@ public class RelToSqlConverterTest {
       return dialect(SqlDialect.DatabaseProduct.VERTICA.getDialect());
     }
 
+    Sql withPostgresqlModifiedTypeSystem() {
+      // Postgresql dialect with max length for varchar set to 256
+      final PostgresqlSqlDialect postgresqlSqlDialect =
+          new PostgresqlSqlDialect(SqlDialect.EMPTY_CONTEXT
+              .withDatabaseProduct(DatabaseProduct.POSTGRESQL)
+              .withIdentifierQuoteString("\"")
+              .withDataTypeSystem(new RelDataTypeSystemImpl() {
+                @Override public int getMaxPrecision(SqlTypeName typeName) {
+                  switch (typeName) {
+                    case VARCHAR:
+                      return 256;
+                    default:
+                      return super.getMaxPrecision(typeName);
+                  }
+                }
+              }));
+      return dialect(postgresqlSqlDialect);
+    }
+
     Sql config(SqlToRelConverter.Config config) {
       return new Sql(schema, sql, dialect, config, transforms);
     }