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

mmior 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 8eb8520  CALCITE-2856-Emulating Cross Join For SparkSqlDialect
8eb8520 is described below

commit 8eb852039db04c132ae7a99943495f87cf39dfd2
Author: stuti.gupta <[email protected]>
AuthorDate: Mon Feb 18 16:29:59 2019 +0530

    CALCITE-2856-Emulating Cross Join For SparkSqlDialect
---
 .../calcite/rel/rel2sql/RelToSqlConverter.java     |  8 ++-
 .../java/org/apache/calcite/sql/SqlDialect.java    |  5 ++
 .../apache/calcite/sql/SqlDialectFactoryImpl.java  |  7 +++
 .../calcite/sql/dialect/SparkSqlDialect.java       | 60 ++++++++++++++++++++++
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java | 12 +++++
 5 files changed, 90 insertions(+), 2 deletions(-)

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 4ee45fd..1e46fd6 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
@@ -126,8 +126,8 @@ public class RelToSqlConverter extends SqlImplementor
     SqlNode sqlCondition = null;
     SqlLiteral condType = JoinConditionType.ON.symbol(POS);
     JoinType joinType = joinType(e.getJoinType());
-    if (e.getJoinType() == JoinRelType.INNER && 
e.getCondition().isAlwaysTrue()) {
-      joinType = JoinType.COMMA;
+    if (isCrossJoin(e)) {
+      joinType = dialect.emulateJoinTypeForCrossJoin();
       condType = JoinConditionType.NONE.symbol(POS);
     } else {
       sqlCondition = convertConditionToSqlNode(e.getCondition(),
@@ -146,6 +146,10 @@ public class RelToSqlConverter extends SqlImplementor
     return result(join, leftResult, rightResult);
   }
 
+  private boolean isCrossJoin(final Join e) {
+    return e.getJoinType() == JoinRelType.INNER && 
e.getCondition().isAlwaysTrue();
+  }
+
   /** @see #dispatch */
   public Result visit(Filter e) {
     final RelNode input = e.getInput();
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 f4de22d..231a9a4 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -741,6 +741,10 @@ public class SqlDialect {
     return null;
   }
 
+  public JoinType emulateJoinTypeForCrossJoin() {
+    return JoinType.COMMA;
+  }
+
   protected SqlNode emulateNullDirectionWithIsNull(SqlNode node,
       boolean nullsFirst, boolean desc) {
     // No need for emulation if the nulls will anyways come out the way we want
@@ -986,6 +990,7 @@ public class SqlDialect {
     HSQLDB("Hsqldb", null, NullCollation.HIGH),
     VERTICA("Vertica", "\"", NullCollation.HIGH),
     SQLSTREAM("SQLstream", "\"", NullCollation.HIGH),
+    SPARK("Spark", null, NullCollation.LOW),
 
     /** Paraccel, now called Actian Matrix. Redshift is based on this, so
      * presumably the dialect capabilities are similar. */
diff --git 
a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java 
b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
index db5b505..f26f940 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
@@ -42,6 +42,7 @@ import org.apache.calcite.sql.dialect.ParaccelSqlDialect;
 import org.apache.calcite.sql.dialect.PhoenixSqlDialect;
 import org.apache.calcite.sql.dialect.PostgresqlSqlDialect;
 import org.apache.calcite.sql.dialect.RedshiftSqlDialect;
+import org.apache.calcite.sql.dialect.SparkSqlDialect;
 import org.apache.calcite.sql.dialect.SybaseSqlDialect;
 import org.apache.calcite.sql.dialect.TeradataSqlDialect;
 import org.apache.calcite.sql.dialect.VerticaSqlDialect;
@@ -116,6 +117,8 @@ public class SqlDialectFactoryImpl implements 
SqlDialectFactory {
       return new MysqlSqlDialect(c);
     case "REDSHIFT":
       return new RedshiftSqlDialect(c);
+    case "SPARK":
+      return new SparkSqlDialect(c);
     }
     // Now the fuzzy matches.
     if (databaseProductName.startsWith("DB2")) {
@@ -144,6 +147,8 @@ public class SqlDialectFactoryImpl implements 
SqlDialectFactory {
       return new H2SqlDialect(c);
     } else if (upperProductName.contains("VERTICA")) {
       return new VerticaSqlDialect(c);
+    } else if (upperProductName.contains("SPARK")) {
+      return new SparkSqlDialect(c);
     } else {
       return new AnsiSqlDialect(c);
     }
@@ -232,6 +237,8 @@ public class SqlDialectFactoryImpl implements 
SqlDialectFactory {
       return TeradataSqlDialect.DEFAULT;
     case VERTICA:
       return VerticaSqlDialect.DEFAULT;
+    case SPARK:
+      return SparkSqlDialect.DEFAULT;
     case SQLSTREAM:
     case UNKNOWN:
     default:
diff --git 
a/core/src/main/java/org/apache/calcite/sql/dialect/SparkSqlDialect.java 
b/core/src/main/java/org/apache/calcite/sql/dialect/SparkSqlDialect.java
new file mode 100644
index 0000000..f37eb5a
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/SparkSqlDialect.java
@@ -0,0 +1,60 @@
+/*
+ * 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.dialect;
+
+import org.apache.calcite.config.NullCollation;
+import org.apache.calcite.sql.JoinType;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+
+/**
+ * A <code>SqlDialect</code> implementation for the APACHE SPARK database.
+ */
+public class SparkSqlDialect extends SqlDialect {
+  public static final SqlDialect DEFAULT =
+      new SparkSqlDialect(EMPTY_CONTEXT
+          .withDatabaseProduct(DatabaseProduct.SPARK)
+          .withNullCollation(NullCollation.LOW));
+
+  /**
+   * Creates a SparkSqlDialect.
+   */
+  public SparkSqlDialect(SqlDialect.Context context) {
+    super(context);
+  }
+
+  @Override protected boolean allowsAs() {
+    return false;
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override public JoinType emulateJoinTypeForCrossJoin() {
+    return JoinType.CROSS;
+  }
+
+
+  @Override public void unparseOffsetFetch(SqlWriter writer, SqlNode offset,
+      SqlNode fetch) {
+    unparseFetchUsingLimit(writer, offset, fetch);
+  }
+}
+
+// End SparkSqlDialect.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 b5afdf5..0932f37 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
@@ -3046,6 +3046,14 @@ public class RelToSqlConverterTest {
     sql(query).ok(expected);
   }
 
+  @Test public void testCrossJoinEmulationForSpark() {
+    String query = "select * from \"employee\", \"department\"";
+    final String expected = "SELECT *\n"
+        + "FROM foodmart.employee\n"
+        + "CROSS JOIN foodmart.department";
+    sql(query).withSpark().ok(expected);
+  }
+
   /** Fluid interface to run tests. */
   static class Sql {
     private final SchemaPlus schema;
@@ -3115,6 +3123,10 @@ public class RelToSqlConverterTest {
       return dialect(SqlDialect.DatabaseProduct.BIG_QUERY.getDialect());
     }
 
+    Sql withSpark() {
+      return dialect(DatabaseProduct.SPARK.getDialect());
+    }
+
     Sql withPostgresqlModifiedTypeSystem() {
       // Postgresql dialect with max length for varchar set to 256
       final PostgresqlSqlDialect postgresqlSqlDialect =

Reply via email to