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 2020/06/11 07:18:07 UTC

[calcite] branch master updated: [CALCITE-3724] Presto dialect implementation

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 f577b7e  [CALCITE-3724] Presto dialect implementation
f577b7e is described below

commit f577b7e3d91191051cfdaade27e0a74f3603648a
Author: XuQianJin-Stars <x1...@163.com>
AuthorDate: Thu Jan 30 14:49:20 2020 +0800

    [CALCITE-3724] Presto dialect implementation
    
    Fixup (by Danny):
    - Add a new tool RelToSqlConverterUtil#specialOperatorByName and remove
    ClickHouseSqlDialect.CLICKHOUSE_SUBSTRING
    - Remove the common code in PrestoSqlDialect and reuse codes from other
    dialect instances
    
    close apache#calcite#1776
---
 .../java/org/apache/calcite/sql/SqlDialect.java    |  11 ++
 .../apache/calcite/sql/SqlDialectFactoryImpl.java  |   3 +
 .../calcite/sql/dialect/ClickHouseSqlDialect.java  |  24 +---
 .../calcite/sql/dialect/PrestoSqlDialect.java      | 129 +++++++++++++++++++++
 .../apache/calcite/util/RelToSqlConverterUtil.java |  23 ++++
 .../calcite/rel/rel2sql/RelToSqlConverterTest.java |  94 +++++++++++++--
 6 files changed, 251 insertions(+), 33 deletions(-)

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 2ef11f9..4924232 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -291,6 +291,8 @@ public class SqlDialect {
       return DatabaseProduct.ORACLE;
     case "PHOENIX":
       return DatabaseProduct.PHOENIX;
+    case "PRESTO":
+      return DatabaseProduct.PRESTO;
     case "MYSQL (INFOBRIGHT)":
       return DatabaseProduct.INFOBRIGHT;
     case "MYSQL":
@@ -921,6 +923,11 @@ public class SqlDialect {
   protected final void unparseFetchUsingLimit(SqlWriter writer, SqlNode offset,
       SqlNode fetch) {
     Preconditions.checkArgument(fetch != null || offset != null);
+    unparseLimit(writer, fetch);
+    unparseOffset(writer, offset);
+  }
+
+  protected final void unparseLimit(SqlWriter writer, SqlNode fetch) {
     if (fetch != null) {
       writer.newlineAndIndent();
       final SqlWriter.Frame fetchFrame =
@@ -929,6 +936,9 @@ public class SqlDialect {
       fetch.unparse(writer, -1, -1);
       writer.endList(fetchFrame);
     }
+  }
+
+  protected final void unparseOffset(SqlWriter writer, SqlNode offset) {
     if (offset != null) {
       writer.newlineAndIndent();
       final SqlWriter.Frame offsetFrame =
@@ -1245,6 +1255,7 @@ public class SqlDialect {
     INTERBASE("Interbase", null, NullCollation.HIGH),
     PHOENIX("Phoenix", "\"", NullCollation.HIGH),
     POSTGRESQL("PostgreSQL", "\"", NullCollation.HIGH),
+    PRESTO("Presto", "\"", NullCollation.LOW),
     NETEZZA("Netezza", "\"", NullCollation.HIGH),
     INFOBRIGHT("Infobright", "`", NullCollation.HIGH),
     NEOVIEW("Neoview", null, NullCollation.HIGH),
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 82d07cf..d150794 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
@@ -43,6 +43,7 @@ import org.apache.calcite.sql.dialect.OracleSqlDialect;
 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.PrestoSqlDialect;
 import org.apache.calcite.sql.dialect.RedshiftSqlDialect;
 import org.apache.calcite.sql.dialect.SnowflakeSqlDialect;
 import org.apache.calcite.sql.dialect.SparkSqlDialect;
@@ -289,6 +290,8 @@ public class SqlDialectFactoryImpl implements SqlDialectFactory {
       return PhoenixSqlDialect.DEFAULT;
     case POSTGRESQL:
       return PostgresqlSqlDialect.DEFAULT;
+    case PRESTO:
+      return PrestoSqlDialect.DEFAULT;
     case REDSHIFT:
       return RedshiftSqlDialect.DEFAULT;
     case SYBASE:
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/ClickHouseSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/ClickHouseSqlDialect.java
index a05799d..c78662c 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/ClickHouseSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/ClickHouseSqlDialect.java
@@ -25,10 +25,8 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDateLiteral;
 import org.apache.calcite.sql.SqlDialect;
-import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlTimeLiteral;
 import org.apache.calcite.sql.SqlTimestampLiteral;
 import org.apache.calcite.sql.SqlWriter;
@@ -36,6 +34,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 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.util.RelToSqlConverterUtil;
 
 import com.google.common.base.Preconditions;
 
@@ -50,24 +49,6 @@ public class ClickHouseSqlDialect extends SqlDialect {
 
   public static final SqlDialect DEFAULT = new ClickHouseSqlDialect(DEFAULT_CONTEXT);
 
-  private static final SqlSpecialOperator CLICKHOUSE_SUBSTRING =
-      new SqlSpecialOperator("substring", SqlKind.OTHER_FUNCTION) {
-        public void unparse(
-            SqlWriter writer,
-            SqlCall call,
-            int leftPrec,
-            int rightPrec) {
-          writer.print(getName());
-          final SqlWriter.Frame frame =
-              writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "(", ")");
-          for (SqlNode operand : call.getOperandList()) {
-            writer.sep(",");
-            operand.unparse(writer, 0, 0);
-          }
-          writer.endList(frame);
-        }
-      };
-
   /** Creates a ClickHouseSqlDialect. */
   public ClickHouseSqlDialect(Context context) {
     super(context);
@@ -166,7 +147,8 @@ public class ClickHouseSqlDialect extends SqlDialect {
   @Override public void unparseCall(SqlWriter writer, SqlCall call,
       int leftPrec, int rightPrec) {
     if (call.getOperator() == SqlStdOperatorTable.SUBSTRING) {
-      CLICKHOUSE_SUBSTRING.unparse(writer, call, 0, 0);
+      RelToSqlConverterUtil.specialOperatorByName("substring")
+          .unparse(writer, call, 0, 0);
     } else {
       switch (call.getKind()) {
       case FLOOR:
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/PrestoSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/PrestoSqlDialect.java
new file mode 100644
index 0000000..2e1b1d4
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/PrestoSqlDialect.java
@@ -0,0 +1,129 @@
+/*
+ * 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.avatica.util.Casing;
+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.SqlCall;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlWriter;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.RelToSqlConverterUtil;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * A <code>SqlDialect</code> implementation for the Presto database.
+ */
+public class PrestoSqlDialect extends SqlDialect {
+  public static final Context DEFAULT_CONTEXT = SqlDialect.EMPTY_CONTEXT
+      .withDatabaseProduct(DatabaseProduct.PRESTO)
+      .withIdentifierQuoteString("\"")
+      .withUnquotedCasing(Casing.UNCHANGED)
+      .withNullCollation(NullCollation.LOW);
+
+  public static final SqlDialect DEFAULT = new PrestoSqlDialect(DEFAULT_CONTEXT);
+
+  /**
+   * Creates a PrestoSqlDialect.
+   */
+  public PrestoSqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override public boolean requiresAliasForFromItems() {
+    return true;
+  }
+
+  @Override public void unparseOffsetFetch(SqlWriter writer, SqlNode offset,
+      SqlNode fetch) {
+    unparseUsingLimit(writer, offset, fetch);
+  }
+
+  /** Unparses offset/fetch using "OFFSET offset LIMIT fetch " syntax. */
+  private void unparseUsingLimit(SqlWriter writer, SqlNode offset,
+      SqlNode fetch) {
+    Preconditions.checkArgument(fetch != null || offset != null);
+    unparseOffset(writer, offset);
+    unparseLimit(writer, fetch);
+  }
+
+  @Override public SqlNode emulateNullDirection(SqlNode node,
+      boolean nullsFirst, boolean desc) {
+    return emulateNullDirectionWithIsNull(node, nullsFirst, desc);
+  }
+
+  @Override public boolean supportsAggregateFunction(SqlKind kind) {
+    switch (kind) {
+    case AVG:
+    case COUNT:
+    case CUBE:
+    case SUM:
+    case MIN:
+    case MAX:
+    case ROLLUP:
+      return true;
+    }
+    return false;
+  }
+
+  @Override public boolean supportsGroupByWithCube() {
+    return true;
+  }
+
+  @Override public boolean supportsNestedAggregations() {
+    return false;
+  }
+
+  @Override public boolean supportsGroupByWithRollup() {
+    return true;
+  }
+
+  @Override public CalendarPolicy getCalendarPolicy() {
+    return CalendarPolicy.SHIFT;
+  }
+
+  @Override public SqlNode getCastSpec(RelDataType type) {
+    return super.getCastSpec(type);
+  }
+
+  @Override public void unparseCall(SqlWriter writer, SqlCall call,
+      int leftPrec, int rightPrec) {
+    if (call.getOperator() == SqlStdOperatorTable.SUBSTRING) {
+      RelToSqlConverterUtil.specialOperatorByName("SUBSTR")
+          .unparse(writer, call, 0, 0);
+    } else {
+      // Current impl is same with Postgresql.
+      PostgresqlSqlDialect.DEFAULT.unparseCall(writer, call, leftPrec, rightPrec);
+    }
+  }
+
+  @Override public void unparseSqlIntervalQualifier(SqlWriter writer,
+      SqlIntervalQualifier qualifier, RelDataTypeSystem typeSystem) {
+    // Current impl is same with MySQL.
+    MysqlSqlDialect.DEFAULT.unparseSqlIntervalQualifier(writer, qualifier, typeSystem);
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/util/RelToSqlConverterUtil.java b/core/src/main/java/org/apache/calcite/util/RelToSqlConverterUtil.java
index bf4a53d..98f4bf7 100644
--- a/core/src/main/java/org/apache/calcite/util/RelToSqlConverterUtil.java
+++ b/core/src/main/java/org/apache/calcite/util/RelToSqlConverterUtil.java
@@ -18,8 +18,10 @@ package org.apache.calcite.util;
 
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSpecialOperator;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
@@ -135,4 +137,25 @@ public abstract class RelToSqlConverterUtil {
     }
     return inputString;
   }
+
+  /** Returns a {@link SqlSpecialOperator} with given operator name, mainly used for
+   * unparse override. */
+  public static SqlSpecialOperator specialOperatorByName(String opName) {
+    return new SqlSpecialOperator(opName, SqlKind.OTHER_FUNCTION) {
+      public void unparse(
+          SqlWriter writer,
+          SqlCall call,
+          int leftPrec,
+          int rightPrec) {
+        writer.print(getName());
+        final SqlWriter.Frame frame =
+            writer.startList(SqlWriter.FrameTypeEnum.FUN_CALL, "(", ")");
+        for (SqlNode operand : call.getOperandList()) {
+          writer.sep(",");
+          operand.unparse(writer, 0, 0);
+        }
+        writer.endList(frame);
+      }
+    };
+  }
 }
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 d4e1060..d8681db 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
@@ -177,6 +177,8 @@ class RelToSqlConverterTest {
             SqlDialect.DatabaseProduct.ORACLE)
         .put(SqlDialect.DatabaseProduct.POSTGRESQL.getDialect(),
             SqlDialect.DatabaseProduct.POSTGRESQL)
+        .put(DatabaseProduct.PRESTO.getDialect(),
+            DatabaseProduct.PRESTO)
         .build();
   }
 
@@ -267,14 +269,20 @@ class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\"";
     final String expectedMySql = "SELECT COUNT(*)\n"
         + "FROM `foodmart`.`product`";
+    final String expectedPresto = "SELECT COUNT(*)\n"
+        + "FROM \"foodmart\".\"product\"";
     sql(sql0)
         .ok(expected)
         .withMysql()
-        .ok(expectedMySql);
+        .ok(expectedMySql)
+        .withPresto()
+        .ok(expectedPresto);
     sql(sql1)
         .ok(expected)
         .withMysql()
-        .ok(expectedMySql);
+        .ok(expectedMySql)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   @Test void testSelectQueryWithGroupByEmpty2() {
@@ -285,10 +293,15 @@ class RelToSqlConverterTest {
     final String expectedMySql = "SELECT 42 AS `C`\n"
         + "FROM `foodmart`.`product`\n"
         + "GROUP BY ()";
+    final String expectedPresto = "SELECT 42 AS \"C\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY ()";
     sql(query)
         .ok(expected)
         .withMysql()
-        .ok(expectedMySql);
+        .ok(expectedMySql)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   /** Test case for
@@ -420,10 +433,17 @@ class RelToSqlConverterTest {
         + "GROUP BY `product_class_id` WITH ROLLUP\n"
         + "ORDER BY `product_class_id` IS NULL, `product_class_id`,"
         + " COUNT(*) IS NULL, COUNT(*)";
+    final String expectedPresto = "SELECT \"product_class_id\", COUNT(*) AS \"C\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY ROLLUP(\"product_class_id\")\n"
+        + "ORDER BY \"product_class_id\" IS NULL, \"product_class_id\", "
+        + "COUNT(*) IS NULL, COUNT(*)";
     sql(query)
         .ok(expected)
         .withMysql()
-        .ok(expectedMySql);
+        .ok(expectedMySql)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   /** As {@link #testSelectQueryWithSingletonCube()}, but no ORDER BY
@@ -438,10 +458,15 @@ class RelToSqlConverterTest {
     final String expectedMySql = "SELECT `product_class_id`, COUNT(*) AS `C`\n"
         + "FROM `foodmart`.`product`\n"
         + "GROUP BY `product_class_id` WITH ROLLUP";
+    final String expectedPresto = "SELECT \"product_class_id\", COUNT(*) AS \"C\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY ROLLUP(\"product_class_id\")";
     sql(query)
         .ok(expected)
         .withMysql()
-        .ok(expectedMySql);
+        .ok(expectedMySql)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   /** Cannot rewrite if ORDER BY contains a column not in GROUP BY (in this
@@ -486,10 +511,16 @@ class RelToSqlConverterTest {
         + "FROM `foodmart`.`product`\n"
         + "GROUP BY `product_class_id` WITH ROLLUP\n"
         + "LIMIT 5";
+    final String expectedPresto = "SELECT \"product_class_id\", COUNT(*) AS \"C\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY ROLLUP(\"product_class_id\")\n"
+        + "LIMIT 5";
     sql(query)
         .ok(expected)
         .withMysql()
-        .ok(expectedMySql);
+        .ok(expectedMySql)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   @Test void testSelectQueryWithMinAggregateFunction() {
@@ -2090,6 +2121,15 @@ class RelToSqlConverterTest {
         .ok(expected)
         .withClickHouse()
         .ok(expectedClickHouse);
+
+    final String expectedPresto = "SELECT \"product_id\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "OFFSET 10\n"
+        + "LIMIT 100";
+    sql(query)
+        .ok(expected)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   @Test void testSelectQueryWithLimitOffsetClause() {
@@ -2872,6 +2912,14 @@ class RelToSqlConverterTest {
         .ok(expected);
   }
 
+  @Test void testFloorPresto() {
+    String query = "SELECT floor(\"hire_date\" TO MINUTE) FROM \"employee\"";
+    String expected = "SELECT DATE_TRUNC('MINUTE', \"hire_date\")\nFROM \"foodmart\".\"employee\"";
+    sql(query)
+        .withPresto()
+        .ok(expected);
+  }
+
   @Test void testFloorMssqlWeek() {
     String query = "SELECT floor(\"hire_date\" TO WEEK) FROM \"employee\"";
     String expected = "SELECT CONVERT(DATETIME, CONVERT(VARCHAR(10), "
@@ -2900,7 +2948,7 @@ class RelToSqlConverterTest {
         .ok(expected);
   }
 
-  @Test public void testFloorWeek() {
+  @Test void testFloorWeek() {
     final String query = "SELECT floor(\"hire_date\" TO WEEK) FROM \"employee\"";
     final String expectedClickHouse = "SELECT toMonday(`hire_date`)\n"
         + "FROM `foodmart`.`employee`";
@@ -3042,7 +3090,7 @@ class RelToSqlConverterTest {
         .ok(expected);
   }
 
-  @Test public void testFloorMonth() {
+  @Test void testFloorMonth() {
     final String query = "SELECT floor(\"hire_date\" TO MONTH) FROM \"employee\"";
     final String expectedClickHouse = "SELECT toStartOfMonth(`hire_date`)\n"
         + "FROM `foodmart`.`employee`";
@@ -3132,6 +3180,8 @@ class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\"";
     final String expectedPostgresql = "SELECT SUBSTRING(\"brand_name\" FROM 2)\n"
         + "FROM \"foodmart\".\"product\"";
+    final String expectedPresto = "SELECT SUBSTR(\"brand_name\", 2)\n"
+        + "FROM \"foodmart\".\"product\"";
     final String expectedSnowflake = expectedPostgresql;
     final String expectedRedshift = expectedPostgresql;
     final String expectedMysql = "SELECT SUBSTRING(`brand_name` FROM 2)\n"
@@ -3143,6 +3193,8 @@ class RelToSqlConverterTest {
         .ok(expectedOracle)
         .withPostgresql()
         .ok(expectedPostgresql)
+        .withPresto()
+        .ok(expectedPresto)
         .withSnowflake()
         .ok(expectedSnowflake)
         .withRedshift()
@@ -3163,6 +3215,8 @@ class RelToSqlConverterTest {
         + "FROM \"foodmart\".\"product\"";
     final String expectedPostgresql = "SELECT SUBSTRING(\"brand_name\" FROM 2 FOR 3)\n"
         + "FROM \"foodmart\".\"product\"";
+    final String expectedPresto = "SELECT SUBSTR(\"brand_name\", 2, 3)\n"
+        + "FROM \"foodmart\".\"product\"";
     final String expectedSnowflake = expectedPostgresql;
     final String expectedRedshift = expectedPostgresql;
     final String expectedMysql = "SELECT SUBSTRING(`brand_name` FROM 2 FOR 3)\n"
@@ -3176,6 +3230,8 @@ class RelToSqlConverterTest {
         .ok(expectedOracle)
         .withPostgresql()
         .ok(expectedPostgresql)
+        .withPresto()
+        .ok(expectedPresto)
         .withSnowflake()
         .ok(expectedSnowflake)
         .withRedshift()
@@ -4690,7 +4746,7 @@ class RelToSqlConverterTest {
     sql(query).ok(expected);
   }
 
-  @Test void testCubeInSpark() {
+  @Test void testCubeWithGroupBy() {
     final String query = "select count(*) "
         + "from \"foodmart\".\"product\" "
         + "group by cube(\"product_id\",\"product_class_id\")";
@@ -4700,13 +4756,18 @@ class RelToSqlConverterTest {
     final String expectedInSpark = "SELECT COUNT(*)\n"
         + "FROM foodmart.product\n"
         + "GROUP BY product_id, product_class_id WITH CUBE";
+    final String expectedPresto = "SELECT COUNT(*)\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY CUBE(\"product_id\", \"product_class_id\")";
     sql(query)
         .ok(expected)
         .withSpark()
-        .ok(expectedInSpark);
+        .ok(expectedInSpark)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
-  @Test void testRollupInSpark() {
+  @Test void testRollupWithGroupBy() {
     final String query = "select count(*) "
         + "from \"foodmart\".\"product\" "
         + "group by rollup(\"product_id\",\"product_class_id\")";
@@ -4716,10 +4777,15 @@ class RelToSqlConverterTest {
     final String expectedInSpark = "SELECT COUNT(*)\n"
         + "FROM foodmart.product\n"
         + "GROUP BY product_id, product_class_id WITH ROLLUP";
+    final String expectedPresto = "SELECT COUNT(*)\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "GROUP BY ROLLUP(\"product_id\", \"product_class_id\")";
     sql(query)
         .ok(expected)
         .withSpark()
-        .ok(expectedInSpark);
+        .ok(expectedInSpark)
+        .withPresto()
+        .ok(expectedPresto);
   }
 
   @Test void testJsonType() {
@@ -5266,6 +5332,10 @@ class RelToSqlConverterTest {
       return dialect(SqlDialect.DatabaseProduct.POSTGRESQL.getDialect());
     }
 
+    Sql withPresto() {
+      return dialect(DatabaseProduct.PRESTO.getDialect());
+    }
+
     Sql withRedshift() {
       return dialect(DatabaseProduct.REDSHIFT.getDialect());
     }