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 2015/12/10 05:53:11 UTC

[2/5] calcite git commit: Deduce dialect's null collation from DatabaseMetaData

Deduce dialect's null collation from DatabaseMetaData


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

Branch: refs/heads/master
Commit: eedd3dc4cc2c28eb05bc42edeaaeb196920da864
Parents: a8a8878
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Dec 7 16:23:24 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 9 13:35:29 2015 -0800

----------------------------------------------------------------------
 .../apache/calcite/adapter/jdbc/JdbcUtils.java  |   8 +-
 .../calcite/rel/rel2sql/RelToSqlConverter.java  |  27 ++--
 .../java/org/apache/calcite/sql/SqlDialect.java | 145 ++++++++++++++-----
 .../rel/rel2sql/RelToSqlConverterTest.java      |   2 +
 .../org/apache/calcite/tools/PlannerTest.java   |   2 +-
 5 files changed, 130 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/eedd3dc4/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
index da5eef8..0bbd023 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcUtils.java
@@ -77,13 +77,7 @@ final class JdbcUtils {
         List key = ImmutableList.of(productName, productVersion);
         SqlDialect dialect = map.get(key);
         if (dialect == null) {
-          final SqlDialect.DatabaseProduct product =
-              SqlDialect.getProduct(productName, productVersion);
-          dialect =
-              new SqlDialect(
-                  product,
-                  productName,
-                  metaData.getIdentifierQuoteString());
+          dialect = SqlDialect.create(metaData);
           map.put(key, dialect);
           map0.put(dataSource, dialect);
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/eedd3dc4/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
index 12f9ab0..2a5fdba 100644
--- a/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rel2sql/RelToSqlConverter.java
@@ -102,7 +102,7 @@ public class RelToSqlConverter {
   public static final SqlParserPos POS = SqlParserPos.ZERO;
 
   final SqlDialect dialect;
-  private final Set<String> aliasSet = new LinkedHashSet<String>();
+  private final Set<String> aliasSet = new LinkedHashSet<>();
 
   private final Map<String, SqlNode> ordinalMap = new HashMap<>();
 
@@ -130,8 +130,7 @@ public class RelToSqlConverter {
    * relational expressions.)
    */
   public Result result(SqlNode join, Result leftResult, Result rightResult) {
-    final List<Pair<String, RelDataType>> list =
-        new ArrayList<Pair<String, RelDataType>>();
+    final List<Pair<String, RelDataType>> list = new ArrayList<>();
     list.addAll(leftResult.aliases);
     list.addAll(rightResult.aliases);
     return new Result(join, Expressions.list(Clause.FROM), null, list);
@@ -351,7 +350,8 @@ public class RelToSqlConverter {
             ISNULL_FUNCTION.createCall(POS,
                 builder.context.field(fieldCollation.getFieldIndex())));
         fieldCollation = new RelFieldCollation(fieldCollation.getFieldIndex(),
-            fieldCollation.getDirection());
+            fieldCollation.getDirection(),
+            RelFieldCollation.NullDirection.UNSPECIFIED);
       }
       orderByList.add(builder.context.toSql(fieldCollation));
     }
@@ -796,13 +796,15 @@ public class RelToSqlConverter {
       case STRICTLY_DESCENDING:
         node = SqlStdOperatorTable.DESC.createCall(POS, node);
       }
-      switch (collation.nullDirection) {
-      case FIRST:
-        node = SqlStdOperatorTable.NULLS_FIRST.createCall(POS, node);
-        break;
-      case LAST:
-        node = SqlStdOperatorTable.NULLS_LAST.createCall(POS, node);
-        break;
+      if (collation.nullDirection != dialect.defaultNullDirection(collation.direction)) {
+        switch (collation.nullDirection) {
+        case FIRST:
+          node = SqlStdOperatorTable.NULLS_FIRST.createCall(POS, node);
+          break;
+        case LAST:
+          node = SqlStdOperatorTable.NULLS_LAST.createCall(POS, node);
+          break;
+        }
       }
       return node;
     }
@@ -918,7 +920,6 @@ public class RelToSqlConverter {
       final SqlNodeList selectList = select.getSelectList();
       if (selectList != null) {
         newContext = new Context(selectList.size()) {
-          @Override
           public SqlNode field(int ordinal) {
             final SqlNode selectItem = selectList.get(ordinal);
             switch (selectItem.getKind()) {
@@ -1056,3 +1057,5 @@ public class RelToSqlConverter {
     FROM, WHERE, GROUP_BY, HAVING, SELECT, SET_OP, ORDER_BY, FETCH, OFFSET
   }
 }
+
+// End RelToSqlConverter.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/eedd3dc4/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 f33fd1e..bacf239 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -16,6 +16,11 @@
  */
 package org.apache.calcite.sql;
 
+import org.apache.calcite.config.NullCollation;
+import org.apache.calcite.rel.RelFieldCollation;
+
+import com.google.common.base.Preconditions;
+
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.sql.Timestamp;
@@ -56,6 +61,7 @@ public class SqlDialect {
   private final String identifierEndQuoteString;
   private final String identifierEscapedQuote;
   private final DatabaseProduct databaseProduct;
+  private final NullCollation nullCollation;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -82,10 +88,32 @@ public class SqlDialect {
     } catch (SQLException e) {
       throw FakeUtil.newInternal(e, "while detecting database product");
     }
+    String databaseProductVersion;
+    try {
+      databaseProductVersion = databaseMetaData.getDatabaseProductVersion();
+    } catch (SQLException e) {
+      throw FakeUtil.newInternal(e, "while detecting database version");
+    }
     final DatabaseProduct databaseProduct =
-        getProduct(databaseProductName, null);
-    return new SqlDialect(
-        databaseProduct, databaseProductName, identifierQuoteString);
+        getProduct(databaseProductName, databaseProductVersion);
+    NullCollation nullCollation;
+    try {
+      if (databaseMetaData.nullsAreSortedAtEnd()) {
+        nullCollation = NullCollation.LAST;
+      } else if (databaseMetaData.nullsAreSortedAtStart()) {
+        nullCollation = NullCollation.FIRST;
+      } else if (databaseMetaData.nullsAreSortedLow()) {
+        nullCollation = NullCollation.LOW;
+      } else if (databaseMetaData.nullsAreSortedHigh()) {
+        nullCollation = NullCollation.HIGH;
+      } else {
+        throw new IllegalArgumentException("cannot deduce null collation");
+      }
+    } catch (SQLException e) {
+      throw new IllegalArgumentException("cannot deduce null collation", e);
+    }
+    return new SqlDialect(databaseProduct, databaseProductName,
+        identifierQuoteString, nullCollation);
   }
 
   /**
@@ -97,13 +125,32 @@ public class SqlDialect {
    *                              is not supported. If "[", close quote is
    *                              deemed to be "]".
    */
+  @Deprecated // to be removed before 2.0
   public SqlDialect(
       DatabaseProduct databaseProduct,
       String databaseProductName,
       String identifierQuoteString) {
-    assert databaseProduct != null;
-    assert databaseProductName != null;
-    this.databaseProduct = databaseProduct;
+    this(databaseProduct, databaseProductName, identifierQuoteString,
+        NullCollation.HIGH);
+  }
+
+  /**
+   * Creates a SqlDialect.
+   *
+   * @param databaseProduct       Database product; may be UNKNOWN, never null
+   * @param databaseProductName   Database product name from JDBC driver
+   * @param identifierQuoteString String to quote identifiers. Null if quoting
+   *                              is not supported. If "[", close quote is
+   *                              deemed to be "]".
+   * @param nullCollation         Whether NULL values appear first or last
+   */
+  public SqlDialect(
+      DatabaseProduct databaseProduct,
+      String databaseProductName,
+      String identifierQuoteString, NullCollation nullCollation) {
+    Preconditions.checkNotNull(this.nullCollation = nullCollation);
+    Preconditions.checkNotNull(databaseProductName);
+    this.databaseProduct = Preconditions.checkNotNull(databaseProduct);
     if (identifierQuoteString != null) {
       identifierQuoteString = identifierQuoteString.trim();
       if (identifierQuoteString.equals("")) {
@@ -419,6 +466,32 @@ public class SqlDialect {
     }
   }
 
+  /** Returns how NULL values are sorted if an ORDER BY item does not contain
+   * NULLS ASCENDING or NULLS DESCENDING. */
+  public NullCollation getNullCollation() {
+    return nullCollation;
+  }
+
+  /** Returns whether NULL values are sorted first or last, in this dialect,
+   * in an ORDER BY item of a given direction. */
+  public RelFieldCollation.NullDirection defaultNullDirection(
+      RelFieldCollation.Direction direction) {
+    switch (direction) {
+    case ASCENDING:
+    case STRICTLY_ASCENDING:
+      return getNullCollation().last(false)
+          ? RelFieldCollation.NullDirection.LAST
+          : RelFieldCollation.NullDirection.FIRST;
+    case DESCENDING:
+    case STRICTLY_DESCENDING:
+      return getNullCollation().last(true)
+          ? RelFieldCollation.NullDirection.LAST
+          : RelFieldCollation.NullDirection.FIRST;
+    default:
+      return RelFieldCollation.NullDirection.UNSPECIFIED;
+    }
+  }
+
   /**
    * A few utility functions copied from org.apache.calcite.util.Util. We have
    * copied them because we wish to keep SqlDialect's dependencies to a
@@ -478,31 +551,31 @@ public class SqlDialect {
    * whether the database allows expressions to appear in the GROUP BY clause.
    */
   public enum DatabaseProduct {
-    ACCESS("Access", "\""),
-    CALCITE("Apache Calcite", "\""),
-    MSSQL("Microsoft SQL Server", "["),
-    MYSQL("MySQL", "`"),
-    ORACLE("Oracle", "\""),
-    DERBY("Apache Derby", null),
-    DB2("IBM DB2", null),
-    FIREBIRD("Firebird", null),
-    H2("H2", "\""),
-    HIVE("Apache Hive", null),
-    INFORMIX("Informix", null),
-    INGRES("Ingres", null),
-    LUCIDDB("LucidDB", "\""),
-    INTERBASE("Interbase", null),
-    PHOENIX("Phoenix", "\""),
-    POSTGRESQL("PostgreSQL", "\""),
-    NETEZZA("Netezza", "\""),
-    INFOBRIGHT("Infobright", "`"),
-    NEOVIEW("Neoview", null),
-    SYBASE("Sybase", null),
-    TERADATA("Teradata", "\""),
-    HSQLDB("Hsqldb", null),
-    VERTICA("Vertica", "\""),
-    SQLSTREAM("SQLstream", "\""),
-    PARACCEL("Paraccel", "\""),
+    ACCESS("Access", "\"", NullCollation.HIGH),
+    CALCITE("Apache Calcite", "\"", NullCollation.HIGH),
+    MSSQL("Microsoft SQL Server", "[", NullCollation.HIGH),
+    MYSQL("MySQL", "`", NullCollation.HIGH),
+    ORACLE("Oracle", "\"", NullCollation.HIGH),
+    DERBY("Apache Derby", null, NullCollation.HIGH),
+    DB2("IBM DB2", null, NullCollation.HIGH),
+    FIREBIRD("Firebird", null, NullCollation.HIGH),
+    H2("H2", "\"", NullCollation.HIGH),
+    HIVE("Apache Hive", null, NullCollation.HIGH),
+    INFORMIX("Informix", null, NullCollation.HIGH),
+    INGRES("Ingres", null, NullCollation.HIGH),
+    LUCIDDB("LucidDB", "\"", NullCollation.HIGH),
+    INTERBASE("Interbase", null, NullCollation.HIGH),
+    PHOENIX("Phoenix", "\"", NullCollation.HIGH),
+    POSTGRESQL("PostgreSQL", "\"", NullCollation.HIGH),
+    NETEZZA("Netezza", "\"", NullCollation.HIGH),
+    INFOBRIGHT("Infobright", "`", NullCollation.HIGH),
+    NEOVIEW("Neoview", null, NullCollation.HIGH),
+    SYBASE("Sybase", null, NullCollation.HIGH),
+    TERADATA("Teradata", "\"", NullCollation.HIGH),
+    HSQLDB("Hsqldb", null, NullCollation.HIGH),
+    VERTICA("Vertica", "\"", NullCollation.HIGH),
+    SQLSTREAM("SQLstream", "\"", NullCollation.HIGH),
+    PARACCEL("Paraccel", "\"", NullCollation.HIGH),
     /**
      * Placeholder for the unknown database.
      *
@@ -510,15 +583,18 @@ public class SqlDialect {
      * do something database-specific like quoting identifiers, don't rely
      * on this dialect to do what you want.
      */
-    UNKNOWN("Unknown", "`");
+    UNKNOWN("Unknown", "`", NullCollation.HIGH);
 
     private SqlDialect dialect = null;
     private String databaseProductName;
     private String quoteString;
+    private final NullCollation nullCollation;
 
-    DatabaseProduct(String databaseProductName, String quoteString) {
+    DatabaseProduct(String databaseProductName, String quoteString,
+        NullCollation nullCollation) {
       this.databaseProductName = databaseProductName;
       this.quoteString = quoteString;
+      this.nullCollation = nullCollation;
     }
 
     /**
@@ -535,7 +611,8 @@ public class SqlDialect {
     public SqlDialect getDialect() {
       if (dialect == null) {
         dialect =
-            new SqlDialect(this, databaseProductName, quoteString);
+            new SqlDialect(this, databaseProductName, quoteString,
+                nullCollation);
       }
       return dialect;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/eedd3dc4/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 33858fa..7514791 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
@@ -345,3 +345,5 @@ public class RelToSqlConverterTest {
             + "ON \"department\".\"department_id\" = \"t1\".\"department_id\"");
   }
 }
+
+// End RelToSqlConverterTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/eedd3dc4/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index 01816df..b769653 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -578,7 +578,7 @@ public class PlannerTest {
         "select * from (select * from \"emps\") as t\n"
             + "where \"name\" like '%e%'");
     final SqlDialect hiveDialect =
-        new SqlDialect(SqlDialect.DatabaseProduct.HIVE, "Hive", null);
+        SqlDialect.DatabaseProduct.HIVE.getDialect();
     assertThat(Util.toLinux(parse.toSqlString(hiveDialect).getSql()),
         equalTo("SELECT *\n"
             + "FROM (SELECT *\n"