From d12029cf7973e2bae6058ff194931d6662f5c5fe Mon Sep 17 00:00:00 2001 From: Julian Hyde Date: Wed, 15 Feb 2023 00:50:52 -0800 Subject: [PATCH] [CALCITE-5529] Improve dialect tests, part 1: check each query against a reference dialect Create class RelToSqlFixture (was inner class RelToSqlConverterTest.Sql). This fixture has a list of dialects for which the test is enabled, and a reference dialect (currently always Calcite). When the test calls 'done()' on the fixture, the fixture checks the query against the reference dialect. In future, it will also execute the query against each enabled dialect. To ensure that each test remembers to call 'done'()', add a 'token pool' to the test framework. If a token has been opened but not closed, the framework prints the call stack of the 'open' call and fails the test. Add enum DialectCode, which defines the dialects (or dialect configurations) that are possible to test. Adding a DialectCode has a cost (e.g. a bunch of new Quidem recording files) so we should not add one without a good reason. After this change, all queries succeed against the reference dialect (Calcite). A few cases are disabled due to bugs; we should fix these bugs and enable the tests. This is just part 1. Part 2 will be to get the tests to succeed against another dialect (probably a local instance of Postgres). Part 3 will be to use a Quidem recording rather than a live Postgres instance. Part 4 will be to enable this framework for other tests (e.g. SqlOperatorTest). --- .../calcite/jdbc/CalciteConnectionImpl.java | 10 + .../java/org/apache/calcite/util/Token.java | 98 + .../calcite/rel/rel2sql/DialectCode.java | 85 + .../rel/rel2sql/DialectTestConfig.java | 299 ++ .../rel/rel2sql/DialectTestConfigs.java | 204 + .../calcite/rel/rel2sql/MockSqlDialect.java | 49 + .../rel2sql/RelToSqlConverterStructsTest.java | 39 +- .../rel/rel2sql/RelToSqlConverterTest.java | 4271 +++++++++-------- .../calcite/rel/rel2sql/RelToSqlFixture.java | 590 +++ .../org/apache/calcite/util/UtilTest.java | 19 + .../apache/calcite/test/CalciteAssert.java | 4 +- .../org/apache/calcite/test/Matchers.java | 8 +- 12 files changed, 3647 insertions(+), 2029 deletions(-) create mode 100644 core/src/main/java/org/apache/calcite/util/Token.java create mode 100644 core/src/test/java/org/apache/calcite/rel/rel2sql/DialectCode.java create mode 100644 core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfig.java create mode 100644 core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfigs.java create mode 100644 core/src/test/java/org/apache/calcite/rel/rel2sql/MockSqlDialect.java create mode 100644 core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlFixture.java diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java index ac2fbc127f51..4b8966542a58 100644 --- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java +++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java @@ -150,6 +150,16 @@ protected CalciteConnectionImpl(Driver driver, AvaticaFactory factory, requireNonNull(rootSchema != null ? rootSchema : CalciteSchema.createRootSchema(true)); + + final String schema = cfg.schema(); + if (schema != null && !schema.isEmpty()) { + try { + setSchema(schema); + } catch (SQLException e) { + throw new AssertionError(e); // not possible + } + } + // Add dual table metadata when isSupportedDualTable return true if (cfg.conformance().isSupportedDualTable()) { SchemaPlus schemaPlus = this.rootSchema.plus(); diff --git a/core/src/main/java/org/apache/calcite/util/Token.java b/core/src/main/java/org/apache/calcite/util/Token.java new file mode 100644 index 000000000000..6b7ebd6ea4f0 --- /dev/null +++ b/core/src/main/java/org/apache/calcite/util/Token.java @@ -0,0 +1,98 @@ +/* + * 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.util; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** Hands out tokens, and throws if they are not all released. + * + *

Typical use: + * + *

{@code
+ * Token.Pool pool = Token.pool();
+ * Token token1 = pool.token();
+ * Token token2 = pool.token();
+ * token1.close();
+ * pool.assertEmpty(); // throws because token2 has not been closed
+ * }
+ * */ +public class Token implements AutoCloseable { + private final Pool pool; + private final int id; + private final StackTraceElement[] stackElements; + + /** Creates a Token. Should only be called from {@link Pool#token()}. */ + private Token(Pool pool, int id, StackTraceElement[] stackElements) { + this.pool = pool; + this.id = id; + this.stackElements = stackElements; + } + + @Override public String toString() { + return Integer.toString(id); + } + + /** Releases this Token. */ + @Override public void close() { + if (!pool.release(id)) { + final RuntimeException x = + new RuntimeException("token " + id + " has already released"); + x.setStackTrace(stackElements); + throw x; + } + } + + /** Creates a pool. */ + public static Pool pool() { + return new Pool(); + } + + /** A collection of tokens. + * + *

It is thread-safe. */ + public static class Pool { + private final Map map = new ConcurrentHashMap<>(); + private final AtomicInteger ordinal = new AtomicInteger(); + + /** Creates a token. */ + public Token token() { + return map.computeIfAbsent(ordinal.getAndIncrement(), + id -> + new Token(Pool.this, id, Thread.currentThread().getStackTrace())); + } + + /** Releases a token id. Should be called from {@link Token#close()}. */ + @SuppressWarnings("resource") + private boolean release(int id) { + return map.remove(id) != null; + } + + /** Throws if not all fixtures have been released. */ + public void assertEmpty() { + int size = map.size(); + if (!map.isEmpty()) { + final RuntimeException x = + new RuntimeException("map should be empty, but contains " + size + + " tokens"); + x.setStackTrace(map.values().iterator().next().stackElements); + throw x; + } + } + } +} diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectCode.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectCode.java new file mode 100644 index 000000000000..becadc3cb3f4 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectCode.java @@ -0,0 +1,85 @@ +/* + * 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.rel.rel2sql; + +import org.apache.calcite.config.NullCollation; +import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.dialect.AnsiSqlDialect; + +import static org.apache.calcite.rel.rel2sql.DialectTestConfigs.JETHRO_DIALECT_SUPPLIER; + +/** Dialect code. */ +enum DialectCode { + ANSI(new AnsiSqlDialect(SqlDialect.EMPTY_CONTEXT)), + BIG_QUERY(SqlDialect.DatabaseProduct.BIG_QUERY), + CALCITE(SqlDialect.DatabaseProduct.CALCITE), + CLICKHOUSE(SqlDialect.DatabaseProduct.CLICKHOUSE), + DB2(SqlDialect.DatabaseProduct.DB2), + EXASOL(SqlDialect.DatabaseProduct.EXASOL), + FIREBOLT(SqlDialect.DatabaseProduct.FIREBOLT), + HIVE(SqlDialect.DatabaseProduct.HIVE), + HIVE_2_0(DialectTestConfigs.hiveDialect(2, 0)), + HIVE_2_1(DialectTestConfigs.hiveDialect(2, 1)), + HIVE_2_2(DialectTestConfigs.hiveDialect(2, 2)), + HSQLDB(SqlDialect.DatabaseProduct.HSQLDB), + INFORMIX(SqlDialect.DatabaseProduct.INFORMIX), + JETHRO(JETHRO_DIALECT_SUPPLIER.get()), + MOCK(new MockSqlDialect()), + MSSQL_2008(DialectTestConfigs.mssqlDialect(10)), + MSSQL_2012(DialectTestConfigs.mssqlDialect(11)), + MSSQL_2017(DialectTestConfigs.mssqlDialect(14)), + MYSQL(SqlDialect.DatabaseProduct.MYSQL), + MYSQL_8(DialectTestConfigs.mysqlDialect(8, null)), + MYSQL_FIRST(DialectTestConfigs.mysqlDialect(8, NullCollation.FIRST)), + MYSQL_HIGH(DialectTestConfigs.mysqlDialect(8, NullCollation.HIGH)), + MYSQL_LAST(DialectTestConfigs.mysqlDialect(8, NullCollation.LAST)), + NON_ORDINAL(DialectTestConfigs.nonOrdinalDialect()), + ORACLE(SqlDialect.DatabaseProduct.ORACLE), + ORACLE_11(DialectTestConfigs.oracleDialect(11, null)), + ORACLE_12(DialectTestConfigs.oracleDialect(12, null)), + ORACLE_19(DialectTestConfigs.oracleDialect(19, null)), + ORACLE_23(DialectTestConfigs.oracleDialect(23, null)), + /** Oracle dialect with max length for varchar set to 512. */ + ORACLE_MODIFIED(DialectTestConfigs.oracleDialect(12, 512)), + POSTGRESQL(SqlDialect.DatabaseProduct.POSTGRESQL), + /** Postgresql dialect with max length for varchar set to 256. */ + POSTGRESQL_MODIFIED(DialectTestConfigs.postgresqlDialect(256, false)), + /** Postgresql dialect with modified decimal type. */ + POSTGRESQL_MODIFIED_DECIMAL( + DialectTestConfigs.postgresqlDialect(null, true)), + PRESTO(SqlDialect.DatabaseProduct.PRESTO), + REDSHIFT(SqlDialect.DatabaseProduct.REDSHIFT), + SNOWFLAKE(SqlDialect.DatabaseProduct.SNOWFLAKE), + SPARK(SqlDialect.DatabaseProduct.SPARK), + STARROCKS(SqlDialect.DatabaseProduct.STARROCKS), + SYBASE(SqlDialect.DatabaseProduct.SYBASE), + VERTICA(SqlDialect.DatabaseProduct.VERTICA); + + private final DialectTestConfig.Dialect dialect; + + DialectCode(SqlDialect.DatabaseProduct databaseProduct) { + dialect = DialectTestConfig.Dialect.of(this, databaseProduct); + } + + DialectCode(SqlDialect sqlDialect) { + dialect = DialectTestConfig.Dialect.of(this, sqlDialect); + } + + DialectTestConfig.Dialect toDialect() { + return dialect; + } +} diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfig.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfig.java new file mode 100644 index 000000000000..c284303addde --- /dev/null +++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfig.java @@ -0,0 +1,299 @@ +/* + * 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.rel.rel2sql; + +import org.apache.calcite.config.CalciteConnectionProperty; +import org.apache.calcite.jdbc.CalciteJdbc41Factory; +import org.apache.calcite.jdbc.CalciteSchema; +import org.apache.calcite.jdbc.Driver; +import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.validate.SqlConformanceEnum; +import org.apache.calcite.test.CalciteAssert; + +import com.google.common.collect.ImmutableMap; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.sql.Connection; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.UnaryOperator; + +import static java.util.Objects.requireNonNull; + +/** Description of the dialects that are enabled for a particular test. + * + *

Each dialect has a name, optionally a connection factory, + * and a state (enabled, recording, replaying). + * + *

It is immutable. + */ +class DialectTestConfig { + final ImmutableMap dialectMap; + + /** The code of the reference dialect. If not null, the queries from this + * dialect as used as exemplars for other dialects: the other dialects are + * expected to return the same set of rows as the reference. */ + final @Nullable DialectCode refDialectCode; + + /** The name of the class relative to which the resource file containing + * query responses is located. */ + @SuppressWarnings("rawtypes") + private final Class testClass; + + /** A function that maps a dialect name to the name of the file containing + * its query responses. */ + private final Function function; + + private DialectTestConfig(Map dialectMap, + @Nullable DialectCode refDialectCode, + @SuppressWarnings("rawtypes") Class testClass, + Function function) { + this.dialectMap = ImmutableMap.copyOf(dialectMap); + this.refDialectCode = refDialectCode; + this.testClass = requireNonNull(testClass, "testClass"); + this.function = requireNonNull(function, "function"); + } + + /** Creates a DialectTestConfig. */ + static DialectTestConfig of(Iterable dialects) { + final ImmutableMap.Builder map = ImmutableMap.builder(); + dialects.forEach(dialect -> map.put(dialect.name, dialect)); + return new DialectTestConfig(map.build(), null, RelToSqlConverterTest.class, + UnaryOperator.identity()); + } + + /** Applies a transform to the dialect with a given code. + * + *

Throws if there is no such dialect. */ + public DialectTestConfig withDialect(DialectCode code, + UnaryOperator dialectTransform) { + return withDialect(code.name(), dialectTransform); + } + + /** Applies a transform to each dialect. */ + public DialectTestConfig withDialects( + UnaryOperator dialectTransform) { + final ImmutableMap.Builder b = + ImmutableMap.builder(); + dialectMap.forEach((name, dialect) -> + b.put(dialect.name, dialectTransform.apply(dialect))); + final ImmutableMap dialectMap2 = b.build(); + if (dialectMap2.equals(dialectMap)) { + return this; + } + return new DialectTestConfig(dialectMap2, refDialectCode, testClass, + function); + } + + /** Applies a transform to the dialect with a given name. + * + *

Throws if there is no such dialect. */ + public DialectTestConfig withDialect(String name, + UnaryOperator dialectTransform) { + final Dialect dialect = dialectMap.get(name); + final Dialect dialect2 = dialectTransform.apply(dialect); + if (dialect == dialect2) { + return this; + } + final Map dialectMap2 = new LinkedHashMap<>(dialectMap); + dialectMap2.put(name, dialect2); + return new DialectTestConfig(dialectMap2, refDialectCode, testClass, + function); + } + + /** Sets the name of the reference dialect. */ + public DialectTestConfig withReference(DialectCode refDialectCode) { + if (refDialectCode == this.refDialectCode) { + return this; + } + return new DialectTestConfig(dialectMap, refDialectCode, testClass, + function); + } + + /** Sets the path for any given dialect's corpus. */ + public DialectTestConfig withPath( + @SuppressWarnings("rawtypes") Class testClass, + Function function) { + if (testClass == this.testClass && function == this.function) { + return this; + } + return new DialectTestConfig(dialectMap, refDialectCode, testClass, + function); + } + + /** Returns the dialect with the given code. */ + public Dialect get(DialectCode dialectCode) { + return requireNonNull(dialectMap.get(dialectCode.name()), + () -> "dialect " + dialectCode); + } + + /** Which phase of query execution. */ + public enum Phase { + /** Parses the query but does not validate. */ + PARSE, + PREPARE, + EXECUTE, + } + + /** Definition of a dialect. */ + static class Dialect { + /** The name of this dialect. */ + final String name; + + /** The code of this dialect. + * Having a code isn't strictly necessary, but it makes tests more concise. */ + final DialectCode code; + + /** The dialect object. */ + final SqlDialect sqlDialect; + + /** Whether the dialect is enabled in the test. */ + final boolean enabled; + + /** Whether the test should execute queries in this dialect. If there is a + * reference, compares the results to the reference. */ + final boolean execute; + + /** The query that we expect to be generated for this dialect in this test + * run. Is only set during a test run, and is always null in the base + * configuration. */ + final @Nullable String expectedQuery; + + /** The error that we expect to be thrown for this dialect in this test + * run. Is only set during a test run, and is always null in the base + * configuration. */ + final @Nullable String expectedError; + + Dialect(String name, DialectCode code, SqlDialect sqlDialect, + boolean enabled, boolean execute, @Nullable String expectedQuery, + @Nullable String expectedError) { + this.name = requireNonNull(name, "name"); + this.code = requireNonNull(code, "code"); + this.sqlDialect = requireNonNull(sqlDialect, "sqlDialect"); + this.enabled = enabled; + this.execute = execute; + this.expectedQuery = expectedQuery; + this.expectedError = expectedError; + } + + /** Creates a Dialect based on a + * {@link org.apache.calcite.sql.SqlDialect.DatabaseProduct}. */ + public static Dialect of(DialectCode dialectCode, + SqlDialect.DatabaseProduct databaseProduct) { + return of(dialectCode, databaseProduct.getDialect()); + } + + /** Creates a Dialect. */ + public static Dialect of(DialectCode dialectCode, SqlDialect dialect) { + return new Dialect(dialectCode.name(), dialectCode, dialect, true, false, + null, null); + } + + @Override public String toString() { + return name; + } + + public Dialect withEnabled(boolean enabled) { + if (enabled == this.enabled) { + return this; + } + return new Dialect(name, code, sqlDialect, enabled, execute, + expectedQuery, expectedError); + } + + public Dialect withExecute(boolean execute) { + if (execute == this.execute) { + return this; + } + return new Dialect(name, code, sqlDialect, enabled, execute, + expectedQuery, expectedError); + } + + public Dialect withExpectedQuery(String expectedQuery) { + if (Objects.equals(expectedQuery, this.expectedQuery)) { + return this; + } + return new Dialect(name, code, sqlDialect, enabled, execute, + expectedQuery, expectedError); + } + + public Dialect withExpectedError(String expectedError) { + if (Objects.equals(expectedError, this.expectedError)) { + return this; + } + return new Dialect(name, code, sqlDialect, enabled, execute, + expectedQuery, expectedError); + } + + /** Performs an action with the dialect's connection. */ + public void withConnection(CalciteAssert.SchemaSpec schemaSpec, + Consumer consumer) { + switch (code) { + case CALCITE: + final CalciteJdbc41Factory factory = new CalciteJdbc41Factory(); + final Driver driver = new Driver(); + final String url = "jdbc:calcite:"; + final CalciteSchema rootSchema = CalciteSchema.createRootSchema(false); + CalciteAssert.addSchema(rootSchema.plus(), + CalciteAssert.SchemaSpec.BOOKSTORE, + CalciteAssert.SchemaSpec.JDBC_FOODMART, + CalciteAssert.SchemaSpec.POST, + CalciteAssert.SchemaSpec.SCOTT, + CalciteAssert.SchemaSpec.SCOTT_WITH_TEMPORAL, + CalciteAssert.SchemaSpec.TPCH); + final Properties info = new Properties(); + // Hive for RLIKE, Postgres for ILIKE, Spark for EXISTS, etc. + info.put(CalciteConnectionProperty.FUN.name(), + "standard,postgresql,bigquery,hive,spark"); + info.put(CalciteConnectionProperty.SCHEMA.name(), + schemaSpec.schemaName); + info.put(CalciteConnectionProperty.CONFORMANCE.name(), + SqlConformanceEnum.LENIENT.name()); + try (Connection connection = + factory.newConnection(driver, factory, url, info, + rootSchema, null)) { + consumer.accept(connection); + return; + } catch (SQLException e) { + throw new RuntimeException(e); + } + default: + return; + } + } + + /** Performs an action with a statement from the dialect's connection, + * or no-ops if no connection. */ + public void withStatement(CalciteAssert.SchemaSpec schemaSpec, + Consumer consumer) { + withConnection(schemaSpec, connection -> { + try (Statement statement = connection.createStatement()) { + consumer.accept(statement); + } catch (SQLException e) { + throw new RuntimeException(e); + } + }); + } + } +} diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfigs.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfigs.java new file mode 100644 index 000000000000..af55a20474a9 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/DialectTestConfigs.java @@ -0,0 +1,204 @@ +/* + * 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.rel.rel2sql; + +import org.apache.calcite.config.NullCollation; +import org.apache.calcite.rel.type.RelDataTypeSystemImpl; +import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.dialect.HiveSqlDialect; +import org.apache.calcite.sql.dialect.JethroDataSqlDialect; +import org.apache.calcite.sql.dialect.MssqlSqlDialect; +import org.apache.calcite.sql.dialect.MysqlSqlDialect; +import org.apache.calcite.sql.dialect.OracleSqlDialect; +import org.apache.calcite.sql.dialect.PostgresqlSqlDialect; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.calcite.sql.validate.SqlConformance; +import org.apache.calcite.sql.validate.SqlConformanceEnum; +import org.apache.calcite.util.Util; + +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Ordering; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.util.function.Supplier; + +import static org.apache.calcite.util.Util.first; + +/** Utilities for {@link DialectTestConfig}. */ +class DialectTestConfigs { + private DialectTestConfigs() { + } + + static final Supplier INSTANCE_SUPPLIER = + Suppliers.memoize(() -> { + final ImmutableList.Builder b = + ImmutableList.builder(); + for (DialectCode dialectCode : DialectCode.values()) { + b.add(dialectCode.toDialect()); + } + final ImmutableList list = b.build(); + final Iterable dialectNames = + Util.transform(list, dialect -> dialect.name); + if (!Ordering.natural().isOrdered(dialectNames)) { + throw new AssertionError("not ordered: " + dialectNames); + } + return DialectTestConfig.of(list); + })::get; + + + @SuppressWarnings("SameParameterValue") + static HiveSqlDialect hiveDialect(int majorVersion, int minorVersion) { + return new HiveSqlDialect(HiveSqlDialect.DEFAULT_CONTEXT + .withDatabaseMajorVersion(majorVersion) + .withDatabaseMinorVersion(minorVersion) + .withNullCollation(NullCollation.LOW)); + } + + static SqlDialect mysqlDialect(@Nullable Integer majorVersion, + @Nullable NullCollation nullCollation) { + final SqlDialect d = SqlDialect.DatabaseProduct.MYSQL.getDialect(); + SqlDialect.Context context = + MysqlSqlDialect.DEFAULT_CONTEXT + .withIdentifierQuoteString(d.quoteIdentifier("").substring(0, 1)) + .withNullCollation(first(nullCollation, d.getNullCollation())); + if (majorVersion != null) { + context = context.withDatabaseMajorVersion(majorVersion); + } + if (nullCollation == null) { + // Historically, the MYSQL_8 dialect used in tests was an instance of + // SqlDialect, not MysqlSqlDialect. Preserve that behavior for now. + return new SqlDialect(context); + } + return new MysqlSqlDialect(context); + } + + static SqlDialect oracleDialect(final @Nullable Integer majorVersion, + final @Nullable Integer maxVarcharLength) { + final SqlDialect oracleDialect = OracleSqlDialect.DEFAULT; + SqlDialect.Context context = + OracleSqlDialect.DEFAULT_CONTEXT + .withIdentifierQuoteString(oracleDialect.quoteIdentifier("") + .substring(0, 1)) + .withNullCollation(oracleDialect.getNullCollation()); + if (maxVarcharLength != null) { + context = context.withDataTypeSystem(new RelDataTypeSystemImpl() { + @Override public int getMaxPrecision(SqlTypeName typeName) { + switch (typeName) { + case VARCHAR: + return maxVarcharLength; + default: + return super.getMaxPrecision(typeName); + } + } + }); + } + + if (majorVersion != null) { + context = + context.withDatabaseMajorVersion(majorVersion); + } + return new OracleSqlDialect(context); + } + + static SqlDialect postgresqlDialect(final @Nullable Integer maxVarcharLength, + final boolean modifyDecimal) { + SqlDialect.Context context = PostgresqlSqlDialect.DEFAULT_CONTEXT; + if (maxVarcharLength != null) { + context = + context + .withDataTypeSystem(new RelDataTypeSystemImpl() { + @Override public int getMaxPrecision(SqlTypeName typeName) { + switch (typeName) { + case VARCHAR: + return maxVarcharLength; + default: + return super.getMaxPrecision(typeName); + } + } + }); + } + if (modifyDecimal) { + context = + context.withDataTypeSystem( + new RelDataTypeSystemImpl() { + @Override public int getMaxNumericScale() { + return getMaxScale(SqlTypeName.DECIMAL); + } + + @Override public int getMaxScale(SqlTypeName typeName) { + switch (typeName) { + case DECIMAL: + return 10; + default: + return super.getMaxScale(typeName); + } + } + + @Override public int getMaxNumericPrecision() { + return getMaxPrecision(SqlTypeName.DECIMAL); + } + + @Override public int getMaxPrecision(SqlTypeName typeName) { + switch (typeName) { + case DECIMAL: + return 39; + default: + return super.getMaxPrecision(typeName); + } + } + }); + } + return new PostgresqlSqlDialect(context); + } + + /** Creates a dialect for Microsoft SQL Server. + * + *

MSSQL 2008 has version 10.0, 2012 has 11.0, 2017 has 14.0. */ + static SqlDialect mssqlDialect(int majorVersion) { + final SqlDialect mssqlDialect = + SqlDialect.DatabaseProduct.MSSQL.getDialect(); + return new MssqlSqlDialect(MssqlSqlDialect.DEFAULT_CONTEXT + .withDatabaseMajorVersion(majorVersion) + .withIdentifierQuoteString(mssqlDialect.quoteIdentifier("") + .substring(0, 1)) + .withNullCollation(mssqlDialect.getNullCollation())); + } + + /** Creates a dialect that doesn't treat integer literals in the ORDER BY as + * field references. */ + static SqlDialect nonOrdinalDialect() { + return new SqlDialect(SqlDialect.EMPTY_CONTEXT) { + @Override public SqlConformance getConformance() { + return SqlConformanceEnum.STRICT_99; + } + }; + } + + static final Supplier JETHRO_DIALECT_SUPPLIER = + Suppliers.memoize(() -> + new JethroDataSqlDialect( + SqlDialect.EMPTY_CONTEXT + .withDatabaseProduct(SqlDialect.DatabaseProduct.JETHRO) + .withDatabaseMajorVersion(1) + .withDatabaseMinorVersion(0) + .withDatabaseVersion("1.0") + .withIdentifierQuoteString("\"") + .withNullCollation(NullCollation.HIGH) + .withJethroInfo(JethroDataSqlDialect.JethroInfo.EMPTY))); +} diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/MockSqlDialect.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/MockSqlDialect.java new file mode 100644 index 000000000000..0d3bd234e242 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/MockSqlDialect.java @@ -0,0 +1,49 @@ +/* + * 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.rel.rel2sql; + +import org.apache.calcite.sql.SqlCall; +import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.SqlSelect; +import org.apache.calcite.sql.SqlWriter; + +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Mock dialect for testing. + * + *

Available under {@link DialectCode#MOCK}. + * + *

If you are writing tests, feel free to add fields and modify behavior + * for particular tests. + */ +class MockSqlDialect extends SqlDialect { + public static final ThreadLocal THREAD_UNPARSE_SELECT_COUNT = + ThreadLocal.withInitial(() -> new AtomicInteger(0)); + + MockSqlDialect() { + super(SqlDialect.EMPTY_CONTEXT); + } + + @Override public void unparseCall(SqlWriter writer, SqlCall call, + int leftPrec, int rightPrec) { + if (call instanceof SqlSelect) { + THREAD_UNPARSE_SELECT_COUNT.get().incrementAndGet(); + } + super.unparseCall(writer, call, leftPrec, rightPrec); + } +} diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java index 0a576f24d6d5..109579bb5ae4 100644 --- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java +++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterStructsTest.java @@ -16,27 +16,44 @@ */ package org.apache.calcite.rel.rel2sql; -import org.apache.calcite.sql.dialect.CalciteSqlDialect; import org.apache.calcite.sql.parser.SqlParser; import org.apache.calcite.test.CalciteAssert; +import org.apache.calcite.util.Token; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import java.util.function.UnaryOperator; +import static org.apache.calcite.rel.rel2sql.DialectCode.CALCITE; + /** * Tests for {@link RelToSqlConverter} on a schema that has nested structures of multiple * levels. */ class RelToSqlConverterStructsTest { - private RelToSqlConverterTest.Sql sql(String sql) { - return new RelToSqlConverterTest.Sql(CalciteAssert.SchemaSpec.MY_DB, sql, - CalciteSqlDialect.DEFAULT, SqlParser.Config.DEFAULT, ImmutableSet.of(), - UnaryOperator.identity(), null, ImmutableList.of()); + /** Creates a fixture. */ + private static RelToSqlFixture fixture() { + final Token token = RelToSqlFixture.POOL.token(); + final DialectTestConfig testConfig = + DialectTestConfigs.INSTANCE_SUPPLIER.get(); + final DialectTestConfig.Dialect calcite = testConfig.get(CALCITE); + final DialectTestConfig.Phase phase = DialectTestConfig.Phase.PREPARE; + return new RelToSqlFixture(token, + CalciteAssert.SchemaSpec.MY_DB, "?", + calcite, phase, SqlParser.Config.DEFAULT, ImmutableSet.of(), + UnaryOperator.identity(), null, ImmutableList.of(), + DialectTestConfigs.INSTANCE_SUPPLIER.get(), + RelToSqlFixture::transformWriter); + } + + /** Creates a fixture and initializes it with a SQL query. */ + private RelToSqlFixture sql(String sql) { + return fixture().withSql(sql); } @Test void testNestedSchemaSelectStar() { @@ -46,7 +63,7 @@ private RelToSqlConverterTest.Sql sql(String sql) { + "ROW(\"n2\".\"d\") AS \"n2\", \"xs\", " + "\"e\"\n" + "FROM \"myDb\".\"myTable\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testNestedSchemaRootColumns() { @@ -54,7 +71,7 @@ private RelToSqlConverterTest.Sql sql(String sql) { String expected = "SELECT \"a\", " + "\"e\"\n" + "FROM \"myDb\".\"myTable\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testNestedSchemaNestedColumns() { @@ -67,12 +84,13 @@ private RelToSqlConverterTest.Sql sql(String sql) { + "\"n1\".\"n11\".\"b\", " + "\"n2\".\"d\"\n" + "FROM \"myDb\".\"myTable\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-6218] * RelToSqlConverter fails to convert correlated lateral joins. */ + @Disabled // TODO fix UnsupportedOperationException in getAliasContext @Test void testUncollectLateralJoin() { final String query = "select \"a\",\n" + "\"x\"\n" @@ -83,6 +101,9 @@ private RelToSqlConverterTest.Sql sql(String sql) { + "FROM \"myDb\".\"myTable\") AS \"$cor0\",\n" + "LATERAL UNNEST((SELECT \"$cor0\".\"xs\"\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\"))) AS \"t10\" (\"xs\")"; - sql(query).schema(CalciteAssert.SchemaSpec.MY_DB).ok(expected); + sql(query).schema(CalciteAssert.SchemaSpec.MY_DB) + .withPhase(DialectTestConfig.Phase.PARSE) + .ok(expected) + .done(); } } 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 fd5600ad1ad9..620559f6c951 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 @@ -16,10 +16,6 @@ */ package org.apache.calcite.rel.rel2sql; -import org.apache.calcite.config.NullCollation; -import org.apache.calcite.plan.RelOptPlanner; -import org.apache.calcite.plan.RelOptRule; -import org.apache.calcite.plan.RelTraitDef; import org.apache.calcite.plan.hep.HepPlanner; import org.apache.calcite.plan.hep.HepProgramBuilder; import org.apache.calcite.rel.RelCollations; @@ -38,32 +34,14 @@ import org.apache.calcite.rel.rules.CoreRules; import org.apache.calcite.rel.rules.FilterJoinRule; import org.apache.calcite.rel.rules.ProjectOverSumToSum0Rule; -import org.apache.calcite.rel.rules.ProjectToWindowRule; import org.apache.calcite.rel.rules.PruneEmptyRules; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rel.type.RelDataTypeSystem; -import org.apache.calcite.rel.type.RelDataTypeSystemImpl; -import org.apache.calcite.runtime.FlatLists; import org.apache.calcite.runtime.Hook; -import org.apache.calcite.schema.SchemaPlus; -import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlDialect; -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; -import org.apache.calcite.sql.SqlWriterConfig; -import org.apache.calcite.sql.dialect.AnsiSqlDialect; import org.apache.calcite.sql.dialect.BigQuerySqlDialect; -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.MssqlSqlDialect; -import org.apache.calcite.sql.dialect.MysqlSqlDialect; -import org.apache.calcite.sql.dialect.OracleSqlDialect; -import org.apache.calcite.sql.dialect.PostgresqlSqlDialect; -import org.apache.calcite.sql.dialect.PrestoSqlDialect; import org.apache.calcite.sql.fun.SqlLibrary; import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.apache.calcite.sql.parser.SqlParser; @@ -72,42 +50,59 @@ import org.apache.calcite.sql.util.SqlShuttle; import org.apache.calcite.sql.validate.SqlConformance; import org.apache.calcite.sql.validate.SqlConformanceEnum; -import org.apache.calcite.sql2rel.SqlToRelConverter; import org.apache.calcite.test.CalciteAssert; -import org.apache.calcite.test.MockSqlOperatorTable; import org.apache.calcite.test.RelBuilderTest; -import org.apache.calcite.tools.FrameworkConfig; -import org.apache.calcite.tools.Frameworks; -import org.apache.calcite.tools.Planner; -import org.apache.calcite.tools.Program; -import org.apache.calcite.tools.Programs; import org.apache.calcite.tools.RelBuilder; import org.apache.calcite.tools.RuleSet; import org.apache.calcite.tools.RuleSets; import org.apache.calcite.util.ConversionUtil; import org.apache.calcite.util.ImmutableBitSet; -import org.apache.calcite.util.TestUtil; -import org.apache.calcite.util.Util; +import org.apache.calcite.util.Token; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Test; import java.math.BigDecimal; -import java.util.Collection; import java.util.List; -import java.util.Map; -import java.util.Set; import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.Supplier; import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.IntStream; - -import static org.apache.calcite.test.Matchers.isLinux; +import java.util.stream.Stream; + +import static com.google.common.collect.ImmutableList.toImmutableList; + +import static org.apache.calcite.rel.rel2sql.DialectCode.ANSI; +import static org.apache.calcite.rel.rel2sql.DialectCode.BIG_QUERY; +import static org.apache.calcite.rel.rel2sql.DialectCode.CALCITE; +import static org.apache.calcite.rel.rel2sql.DialectCode.CLICKHOUSE; +import static org.apache.calcite.rel.rel2sql.DialectCode.DB2; +import static org.apache.calcite.rel.rel2sql.DialectCode.EXASOL; +import static org.apache.calcite.rel.rel2sql.DialectCode.HIVE; +import static org.apache.calcite.rel.rel2sql.DialectCode.HIVE_2_0; +import static org.apache.calcite.rel.rel2sql.DialectCode.HIVE_2_1; +import static org.apache.calcite.rel.rel2sql.DialectCode.HIVE_2_2; +import static org.apache.calcite.rel.rel2sql.DialectCode.HSQLDB; +import static org.apache.calcite.rel.rel2sql.DialectCode.JETHRO; +import static org.apache.calcite.rel.rel2sql.DialectCode.MOCK; +import static org.apache.calcite.rel.rel2sql.DialectCode.MSSQL_2008; +import static org.apache.calcite.rel.rel2sql.DialectCode.MSSQL_2012; +import static org.apache.calcite.rel.rel2sql.DialectCode.MSSQL_2017; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL_HIGH; +import static org.apache.calcite.rel.rel2sql.DialectCode.NON_ORDINAL; +import static org.apache.calcite.rel.rel2sql.DialectCode.ORACLE_12; +import static org.apache.calcite.rel.rel2sql.DialectCode.POSTGRESQL; +import static org.apache.calcite.rel.rel2sql.DialectCode.PRESTO; +import static org.apache.calcite.rel.rel2sql.DialectCode.STARROCKS; +import static org.apache.calcite.rel.rel2sql.DialectCode.SYBASE; +import static org.apache.calcite.rel.rel2sql.DialectTestConfig.Phase.PARSE; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; @@ -116,82 +111,73 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; +import static java.util.Objects.requireNonNull; + /** * Tests for {@link RelToSqlConverter}. */ class RelToSqlConverterTest { - private Sql fixture() { - return new Sql(CalciteAssert.SchemaSpec.JDBC_FOODMART, "?", - CalciteSqlDialect.DEFAULT, SqlParser.Config.DEFAULT, ImmutableSet.of(), - UnaryOperator.identity(), null, ImmutableList.of()); - } - - /** Initiates a test case with a given SQL query. */ - private Sql sql(String sql) { + private static final Supplier CONFIG_SUPPLIER = + Suppliers.memoize(() -> + DialectTestConfigs.INSTANCE_SUPPLIER.get() + .withReference(CALCITE) + .withDialects(d -> d.withEnabled(false)) + .withPath(RelToSqlConverterTest.class, + dialectName -> dialectName + ".json") + .withDialect(HSQLDB, d -> d.withExecute(true))); + + private static final SqlParser.Config BIG_QUERY_PARSER_CONFIG = + BigQuerySqlDialect.DEFAULT.configureParser(SqlParser.config()); + + @AfterAll + static void assertFixtureTrackerIsEmpty() { + RelToSqlFixture.POOL.assertEmpty(); + } + + /** Creates a fixture. */ + private RelToSqlFixture fixture() { + Token id = RelToSqlFixture.POOL.token(); + final DialectTestConfig dialectTestConfig = CONFIG_SUPPLIER.get(); + final DialectTestConfig.Dialect dialect = + dialectTestConfig.get(CALCITE); + final DialectTestConfig.Phase phase = DialectTestConfig.Phase.PREPARE; + return new RelToSqlFixture(id, CalciteAssert.SchemaSpec.JDBC_FOODMART, "?", + dialect, phase, SqlParser.Config.DEFAULT, ImmutableSet.of(), + UnaryOperator.identity(), null, ImmutableList.of(), dialectTestConfig, + RelToSqlFixture::transformWriter); + } + + /** Creates a fixture and initializes it with a SQL query. */ + private RelToSqlFixture sql(String sql) { return fixture().withSql(sql); } /** Initiates a test case with a given {@link RelNode} supplier. */ - private Sql relFn(Function relFn) { + private RelToSqlFixture relFn(Function relFn) { return fixture() .schema(CalciteAssert.SchemaSpec.SCOTT_WITH_TEMPORAL) .relFn(relFn); } - private static Planner getPlanner(List traitDefs, - SqlParser.Config parserConfig, SchemaPlus schema, - SqlToRelConverter.Config sqlToRelConf, Collection librarySet, - RelDataTypeSystem typeSystem, Program... programs) { - final FrameworkConfig config = Frameworks.newConfigBuilder() - .parserConfig(parserConfig) - .defaultSchema(schema) - .traitDefs(traitDefs) - .sqlToRelConverterConfig(sqlToRelConf) - .programs(programs) - .operatorTable(MockSqlOperatorTable.standard() - .plus(librarySet) - .extend()) - .typeSystem(typeSystem) - .build(); - return Frameworks.getPlanner(config); - } - - private static JethroDataSqlDialect jethroDataSqlDialect() { - SqlDialect.Context dummyContext = SqlDialect.EMPTY_CONTEXT - .withDatabaseProduct(DatabaseProduct.JETHRO) - .withDatabaseMajorVersion(1) - .withDatabaseMinorVersion(0) - .withDatabaseVersion("1.0") - .withIdentifierQuoteString("\"") - .withNullCollation(NullCollation.HIGH) - .withJethroInfo(JethroDataSqlDialect.JethroInfo.EMPTY); - return new JethroDataSqlDialect(dummyContext); - } - - private static MysqlSqlDialect mySqlDialect(NullCollation nullCollation) { - return new MysqlSqlDialect(MysqlSqlDialect.DEFAULT_CONTEXT - .withNullCollation(nullCollation)); - } - /** Returns a collection of common dialects, and the database products they * represent. */ - private static Map dialects() { - return ImmutableMap.builder() - .put(DatabaseProduct.BIG_QUERY.getDialect(), DatabaseProduct.BIG_QUERY) - .put(DatabaseProduct.CALCITE.getDialect(), DatabaseProduct.CALCITE) - .put(DatabaseProduct.DB2.getDialect(), DatabaseProduct.DB2) - .put(DatabaseProduct.EXASOL.getDialect(), DatabaseProduct.EXASOL) - .put(DatabaseProduct.HIVE.getDialect(), DatabaseProduct.HIVE) - .put(jethroDataSqlDialect(), DatabaseProduct.JETHRO) - .put(DatabaseProduct.MSSQL.getDialect(), DatabaseProduct.MSSQL) - .put(DatabaseProduct.MYSQL.getDialect(), DatabaseProduct.MYSQL) - .put(mySqlDialect(NullCollation.HIGH), DatabaseProduct.MYSQL) - .put(DatabaseProduct.ORACLE.getDialect(), DatabaseProduct.ORACLE) - .put(DatabaseProduct.POSTGRESQL.getDialect(), DatabaseProduct.POSTGRESQL) - .put(DatabaseProduct.PRESTO.getDialect(), DatabaseProduct.PRESTO) - .put(DatabaseProduct.STARROCKS.getDialect(), DatabaseProduct.STARROCKS) - .build(); + private static List dialects() { + return Stream.of(BIG_QUERY, + CALCITE, + DB2, + EXASOL, + HIVE, + JETHRO, + MSSQL_2017, + MYSQL, + MYSQL_HIGH, + ORACLE_12, + POSTGRESQL, + PRESTO, + STARROCKS) + .map(dialectCode -> CONFIG_SUPPLIER.get().get(dialectCode)) + .collect(toImmutableList()); } /** Creates a RelBuilder. */ @@ -199,29 +185,6 @@ private static RelBuilder relBuilder() { return RelBuilder.create(RelBuilderTest.config().build()); } - /** Converts a relational expression to SQL. */ - private String toSql(RelNode root) { - return toSql(root, DatabaseProduct.CALCITE.getDialect()); - } - - /** Converts a relational expression to SQL in a given dialect. */ - private static String toSql(RelNode root, SqlDialect dialect) { - return toSql(root, dialect, c -> - c.withAlwaysUseParentheses(false) - .withSelectListItemsOnSeparateLines(false) - .withUpdateSetListNewline(false) - .withIndentation(0)); - } - - /** Converts a relational expression to SQL in a given dialect - * and with a particular writer configuration. */ - private static String toSql(RelNode root, SqlDialect dialect, - UnaryOperator transform) { - final RelToSqlConverter converter = new RelToSqlConverter(dialect); - final SqlNode sqlNode = converter.visitRoot(root).asStatement(); - return sqlNode.toSqlString(c -> transform.apply(c.withDialect(dialect))) - .getSql(); - } /** Test case for * [CALCITE-5988] @@ -233,18 +196,27 @@ private static String toSql(RelNode root, SqlDialect dialect, // the expected string is a bit too verbose: // "SELECT *\n" // + "FROM (VALUES (X'ABCD')) AS \"t\" (\"EXPR$0\")" - sql(query).withMysql().ok(expected); - sql("SELECT cast(null as binary)").withMysql().ok("SELECT NULL"); + sql(query) + .withMysql().ok(expected) + .done(); + + final String query2 = "SELECT cast(null as binary)"; + final String expected2 = "SELECT NULL"; + sql(query2) + .withMysql().ok(expected2) + .done(); } @Test void testFloatingPointLiteral() { String query = "SELECT CAST(0.1E0 AS DOUBLE), CAST(0.1E0 AS REAL), CAST(0.1E0 AS DOUBLE)"; String expected = "SELECT 1E-1, 1E-1, 1E-1"; - sql(query).withMysql().ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testGroupByBooleanLiteral() { - String query = "select avg(\"salary\") from \"employee\" group by true"; + String query = "select avg(\"salary\")\n" + + "from \"employee\"\n" + + "group by true"; String expectedRedshift = "SELECT AVG(\"employee\".\"salary\")\n" + "FROM \"foodmart\".\"employee\",\n" + "(SELECT TRUE AS \"$f0\") AS \"t\"\n" @@ -255,11 +227,13 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY t.$f0"; sql(query) .withRedshift().ok(expectedRedshift) - .withInformix().ok(expectedInformix); + .withInformix().ok(expectedInformix).done(); } @Test void testGroupByDateLiteral() { - String query = "select avg(\"salary\") from \"employee\" group by DATE '2022-01-01'"; + String query = "select avg(\"salary\")\n" + + "from \"employee\"\n" + + "group by DATE '2022-01-01'"; String expectedRedshift = "SELECT AVG(\"employee\".\"salary\")\n" + "FROM \"foodmart\".\"employee\",\n" + "(SELECT DATE '2022-01-01' AS \"$f0\") AS \"t\"\n" @@ -270,14 +244,14 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY t.$f0"; sql(query) .withRedshift().ok(expectedRedshift) - .withInformix().ok(expectedInformix); + .withInformix().ok(expectedInformix).done(); } @Test void testSimpleSelectStarFromProductTable() { String query = "select * from \"product\""; String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -287,26 +261,30 @@ private static String toSql(RelNode root, SqlDialect dialect, // Before 4901 was fixed, the generated query would have "product_id" in its // SELECT clause. String query = "select count(1) as c\n" - + "from \"foodmart\".\"product\"\n" + + "from \"product\"\n" + "group by \"product_id\"\n" + "order by \"product_id\" desc"; final String expected = "SELECT COUNT(*) AS \"C\"\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_id\"\n" + "ORDER BY \"product_id\" DESC"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-6006] * RelToSqlConverter loses charset information. */ + @SuppressWarnings("UnnecessaryUnicodeEscape") @Test void testCharset() { sql("select _UTF8'\u4F60\u597D'") .withMysql() // produces a simpler output query - .ok("SELECT _UTF-8'\u4F60\u597D'"); + .ok("SELECT _UTF-8'\u4F60\u597D'") + .done(); + sql("select _UTF16'" + ConversionUtil.TEST_UNICODE_STRING + "'") .withMysql() - .ok("SELECT _UTF-16LE'" + ConversionUtil.TEST_UNICODE_STRING + "'"); + .ok("SELECT _UTF-16LE'" + ConversionUtil.TEST_UNICODE_STRING + "'") + .done(); } /** Test case for @@ -323,7 +301,7 @@ private static String toSql(RelNode root, SqlDialect dialect, String query = "select\n" + " sum(\"shelf_width\") filter (where \"net_weight\" > 0),\n" + " sum(\"shelf_width\")\n" - + "from \"foodmart\".\"product\"\n" + + "from \"product\"\n" + "where \"product_id\" > 0\n" + "group by \"product_id\""; final String expectedDefault = "SELECT" @@ -362,7 +340,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .withBigQuery().ok(expectedBigQuery) .withFirebolt().ok(expectedFirebolt) .withMysql().ok(expectedMysql) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for @@ -392,7 +371,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .withMysql().ok(expectedMysql) .withClickHouse().ok(expectedClickHouse) .withPresto().ok(expectedPresto) - .withOracle().ok(expectedOracle); + .withOracle().ok(expectedOracle) + .done(); } @Test void testPiFunctionWithoutParentheses() { @@ -418,7 +398,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .withMysql().ok(expectedMysql) .withClickHouse().ok(expectedClickHouse) .withPresto().ok(expectedPresto) - .withOracle().ok(expectedOracle); + .withOracle().ok(expectedOracle) + .done(); } @Test void testNiladicCurrentDateFunction() { @@ -437,13 +418,14 @@ private static String toSql(RelNode root, SqlDialect dialect, .withPostgresql().ok(expectedPostgresql) .withSpark().ok(expectedSpark) .withMysql().ok(expectedMysql) - .withOracle().ok(expectedOracle); + .withOracle().ok(expectedOracle) + .done(); } @Test void testPivotToSqlFromProductTable() { String query = "select * from (\n" + " select \"shelf_width\", \"net_weight\", \"product_id\"\n" - + " from \"foodmart\".\"product\")\n" + + " from \"product\")\n" + " pivot (sum(\"shelf_width\") as w, count(*) as c\n" + " for (\"product_id\") in (10, 20))"; final String expected = "SELECT \"net_weight\"," @@ -465,23 +447,25 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM foodmart.product\n" + "GROUP BY net_weight"; sql(query).ok(expected) - .withBigQuery().ok(expectedBigQuery); + .withBigQuery().ok(expectedBigQuery).done(); } @Test void testSimpleSelectQueryFromProductTable() { - String query = "select \"product_id\", \"product_class_id\" from \"product\""; + String query = "select \"product_id\", \"product_class_id\"\n" + + "from \"product\""; final String expected = "SELECT \"product_id\", \"product_class_id\"\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithWhereClauseOfLessThan() { String query = "select \"product_id\", \"shelf_width\"\n" - + "from \"product\" where \"product_id\" < 10"; + + "from \"product\"\n" + + "where \"product_id\" < 10"; final String expected = "SELECT \"product_id\", \"shelf_width\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" < 10"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectWhereNotEqualsOrNull() { @@ -491,47 +475,57 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT \"product_id\", \"shelf_width\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"net_weight\" <> CAST(10 AS DOUBLE) OR \"net_weight\" IS NULL"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-5906] * JDBC adapter should generate TABLESAMPLE. */ @Test void testTableSampleBernoulli() { - String query = "select * from \"product\" tablesample bernoulli(11)"; + String query = "select *\n" + + "from \"product\" tablesample bernoulli(11)"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\" TABLESAMPLE BERNOULLI(11.00)"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-5906] * JDBC adapter should generate TABLESAMPLE. */ @Test void testTableSampleBernoulliRepeatable() { - String query = "select * from \"product\" tablesample bernoulli(15) repeatable(10)"; + String query = "select *\n" + + "from \"product\" tablesample bernoulli(15) repeatable(10)"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\" TABLESAMPLE BERNOULLI(15.00) REPEATABLE(10)"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-5906] * JDBC adapter should generate TABLESAMPLE. */ @Test void testTableSampleSystem() { - String query = "select * from \"product\" tablesample system(11)"; + String query = "select *\n" + + "from \"product\" tablesample system(11)"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\" TABLESAMPLE SYSTEM(11.00)"; - sql(query).ok(expected); + sql(query) + // only PARSE; execution hits 'There are not enough rules' + .withPhase(PARSE) + .ok(expected).done(); } /** Test case for * [CALCITE-5906] * JDBC adapter should generate TABLESAMPLE. */ @Test void testTableSampleSystemRepeatable() { - String query = "select * from \"product\" TABLESAMPLE system(11) repeatable(10)"; + String query = "select *\n" + + "from \"product\" TABLESAMPLE system(11) repeatable(10)"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\" TABLESAMPLE SYSTEM(11.00) REPEATABLE(10)"; - sql(query).ok(expected); + sql(query) + // only PARSE; execution hits 'There are not enough rules' + .withPhase(PARSE) + .ok(expected).done(); } /** Test case for @@ -548,7 +542,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IS NULL OR \"COMM\" NOT IN (1, 2)"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } @Test void testSelectWhereNotEquals() { @@ -561,7 +555,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IS NULL OR \"COMM\" <> 1"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } @Test void testSelectWhereIn() { @@ -572,7 +566,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IN (1, 2)"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } @Test void testSelectWhereIn2() { @@ -583,7 +577,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IN (1, 2)"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } @Test void testSelectWhereIn3() { @@ -594,7 +588,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IN (1, 2)"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } @Test void testUsesSubqueryWhenSortingByIdThenOrdinal() { @@ -612,54 +606,60 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY \"JOB\"\n" + "ORDER BY \"JOB\", 2) AS \"t0\""; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } @Test void testSelectQueryWithWhereClauseOfBasicOperators() { - String query = "select * from \"product\" " - + "where (\"product_id\" = 10 OR \"product_id\" <= 5) " + String query = "select *\n" + + "from \"product\"\n" + + "where (\"product_id\" = 10 OR \"product_id\" <= 5)\n" + "AND (80 >= \"shelf_width\" OR \"shelf_width\" > 30)"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE (\"product_id\" = 10 OR \"product_id\" <= 5) " + "AND (CAST(80 AS DOUBLE) >= \"shelf_width\" OR \"shelf_width\" > CAST(30 AS DOUBLE))"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithGroupBy() { - String query = "select count(*) from \"product\" group by \"product_class_id\", \"product_id\""; + String query = "select count(*)\n" + + "from \"product\"\n" + + "group by \"product_class_id\", \"product_id\""; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\", \"product_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithHiveCube() { - String query = "select \"product_class_id\", \"product_id\", count(*) " - + "from \"product\" group by cube(\"product_class_id\", \"product_id\")"; + String query = "select \"product_class_id\", \"product_id\", count(*)\n" + + "from \"product\"\n" + + "group by cube(\"product_class_id\", \"product_id\")"; String expected = "SELECT `product_class_id`, `product_id`, COUNT(*)\n" - + "FROM `foodmart`.`product`\n" - + "GROUP BY `product_class_id`, `product_id` WITH CUBE"; - sql(query).withHive().ok(expected); - SqlDialect sqlDialect = sql(query).withHive().dialect; - assertTrue(sqlDialect.supportsGroupByWithCube()); + + "FROM `foodmart`.`product`\n" + + "GROUP BY `product_class_id`, `product_id` WITH CUBE"; + final RelToSqlFixture f = sql(query).withHive().ok(expected).done(); + assertThat(f.sqlDialect().supportsGroupByWithCube(), is(true)); } @Test void testSelectQueryWithHiveRollup() { - String query = "select \"product_class_id\", \"product_id\", count(*) " - + "from \"product\" group by rollup(\"product_class_id\", \"product_id\")"; + String query = "select \"product_class_id\", \"product_id\", count(*)\n" + + "from \"product\"\n" + + "group by rollup(\"product_class_id\", \"product_id\")"; String expected = "SELECT `product_class_id`, `product_id`, COUNT(*)\n" - + "FROM `foodmart`.`product`\n" - + "GROUP BY `product_class_id`, `product_id` WITH ROLLUP"; - sql(query).withHive().ok(expected); - SqlDialect sqlDialect = sql(query).withHive().dialect; - assertTrue(sqlDialect.supportsGroupByWithRollup()); + + "FROM `foodmart`.`product`\n" + + "GROUP BY `product_class_id`, `product_id` WITH ROLLUP"; + final RelToSqlFixture f = sql(query).withHive().ok(expected).done(); + assertThat(f.sqlDialect().supportsGroupByWithRollup(), is(true)); } @Test void testSelectQueryWithGroupByEmpty() { - final String sql0 = "select count(*) from \"product\" group by ()"; - final String sql1 = "select count(*) from \"product\""; + final String sql0 = "select count(*)\n" + + "from \"product\"\n" + + "group by ()"; + final String sql1 = "select count(*)\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\""; final String expectedMysql = "SELECT COUNT(*)\n" @@ -672,23 +672,27 @@ private static String toSql(RelNode root, SqlDialect dialect, .ok(expected) .withMysql().ok(expectedMysql) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); sql(sql1) .ok(expected) .withMysql().ok(expectedMysql) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testSelectQueryWithGroupByEmpty2() { - final String query = "select 42 as c from \"product\" group by ()"; + final String query = "select 42 as c from emp group by ()"; final String expected = "SELECT *\n" + "FROM (VALUES (42)) AS \"t\" (\"C\")"; final String expectedMysql = "SELECT 42 AS `C`"; sql(query) + .schema(CalciteAssert.SchemaSpec.SCOTT) .ok(expected) .withMysql().ok(expectedMysql) - .withPresto().ok(expected); + .withPresto().ok(expected) + .done(); } /** When ceiling/flooring an integer, BigQuery returns a double while Calcite and other dialects @@ -697,30 +701,30 @@ private static String toSql(RelNode root, SqlDialect dialect, final String query = "SELECT\n" + " TIMESTAMP_SECONDS(CAST(CEIL(CAST(3 AS BIGINT)) AS BIGINT))\n" + " as created_thing\n" - + "FROM `foodmart`.`product`"; - final SqlParser.Config parserConfig = - BigQuerySqlDialect.DEFAULT.configureParser(SqlParser.config()); - final Sql sql = fixture() - .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).parserConfig(parserConfig); + + "FROM emp"; final String expected = "SELECT" + " TIMESTAMP_SECONDS(CAST(CEIL(3) AS INT64)) AS created_thing\n" - + "FROM foodmart.product"; - sql.withSql(query).ok(expected); + + "FROM scott.EMP"; + sql(query) + .schema(CalciteAssert.SchemaSpec.SCOTT) + .parserConfig(BIG_QUERY_PARSER_CONFIG) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY) + .ok(expected).done(); } @Test void testBigQueryFloorPreservesCast() { final String query = "SELECT\n" + " TIMESTAMP_SECONDS(CAST(FLOOR(CAST(3 AS BIGINT)) AS BIGINT))\n" + " as created_thing\n" - + "FROM `foodmart`.`product`"; - final SqlParser.Config parserConfig = - BigQuerySqlDialect.DEFAULT.configureParser(SqlParser.config()); - final Sql sql = fixture() - .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).parserConfig(parserConfig); + + "FROM emp"; final String expected = "SELECT" + " TIMESTAMP_SECONDS(CAST(FLOOR(3) AS INT64)) AS created_thing\n" - + "FROM foodmart.product"; - sql.withSql(query).ok(expected); + + "FROM scott.EMP"; + sql(query) + .schema(CalciteAssert.SchemaSpec.SCOTT) + .parserConfig(BIG_QUERY_PARSER_CONFIG) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY) + .ok(expected).done(); } /** Test case for @@ -728,13 +732,14 @@ private static String toSql(RelNode root, SqlDialect dialect, * Incorrect return type for BigQuery TRUNC. */ @Test void testBigQueryTruncPreservesCast() { final String query = "SELECT CAST(TRUNC(3) AS BIGINT) as created_thing\n" - + " FROM `foodmart`.`product`"; - final SqlParser.Config parserConfig = - BigQuerySqlDialect.DEFAULT.configureParser(SqlParser.config()); - final Sql sql = fixture() - .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).parserConfig(parserConfig); - sql.withSql(query).ok("SELECT CAST(TRUNC(3) AS INT64) AS created_thing\n" - + "FROM foodmart.product"); + + " FROM emp"; + final String expected = "SELECT CAST(TRUNC(3) AS INT64) AS created_thing\n" + + "FROM scott.EMP"; + sql(query) + .schema(CalciteAssert.SchemaSpec.SCOTT) + .parserConfig(BIG_QUERY_PARSER_CONFIG) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY) + .ok(expected).done(); } @Test void testSelectLiteralAgg() { @@ -752,7 +757,8 @@ private static String toSql(RelNode root, SqlDialect dialect, relFn(relFn) .ok(expected) .withMysql().ok(expectedMysql) - .withPresto().ok(expected); + .withPresto().ok(expected) + .done(); } /** Test case for @@ -771,7 +777,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + " \"product_class_id\")\n" + "ORDER BY \"brand_name\", \"product_class_id\""; sql(query) - .withPostgresql().ok(expected); + .withPostgresql().ok(expected).done(); } /** Test case for @@ -798,7 +804,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + " (\"EMPNO\", \"ENAME\"), \"EMPNO\")\n" + "HAVING GROUPING(\"EMPNO\", \"ENAME\", \"JOB\") <> 0" + " AND \"JOB\" = 'DEVELOP') AS \"t\""; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } /** As {@link #testGroupSuperset()}, @@ -828,7 +834,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + " AND \"C\" > 10) AS \"t\") " + "AS \"t0\"\n" + "WHERE \"JOB\" = 'DEVELOP'"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } /** As {@link #testGroupSuperset()}, @@ -862,7 +868,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + " AND (\"C\" > 10 OR \"S\" < 3000)) AS \"t\") " + "AS \"t0\"\n" + "WHERE \"JOB\" = 'DEVELOP'"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } /** As {@link #testGroupSuperset()}, but with no Filter between the Aggregate @@ -882,7 +888,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY GROUPING SETS((\"EMPNO\", \"ENAME\", \"JOB\")," + " (\"EMPNO\", \"ENAME\"), \"EMPNO\")\n" + "HAVING GROUPING(\"EMPNO\", \"ENAME\", \"JOB\") <> 0"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } /** As {@link #testGroupSuperset()}, but with no Filter between the Aggregate @@ -904,7 +910,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + " (\"EMPNO\", \"ENAME\"), \"EMPNO\")\n" + "HAVING GROUPING(\"EMPNO\", \"ENAME\", \"JOB\") <> 0\n" + "ORDER BY 4"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } /** As {@link #testGroupSuperset()}, but with Filter condition and Where condition. */ @@ -919,8 +925,7 @@ private static String toSql(RelNode root, SqlDialect dialect, b.count(false, "C"), b.sum(false, "S", b.field("SAL"))) .filter( - b.lessThan( - b.call(SqlStdOperatorTable.GROUP_ID, b.field("EMPNO")), + b.lessThan(b.call(SqlStdOperatorTable.GROUPING, b.field("EMPNO")), b.literal(1))) .filter(b.equals(b.field("JOB"), b.literal("DEVELOP"))) .project(b.field("JOB")) @@ -932,10 +937,10 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY GROUPING SETS((\"EMPNO\", \"ENAME\", \"JOB\")," + " (\"EMPNO\", \"ENAME\"), \"EMPNO\", ())\n" + "HAVING GROUPING(\"EMPNO\", \"ENAME\", \"JOB\") <> 0" - + " AND GROUP_ID(\"EMPNO\") < 1) AS \"t\") " + + " AND GROUPING(\"EMPNO\") < 1) AS \"t\") " + "AS \"t0\"\n" + "WHERE \"JOB\" = 'DEVELOP'"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @@ -957,10 +962,11 @@ private static String toSql(RelNode root, SqlDialect dialect, .project(b.alias(b.field(1), "l_v")) .build(); // RelFieldTrimmer maybe build the RelNode. - relFn(fn).ok("SELECT \"t\".\"V\" AS \"l_v\"\n" + final String expected = "SELECT \"t\".\"V\" AS \"l_v\"\n" + "FROM (VALUES (1, 2)) AS \"t\" (\"K\", \"V\")\n" + "INNER JOIN " - + "(VALUES (1)) AS \"t0\" (\"K\") ON \"t\".\"K\" = \"t0\".\"K\""); + + "(VALUES (1)) AS \"t0\" (\"K\") ON \"t\".\"K\" = \"t0\".\"K\""; + relFn(fn).ok(expected).done(); } /** As {@link #testTrimmedAggregateUnderProject()} @@ -980,10 +986,11 @@ private static String toSql(RelNode root, SqlDialect dialect, .project(b.alias(b.field(1), "l_v")) .build(); // RelFieldTrimmer maybe build the RelNode. - relFn(fn).ok("SELECT \"t\".\"V\" AS \"l_v\"\n" + final String expected = "SELECT \"t\".\"V\" AS \"l_v\"\n" + "FROM (VALUES (1, 2)) AS \"t\" (\"K\", \"V\")\n" + "INNER JOIN (SELECT COUNT(\"ENAME\") AS \"DUMMY\"\n" - + "FROM \"scott\".\"EMP\") AS \"t0\" ON \"t\".\"K\" = \"t0\".\"DUMMY\""); + + "FROM \"scott\".\"EMP\") AS \"t0\" ON \"t\".\"K\" = \"t0\".\"DUMMY\""; + relFn(fn).ok(expected).done(); } /** Tests GROUP BY ROLLUP of two columns. The SQL for MySQL has @@ -1013,7 +1020,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .ok(expected) .withMysql().ok(expectedMysql) .withMysql8().ok(expectedMysql8) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** As {@link #testSelectQueryWithGroupByRollup()}, @@ -1040,7 +1048,8 @@ private static String toSql(RelNode root, SqlDialect dialect, sql(query) .ok(expected) .withMysql().ok(expectedMysql) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for @@ -1055,10 +1064,11 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"foodmart\".\"product\"\n" + "GROUP BY ROLLUP(\"brand_name\", \"product_class_id\")"; sql(query1) - .withPostgresql().ok(expected1); + .withPostgresql().ok(expected1).done(); - final String query2 = "select \"product_class_id\", \"brand_name\", \"product_id\"\n" - + "from \"product\"\n" + final String query2 = "select\n" + + " \"product_class_id\", \"brand_name\", \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "group by GROUPING SETS (" + " (\"product_class_id\", \"brand_name\", \"product_id\")," + " (\"product_class_id\", \"brand_name\")," @@ -1067,7 +1077,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"foodmart\".\"product\"\n" + "GROUP BY ROLLUP(\"brand_name\", \"product_class_id\", \"product_id\")"; sql(query2) - .withPostgresql().ok(expected2); + .withPostgresql().ok(expected2).done(); } /** Tests a query with GROUP BY and a sub-query which is also with GROUP BY. @@ -1075,25 +1085,28 @@ private static String toSql(RelNode root, SqlDialect dialect, * incorrect. */ @Test void testSelectQueryWithGroupBySubQuery1() { final String query = "select \"product_class_id\", avg(\"product_id\")\n" - + "from (select \"product_class_id\", \"product_id\", avg(\"product_class_id\")\n" - + "from \"product\"\n" - + "group by \"product_class_id\", \"product_id\") as t\n" + + "from (\n" + + " select \"product_class_id\", \"product_id\",\n" + + " avg(\"product_class_id\")\n" + + " from \"foodmart\".\"product\"\n" + + " group by \"product_class_id\", \"product_id\") as t\n" + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\", AVG(\"product_id\")\n" + "FROM (SELECT \"product_class_id\", \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\", \"product_id\") AS \"t1\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Tests query without GROUP BY but an aggregate function * and a sub-query which is with GROUP BY. */ @Test void testSelectQueryWithGroupBySubQuery2() { final String query = "select sum(\"product_id\")\n" - + "from (select \"product_class_id\", \"product_id\"\n" - + "from \"product\"\n" - + "group by \"product_class_id\", \"product_id\") as t"; + + "from (\n" + + " select \"product_class_id\", \"product_id\"\n" + + " from \"foodmart\".\"product\"\n" + + " group by \"product_class_id\", \"product_id\") as t"; final String expected = "SELECT SUM(\"product_id\")\n" + "FROM (SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" @@ -1104,22 +1117,22 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY `product_class_id`, `product_id`) AS `t1`"; sql(query) .ok(expected) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql).done(); // Equivalent sub-query that uses SELECT DISTINCT final String query2 = "select sum(\"product_id\")\n" + "from (select distinct \"product_class_id\", \"product_id\"\n" - + " from \"product\") as t"; + + " from \"foodmart\".\"product\") as t"; sql(query2) .ok(expected) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql).done(); } /** CUBE of one column is equivalent to ROLLUP, and Calcite recognizes * this. */ @Test void testSelectQueryWithSingletonCube() { final String query = "select \"product_class_id\", count(*) as c\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by cube(\"product_class_id\")\n" + "order by 1, 2"; final String expected = "SELECT \"product_class_id\", COUNT(*) AS \"C\"\n" @@ -1143,14 +1156,15 @@ private static String toSql(RelNode root, SqlDialect dialect, .ok(expected) .withMysql().ok(expectedMysql) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** As {@link #testSelectQueryWithSingletonCube()}, but no ORDER BY * clause. */ @Test void testSelectQueryWithSingletonCubeNoOrderBy() { final String query = "select \"product_class_id\", count(*) as c\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by cube(\"product_class_id\")"; final String expected = "SELECT \"product_class_id\", COUNT(*) AS \"C\"\n" + "FROM \"foodmart\".\"product\"\n" @@ -1168,7 +1182,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .ok(expected) .withMysql().ok(expectedMysql) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Cannot rewrite if ORDER BY contains a column not in GROUP BY (in this @@ -1176,7 +1191,7 @@ private static String toSql(RelNode root, SqlDialect dialect, @Test void testSelectQueryWithRollupOrderByCount() { final String query = "select \"product_class_id\", \"brand_name\",\n" + " count(*) as c\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by rollup(\"product_class_id\", \"brand_name\")\n" + "order by 1, 2, 3"; final String expected = "SELECT \"product_class_id\", \"brand_name\"," @@ -1199,13 +1214,14 @@ private static String toSql(RelNode root, SqlDialect dialect, sql(query) .ok(expected) .withMysql().ok(expectedMysql) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** As {@link #testSelectQueryWithSingletonCube()}, but with LIMIT. */ @Test void testSelectQueryWithCubeLimit() { final String query = "select \"product_class_id\", count(*) as c\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by cube(\"product_class_id\")\n" + "limit 5"; final String expected = "SELECT \"product_class_id\", COUNT(*) AS \"C\"\n" @@ -1230,7 +1246,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .ok(expected) .withMysql().ok(expectedMysql) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for @@ -1238,51 +1255,42 @@ private static String toSql(RelNode root, SqlDialect dialect, * RelToSqlConverter[ORDER BY] generates an incorrect field alias * when 2 projection fields have the same name. */ @Test void testOrderByFieldNotInTheProjectionWithASameAliasAsThatInTheProjection() { - final RelBuilder builder = relBuilder(); - final RelNode base = builder - .scan("EMP") - .project( - builder.alias( - builder.call(SqlStdOperatorTable.UPPER, builder.field("ENAME")), "EMPNO"), - builder.field("EMPNO") - ) - .sort(1) - .project(builder.field(0)) - .build(); + final Function relFn = b -> + b.scan("EMP") + .project( + b.alias(b.call(SqlStdOperatorTable.UPPER, b.field("ENAME")), + "EMPNO"), + b.field("EMPNO")) + .sort(1) + .project(b.field(0)) + .build(); // The expected string should deliberately have a subquery to handle a scenario in which // the projection field has an alias with the same name as that of the field used in the // ORDER BY - String expectedSql1 = "" - + "SELECT \"EMPNO\"\n" + String expectedSql1 = "SELECT \"EMPNO\"\n" + "FROM (SELECT UPPER(\"ENAME\") AS \"EMPNO\", \"EMPNO\" AS \"EMPNO0\"\n" + "FROM \"scott\".\"EMP\"\n" + "ORDER BY 2) AS \"t0\""; - String actualSql1 = toSql(base); - assertThat(actualSql1, isLinux(expectedSql1)); + relFn(relFn).ok(expectedSql1).done(); - String actualSql2 = toSql(base, nonOrdinalDialect()); String expectedSql2 = "SELECT UPPER(ENAME) AS EMPNO\n" + "FROM scott.EMP\n" + "ORDER BY EMPNO"; - assertThat(actualSql2, isLinux(expectedSql2)); + relFn(relFn).dialect(NON_ORDINAL).ok(expectedSql2).done(); } @Test void testOrderByExpressionNotInTheProjectionThatRefersToUnderlyingFieldWithSameAlias() { - final RelBuilder builder = relBuilder(); - final RelNode base = builder - .scan("EMP") - .project( - builder.alias( - builder.call(SqlStdOperatorTable.UPPER, builder.field("ENAME")), "EMPNO"), - builder.call( - SqlStdOperatorTable.PLUS, builder.field("EMPNO"), - builder.literal(1) - ) - ) - .sort(1) - .project(builder.field(0)) - .build(); + final Function relFn = b -> + b.scan("EMP") + .project( + b.alias(b.call(SqlStdOperatorTable.UPPER, b.field("ENAME")), + "EMPNO"), + b.call(SqlStdOperatorTable.PLUS, b.field("EMPNO"), + b.literal(1))) + .sort(1) + .project(b.field(0)) + .build(); // An output such as // "SELECT UPPER(\"ENAME\") AS \"EMPNO\"\n" @@ -1290,123 +1298,127 @@ private static String toSql(RelNode root, SqlDialect dialect, // + "ORDER BY \"EMPNO\" + 1" // would be incorrect since the rel is sorting by the field \"EMPNO\" + 1 in which EMPNO // refers to the physical column EMPNO and not the alias - String actualSql1 = toSql(base); - String expectedSql1 = "" - + "SELECT \"EMPNO\"\n" + String expectedSql1 = "SELECT \"EMPNO\"\n" + "FROM (SELECT UPPER(\"ENAME\") AS \"EMPNO\", \"EMPNO\" + 1 AS \"$f1\"\n" + "FROM \"scott\".\"EMP\"\n" + "ORDER BY 2) AS \"t0\""; - assertThat(actualSql1, isLinux(expectedSql1)); + relFn(relFn).ok(expectedSql1).done(); - String actualSql2 = toSql(base, nonOrdinalDialect()); String expectedSql2 = "SELECT UPPER(ENAME) AS EMPNO\n" + "FROM scott.EMP\n" + "ORDER BY EMPNO + 1"; - assertThat(actualSql2, isLinux(expectedSql2)); + relFn(relFn).dialect(NON_ORDINAL).ok(expectedSql2).done(); } @Test void testSelectQueryWithMinAggregateFunction() { - String query = "select min(\"net_weight\") from \"product\" group by \"product_class_id\" "; + String query = "select min(\"net_weight\")\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\" "; final String expected = "SELECT MIN(\"net_weight\")\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithMinAggregateFunction1() { - String query = "select \"product_class_id\", min(\"net_weight\") from" - + " \"product\" group by \"product_class_id\""; + String query = "select \"product_class_id\", min(\"net_weight\")\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\", MIN(\"net_weight\")\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithSumAggregateFunction() { - String query = - "select sum(\"net_weight\") from \"product\" group by \"product_class_id\" "; + String query = "select sum(\"net_weight\")\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\" "; final String expected = "SELECT SUM(\"net_weight\")\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithMultipleAggregateFunction() { - String query = "select sum(\"net_weight\"), min(\"low_fat\"), count(*)" - + " from \"product\" group by \"product_class_id\" "; + String query = "select sum(\"net_weight\"), min(\"low_fat\"), count(*)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT SUM(\"net_weight\"), MIN(\"low_fat\")," + " COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithMultipleAggregateFunction1() { - String query = "select \"product_class_id\"," - + " sum(\"net_weight\"), min(\"low_fat\"), count(*)" - + " from \"product\" group by \"product_class_id\" "; + String query = "select \"product_class_id\",\n" + + " sum(\"net_weight\"), min(\"low_fat\"), count(*)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\"," + " SUM(\"net_weight\"), MIN(\"low_fat\"), COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithGroupByAndProjectList() { - String query = "select \"product_class_id\", \"product_id\", count(*) " - + "from \"product\" group by \"product_class_id\", \"product_id\" "; + String query = "select \"product_class_id\", \"product_id\", count(*)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\", \"product_id\""; final String expected = "SELECT \"product_class_id\", \"product_id\"," + " COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\", \"product_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testCastDecimal1() { final String query = "select -0.0000000123\n" - + " from \"expense_fact\""; + + "from \"foodmart\".\"expense_fact\""; final String expected = "SELECT -0.0000000123\n" + "FROM \"foodmart\".\"expense_fact\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-4706] * JDBC adapter generates casts exceeding Redshift's data types bounds. */ @Test void testCastDecimalBigPrecision() { - final String query = "select cast(\"product_id\" as decimal(60,2)) " - + "from \"product\" "; + final String query = "select cast(\"product_id\" as decimal(60,2))\n" + + "from \"foodmart\".\"product\""; final String expectedRedshift = "SELECT CAST(\"product_id\" AS DECIMAL(38, 2))\n" + "FROM \"foodmart\".\"product\""; sql(query) .withRedshift() - .ok(expectedRedshift); + .ok(expectedRedshift).done(); } /** Test case for * [CALCITE-4706] * JDBC adapter generates casts exceeding Redshift's data types bounds. */ @Test void testCastDecimalBigScale() { - final String query = "select cast(\"product_id\" as decimal(2,90)) " - + "from \"product\" "; + final String query = "select cast(\"product_id\" as decimal(2,90))\n" + + "from \"foodmart\".\"product\""; final String expectedRedshift = "SELECT CAST(\"product_id\" AS DECIMAL(2, 37))\n" + "FROM \"foodmart\".\"product\""; sql(query) .withRedshift() - .ok(expectedRedshift); + .ok(expectedRedshift).done(); } /** Test case for * [CALCITE-4706] * JDBC adapter generates casts exceeding Redshift's data types bounds. */ @Test void testCastLongChar() { - final String query = "select cast(\"product_id\" as char(9999999)) " - + "from \"product\" "; + final String query = "select cast(\"product_id\" as char(9999999))\n" + + "from \"foodmart\".\"product\""; final String expectedRedshift = "SELECT CAST(\"product_id\" AS CHAR(4096))\n" + "FROM \"foodmart\".\"product\""; sql(query) .withRedshift() - .ok(expectedRedshift); + .ok(expectedRedshift).done(); } /** Test case for @@ -1415,7 +1427,7 @@ private static String toSql(RelNode root, SqlDialect dialect, * max length. */ @Test void testCastLongVarchar1() { final String query = "select cast(\"store_id\" as VARCHAR(10485761))\n" - + " from \"expense_fact\""; + + "from \"foodmart\".\"expense_fact\""; final String expectedPostgresql = "SELECT CAST(\"store_id\" AS VARCHAR(256))\n" + "FROM \"foodmart\".\"expense_fact\""; final String expectedOracle = "SELECT CAST(\"store_id\" AS VARCHAR(512))\n" @@ -1428,7 +1440,7 @@ private static String toSql(RelNode root, SqlDialect dialect, .withOracleModifiedTypeSystem() .ok(expectedOracle) .withRedshift() - .ok(expectedRedshift); + .ok(expectedRedshift).done(); } /** Test case for @@ -1437,18 +1449,18 @@ private static String toSql(RelNode root, SqlDialect dialect, * max length. */ @Test void testCastLongVarchar2() { final String query = "select cast(\"store_id\" as VARCHAR(175))\n" - + " from \"expense_fact\""; + + "from \"foodmart\".\"expense_fact\""; final String expectedPostgresql = "SELECT CAST(\"store_id\" AS VARCHAR(175))\n" + "FROM \"foodmart\".\"expense_fact\""; sql(query) .withPostgresqlModifiedTypeSystem() - .ok(expectedPostgresql); + .ok(expectedPostgresql).done(); final String expectedOracle = "SELECT CAST(\"store_id\" AS VARCHAR(175))\n" + "FROM \"foodmart\".\"expense_fact\""; sql(query) .withOracleModifiedTypeSystem() - .ok(expectedOracle); + .ok(expectedOracle).done(); } /** Test case for @@ -1466,8 +1478,11 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expectedPostgresql = "SELECT COALESCE(SUM(\"MGR\"), 0) AS \"s\"\n" + "FROM \"scott\".\"EMP\""; relFn(relFn) + // only PARSE; [CALCITE-6834] logged + .withPhase(PARSE) .withPostgresql().ok(expectedPostgresql) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql) + .done(); } /** As {@link #testSum0BecomesCoalesce()} but for windowed aggregates. */ @@ -1489,7 +1504,9 @@ private static String toSql(RelNode root, SqlDialect dialect, RuleSet rules = RuleSets.ofList(CoreRules.PROJECT_OVER_SUM_TO_SUM0_RULE); - sql(query).withPostgresql().optimize(rules, hepPlanner).ok(expectedPostgresql); + sql(query) + .withPostgresql().optimize(rules, hepPlanner).ok(expectedPostgresql) + .done(); } /** Test case for @@ -1504,7 +1521,8 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"foodmart\".\"product\"\n" + "WHERE (\"product_id\" = 0) = (\"product_class_id\" = 0)"; sql(query) - .ok(expectedQuery); + .ok(expectedQuery) + .done(); } @Test void testMissingParenthesesWithCondition2() { @@ -1518,7 +1536,8 @@ private static String toSql(RelNode root, SqlDialect dialect, + "(SELECT \"product_id\" = 0\n" + "FROM \"foodmart\".\"product\")"; sql(query) - .ok(expectedQuery); + .ok(expectedQuery) + .done(); } @Test void testMissingParenthesesWithProject() { @@ -1527,7 +1546,8 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expectedQuery = "SELECT (\"product_id\" = 0) = (\"product_class_id\" = 0)\n" + "FROM \"foodmart\".\"product\""; sql(query) - .ok(expectedQuery); + .ok(expectedQuery) + .done(); } @Test void testMissingParenthesesWithSubquery1() { @@ -1542,8 +1562,10 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"foodmart\".\"product\")\n" + "FROM \"foodmart\".\"product\""; sql(query) + .withPhase(PARSE) // only PARSE; execution hits [CALCITE-6458] .withConfig(c -> c.withExpand(false)) - .ok(expectedQuery); + .ok(expectedQuery) + .done(); } @Test void testMissingParenthesesWithSubquery2() { @@ -1558,8 +1580,10 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"foodmart\".\"product\")\n" + "FROM \"foodmart\".\"product\""; sql(query) + .withPhase(PARSE) // only PARSE; execution hits [CALCITE-6458] .withConfig(c -> c.withExpand(false)) - .ok(expectedQuery); + .ok(expectedQuery) + .done(); } @Test void testMissingParenthesesWithSubquery3() { @@ -1576,44 +1600,78 @@ private static String toSql(RelNode root, SqlDialect dialect, + "IN (SELECT \"product_class_id\" = 0\n" + "FROM \"foodmart\".\"product\")"; sql(query) + // only PARSE; Calcite planner hits the following bug: + // [CALCITE-6824] FieldTrimmer corrupts plan in query that compares + // two BOOLEAN subqueries + .withPhase(PARSE) .withConfig(c -> c.withExpand(false)) - .ok(expectedQuery); + .ok(expectedQuery) + .done(); } /** Test case for * [CALCITE-5955] * BigQuery PERCENTILE functions are unparsed incorrectly. */ @Test void testPercentileContWindow() { - final String partitionQuery = "select percentile_cont(\"product_id\", 0.5)\n" - + "over(partition by \"product_id\")\n" - + "from \"foodmart\".\"product\""; - final String expectedPartition = "SELECT PERCENTILE_CONT(product_id, 0.5) " - + "OVER (PARTITION BY product_id)\n" - + "FROM foodmart.product"; final String query = "select percentile_cont(\"product_id\", 0.5) over()\n" + "from \"foodmart\".\"product\""; final String expectedQuery = "SELECT PERCENTILE_CONT(product_id, 0.5) OVER ()\n" + "FROM foodmart.product"; - sql(partitionQuery).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedPartition); - sql(query).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedQuery); + sql(query) + // Execution disabled due to error: + // "IllegalStateException: Unable to implement EnumerableCalc" + .withPhase(PARSE) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedQuery) + .done(); + } + + /** As {@link #testPercentileContWindow()} but with {@code PARTITION BY}. */ + @Test void testPercentileContWindowPartition() { + final String query = "select percentile_cont(\"product_id\", 0.5)\n" + + "over(partition by \"product_id\")\n" + + "from \"foodmart\".\"product\""; + final String expected = "SELECT PERCENTILE_CONT(product_id, 0.5) " + + "OVER (PARTITION BY product_id)\n" + + "FROM foodmart.product"; + sql(query) + // Execution disabled due to error: + // "IllegalStateException: Unable to implement EnumerableCalc" + .withPhase(PARSE) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expected) + .done(); } /** Test case for * [CALCITE-5955] * BigQuery PERCENTILE functions are unparsed incorrectly. */ @Test void testPercentileDiscWindowFrameClause() { + final String query = "select percentile_disc(\"product_id\", 0.5) over()\n" + + "from \"foodmart\".\"product\""; + final String expectedQuery = "SELECT PERCENTILE_DISC(product_id, 0.5) OVER ()\n" + + "FROM foodmart.product"; + sql(query) + // Execution disabled due to error: + // "IllegalStateException: Unable to implement EnumerableCalc" + .withPhase(PARSE) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedQuery) + .done(); + } + + /** As {@link #testPercentileDiscWindowFrameClause()} but with + * {@code PARTITION BY}. */ + @Test void testPercentileDiscWindowFrameClausePartition() { final String partitionQuery = "select percentile_disc(\"product_id\", 0.5)\n" + "over(partition by \"product_id\")\n" + "from \"foodmart\".\"product\""; final String expectedPartition = "SELECT PERCENTILE_DISC(product_id, 0.5) " + "OVER (PARTITION BY product_id)\n" + "FROM foodmart.product"; - final String query = "select percentile_disc(\"product_id\", 0.5) over()\n" - + "from \"foodmart\".\"product\""; - final String expectedQuery = "SELECT PERCENTILE_DISC(product_id, 0.5) OVER ()\n" - + "FROM foodmart.product"; - sql(partitionQuery).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedPartition); - sql(query).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedQuery); + sql(partitionQuery) + // Execution disabled due to error: + // "IllegalStateException: Unable to implement EnumerableCalc" + .withPhase(PARSE) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedPartition) + .done(); } /** Test case for @@ -1628,32 +1686,31 @@ private static String toSql(RelNode root, SqlDialect dialect, .mapToObj(i -> b.equals(b.field("EMPNO"), b.literal(i))) .collect(Collectors.toList()))) .build(); - final SqlDialect dialect = DatabaseProduct.CALCITE.getDialect(); + final SqlDialect dialect = CONFIG_SUPPLIER.get().get(CALCITE).sqlDialect; final RelNode root = relFn.apply(relBuilder()); final RelToSqlConverter converter = new RelToSqlConverter(dialect); final SqlNode sqlNode = converter.visitRoot(root).asStatement(); - final String sqlString = sqlNode.accept(new SqlShuttle()) - .toSqlString(dialect).getSql(); + final String sqlString = + requireNonNull(sqlNode.accept(new SqlShuttle())) + .toSqlString(dialect).getSql(); assertThat(sqlString, notNullValue()); } @Test void testAntiJoin() { - final RelBuilder builder = relBuilder(); - final RelNode root = builder - .scan("DEPT") - .scan("EMP") - .join( - JoinRelType.ANTI, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function relFn = b -> + b.scan("DEPT") + .scan("EMP") + .join(JoinRelType.ANTI, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPTNO\"\n" + "FROM \"scott\".\"DEPT\"\n" + "WHERE NOT EXISTS (SELECT 1\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"DEPT\".\"DEPTNO\" = \"EMP\".\"DEPTNO\")"; - assertThat(toSql(root), isLinux(expectedSql)); + relFn(relFn).ok(expectedSql).done(); } /** Test case for @@ -1679,56 +1736,56 @@ private static String toSql(RelNode root, SqlDialect dialect, b.field("c"), b.literal(10))) .build(); - // PostgreSQL does not not support nested aggregations + // PostgreSQL does not support nested aggregations final String expectedPostgresql = "SELECT COUNT(DISTINCT \"rank\") AS \"c\"\n" + "FROM (SELECT RANK() OVER (ORDER BY \"SAL\") AS \"rank\"\n" + "FROM \"scott\".\"EMP\") AS \"t\"\n" + "HAVING COUNT(DISTINCT \"rank\") >= 10"; - relFn(relFn).withPostgresql().ok(expectedPostgresql); // Oracle does support nested aggregations final String expectedOracle = "SELECT COUNT(DISTINCT RANK() OVER (ORDER BY \"SAL\")) \"c\"\n" + "FROM \"scott\".\"EMP\"\n" + "HAVING COUNT(DISTINCT RANK() OVER (ORDER BY \"SAL\")) >= 10"; - relFn(relFn).withOracle().ok(expectedOracle); + + relFn(relFn) + .withPhase(PARSE) // only PARSE; Calcite considers the query invalid + .withPostgresql().ok(expectedPostgresql) + .withOracle().ok(expectedOracle) + .done(); } @Test void testSemiJoin() { - final RelBuilder builder = relBuilder(); - final RelNode root = builder - .scan("DEPT") - .scan("EMP") - .join( - JoinRelType.SEMI, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function relFn = b -> + b.scan("DEPT") + .scan("EMP") + .join(JoinRelType.SEMI, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPTNO\"\n" + "FROM \"scott\".\"DEPT\"\n" + "WHERE EXISTS (SELECT 1\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"DEPT\".\"DEPTNO\" = \"EMP\".\"DEPTNO\")"; - assertThat(toSql(root), isLinux(expectedSql)); + relFn(relFn).ok(expectedSql).done(); } @Test void testSemiJoinFilter() { - final RelBuilder builder = relBuilder(); - final RelNode root = builder - .scan("DEPT") - .scan("EMP") - .filter( - builder.call(SqlStdOperatorTable.GREATER_THAN, - builder.field("EMPNO"), - builder.literal((short) 10))) - .join( - JoinRelType.SEMI, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function relFn = b -> + b.scan("DEPT") + .scan("EMP") + .filter( + b.call(SqlStdOperatorTable.GREATER_THAN, + b.field("EMPNO"), + b.literal((short) 10))) + .join(JoinRelType.SEMI, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPTNO\"\n" + "FROM \"scott\".\"DEPT\"\n" + "WHERE EXISTS (SELECT 1\n" @@ -1736,30 +1793,27 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"scott\".\"EMP\"\n" + "WHERE \"EMPNO\" > 10) AS \"t\"\n" + "WHERE \"DEPT\".\"DEPTNO\" = \"t\".\"DEPTNO\")"; - assertThat(toSql(root), isLinux(expectedSql)); + relFn(relFn).ok(expectedSql).done(); } @Test void testSemiJoinProject() { - final RelBuilder builder = relBuilder(); - final RelNode root = builder - .scan("DEPT") - .scan("EMP") - .project( - builder.field(builder.peek().getRowType().getField("EMPNO", false, false).getIndex()), - builder.field(builder.peek().getRowType().getField("DEPTNO", false, false).getIndex())) - .join( - JoinRelType.SEMI, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function relFn = b -> + b.scan("DEPT") + .scan("EMP") + .project(b.field("EMPNO"), + b.field("DEPTNO")) + .join(JoinRelType.SEMI, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPTNO\"\n" + "FROM \"scott\".\"DEPT\"\n" + "WHERE EXISTS (SELECT 1\n" + "FROM (SELECT \"EMPNO\", \"DEPTNO\"\n" + "FROM \"scott\".\"EMP\") AS \"t\"\n" + "WHERE \"DEPT\".\"DEPTNO\" = \"t\".\"DEPTNO\")"; - assertThat(toSql(root), isLinux(expectedSql)); + relFn(relFn).ok(expectedSql).done(); } /** Test case for @@ -1782,28 +1836,23 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"foodmart\".\"department\") AS \"$cor0\",\n" + "LATERAL UNNEST((SELECT \"$cor0\".\"$f2\"\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\"))) AS \"t10\" (\"col_0\")"; - sql(sql).withLibrary(SqlLibrary.BIG_QUERY).ok(expected); + sql(sql).withLibrary(SqlLibrary.BIG_QUERY).ok(expected).done(); } /** Test case for * [CALCITE-5395] * RelToSql converter fails when SELECT * is under a semi-join node. */ @Test void testUnionUnderSemiJoinNode() { - final RelBuilder builder = relBuilder(); - final RelNode base = builder - .scan("EMP") - .scan("EMP") - .union(true) - .build(); - final RelNode root = builder - .push(base) - .scan("DEPT") - .join( - JoinRelType.SEMI, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function relFn = b -> + b.scan("EMP") + .scan("EMP") + .union(true) + .scan("DEPT") + .join(JoinRelType.SEMI, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPTNO\"\n" + "FROM (SELECT *\n" + "FROM (SELECT *\n" @@ -1814,68 +1863,66 @@ private static String toSql(RelNode root, SqlDialect dialect, + "WHERE EXISTS (SELECT 1\n" + "FROM \"scott\".\"DEPT\"\n" + "WHERE \"t\".\"DEPTNO\" = \"DEPT\".\"DEPTNO\")) AS \"t\""; - assertThat(toSql(root), isLinux(expectedSql)); + relFn(relFn) + // cannot validate because generated SQL is invalid; see [CALCITE-6833] + .withPhase(PARSE) + .ok(expectedSql).done(); } @Test void testSemiNestedJoin() { - final RelBuilder builder = relBuilder(); - final RelNode base = builder - .scan("EMP") - .scan("EMP") - .join( - JoinRelType.INNER, builder.equals( - builder.field(2, 0, "EMPNO"), - builder.field(2, 1, "EMPNO"))) - .build(); - final RelNode root = builder - .scan("DEPT") - .push(base) - .join( - JoinRelType.SEMI, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function baseFn = b -> + b.scan("EMP") + .scan("EMP") + .join(JoinRelType.INNER, + b.equals(b.field(2, 0, "EMPNO"), + b.field(2, 1, "EMPNO"))) + .build(); + final Function relFn = b -> + b.scan("DEPT") + .push(baseFn.apply(b)) + .join(JoinRelType.SEMI, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPTNO\"\n" + "FROM \"scott\".\"DEPT\"\n" + "WHERE EXISTS (SELECT 1\n" + "FROM \"scott\".\"EMP\"\n" - + "INNER JOIN \"scott\".\"EMP\" AS \"EMP0\" ON \"EMP\".\"EMPNO\" = \"EMP0\".\"EMPNO\"\n" + + "INNER JOIN \"scott\".\"EMP\" AS \"EMP0\"" + + " ON \"EMP\".\"EMPNO\" = \"EMP0\".\"EMPNO\"\n" + "WHERE \"DEPT\".\"DEPTNO\" = \"EMP\".\"DEPTNO\")"; - assertThat(toSql(root), isLinux(expectedSql)); + relFn(relFn).ok(expectedSql).done(); } /** Test case for * [CALCITE-5394] * RelToSql converter fails when semi-join is under a join node. */ @Test void testSemiJoinUnderJoin() { - final RelBuilder builder = relBuilder(); - final RelNode base = builder - .scan("EMP") - .scan("EMP") - .join( - JoinRelType.SEMI, builder.equals( - builder.field(2, 0, "EMPNO"), - builder.field(2, 1, "EMPNO"))) - .build(); - final RelNode root = builder - .scan("DEPT") - .push(base) - .join( - JoinRelType.INNER, builder.equals( - builder.field(2, 1, "DEPTNO"), - builder.field(2, 0, "DEPTNO"))) - .project(builder.field("DEPTNO")) - .build(); + final Function baseFn = b -> + b.scan("EMP") + .scan("EMP") + .join(JoinRelType.SEMI, + b.equals(b.field(2, 0, "EMPNO"), + b.field(2, 1, "EMPNO"))) + .build(); + final Function relFn = b -> + b.scan("DEPT") + .push(baseFn.apply(b)) + .join(JoinRelType.INNER, + b.equals(b.field(2, 1, "DEPTNO"), + b.field(2, 0, "DEPTNO"))) + .project(b.field("DEPTNO")) + .build(); final String expectedSql = "SELECT \"DEPT\".\"DEPTNO\"\n" + "FROM \"scott\".\"DEPT\"\n" + "INNER JOIN (SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE EXISTS (SELECT 1\n" + "FROM \"scott\".\"EMP\" AS \"EMP0\"\n" - + "WHERE \"EMP\".\"EMPNO\" = \"EMP0\".\"EMPNO\")) AS \"t\" ON \"DEPT\".\"DEPTNO\" = \"t\"" - + ".\"DEPTNO\""; - assertThat(toSql(root), isLinux(expectedSql)); + + "WHERE \"EMP\".\"EMPNO\" = \"EMP0\".\"EMPNO\")) AS \"t\"" + + " ON \"DEPT\".\"DEPTNO\" = \"t\".\"DEPTNO\""; + relFn(relFn).ok(expectedSql).done(); } /** Test case for @@ -1897,7 +1944,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"EMPNO\" IN (0, 1, 2, 3) AND \"DEPTNO\" IN (5, 6, 7)"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } /** Test case for @@ -1917,7 +1964,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"EMPNO\" >= 6 AND \"EMPNO\" < 8 OR \"EMPNO\" >= 10 AND \"EMPNO\" < 12"; - relFn(relFn).optimize(rules, null).ok(expected); + relFn(relFn).optimize(rules, null).ok(expected).done(); } /** Test case for @@ -1936,7 +1983,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expected = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IS NULL OR \"COMM\" IN (1.0, 20000.0)"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } /** Test case for @@ -1957,7 +2004,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + "FROM \"scott\".\"EMP\"\n" + "WHERE \"COMM\" IS NULL OR \"COMM\" >= 1.0 AND \"COMM\" <= 20000.0"; - relFn(relFn).ok(expected); + relFn(relFn).ok(expected).done(); } /** Test case for @@ -1970,7 +2017,7 @@ private static String toSql(RelNode root, SqlDialect dialect, // Oracle can do it in a single SELECT. final String query = "select\n" + " SUM(\"net_weight1\") as \"net_weight_converted\"\n" - + " from (" + + "from (\n" + " select\n" + " SUM(\"net_weight\") as \"net_weight1\"\n" + " from \"foodmart\".\"product\"\n" @@ -2010,7 +2057,8 @@ private static String toSql(RelNode root, SqlDialect dialect, .withPostgresql().ok(expectedPostgresql) .withSpark().ok(expectedSpark) .withVertica().ok(expectedVertica) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for @@ -2030,7 +2078,7 @@ private static String toSql(RelNode root, SqlDialect dialect, .build(); final String expectedSql = "SELECT COUNT(`MGR`) AS `c`\n" + "FROM `scott`.`EMP`"; - relFn(relFn).withMysql().ok(expectedSql); + relFn(relFn).withMysql().ok(expectedSql).done(); } /** As {@link #testNestedAggregatesMySqlTable()}, but input is a sub-query, @@ -2045,7 +2093,7 @@ private static String toSql(RelNode root, SqlDialect dialect, final String expectedSql = "SELECT COUNT(`MGR`) AS `c`\n" + "FROM `scott`.`EMP`\n" + "WHERE `DEPTNO` = 10"; - relFn(relFn).withMysql().ok(expectedSql); + relFn(relFn).withMysql().ok(expectedSql).done(); } /** Test case for @@ -2068,40 +2116,44 @@ private static String toSql(RelNode root, SqlDialect dialect, + "LEFT JOIN \"scott\".\"DEPT\" " + "ON \"EMP\".\"DEPTNO\" = \"DEPT\".\"DEPTNO\" " + "AND \"DEPT\".\"DNAME\" LIKE 'ACCOUNTING'"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @Test void testSelectQueryWithGroupByAndProjectList1() { - String query = "select count(*) from \"product\"\n" + String query = "select count(*)\n" + + "from \"foodmart\".\"product\"\n" + "group by \"product_class_id\", \"product_id\""; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\", \"product_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithGroupByHaving() { - String query = "select count(*) from \"product\" group by \"product_class_id\"," - + " \"product_id\" having \"product_id\" > 10"; + String query = "select count(*)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\", \"product_id\"\n" + + "having \"product_id\" > 10"; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\", \"product_id\"\n" + "HAVING \"product_id\" > 10"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-1665] * Aggregates and having cannot be combined. */ @Test void testSelectQueryWithGroupByHaving2() { - String query = " select \"product\".\"product_id\",\n" + String query = "select \"product\".\"product_id\",\n" + " min(\"sales_fact_1997\".\"store_id\")\n" - + " from \"product\"\n" - + " inner join \"sales_fact_1997\"\n" - + " on \"product\".\"product_id\" = \"sales_fact_1997\".\"product_id\"\n" - + " group by \"product\".\"product_id\"\n" - + " having count(*) > 1"; + + "from \"foodmart\".\"product\"\n" + + "inner join \"foodmart\".\"sales_fact_1997\"\n" + + " on \"product\".\"product_id\" =\n" + + " \"sales_fact_1997\".\"product_id\"\n" + + "group by \"product\".\"product_id\"\n" + + "having count(*) > 1"; String expected = "SELECT \"product\".\"product_id\", " + "MIN(\"sales_fact_1997\".\"store_id\")\n" @@ -2110,20 +2162,23 @@ private static String toSql(RelNode root, SqlDialect dialect, + "ON \"product\".\"product_id\" = \"sales_fact_1997\".\"product_id\"\n" + "GROUP BY \"product\".\"product_id\"\n" + "HAVING COUNT(*) > 1"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-1665] * Aggregates and having cannot be combined. */ @Test void testSelectQueryWithGroupByHaving3() { - String query = " select * from (select \"product\".\"product_id\",\n" - + " min(\"sales_fact_1997\".\"store_id\")\n" - + " from \"product\"\n" - + " inner join \"sales_fact_1997\"\n" - + " on \"product\".\"product_id\" = \"sales_fact_1997\".\"product_id\"\n" + String query = " select *\n" + + "from (select \"product\".\"product_id\",\n" + + " min(\"sales_fact_1997\".\"store_id\")\n" + + " from \"foodmart\".\"product\"\n" + + " inner join \"foodmart\".\"sales_fact_1997\"\n" + + " on \"product\".\"product_id\"\n" + + " = \"sales_fact_1997\".\"product_id\"\n" + " group by \"product\".\"product_id\"\n" - + " having count(*) > 1) where \"product_id\" > 100"; + + " having count(*) > 1)\n" + + "where \"product_id\" > 100"; String expected = "SELECT *\n" + "FROM (SELECT \"product\".\"product_id\"," @@ -2134,7 +2189,7 @@ private static String toSql(RelNode root, SqlDialect dialect, + "GROUP BY \"product\".\"product_id\"\n" + "HAVING COUNT(*) > 1) AS \"t2\"\n" + "WHERE \"t2\".\"product_id\" > 100"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -2191,7 +2246,7 @@ private static String toSql(RelNode root, SqlDialect dialect, relFn(b -> root) .withBigQuery().ok(expectedBigQuery) .withMysql().ok(expectedMysql) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql).done(); } /** Test case for @@ -2214,7 +2269,7 @@ private void checkHavingAliasSameAsColumn(boolean upperAlias) { final String alias = upperAlias ? "GROSS_WEIGHT" : "gross_weight"; final String query = "select \"product_id\" + 1,\n" + " sum(\"gross_weight\") as \"" + alias + "\"\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by \"product_id\"\n" + "having sum(\"product\".\"gross_weight\") < 2.000E2"; // PostgreSQL has isHavingAlias=false, case-sensitive=true @@ -2247,14 +2302,14 @@ private void checkHavingAliasSameAsColumn(boolean upperAlias) { sql(query) .withBigQuery().ok(expectedBigQuery) .withPostgresql().ok(expectedPostgresql) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql).done(); } @Test void testHaving4() { final String query = "select \"product_id\"\n" + "from (\n" + " select \"product_id\", avg(\"gross_weight\") as agw\n" - + " from \"product\"\n" + + " from \"foodmart\".\"product\"\n" + " where \"net_weight\" < 100\n" + " group by \"product_id\")\n" + "where agw > 50\n" @@ -2268,53 +2323,48 @@ private void checkHavingAliasSameAsColumn(boolean upperAlias) { + "HAVING AVG(\"gross_weight\") > CAST(50 AS DOUBLE)) AS \"t2\"\n" + "GROUP BY \"product_id\"\n" + "HAVING AVG(\"AGW\") > 6.00E1"; - sql(query).ok(expected); + sql(query).ok(expected) + .done(); } @Test void testSelectQueryWithOrderByClause() { - String query = "select \"product_id\" from \"product\"\n" + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"net_weight\""; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"net_weight\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithOrderByClause1() { - String query = - "select \"product_id\", \"net_weight\" from \"product\" order by \"net_weight\""; + String query = "select \"product_id\", \"net_weight\"\n" + + "from \"foodmart\".\"product\"\n" + + "order by \"net_weight\""; final String expected = "SELECT \"product_id\", \"net_weight\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"net_weight\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithTwoOrderByClause() { - String query = "select \"product_id\" from \"product\"\n" + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"net_weight\", \"gross_weight\""; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"net_weight\", \"gross_weight\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithAscDescOrderByClause() { - String query = "select \"product_id\" from \"product\" " + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"net_weight\" asc, \"gross_weight\" desc, \"low_fat\""; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"net_weight\", \"gross_weight\" DESC, \"low_fat\""; - sql(query).ok(expected); - } - - /** A dialect that doesn't treat integer literals in the ORDER BY as field - * references. */ - private SqlDialect nonOrdinalDialect() { - return new SqlDialect(SqlDialect.EMPTY_CONTEXT) { - @Override public SqlConformance getConformance() { - return SqlConformanceEnum.STRICT_99; - } - }; + sql(query).ok(expected).done(); } /** Test case for @@ -2343,18 +2393,24 @@ private SqlDialect nonOrdinalDialect() { .ok("SELECT \"JOB\", \"ENAME\"\n" + "FROM \"scott\".\"EMP\"\n" + "ORDER BY '1', '23', '12', \"ENAME\", '34' DESC NULLS LAST") - .dialect(nonOrdinalDialect()) + .dialect(NON_ORDINAL) .ok("SELECT JOB, ENAME\n" + "FROM scott.EMP\n" - + "ORDER BY 1, '23', 12, ENAME, 34 DESC NULLS LAST"); + + "ORDER BY 1, '23', 12, ENAME, 34 DESC NULLS LAST").done(); } @Test void testNoNeedRewriteOrderByConstantsForOver() { - final String query = "select row_number() over " - + "(order by 1 nulls last) from \"employee\""; + final String query = "select row_number() over (order by 1 nulls last)\n" + + "from \"foodmart\".\"employee\""; // Default dialect keep numeric constant keys in the over of order-by. - sql(query).ok("SELECT ROW_NUMBER() OVER (ORDER BY 1)\n" - + "FROM \"foodmart\".\"employee\""); + final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY 1)\n" + + "FROM \"foodmart\".\"employee\""; + sql(query) + // Disabled due to bug: + // [CALCITE-6837] Invalid code generated for ROW_NUMBER + // function in Enumerable convention + .withPhase(PARSE) + .ok(expected).done(); } /** Test case for @@ -2363,7 +2419,7 @@ private SqlDialect nonOrdinalDialect() { * expression. */ @Test void testOrderByOrdinalWithExpression() { final String query = "select \"product_id\", count(*) as \"c\"\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by \"product_id\"\n" + "order by 2"; final String ordinalExpected = "SELECT \"product_id\", COUNT(*) AS \"c\"\n" @@ -2380,10 +2436,10 @@ private SqlDialect nonOrdinalDialect() { + "ORDER BY 2"; sql(query) .ok(ordinalExpected) - .dialect(nonOrdinalDialect()) + .dialect(NON_ORDINAL) .ok(nonOrdinalExpected) - .dialect(PrestoSqlDialect.DEFAULT) - .ok(prestoExpected); + .dialect(PRESTO) + .ok(prestoExpected).done(); } /** Test case for the base case of @@ -2402,7 +2458,7 @@ private SqlDialect nonOrdinalDialect() { + "WHERE \"net_weight\" IS NOT NULL\n" + "GROUP BY \"product_id\"\n" + "ORDER BY 2 DESC) AS \"t3\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for the problematic case of @@ -2421,7 +2477,7 @@ private SqlDialect nonOrdinalDialect() { + "WHERE \"net_weight\" IS NOT NULL\n" + "GROUP BY \"product_id\"\n" + "ORDER BY 2 DESC NULLS LAST) AS \"t3\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -2430,13 +2486,13 @@ private SqlDialect nonOrdinalDialect() { @Test void testOrderByColumnWithSameNameAsAlias() { String query = "select \"product_id\" as \"p\",\n" + " \"net_weight\" as \"product_id\"\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "order by 1"; final String expected = "SELECT \"product_id\" AS \"p\"," + " \"net_weight\" AS \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY 1"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testOrderByColumnWithSameNameAsAlias2() { @@ -2444,7 +2500,7 @@ private SqlDialect nonOrdinalDialect() { // by alias "product_id". String query = "select \"net_weight\" as \"product_id\",\n" + " \"product_id\" as \"product_id\"\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product\".\"product_id\""; final String expected = "SELECT \"net_weight\" AS \"product_id\"," + " \"product_id\" AS \"product_id0\"\n" @@ -2455,7 +2511,7 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL, 2"; sql(query).ok(expected) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql).done(); } @Test void testHiveSelectCharset() { @@ -2463,7 +2519,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"foodmart\".\"reserve_employee\""; final String expected = "SELECT `hire_date`, CAST(`hire_date` AS VARCHAR(10))\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } /** Test case for @@ -2474,23 +2530,33 @@ private SqlDialect nonOrdinalDialect() { final String query = "select cast(array['a', 'b', 'c'] as varchar array)"; final String expected = "SELECT CAST(ARRAY ('a', 'b', 'c') AS VARCHAR ARRAY)"; sql(query) - .withHive().ok(expected); + .withHive().ok(expected).done(); } /** Test case for * [CALCITE-6088] * SqlItemOperator fails in RelToSqlConverter. */ @Test void testSqlItemOperator() { - sql("SELECT foo[0].\"EXPR$1\" FROM (SELECT ARRAY[ROW('a', 'b')] AS foo)") - .ok("SELECT \"ARRAY[ROW('a', 'b')][0]\".\"EXPR$1\"\n" - + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"); - sql("SELECT foo['k'].\"EXPR$1\" FROM (SELECT MAP['k', ROW('a', 'b')] AS foo)") - .ok("SELECT \"MAP['k', ROW('a', 'b')]['k']\".\"EXPR$1\"\n" - + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"); - sql("select\"books\"[0].\"title\" from \"authors\"") + final String query1 = "SELECT foo[0].\"EXPR$1\"\n" + + "FROM (SELECT ARRAY[ROW('a', 'b')] AS foo)"; + final String expected1 = "SELECT \"ARRAY[ROW('a', 'b')][0]\".\"EXPR$1\"\n" + + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; + sql(query1).ok(expected1).done(); + + final String query2 = "SELECT foo['k'].\"EXPR$1\"\n" + + "FROM (SELECT MAP['k', ROW('a', 'b')] AS foo)"; + final String expected2 = "SELECT" + + " \"MAP['k', ROW('a', 'b')]['k']\".\"EXPR$1\"\n" + + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; + sql(query2).ok(expected2).done(); + + final String query3 = "select\"books\"[0].\"title\" from \"authors\""; + final String expected3 = "SELECT \"`books`[0]\".\"title\"\n" + + "FROM \"bookstore\".\"authors\""; + sql(query3) .schema(CalciteAssert.SchemaSpec.BOOKSTORE) - .ok("SELECT \"`books`[0]\".\"title\"\n" - + "FROM \"bookstore\".\"authors\""); + .ok(expected3) + .done(); } /** Test case for @@ -2502,7 +2568,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"foodmart\".\"reserve_employee\" "; final String expected = "SELECT `employee_id`\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testBigQueryCast() { @@ -2536,7 +2602,7 @@ private SqlDialect nonOrdinalDialect() { + "CAST(CAST(employee_id AS STRING) AS TIME), " + "CAST(CAST(employee_id AS STRING) AS BOOL)\n" + "FROM foodmart.reserve_employee"; - sql(query).withBigQuery().ok(expected); + sql(query).withBigQuery().ok(expected).done(); } @Test void testBigQueryParseDatetimeFunctions() { @@ -2545,14 +2611,18 @@ private SqlDialect nonOrdinalDialect() { final String expectedTimestampTrunc = "SELECT PARSE_TIME('%I:%M:%S', '07:30:00')\n" + "FROM \"foodmart\".\"product\""; - sql(parseTime).withLibrary(SqlLibrary.BIG_QUERY).ok(expectedTimestampTrunc); + sql(parseTime) + .withLibrary(SqlLibrary.BIG_QUERY).ok(expectedTimestampTrunc) + .done(); String parseDate = "select parse_date('%A %b %e %Y', 'Thursday Dec 25 2008')\n" + "from \"foodmart\".\"product\"\n"; final String expectedParseDate = "SELECT PARSE_DATE('%A %b %e %Y', 'Thursday Dec 25 2008')\n" + "FROM \"foodmart\".\"product\""; - sql(parseDate).withLibrary(SqlLibrary.BIG_QUERY).ok(expectedParseDate); + sql(parseDate) + .withLibrary(SqlLibrary.BIG_QUERY).ok(expectedParseDate) + .done(); String parseTimestamp = "select parse_timestamp('%a %b %e %I:%M:%S %Y', 'Thu Dec 25 07:30:00 2008')\n" @@ -2560,7 +2630,9 @@ private SqlDialect nonOrdinalDialect() { final String expectedParseTimestamp = "SELECT PARSE_TIMESTAMP('%a %b %e %I:%M:%S %Y', 'Thu Dec 25 07:30:00 2008')\n" + "FROM \"foodmart\".\"product\""; - sql(parseTimestamp).withLibrary(SqlLibrary.BIG_QUERY).ok(expectedParseTimestamp); + sql(parseTimestamp) + .withLibrary(SqlLibrary.BIG_QUERY).ok(expectedParseTimestamp) + .done(); String parseDatetime = "select parse_datetime('%a %b %e %I:%M:%S %Y', 'Thu Dec 25 07:30:00 2008')\n" @@ -2568,7 +2640,9 @@ private SqlDialect nonOrdinalDialect() { final String expectedParseDatetime = "SELECT PARSE_DATETIME('%a %b %e %I:%M:%S %Y', 'Thu Dec 25 07:30:00 2008')\n" + "FROM \"foodmart\".\"product\""; - sql(parseDatetime).withLibrary(SqlLibrary.BIG_QUERY).ok(expectedParseDatetime); + sql(parseDatetime) + .withLibrary(SqlLibrary.BIG_QUERY).ok(expectedParseDatetime) + .done(); } @Test void testBigQueryTimeTruncFunctions() { @@ -2577,13 +2651,15 @@ private SqlDialect nonOrdinalDialect() { final String expectedTimestampTrunc = "SELECT TIMESTAMP_TRUNC(TIMESTAMP '2012-02-03 15:30:00', MONTH)\n" + "FROM \"foodmart\".\"product\""; - sql(timestampTrunc).withLibrary(SqlLibrary.BIG_QUERY).ok(expectedTimestampTrunc); + sql(timestampTrunc).withLibrary(SqlLibrary.BIG_QUERY) + .ok(expectedTimestampTrunc).done(); String timeTrunc = "select time_trunc(time '15:30:00', minute)\n" + "from \"foodmart\".\"product\"\n"; final String expectedTimeTrunc = "SELECT TIME_TRUNC(TIME '15:30:00', MINUTE)\n" + "FROM \"foodmart\".\"product\""; - sql(timeTrunc).withLibrary(SqlLibrary.BIG_QUERY).ok(expectedTimeTrunc); + sql(timeTrunc).withLibrary(SqlLibrary.BIG_QUERY) + .ok(expectedTimeTrunc).done(); } @Test void testBigQueryDatetimeFormatFunctions() { @@ -2610,15 +2686,17 @@ private SqlDialect nonOrdinalDialect() { final String expectedBqFormatDatetime = "SELECT FORMAT_DATETIME('%R', TIMESTAMP '2012-02-03 12:34:34')\n" + "FROM foodmart.product"; - final Sql sql = fixture().withBigQuery().withLibrary(SqlLibrary.BIG_QUERY); - sql.withSql(formatTime) - .ok(expectedBqFormatTime); - sql.withSql(formatDate) - .ok(expectedBqFormatDate); - sql.withSql(formatTimestamp) - .ok(expectedBqFormatTimestamp); - sql.withSql(formatDatetime) - .ok(expectedBqFormatDatetime); + + final Function factory = sql -> + fixture().withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).withSql(sql); + factory.apply(formatTime) + .ok(expectedBqFormatTime).done(); + factory.apply(formatDate) + .ok(expectedBqFormatDate).done(); + factory.apply(formatTimestamp) + .ok(expectedBqFormatTimestamp).done(); + factory.apply(formatDatetime) + .ok(expectedBqFormatDatetime).done(); } /** Test case for @@ -2631,7 +2709,9 @@ private SqlDialect nonOrdinalDialect() { final String expected = "SELECT SAFE_CAST(\"product_name\" AS DATE)\n" + "FROM \"foodmart\".\"product\""; - sql(query).withLibrary(SqlLibrary.BIG_QUERY).ok(expected); + sql(query) + .withLibrary(SqlLibrary.BIG_QUERY).ok(expected) + .done(); } /** Test case for @@ -2724,7 +2804,8 @@ private SqlDialect nonOrdinalDialect() { .withStarRocks().ok(expectedStarRocks) .withHive().ok(expectedHive) .withPostgresql().ok(expectedPostgresql) - .withHsqldb().ok(expectedHsqldb); + .withHsqldb().ok(expectedHsqldb) + .done(); } /** Test case for @@ -2746,7 +2827,8 @@ private SqlDialect nonOrdinalDialect() { sql(query) .withBigQuery().ok(expectedBigQuery) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected) + .done(); } @Test void testHiveSparkAndBqTrimWithBoth() { @@ -2759,7 +2841,8 @@ private SqlDialect nonOrdinalDialect() { sql(query) .withBigQuery().ok(expectedBigQuery) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected) + .done(); } @Test void testHiveSparkAndBqTrimWithLeading() { @@ -2772,7 +2855,8 @@ private SqlDialect nonOrdinalDialect() { sql(query) .withBigQuery().ok(expectedBigQuery) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected) + .done(); } @Test void testHiveSparkAndBqTrimWithTailing() { @@ -2785,7 +2869,8 @@ private SqlDialect nonOrdinalDialect() { sql(query) .withBigQuery().ok(expectedBigQuery) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected) + .done(); } /** Test case for @@ -2800,7 +2885,8 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.reserve_employee"; sql(query) .withBigQuery().ok(expected) - .withHsqldb().ok(expectedHsqldb); + .withHsqldb().ok(expectedHsqldb) + .done(); } /** Test case for @@ -2814,7 +2900,8 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`reserve_employee`"; sql(query) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected) + .done(); } @Test void testBqTrimWithBothChar() { @@ -2823,7 +2910,8 @@ private SqlDialect nonOrdinalDialect() { final String expected = "SELECT TRIM('abcda', 'a')\n" + "FROM foodmart.reserve_employee"; sql(query) - .withBigQuery().ok(expected); + .withBigQuery().ok(expected) + .done(); } @Test void testHiveAndSparkTrimWithBothChar() { @@ -2833,7 +2921,8 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`reserve_employee`"; sql(query) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected) + .done(); } @Test void testHiveBqTrimWithTailingChar() { @@ -2842,7 +2931,8 @@ private SqlDialect nonOrdinalDialect() { final String expected = "SELECT RTRIM('abcd', 'a')\n" + "FROM foodmart.reserve_employee"; sql(query) - .withBigQuery().ok(expected); + .withBigQuery().ok(expected) + .done(); } @Test void testHiveAndSparkTrimWithTailingChar() { @@ -2852,7 +2942,7 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`reserve_employee`"; sql(query) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected).done(); } @Test void testBqTrimWithBothSpecialCharacter() { @@ -2862,7 +2952,7 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.reserve_employee"; sql(query) .withBigQuery() - .ok(expected); + .ok(expected).done(); } @Test void testHiveAndSparkTrimWithBothSpecialCharacter() { @@ -2873,7 +2963,7 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`reserve_employee`"; sql(query) .withHive().ok(expected) - .withSpark().ok(expected); + .withSpark().ok(expected).done(); } /** Test case for @@ -2897,7 +2987,7 @@ private SqlDialect nonOrdinalDialect() { sql(query) .withExasol().ok(expectedExasol) .withMssql().ok(expectedMssql) - .withRedshift().ok(expectedRedshift); + .withRedshift().ok(expectedRedshift).done(); } /** Test case for @@ -2922,17 +3012,19 @@ private SqlDialect nonOrdinalDialect() { .withSpark().ok(expectedSpark) .withPresto().ok(expectedPresto) .withStarRocks().ok(expectedStarRocks) - .withHive().ok(expectedHive); + .withHive().ok(expectedHive).done(); } @Test void testExasolCastToTimestamp() { - final String query = "select * from \"employee\" where \"hire_date\" - " - + "INTERVAL '19800' SECOND(5) > cast(\"hire_date\" as TIMESTAMP(0))"; + final String query = "select *\n" + + "from \"foodmart\".\"employee\"\n" + + "where \"hire_date\" - INTERVAL '19800' SECOND(5)\n" + + " > cast(\"hire_date\" as TIMESTAMP(0))"; final String expected = "SELECT *\n" + "FROM foodmart.employee\n" + "WHERE (hire_date - INTERVAL '19800' SECOND(5))" + " > CAST(hire_date AS TIMESTAMP)"; - sql(query).withExasol().ok(expected); + sql(query).withExasol().ok(expected).done(); } /** @@ -2949,7 +3041,7 @@ private SqlDialect nonOrdinalDialect() { final String expectedSql = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"DEPTNO\" = 21"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @Test void testUnparseIn2() { @@ -2960,7 +3052,7 @@ private SqlDialect nonOrdinalDialect() { final String expectedSql = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE \"DEPTNO\" IN (20, 21)"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @Test void testUnparseInStruct1() { @@ -2976,7 +3068,7 @@ private SqlDialect nonOrdinalDialect() { final String expectedSql = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE ROW(\"DEPTNO\", \"JOB\") = ROW(1, 'PRESIDENT')"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @Test void testUnparseInStruct2() { @@ -2994,7 +3086,7 @@ private SqlDialect nonOrdinalDialect() { final String expectedSql = "SELECT *\n" + "FROM \"scott\".\"EMP\"\n" + "WHERE ROW(\"DEPTNO\", \"JOB\") IN (ROW(1, 'PRESIDENT'), ROW(2, 'PRESIDENT'))"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } /** Test case for @@ -3018,11 +3110,13 @@ private SqlDialect nonOrdinalDialect() { + "FROM \"scott\".\"EMP\")\n" + "OFFSET 1 ROWS\n" + "FETCH NEXT 3 ROWS ONLY"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @Test void testSelectQueryWithLimitClause() { - String query = "select \"product_id\" from \"product\" limit 100 offset 10"; + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + + "limit 100 offset 10"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "LIMIT 100\n" @@ -3042,28 +3136,31 @@ private SqlDialect nonOrdinalDialect() { sql(query).withHive().ok(expected) .withVertica().ok(expectedVertica) .withStarRocks().ok(expectedStarRocks) - .withSnowflake().ok(expectedSnowflake); + .withSnowflake().ok(expectedSnowflake) + .done(); } @Test void testPositionFunctionForHive() { - final String query = "select position('A' IN 'ABC') from \"product\""; + final String query = "select position('A' IN 'ABC')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT INSTR('ABC', 'A')\n" + "FROM `foodmart`.`product`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testPositionFunctionForMySql() { final String query = "select position('A' IN 'ABC') from \"product\""; final String expected = "SELECT INSTR('ABC', 'A')\n" + "FROM `foodmart`.`product`"; - sql(query).withMysql().ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testPositionFunctionForBigQuery() { - final String query = "select position('A' IN 'ABC') from \"product\""; + final String query = "select position('A' IN 'ABC')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT INSTR('ABC', 'A')\n" + "FROM foodmart.product"; - sql(query).withBigQuery().ok(expected); + sql(query).withBigQuery().ok(expected).done(); } /** Test case for @@ -3074,12 +3171,12 @@ private SqlDialect nonOrdinalDialect() { final String query = "SELECT POSITION('a' IN 'abc')"; final String expected = "SELECT POSITION('a', 'abc')\n" + "FROM (VALUES (0)) `t` (`ZERO`)"; - sql(query).withSpark().ok(expected); + sql(query).withSpark().ok(expected).done(); final String query2 = "SELECT POSITION('a' IN 'abc' FROM 1)"; final String expected2 = "SELECT POSITION('a', 'abc', 1)\n" + "FROM (VALUES (0)) `t` (`ZERO`)"; - sql(query2).withSpark().ok(expected2); + sql(query2).withSpark().ok(expected2).done(); } @Test void testInstrFunction4Operands() { @@ -3088,14 +3185,13 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.product"; final String expectedHive = "SELECT INSTR('ABC', 'A', 1, 1)\n" + "FROM `foodmart`.`product`"; - final String expected_oracle = "SELECT INSTR('ABC', 'A', 1, 1)\n" + final String expectedOracle = "SELECT INSTR('ABC', 'A', 1, 1)\n" + "FROM \"foodmart\".\"product\""; - final Sql sqlOracle = fixture().withOracle().withLibrary(SqlLibrary.ORACLE); - sqlOracle.withSql(query).withOracle().ok(expected_oracle); - final Sql sqlBQ = fixture().withBigQuery().withLibrary(SqlLibrary.BIG_QUERY); - sqlBQ.withSql(query).withBigQuery().ok(expectedBQ); - final Sql sqlHive = fixture().withHive().withLibrary(SqlLibrary.HIVE); - sqlHive.withSql(query).withHive().ok(expectedHive); + sql(query) + .withOracle().withLibrary(SqlLibrary.ORACLE).ok(expectedOracle) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedBQ) + .withHive().withLibrary(SqlLibrary.HIVE).ok(expectedHive) + .done(); } @Test void testInstrFunction3Operands() { @@ -3106,12 +3202,11 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`product`"; final String expectedOracle = "SELECT INSTR('ABC', 'A', 1)\n" + "FROM \"foodmart\".\"product\""; - final Sql sqlOracle = fixture().withOracle().withLibrary(SqlLibrary.ORACLE); - sqlOracle.withSql(query).withOracle().ok(expectedOracle); - final Sql sqlBQ = fixture().withBigQuery().withLibrary(SqlLibrary.BIG_QUERY); - sqlBQ.withSql(query).withBigQuery().ok(expectedBQ); - final Sql sqlHive = fixture().withHive().withLibrary(SqlLibrary.HIVE); - sqlHive.withSql(query).withHive().ok(expectedHive); + sql(query) + .withOracle().withLibrary(SqlLibrary.ORACLE).ok(expectedOracle) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedBQ) + .withHive().withLibrary(SqlLibrary.HIVE).ok(expectedHive) + .done(); } /** Test case for @@ -3125,25 +3220,25 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`product`"; final String expectedOracle = "SELECT INSTR('ABC', 'A')\n" + "FROM \"foodmart\".\"product\""; - final Sql sqlOracle = fixture().withOracle().withLibrary(SqlLibrary.ORACLE); - sqlOracle.withSql(query).withOracle().ok(expectedOracle); - final Sql sqlBQ = fixture().withBigQuery().withLibrary(SqlLibrary.BIG_QUERY); - sqlBQ.withSql(query).withBigQuery().ok(expectedBQ); - final Sql sqlHive = fixture().withHive().withLibrary(SqlLibrary.HIVE); - sqlHive.withSql(query).withHive().ok(expectedHive); + sql(query) + .withOracle().withLibrary(SqlLibrary.ORACLE).ok(expectedOracle) + .withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expectedBQ) + .withHive().withLibrary(SqlLibrary.HIVE).ok(expectedHive) + .done(); } /** Tests that we escape single-quotes in character literals using back-slash * in BigQuery. The norm is to escape single-quotes with single-quotes. */ @Test void testCharLiteralForBigQuery() { - final String query = "select 'that''s all folks!' from \"product\""; + final String query = "select 'that''s all folks!'\n" + + "from \"foodmart\".\"product\""; final String expectedPostgresql = "SELECT 'that''s all folks!'\n" + "FROM \"foodmart\".\"product\""; final String expectedBigQuery = "SELECT 'that\\'s all folks!'\n" + "FROM foodmart.product"; sql(query) .withPostgresql().ok(expectedPostgresql) - .withBigQuery().ok(expectedBigQuery); + .withBigQuery().ok(expectedBigQuery).done(); } @Test void testIdentifier() { @@ -3180,62 +3275,73 @@ private SqlDialect nonOrdinalDialect() { .withMysql().ok(expectedMysql) .withOracle().ok(expectedOracle) .withPostgresql().ok(expectedPostgresql) - .withExasol().ok(expectedExasol); + .withExasol().ok(expectedExasol).done(); } @Test void testModFunctionForHive() { - final String query = "select mod(11,3) from \"product\""; + final String query = "select mod(11,3)\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT 11 % 3\n" + "FROM `foodmart`.`product`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testUnionOperatorForBigQuery() { - final String query = "select mod(11,3) from \"product\"\n" - + "UNION select 1 from \"product\""; + final String query = "select mod(11,3)\n" + + "from \"foodmart\".\"product\"\n" + + "UNION select 1\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT MOD(11, 3)\n" + "FROM foodmart.product\n" + "UNION DISTINCT\n" + "SELECT 1\n" + "FROM foodmart.product"; - sql(query).withBigQuery().ok(expected); + sql(query).withBigQuery().ok(expected).done(); } @Test void testUnionAllOperatorForBigQuery() { - final String query = "select mod(11,3) from \"product\"\n" - + "UNION ALL select 1 from \"product\""; + final String query = "select mod(11,3)\n" + + "from \"foodmart\".\"product\"\n" + + "UNION ALL\n" + + "select 1\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT MOD(11, 3)\n" + "FROM foodmart.product\n" + "UNION ALL\n" + "SELECT 1\n" + "FROM foodmart.product"; - sql(query).withBigQuery().ok(expected); + sql(query).withBigQuery().ok(expected).done(); } @Test void testIntersectOperatorForBigQuery() { - final String query = "select mod(11,3) from \"product\"\n" - + "INTERSECT select 1 from \"product\""; + final String query = "select mod(11,3)\n" + + "from \"foodmart\".\"product\"\n" + + "INTERSECT select 1\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT MOD(11, 3)\n" + "FROM foodmart.product\n" + "INTERSECT DISTINCT\n" + "SELECT 1\n" + "FROM foodmart.product"; - sql(query).withBigQuery().ok(expected); + sql(query).withBigQuery().ok(expected).done(); } @Test void testExceptOperatorForBigQuery() { - final String query = "select mod(11,3) from \"product\"\n" - + "EXCEPT select 1 from \"product\""; + final String query = "select mod(11,3)\n" + + "from \"foodmart\".\"product\"\n" + + "EXCEPT select 1\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT MOD(11, 3)\n" + "FROM foodmart.product\n" + "EXCEPT DISTINCT\n" + "SELECT 1\n" + "FROM foodmart.product"; - sql(query).withBigQuery().ok(expected); + sql(query).withBigQuery().ok(expected).done(); } @Test void testSelectOrderByDescNullsFirst() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; // Hive and MSSQL do not support NULLS FIRST, so need to emulate final String expected = "SELECT `product_id`\n" @@ -3245,12 +3351,13 @@ private SqlDialect nonOrdinalDialect() { + "FROM [foodmart].[product]\n" + "ORDER BY CASE WHEN [product_id] IS NULL THEN 0 ELSE 1 END, [product_id] DESC"; sql(query) - .dialect(HiveSqlDialect.DEFAULT).ok(expected) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected); + .withHive().ok(expected) + .withMssql().ok(mssqlExpected).done(); } @Test void testSelectOrderByAscNullsLast() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls last"; // Hive and MSSQL do not support NULLS LAST, so need to emulate final String expected = "SELECT `product_id`\n" @@ -3260,12 +3367,13 @@ private SqlDialect nonOrdinalDialect() { + "FROM [foodmart].[product]\n" + "ORDER BY CASE WHEN [product_id] IS NULL THEN 1 ELSE 0 END, [product_id]"; sql(query) - .dialect(HiveSqlDialect.DEFAULT).ok(expected) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected); + .withHive().ok(expected) + .withMssql().ok(mssqlExpected).done(); } @Test void testSelectOrderByAscNullsFirst() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls first"; // Hive and MSSQL do not support NULLS FIRST, but nulls sort low, so no // need to emulate @@ -3276,12 +3384,13 @@ private SqlDialect nonOrdinalDialect() { + "FROM [foodmart].[product]\n" + "ORDER BY [product_id]"; sql(query) - .dialect(HiveSqlDialect.DEFAULT).ok(expected) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected); + .withHive().ok(expected) + .withMssql().ok(mssqlExpected).done(); } @Test void testSelectOrderByDescNullsLast() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls last"; // Hive and MSSQL do not support NULLS LAST, but nulls sort low, so no // need to emulate @@ -3292,33 +3401,37 @@ private SqlDialect nonOrdinalDialect() { + "FROM [foodmart].[product]\n" + "ORDER BY [product_id] DESC"; sql(query) - .dialect(HiveSqlDialect.DEFAULT).ok(expectedHive) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected); + .withHive().ok(expectedHive) + .withMssql().ok(mssqlExpected) + .done(); } @Test void testHiveSelectQueryWithOverDescAndNullsFirstShouldBeEmulated() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" desc nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() " + "OVER (ORDER BY `hire_date` IS NULL DESC, `hire_date` DESC)\n" + "FROM `foodmart`.`employee`"; - sql(query).dialect(HiveSqlDialect.DEFAULT).ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSelectQueryWithOverAscAndNullsLastShouldBeEmulated() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" nulls last) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls last)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date` IS NULL, `hire_date`)\n" + "FROM `foodmart`.`employee`"; - sql(query).dialect(HiveSqlDialect.DEFAULT).ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSelectQueryWithOverAscNullsFirstShouldNotAddNullEmulation() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date`)\n" + "FROM `foodmart`.`employee`"; - sql(query).dialect(HiveSqlDialect.DEFAULT).ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSubstring() { @@ -3326,7 +3439,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"foodmart\".\"reserve_employee\""; final String expected = "SELECT SUBSTRING('ABC', 2)\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSubstringWithLength() { @@ -3334,7 +3447,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"foodmart\".\"reserve_employee\""; final String expected = "SELECT SUBSTRING('ABC', 2, 3)\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSubstringWithANSI() { @@ -3342,7 +3455,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"foodmart\".\"reserve_employee\""; final String expected = "SELECT SUBSTRING('ABC', 2)\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSubstringWithANSIAndLength() { @@ -3350,7 +3463,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"foodmart\".\"reserve_employee\""; final String expected = "SELECT SUBSTRING('ABC', 2, 3)\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withHive().ok(expected); + sql(query).withHive().ok(expected).done(); } @Test void testHiveSelectQueryWithOverDescNullsLastShouldNotAddNullEmulation() { @@ -3358,7 +3471,7 @@ private SqlDialect nonOrdinalDialect() { + "(order by \"hire_date\" desc nulls last) FROM \"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date` DESC)\n" + "FROM `foodmart`.`employee`"; - sql(query).dialect(HiveSqlDialect.DEFAULT).ok(expected); + sql(query).withHive().ok(expected).done(); } /** Test case for @@ -3375,8 +3488,10 @@ private SqlDialect nonOrdinalDialect() { final String expectedStarRocks = "SELECT CAST(`product_id` AS BIGINT)\n" + "FROM `foodmart`.`product`"; sql(query) + .schema(CalciteAssert.SchemaSpec.JDBC_FOODMART) .withMysql().ok(expectedMysql) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for @@ -3398,194 +3513,180 @@ private SqlDialect nonOrdinalDialect() { + "FROM `foodmart`.`salary`"; sql(query) .withMysql().ok(expectedMysql) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testHiveSelectQueryWithOrderByDescAndHighNullsWithVersionGreaterThanOrEq21() { - final HiveSqlDialect hive2_1Dialect = - new HiveSqlDialect(HiveSqlDialect.DEFAULT_CONTEXT - .withDatabaseMajorVersion(2) - .withDatabaseMinorVersion(1) - .withNullCollation(NullCollation.LOW)); - - final HiveSqlDialect hive2_2_Dialect = - new HiveSqlDialect(HiveSqlDialect.DEFAULT_CONTEXT - .withDatabaseMajorVersion(2) - .withDatabaseMinorVersion(2) - .withNullCollation(NullCollation.LOW)); - - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` DESC NULLS FIRST"; - sql(query).dialect(hive2_1Dialect).ok(expected); - sql(query).dialect(hive2_2_Dialect).ok(expected); + sql(query) + .dialect(HIVE_2_1).ok(expected) + .dialect(HIVE_2_2).ok(expected) + .done(); } @Test void testHiveSelectQueryWithOverDescAndHighNullsWithVersionGreaterThanOrEq21() { - final HiveSqlDialect hive2_1Dialect = - new HiveSqlDialect(SqlDialect.EMPTY_CONTEXT - .withDatabaseMajorVersion(2) - .withDatabaseMinorVersion(1) - .withNullCollation(NullCollation.LOW)); - - final HiveSqlDialect hive2_2_Dialect = - new HiveSqlDialect(SqlDialect.EMPTY_CONTEXT - .withDatabaseMajorVersion(2) - .withDatabaseMinorVersion(2) - .withNullCollation(NullCollation.LOW)); - - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" desc nulls first) FROM \"employee\""; - final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY hire_date DESC NULLS FIRST)\n" - + "FROM foodmart.employee"; - sql(query).dialect(hive2_1Dialect).ok(expected); - sql(query).dialect(hive2_2_Dialect).ok(expected); + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; + final String expected = "SELECT ROW_NUMBER()" + + " OVER (ORDER BY `hire_date` DESC NULLS FIRST)\n" + + "FROM `foodmart`.`employee`"; + sql(query).dialect(HIVE_2_1).ok(expected).done(); + sql(query).dialect(HIVE_2_2).ok(expected).done(); } @Test void testHiveSelectQueryWithOrderByDescAndHighNullsWithVersion20() { - final HiveSqlDialect hive2_1_0_Dialect = - new HiveSqlDialect(HiveSqlDialect.DEFAULT_CONTEXT - .withDatabaseMajorVersion(2) - .withDatabaseMinorVersion(0) - .withNullCollation(NullCollation.LOW)); - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL DESC, `product_id` DESC"; - sql(query).dialect(hive2_1_0_Dialect).ok(expected); + sql(query).dialect(HIVE_2_0).ok(expected).done(); } @Test void testHiveSelectQueryWithOverDescAndHighNullsWithVersion20() { - final HiveSqlDialect hive2_1_0_Dialect = - new HiveSqlDialect(SqlDialect.EMPTY_CONTEXT - .withDatabaseMajorVersion(2) - .withDatabaseMinorVersion(0) - .withNullCollation(NullCollation.LOW)); - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" desc nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER " - + "(ORDER BY hire_date IS NULL DESC, hire_date DESC)\n" - + "FROM foodmart.employee"; - sql(query).dialect(hive2_1_0_Dialect).ok(expected); + + "(ORDER BY `hire_date` IS NULL DESC, `hire_date` DESC)\n" + + "FROM `foodmart`.`employee`"; + sql(query).dialect(HIVE_2_0).ok(expected).done(); } @Test void testJethroDataSelectQueryWithOrderByDescAndNullsFirstShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"product_id\", \"product_id\" DESC"; - sql(query).dialect(jethroDataSqlDialect()).ok(expected); + sql(query).withJethro().ok(expected).done(); } @Test void testJethroDataSelectQueryWithOverDescAndNullsFirstShouldBeEmulated() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" desc nulls first) FROM \"employee\""; - + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER " - + "(ORDER BY \"hire_date\", \"hire_date\" DESC)\n" - + "FROM \"foodmart\".\"employee\""; - sql(query).dialect(jethroDataSqlDialect()).ok(expected); + + "(ORDER BY \"hire_date\", \"hire_date\" DESC)\n" + + "FROM \"foodmart\".\"employee\""; + sql(query).withJethro().ok(expected).done(); } @Test void testMySqlSelectQueryWithOrderByDescAndNullsFirstShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL DESC, `product_id` DESC"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOverDescAndNullsFirstShouldBeEmulated() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" desc nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER " + "(ORDER BY `hire_date` IS NULL DESC, `hire_date` DESC)\n" + "FROM `foodmart`.`employee`"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOrderByAscAndNullsLastShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL, `product_id`"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOverAscAndNullsLastShouldBeEmulated() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" nulls last) FROM \"employee\""; - final String expected = "SELECT ROW_NUMBER() OVER " - + "(ORDER BY `hire_date` IS NULL, `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls last)\n" + + "FROM \"foodmart\".\"employee\""; + final String expected = "SELECT" + + " ROW_NUMBER() OVER (ORDER BY `hire_date` IS NULL, `hire_date`)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOrderByAscNullsFirstShouldNotAddNullEmulation() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id`"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOverAscNullsFirstShouldNotAddNullEmulation() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + + "FROM `foodmart`.`employee`"; + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOrderByDescNullsLastShouldNotAddNullEmulation() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` DESC"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlSelectQueryWithOverDescNullsLastShouldNotAddNullEmulation() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls last) FROM \"employee\""; - final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date` DESC)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(MysqlSqlDialect.DEFAULT).ok(expected); + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls last)\n" + + "FROM \"foodmart\".\"employee\""; + final String expected = "SELECT" + + " ROW_NUMBER() OVER (ORDER BY `hire_date` DESC)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlCastToVarcharWithLessThanMaxPrecision() { - final String query = "select cast(\"product_id\" as varchar(50)), \"product_id\" " - + "from \"product\" "; + final String query = "select cast(\"product_id\" as varchar(50)), \"product_id\"\n" + + "from \"foodmart\".\"product\" "; final String expected = "SELECT CAST(`product_id` AS CHAR(50)), `product_id`\n" + "FROM `foodmart`.`product`"; - sql(query).withMysql().ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlCastToTimestamp() { - final String query = "select * from \"employee\" where \"hire_date\" - " - + "INTERVAL '19800' SECOND(5) > cast(\"hire_date\" as TIMESTAMP) "; + final String query = "select *\n" + + "from \"foodmart\".\"employee\"\n" + + "where \"hire_date\" - INTERVAL '19800' SECOND(5)\n" + + " > cast(\"hire_date\" as TIMESTAMP) "; final String expected = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` - INTERVAL '19800' SECOND)" + " > CAST(`hire_date` AS DATETIME)"; - sql(query).withMysql().ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlCastToVarcharWithGreaterThanMaxPrecision() { - final String query = "select cast(\"product_id\" as varchar(500)), \"product_id\" " - + "from \"product\" "; + final String query = "select cast(\"product_id\" as varchar(500)), \"product_id\"\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT CAST(`product_id` AS CHAR(255)), `product_id`\n" + "FROM `foodmart`.`product`"; - sql(query).withMysql().ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlUnparseListAggCall() { @@ -3596,7 +3697,7 @@ private SqlDialect nonOrdinalDialect() { + "listagg(distinct \"product_name\", ',') within group(order by \"cases_per_pallet\"),\n" + "listagg(\"product_name\"),\n" + "listagg(\"product_name\", ',')\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "group by \"product_id\"\n"; final String expected = "SELECT GROUP_CONCAT(DISTINCT `product_name` " + "ORDER BY `cases_per_pallet` IS NULL, `cases_per_pallet` SEPARATOR ','), " @@ -3609,217 +3710,243 @@ private SqlDialect nonOrdinalDialect() { + "GROUP_CONCAT(`product_name`), GROUP_CONCAT(`product_name` SEPARATOR ',')\n" + "FROM `foodmart`.`product`\n" + "GROUP BY `product_id`"; - sql(query).withMysql().ok(expected); + sql(query).withMysql().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOrderByAscNullsLastAndNoEmulation() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id`"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOverAscNullsLastAndNoEmulation() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" nulls last) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls last)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOrderByAscNullsFirstAndNullEmulation() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL DESC, `product_id`"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOverAscNullsFirstAndNullEmulation() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() " - + "OVER (ORDER BY `hire_date` IS NULL DESC, `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + + "OVER (ORDER BY `hire_date` IS NULL DESC, `hire_date`)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOrderByDescNullsFirstAndNoEmulation() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` DESC"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOverDescNullsFirstAndNoEmulation() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date` DESC)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOrderByDescNullsLastAndNullEmulation() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL, `product_id` DESC"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithHighNullsSelectWithOverDescNullsLastAndNullEmulation() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls last) FROM \"employee\""; - final String expected = "SELECT ROW_NUMBER() " - + "OVER (ORDER BY `hire_date` IS NULL, `hire_date` DESC)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.HIGH)).ok(expected); + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls last)\n" + + "FROM \"foodmart\".\"employee\""; + final String expected = "SELECT ROW_NUMBER()" + + " OVER (ORDER BY `hire_date` IS NULL, `hire_date` DESC)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlHigh().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOrderByDescAndNullsFirstShouldNotBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` DESC"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOverDescAndNullsFirstShouldNotBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls first) FROM \"employee\""; - final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date` DESC)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + final String query = "SELECT row_number()\n" + + " over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; + final String expected = "SELECT ROW_NUMBER()" + + " OVER (ORDER BY `hire_date` DESC)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOrderByAscAndNullsFirstShouldNotBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id`"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOverAscAndNullsFirstShouldNotBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" nulls first) FROM \"employee\""; + final String query = "SELECT row_number()" + + " over (order by \"hire_date\" nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOrderByDescAndNullsLastShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL, `product_id` DESC"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOverDescAndNullsLastShouldBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls last) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls last)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() " - + "OVER (ORDER BY `hire_date` IS NULL, `hire_date` DESC)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + + "OVER (ORDER BY `hire_date` IS NULL, `hire_date` DESC)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOrderByAscAndNullsLastShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL, `product_id`"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithFirstNullsSelectWithOverAscAndNullsLastShouldBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" nulls last) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls last)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() " - + "OVER (ORDER BY `hire_date` IS NULL, `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.FIRST)).ok(expected); + + "OVER (ORDER BY `hire_date` IS NULL, `hire_date`)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlFirst().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOrderByDescAndNullsFirstShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL DESC, `product_id` DESC"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOverDescAndNullsFirstShouldBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() " + "OVER (ORDER BY `hire_date` IS NULL DESC, `hire_date` DESC)\n" + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOrderByAscAndNullsFirstShouldBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls first"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` IS NULL DESC, `product_id`"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOverAscAndNullsFirstShouldBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" nulls first) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls first)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() " - + "OVER (ORDER BY `hire_date` IS NULL DESC, `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + + "OVER (ORDER BY `hire_date` IS NULL DESC, `hire_date`)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOrderByDescAndNullsLastShouldNotBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" desc nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id` DESC"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOverDescAndNullsLastShouldNotBeEmulated() { - final String query = "SELECT row_number() " - + "over (order by \"hire_date\" desc nulls last) FROM \"employee\""; - final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date` DESC)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" desc nulls last)\n" + + "FROM \"foodmart\".\"employee\""; + final String expected = "SELECT ROW_NUMBER() " + + "OVER (ORDER BY `hire_date` DESC)\n" + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOrderByAscAndNullsLastShouldNotBeEmulated() { - final String query = "select \"product_id\" from \"product\"\n" + final String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" nulls last"; final String expected = "SELECT `product_id`\n" + "FROM `foodmart`.`product`\n" + "ORDER BY `product_id`"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + sql(query).withMysqlLast().ok(expected).done(); } @Test void testMySqlWithLastNullsSelectWithOverAscAndNullsLastShouldNotBeEmulated() { - final String query = "SELECT row_number() over " - + "(order by \"hire_date\" nulls last) FROM \"employee\""; + final String query = "SELECT\n" + + " row_number() over (order by \"hire_date\" nulls last)\n" + + "FROM \"foodmart\".\"employee\""; final String expected = "SELECT ROW_NUMBER() OVER (ORDER BY `hire_date`)\n" - + "FROM `foodmart`.`employee`"; - sql(query).dialect(mySqlDialect(NullCollation.LAST)).ok(expected); + + "FROM `foodmart`.`employee`"; + sql(query).withMysqlLast().ok(expected).done(); } /** Test case for @@ -3830,7 +3957,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"product\" "; final String expected = "SELECT CAST(`product_id` AS VARCHAR(50)), `product_id`\n" + "FROM `foodmart`.`product`"; - sql(query).withStarRocks().ok(expected); + sql(query).withStarRocks().ok(expected).done(); } /** Test case for @@ -3841,7 +3968,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"product\" "; final String expected = "SELECT CAST(`product_id` AS VARCHAR(65533)), `product_id`\n" + "FROM `foodmart`.`product`"; - sql(query).withStarRocks().ok(expected); + sql(query).withStarRocks().ok(expected).done(); } /** Test case for @@ -3852,7 +3979,7 @@ private SqlDialect nonOrdinalDialect() { + "from \"product\" "; final String expected = "SELECT CAST(`product_id` AS VARCHAR), `product_id`\n" + "FROM `foodmart`.`product`"; - sql(query).withStarRocks().ok(expected); + sql(query).withStarRocks().ok(expected).done(); } /** Test case for @@ -3860,7 +3987,8 @@ private SqlDialect nonOrdinalDialect() { * Invalid unparse for VARCHAR without precision in HiveSqlDialect and * SparkSqlDialect. */ @Test void testCastToVarchar() { - String query = "select cast(\"product_id\" as varchar) from \"product\""; + String query = "select cast(\"product_id\" as varchar)\n" + + "from \"foodmart\".\"product\""; final String expectedClickHouse = "SELECT CAST(`product_id` AS `String`)\n" + "FROM `foodmart`.`product`"; final String expectedMysql = "SELECT CAST(`product_id` AS CHAR)\n" @@ -3873,11 +4001,13 @@ private SqlDialect nonOrdinalDialect() { .withClickHouse().ok(expectedClickHouse) .withMysql().ok(expectedMysql) .withHive().ok(expectedHive) - .withSpark().ok(expectedSpark); + .withSpark().ok(expectedSpark) + .done(); } @Test void testCastToVarcharWithPrecision() { - String query = "select cast(\"product_id\" as varchar(5)) from \"product\""; + String query = "select cast(\"product_id\" as varchar(5))\n" + + "from \"foodmart\".\"product\""; final String expectedMysql = "SELECT CAST(`product_id` AS CHAR(5))\n" + "FROM `foodmart`.`product`"; final String expectedHive = "SELECT CAST(`product_id` AS VARCHAR(5))\n" @@ -3887,11 +4017,13 @@ private SqlDialect nonOrdinalDialect() { sql(query) .withMysql().ok(expectedMysql) .withHive().ok(expectedHive) - .withSpark().ok(expectedSpark); + .withSpark().ok(expectedSpark) + .done(); } @Test void testCastToChar() { - String query = "select cast(\"product_id\" as char) from \"product\""; + String query = "select cast(\"product_id\" as char)\n" + + "from \"foodmart\".\"product\""; final String expectedMysql = "SELECT CAST(`product_id` AS CHAR)\n" + "FROM `foodmart`.`product`"; final String expectedMssql = "SELECT CAST([product_id] AS CHAR)\n" @@ -3904,7 +4036,8 @@ private SqlDialect nonOrdinalDialect() { .withMysql().ok(expectedMysql) .withMssql().ok(expectedMssql) .withHive().ok(expectedHive) - .withSpark().ok(expectedSpark); + .withSpark().ok(expectedSpark) + .done(); } @Test void testCastToCharWithPrecision() { @@ -3921,11 +4054,14 @@ private SqlDialect nonOrdinalDialect() { .withMysql().ok(expectedMysql) .withMssql().ok(expectedMssql) .withHive().ok(expectedHive) - .withSpark().ok(expectedSpark); + .withSpark().ok(expectedSpark) + .done(); } @Test void testSelectQueryWithLimitClauseWithoutOrder() { - String query = "select \"product_id\" from \"product\" limit 100 offset 10"; + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + + "limit 100 offset 10"; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "OFFSET 10 ROWS\n" @@ -3945,11 +4081,13 @@ private SqlDialect nonOrdinalDialect() { .ok(expected) .withClickHouse().ok(expectedClickHouse) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testSelectQueryWithLimitOffsetClause() { - String query = "select \"product_id\" from \"product\"\n" + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"net_weight\" asc limit 100 offset 10"; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" @@ -3969,40 +4107,43 @@ private SqlDialect nonOrdinalDialect() { + "OFFSET 10"; sql(query).ok(expected) .withBigQuery().ok(expectedBigQuery) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testSelectQueryWithParameters() { - String query = "select * from \"product\" " - + "where \"product_id\" = ? " + String query = "select *\n" + + "from \"foodmart\".\"product\"\n" + + "where \"product_id\" = ?\n" + "AND ? >= \"shelf_width\""; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" = ? " + "AND ? >= \"shelf_width\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithFetchOffsetClause() { - String query = "select \"product_id\" from \"product\"\n" + String query = "select \"product_id\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" offset 10 rows fetch next 100 rows only"; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"product_id\"\n" + "OFFSET 10 ROWS\n" + "FETCH NEXT 100 ROWS ONLY"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithFetchClause() { String query = "select \"product_id\"\n" - + "from \"product\"\n" + + "from \"foodmart\".\"product\"\n" + "order by \"product_id\" fetch next 100 rows only"; final String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"product_id\"\n" + "FETCH NEXT 100 ROWS ONLY"; - final String expectedMssql10 = "SELECT TOP (100) [product_id]\n" + final String expectedMssql2008 = "SELECT TOP (100) [product_id]\n" + "FROM [foodmart].[product]\n" + "ORDER BY CASE WHEN [product_id] IS NULL THEN 1 ELSE 0 END, [product_id]"; final String expectedMssql = "SELECT TOP (100) [product_id]\n" @@ -4012,44 +4153,48 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.product\n" + "ORDER BY product_id"; sql(query).ok(expected) - .withMssql(10).ok(expectedMssql10) - .withMssql(11).ok(expectedMssql) - .withMssql(14).ok(expectedMssql) - .withSybase().ok(expectedSybase); + .dialect(MSSQL_2008).ok(expectedMssql2008) + .dialect(MSSQL_2012).ok(expectedMssql) + .dialect(MSSQL_2017).ok(expectedMssql) + .withSybase().ok(expectedSybase).done(); } @Test void testSelectQueryComplex() { - String query = - "select count(*), \"units_per_case\" from \"product\" where \"cases_per_pallet\" > 100 " - + "group by \"product_id\", \"units_per_case\" order by \"units_per_case\" desc"; + String query = "select count(*), \"units_per_case\"\n" + + "from \"foodmart\".\"product\"\n" + + "where \"cases_per_pallet\" > 100\n" + + "group by \"product_id\", \"units_per_case\"\n" + + "order by \"units_per_case\" desc"; final String expected = "SELECT COUNT(*), \"units_per_case\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE CAST(\"cases_per_pallet\" AS INTEGER) > 100\n" + "GROUP BY \"product_id\", \"units_per_case\"\n" + "ORDER BY \"units_per_case\" DESC"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectQueryWithGroup() { - String query = "select" - + " count(*), sum(\"employee_id\") from \"reserve_employee\" " - + "where \"hire_date\" > '2015-01-01' " - + "and (\"position_title\" = 'SDE' or \"position_title\" = 'SDM') " + String query = "select count(*), sum(\"employee_id\")\n" + + "from \"foodmart\".\"reserve_employee\"\n" + + "where \"hire_date\" > '2015-01-01'\n" + + "and (\"position_title\" = 'SDE' or \"position_title\" = 'SDM')\n" + "group by \"store_id\", \"position_title\""; final String expected = "SELECT COUNT(*), SUM(\"employee_id\")\n" + "FROM \"foodmart\".\"reserve_employee\"\n" + "WHERE \"hire_date\" > '2015-01-01' " + "AND (\"position_title\" = 'SDE' OR \"position_title\" = 'SDM')\n" + "GROUP BY \"store_id\", \"position_title\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSimpleJoin() { String query = "select *\n" - + "from \"sales_fact_1997\" as s\n" - + "join \"customer\" as c on s.\"customer_id\" = c.\"customer_id\"\n" - + "join \"product\" as p on s.\"product_id\" = p.\"product_id\"\n" - + "join \"product_class\" as pc\n" + + "from \"foodmart\".\"sales_fact_1997\" as s\n" + + "join \"foodmart\".\"customer\" as c\n" + + " on s.\"customer_id\" = c.\"customer_id\"\n" + + "join \"foodmart\".\"product\" as p\n" + + " on s.\"product_id\" = p.\"product_id\"\n" + + "join \"foodmart\".\"product_class\" as pc\n" + " on p.\"product_class_id\" = pc.\"product_class_id\"\n" + "where c.\"city\" = 'San Francisco'\n" + "and pc.\"product_department\" = 'Snacks'\n"; @@ -4122,15 +4267,16 @@ private SqlDialect nonOrdinalDialect() { + ".\"product_class_id\"\n" + "WHERE \"customer\".\"city\" = 'San Francisco' AND " + "\"product_class\".\"product_department\" = 'Snacks'"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSimpleJoinUsing() { String query = "select *\n" - + "from \"sales_fact_1997\" as s\n" - + " join \"customer\" as c using (\"customer_id\")\n" - + " join \"product\" as p using (\"product_id\")\n" - + " join \"product_class\" as pc using (\"product_class_id\")\n" + + "from \"foodmart\".\"sales_fact_1997\" as s\n" + + " join \"foodmart\".\"customer\" as c using (\"customer_id\")\n" + + " join \"foodmart\".\"product\" as p using (\"product_id\")\n" + + " join \"foodmart\".\"product_class\" as pc\n" + + " using (\"product_class_id\")\n" + "where c.\"city\" = 'San Francisco'\n" + "and pc.\"product_department\" = 'Snacks'\n"; final String expected = "SELECT" @@ -4199,7 +4345,7 @@ private SqlDialect nonOrdinalDialect() { + ".\"product_class_id\"\n" + "WHERE \"customer\".\"city\" = 'San Francisco' AND " + "\"product_class\".\"product_department\" = 'Snacks'"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -4207,8 +4353,10 @@ private SqlDialect nonOrdinalDialect() { * JDBC adapter generates wrong SQL for self join with sub-query. */ @Test void testSubQueryAlias() { String query = "select t1.\"customer_id\", t2.\"customer_id\"\n" - + "from (select \"customer_id\" from \"sales_fact_1997\") as t1\n" - + "inner join (select \"customer_id\" from \"sales_fact_1997\") t2\n" + + "from (select \"customer_id\"\n" + + " from \"foodmart\".\"sales_fact_1997\") as t1\n" + + "inner join (select \"customer_id\"\n" + + " from \"foodmart\".\"sales_fact_1997\") t2\n" + "on t1.\"customer_id\" = t2.\"customer_id\""; final String expected = "SELECT *\n" + "FROM (SELECT sales_fact_1997.customer_id\n" @@ -4216,15 +4364,16 @@ private SqlDialect nonOrdinalDialect() { + "INNER JOIN (SELECT sales_fact_19970.customer_id\n" + "FROM foodmart.sales_fact_1997 AS sales_fact_19970) AS t0 ON t.customer_id = t0.customer_id"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testCartesianProductWithCommaSyntax() { - String query = "select * from \"department\" , \"employee\""; + String query = "select *\n" + + "from \"foodmart\".\"department\" , \"foodmart\".\"employee\""; String expected = "SELECT *\n" + "FROM \"foodmart\".\"department\",\n" + "\"foodmart\".\"employee\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -4233,10 +4382,18 @@ private SqlDialect nonOrdinalDialect() { * column. */ @Test void testJoinOnBoolean() { final String sql = "SELECT 1\n" - + "from emps\n" - + "join emp on (emp.deptno = emps.empno and manager)"; - final String s = sql(sql).schema(CalciteAssert.SchemaSpec.POST).exec(); - assertThat(s, notNullValue()); // sufficient that conversion did not throw + + "from (select empno, deptno, job = 'MANAGER' as manager\n" + + " from \"scott\".emp) as emp\n" + + "join \"scott\".dept on (emp.deptno = dept.deptno and manager)"; + final String expected = "SELECT 1\n" + + "FROM (SELECT \"EMPNO\", \"DEPTNO\"," + + " \"JOB\" = 'MANAGER' AS \"MANAGER\"\n" + + "FROM \"scott\".\"EMP\") AS \"t\"\n" + + "INNER JOIN \"scott\".\"DEPT\"" + + " ON \"t\".\"DEPTNO\" = \"DEPT\".\"DEPTNO\" AND \"t\".\"MANAGER\""; + sql(sql).schema(CalciteAssert.SchemaSpec.SCOTT) + .ok(expected) + .done(); } /** Test case for @@ -4260,25 +4417,27 @@ private SqlDialect nonOrdinalDialect() { + "LEFT JOIN \"scott\".\"DEPT\" " + "ON \"EMP\".\"DEPTNO\" = \"DEPT\".\"DEPTNO\" " + "AND \"DEPT\".\"DNAME\" NOT LIKE 'ACCOUNTING'"; - relFn(relFn).ok(expectedSql); + relFn(relFn).ok(expectedSql).done(); } @Test void testCartesianProductWithInnerJoinSyntax() { - String query = "select * from \"department\"\n" + String query = "select *\n" + + "from \"foodmart\".\"department\"\n" + "INNER JOIN \"employee\" ON TRUE"; String expected = "SELECT *\n" + "FROM \"foodmart\".\"department\",\n" + "\"foodmart\".\"employee\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testFullJoinOnTrueCondition() { - String query = "select * from \"department\"\n" + String query = "select *\n" + + "from \"foodmart\".\"department\"\n" + "FULL JOIN \"employee\" ON TRUE"; String expected = "SELECT *\n" + "FROM \"foodmart\".\"department\"\n" + "FULL JOIN \"foodmart\".\"employee\" ON TRUE"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testCaseOnSubQuery() { @@ -4292,12 +4451,15 @@ private SqlDialect nonOrdinalDialect() { + "FROM \"foodmart\".\"customer\") AS \"t\",\n" + "(VALUES (0)) AS \"t0\" (\"G\")\n" + "GROUP BY \"t0\".\"G\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSimpleIn() { - String query = "select * from \"department\" where \"department_id\" in (\n" - + " select \"department_id\" from \"employee\"\n" + String query = "select *\n" + + "from \"foodmart\".\"department\"\n" + + "where \"department_id\" in (\n" + + " select \"department_id\"\n" + + " from \"foodmart\".\"employee\"\n" + " where \"store_id\" < 150)"; final String expected = "SELECT " + "\"department\".\"department_id\", \"department\"" @@ -4309,7 +4471,7 @@ private SqlDialect nonOrdinalDialect() { + "WHERE \"store_id\" < 150\n" + "GROUP BY \"department_id\") AS \"t1\" " + "ON \"department\".\"department_id\" = \"t1\".\"department_id\""; - sql(query).withConfig(c -> c.withExpand(true)).ok(expected); + sql(query).withConfig(c -> c.withExpand(true)).ok(expected).done(); } /** Test case for @@ -4324,7 +4486,7 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.employee AS employee\n" + "INNER JOIN foodmart.department AS department " + "ON employee.department_id = department.department_id"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectSelfJoinStar() { @@ -4335,7 +4497,7 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.employee AS employee\n" + "INNER JOIN foodmart.employee AS employee0 " + "ON employee.department_id = employee0.department_id"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectJoin() { @@ -4347,11 +4509,12 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.employee AS employee\n" + "INNER JOIN foodmart.department AS department " + "ON employee.department_id = department.department_id"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectSelfJoin() { - String query = "select A.\"employee_id\", B.\"employee_id\" from " + String query = "select A.\"employee_id\", B.\"employee_id\"\n" + + "from " + "\"foodmart\".\"employee\" A join \"foodmart\".\"employee\" B\n" + "on A.\"department_id\" = B.\"department_id\""; final String expected = "SELECT" @@ -4359,16 +4522,17 @@ private SqlDialect nonOrdinalDialect() { + "FROM foodmart.employee AS employee\n" + "INNER JOIN foodmart.employee AS employee0 " + "ON employee.department_id = employee0.department_id"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectWhere() { - String query = "select A.\"employee_id\" from " + String query = "select A.\"employee_id\"\n" + + "from " + "\"foodmart\".\"employee\" A where A.\"department_id\" < 1000"; final String expected = "SELECT employee.employee_id\n" + "FROM foodmart.employee AS employee\n" + "WHERE employee.department_id < 1000"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectJoinWhere() { @@ -4382,12 +4546,13 @@ private SqlDialect nonOrdinalDialect() { + "INNER JOIN foodmart.department AS department " + "ON employee.department_id = department.department_id\n" + "WHERE employee.employee_id < 1000"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectSelfJoinWhere() { - String query = "select A.\"employee_id\", B.\"employee_id\" from " - + "\"foodmart\".\"employee\" A join \"foodmart\".\"employee\" B\n" + String query = "select A.\"employee_id\", B.\"employee_id\"\n" + + "from \"foodmart\".\"employee\" A\n" + + "join \"foodmart\".\"employee\" B\n" + "on A.\"department_id\" = B.\"department_id\" " + "where B.\"employee_id\" < 2000"; final String expected = "SELECT " @@ -4396,41 +4561,43 @@ private SqlDialect nonOrdinalDialect() { + "INNER JOIN foodmart.employee AS employee0 " + "ON employee.department_id = employee0.department_id\n" + "WHERE employee0.employee_id < 2000"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectCast() { - String query = "select \"hire_date\", cast(\"hire_date\" as varchar(10)) " + String query = "select \"hire_date\", cast(\"hire_date\" as varchar(10))\n" + "from \"foodmart\".\"reserve_employee\""; final String expected = "SELECT reserve_employee.hire_date, " + "CAST(reserve_employee.hire_date AS VARCHAR(10))\n" + "FROM foodmart.reserve_employee AS reserve_employee"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectSelectQueryWithGroupByHaving() { - String query = "select count(*) from \"product\" " - + "group by \"product_class_id\", \"product_id\" " + String query = "select count(*)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\", \"product_id\"\n" + "having \"product_id\" > 10"; final String expected = "SELECT COUNT(*)\n" + "FROM foodmart.product AS product\n" + "GROUP BY product.product_class_id, product.product_id\n" + "HAVING product.product_id > 10"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectSelectQueryComplex() { - String query = "select count(*), \"units_per_case\" " - + "from \"product\" where \"cases_per_pallet\" > 100 " - + "group by \"product_id\", \"units_per_case\" " + String query = "select count(*), \"units_per_case\"\n" + + "from \"foodmart\".\"product\"\n" + + "where \"cases_per_pallet\" > 100\n" + + "group by \"product_id\", \"units_per_case\"\n" + "order by \"units_per_case\" desc"; final String expected = "SELECT COUNT(*), product.units_per_case\n" + "FROM foodmart.product AS product\n" + "WHERE CAST(product.cases_per_pallet AS INTEGER) > 100\n" + "GROUP BY product.product_id, product.units_per_case\n" + "ORDER BY product.units_per_case DESC"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } /** Test case for @@ -4440,7 +4607,7 @@ private SqlDialect nonOrdinalDialect() { String query = "select count(foo), \"units_per_case\"\n" + "from (select \"units_per_case\", \"cases_per_pallet\",\n" + " \"product_id\", 1 as foo\n" - + " from \"product\")\n" + + " from \"foodmart\".\"product\")\n" + "where \"cases_per_pallet\" > 100\n" + "group by \"product_id\", \"units_per_case\"\n" + "order by \"units_per_case\" desc"; @@ -4451,19 +4618,19 @@ private SqlDialect nonOrdinalDialect() { + "WHERE CAST(t.cases_per_pallet AS INTEGER) > 100\n" + "GROUP BY t.product_id, t.units_per_case\n" + "ORDER BY t.units_per_case DESC"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2SubQueryFromUnion() { String query = "select count(foo), \"units_per_case\"\n" + "from (select \"units_per_case\", \"cases_per_pallet\",\n" + " \"product_id\", 1 as foo\n" - + " from \"product\"\n" + + " from \"foodmart\".\"product\"\n" + " where \"cases_per_pallet\" > 100\n" + " union all\n" + " select \"units_per_case\", \"cases_per_pallet\",\n" + " \"product_id\", 1 as foo\n" - + " from \"product\"\n" + + " from \"foodmart\".\"product\"\n" + " where \"cases_per_pallet\" < 100)\n" + "where \"cases_per_pallet\" > 100\n" + "group by \"product_id\", \"units_per_case\"\n" @@ -4481,14 +4648,14 @@ private SqlDialect nonOrdinalDialect() { + "WHERE CAST(t3.cases_per_pallet AS INTEGER) > 100\n" + "GROUP BY t3.product_id, t3.units_per_case\n" + "ORDER BY t3.units_per_case DESC"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } @Test void testDb2DialectSelectQueryWithGroup() { - String query = "select count(*), sum(\"employee_id\") " - + "from \"reserve_employee\" " - + "where \"hire_date\" > '2015-01-01' " - + "and (\"position_title\" = 'SDE' or \"position_title\" = 'SDM') " + String query = "select count(*), sum(\"employee_id\")\n" + + "from \"foodmart\".\"reserve_employee\"\n" + + "where \"hire_date\" > '2015-01-01'\n" + + "and (\"position_title\" = 'SDE' or \"position_title\" = 'SDM')\n" + "group by \"store_id\", \"position_title\""; final String expected = "SELECT" + " COUNT(*), SUM(reserve_employee.employee_id)\n" @@ -4497,7 +4664,7 @@ private SqlDialect nonOrdinalDialect() { + "AND (reserve_employee.position_title = 'SDE' OR " + "reserve_employee.position_title = 'SDM')\n" + "GROUP BY reserve_employee.store_id, reserve_employee.position_title"; - sql(query).withDb2().ok(expected); + sql(query).withDb2().ok(expected).done(); } /** Test case for @@ -4505,7 +4672,8 @@ private SqlDialect nonOrdinalDialect() { * JDBC adapter generates SQL with wrong field names. */ @Test void testJoinPlan2() { final String sql = "SELECT v1.deptno, v2.deptno\n" - + "FROM dept v1 LEFT JOIN emp v2 ON v1.deptno = v2.deptno\n" + + "FROM dept v1\n" + + "LEFT JOIN emp v2 ON v1.deptno = v2.deptno\n" + "WHERE v2.job LIKE 'PRESIDENT'"; final String expected = "SELECT \"DEPT\".\"DEPTNO\"," + " \"EMP\".\"DEPTNO\" AS \"DEPTNO0\"\n" @@ -4522,7 +4690,7 @@ private SqlDialect nonOrdinalDialect() { sql(sql) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) .ok(expected) - .withDb2().ok(expectedDb2); + .withDb2().ok(expectedDb2).done(); } /** Test case for @@ -4555,7 +4723,7 @@ private SqlDialect nonOrdinalDialect() { // The hook prevents RelBuilder from removing "FALSE AND FALSE" and such try (Hook.Closeable ignore = Hook.REL_BUILDER_SIMPLIFY.addThread(Hook.propertyJ(false))) { - sql(query).ok(expected); + sql(query).ok(expected).done(); } } @@ -4580,7 +4748,7 @@ private SqlDialect nonOrdinalDialect() { + "WHERE \"EMP\".\"JOB\" LIKE 'PRESIDENT'"; sql(sql) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } /** Test case for @@ -4588,7 +4756,8 @@ private SqlDialect nonOrdinalDialect() { * Join on CASE causes AssertionError in RelToSqlConverter. */ @Test void testJoinOnCase() { final String sql = "SELECT d.deptno, e.deptno\n" - + "FROM dept AS d LEFT JOIN emp AS e\n" + + "FROM dept AS d\n" + + "LEFT JOIN emp AS e\n" + " ON CASE WHEN e.job = 'PRESIDENT' THEN true ELSE d.deptno = 10 END\n" + "WHERE e.job LIKE 'PRESIDENT'"; final String expected = "SELECT \"DEPT\".\"DEPTNO\"," @@ -4600,12 +4769,13 @@ private SqlDialect nonOrdinalDialect() { + "WHERE \"EMP\".\"JOB\" LIKE 'PRESIDENT'"; sql(sql) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } @Test void testWhereCase() { final String sql = "SELECT d.deptno, e.deptno\n" - + "FROM dept AS d LEFT JOIN emp AS e ON d.deptno = e.deptno\n" + + "FROM dept AS d\n" + + "LEFT JOIN emp AS e ON d.deptno = e.deptno\n" + "WHERE CASE WHEN e.job = 'PRESIDENT' THEN true\n" + " ELSE d.deptno = 10 END\n"; final String expected = "SELECT \"DEPT\".\"DEPTNO\"," @@ -4617,18 +4787,21 @@ private SqlDialect nonOrdinalDialect() { + " ELSE CAST(\"DEPT\".\"DEPTNO\" AS INTEGER) = 10 END"; sql(sql) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } /** Test case for * [CALCITE-1586] * JDBC adapter generates wrong SQL if UNION has more than two inputs. */ @Test void testThreeQueryUnion() { - String query = "SELECT \"product_id\" FROM \"product\" " - + " UNION ALL " - + "SELECT \"product_id\" FROM \"sales_fact_1997\" " - + " UNION ALL " - + "SELECT \"product_class_id\" AS product_id FROM \"product_class\""; + String query = "SELECT \"product_id\"\n" + + "FROM \"foodmart\".\"product\"\n" + + "UNION ALL\n" + + "SELECT \"product_id\"\n" + + "FROM \"foodmart\".\"sales_fact_1997\"\n" + + "UNION ALL\n" + + "SELECT \"product_class_id\" AS product_id\n" + + "FROM \"foodmart\".\"product_class\""; String expected = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "UNION ALL\n" @@ -4641,7 +4814,7 @@ private SqlDialect nonOrdinalDialect() { final RuleSet rules = RuleSets.ofList(CoreRules.UNION_MERGE); sql(query) .optimize(rules, null) - .ok(expected); + .ok(expected).done(); } /** Test case for @@ -4653,10 +4826,10 @@ private SqlDialect nonOrdinalDialect() { + " as net_weight\n" + "from (\n" + " select \"product_id\", \"net_weight\"\n" - + " from \"product\"\n" + + " from \"foodmart\".\"product\"\n" + " union all\n" + " select \"product_id\", 0 as \"net_weight\"\n" - + " from \"sales_fact_1997\") t0"; + + " from \"foodmart\".\"sales_fact_1997\") t0"; final String expected = "SELECT SUM(CASE WHEN \"product_id\" = 0" + " THEN \"net_weight\" ELSE 0E0 END) AS \"NET_WEIGHT\"\n" + "FROM (SELECT \"product_id\", \"net_weight\"\n" @@ -4664,7 +4837,7 @@ private SqlDialect nonOrdinalDialect() { + "UNION ALL\n" + "SELECT \"product_id\", 0E0 AS \"net_weight\"\n" + "FROM \"foodmart\".\"sales_fact_1997\") AS \"t1\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @@ -4674,11 +4847,13 @@ private SqlDialect nonOrdinalDialect() { * when the operand has limit or offset. */ @Test void testSetOpRetainParentheses() { // Parentheses will be discarded, because semantics not be affected. - final String discardedParenthesesQuery = "SELECT \"product_id\" FROM \"product\"" + final String discardedParenthesesQuery = "" + + "SELECT \"product_id\" FROM \"foodmart\".\"product\"" + "UNION ALL\n" - + "(SELECT \"product_id\" FROM \"product\" WHERE \"product_id\" > 10)\n" + + "(SELECT \"product_id\" FROM \"foodmart\".\"product\"\n" + + " WHERE \"product_id\" > 10)\n" + "INTERSECT ALL\n" - + "(SELECT \"product_id\" FROM \"product\" )"; + + "(SELECT \"product_id\" FROM \"foodmart\".\"product\" )"; final String discardedParenthesesRes = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "UNION ALL\n" @@ -4689,17 +4864,21 @@ private SqlDialect nonOrdinalDialect() { + "INTERSECT ALL\n" + "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\")"; - sql(discardedParenthesesQuery).ok(discardedParenthesesRes); + sql(discardedParenthesesQuery).ok(discardedParenthesesRes) + .withDisable(BIG_QUERY) + .done(); // Parentheses will be retained because sub-query has LIMIT or OFFSET. // If parentheses are discarded the semantics of parsing will be affected. - final String allSetOpQuery = "SELECT \"product_id\" FROM \"product\"" + final String allSetOpQuery = "" + + "SELECT \"product_id\" FROM \"foodmart\".\"product\"" + "UNION ALL\n" - + "(SELECT \"product_id\" FROM \"product\" LIMIT 10)\n" + + "(SELECT \"product_id\" FROM \"foodmart\".\"product\" LIMIT 10)\n" + "INTERSECT ALL\n" - + "(SELECT \"product_id\" FROM \"product\" OFFSET 10)\n" + + "(SELECT \"product_id\" FROM \"foodmart\".\"product\" OFFSET 10)\n" + "EXCEPT ALL\n" - + "(SELECT \"product_id\" FROM \"product\" LIMIT 5 OFFSET 5)"; + + "(SELECT \"product_id\" FROM \"foodmart\".\"product\"\n" + + " LIMIT 5 OFFSET 5)"; final String allSetOpRes = "SELECT *\n" + "FROM (SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" @@ -4717,24 +4896,34 @@ private SqlDialect nonOrdinalDialect() { + "FROM \"foodmart\".\"product\"\n" + "OFFSET 5 ROWS\n" + "FETCH NEXT 5 ROWS ONLY)"; - sql(allSetOpQuery).ok(allSetOpRes); + sql(allSetOpQuery).ok(allSetOpRes) + .withDisable(BIG_QUERY, CLICKHOUSE, MSSQL_2008, SYBASE) + .done(); // After the config is enabled, order by will be retained, so parentheses are required. - final String retainOrderQuery = "SELECT \"product_id\" FROM \"product\"" + final String retainOrderQuery = "SELECT \"product_id\"\n" + + "FROM \"foodmart\".\"product\"\n" + "UNION ALL\n" - + "(SELECT \"product_id\" FROM \"product\" ORDER BY \"product_id\")"; + + "(SELECT \"product_id\"\n" + + " FROM \"foodmart\".\"product\"\n" + + " ORDER BY \"product_id\")"; final String retainOrderResult = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "UNION ALL\n" + "(SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"product_id\")"; - sql(retainOrderQuery).withConfig(c -> c.withRemoveSortInSubQuery(false)).ok(retainOrderResult); + sql(retainOrderQuery) + .withConfig(c -> c.withRemoveSortInSubQuery(false)) + .ok(retainOrderResult).done(); // Parentheses are required to keep ORDER and LIMIT on the sub-query. - final String retainLimitQuery = "SELECT \"product_id\" FROM \"product\"" + final String retainLimitQuery = "SELECT \"product_id\"\n" + + "FROM \"foodmart\".\"product\"" + "UNION ALL\n" - + "(SELECT \"product_id\" FROM \"product\" ORDER BY \"product_id\" LIMIT 2)"; + + "(SELECT \"product_id\"\n" + + " FROM \"foodmart\".\"product\"\n" + + " ORDER BY \"product_id\" LIMIT 2)"; final String retainLimitResult = "SELECT \"product_id\"\n" + "FROM \"foodmart\".\"product\"\n" + "UNION ALL\n" @@ -4742,35 +4931,46 @@ private SqlDialect nonOrdinalDialect() { + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"product_id\"\n" + "FETCH NEXT 2 ROWS ONLY)"; - sql(retainLimitQuery).ok(retainLimitResult); + sql(retainLimitQuery).ok(retainLimitResult).done(); } - /** Test case for * [CALCITE-5570] * Support nested map type for SqlDataTypeSpec. */ @Test void testCastAsMapType() { - sql("SELECT CAST(MAP['A', 1.0] AS MAP)") - .ok("SELECT CAST(MAP['A', 1.0] AS MAP< VARCHAR CHARACTER SET \"ISO-8859-1\", DOUBLE >)\n" - + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"); - sql("SELECT CAST(MAP['A', ARRAY[1, 2, 3]] AS MAP)") - .ok("SELECT CAST(MAP['A', ARRAY[1, 2, 3]] AS " - + "MAP< VARCHAR CHARACTER SET \"ISO-8859-1\", INTEGER ARRAY >)\n" - + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"); - sql("SELECT CAST(MAP[ARRAY['A'], MAP[1, 2]] AS MAP>)") - .ok("SELECT CAST(MAP[ARRAY['A'], MAP[1, 2]] AS " - + "MAP< VARCHAR CHARACTER SET \"ISO-8859-1\" ARRAY, MAP< INTEGER, INTEGER > >)\n" - + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"); + final String query = "SELECT CAST(MAP['A', 1.0] AS MAP)"; + final String expected = "SELECT CAST(MAP['A', 1.0] AS " + + "MAP< VARCHAR CHARACTER SET \"ISO-8859-1\", DOUBLE >)\n" + + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; + sql(query).ok(expected).done(); + } + + @Test void testCastAsMapType2() { + final String query = "SELECT CAST(MAP['A', ARRAY[1, 2, 3]] AS MAP)"; + final String expected = "SELECT CAST(MAP['A', ARRAY[1, 2, 3]] AS " + + "MAP< VARCHAR CHARACTER SET \"ISO-8859-1\", INTEGER ARRAY >)\n" + + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; + sql(query).ok(expected).done(); + } + + @Test void testCastAsMapType3() { + final String query = "SELECT CAST(MAP[ARRAY['A'], MAP[1, 2]] AS " + + "MAP>)"; + final String expected = "SELECT CAST(MAP[ARRAY['A'], MAP[1, 2]] AS " + + "MAP< VARCHAR CHARACTER SET \"ISO-8859-1\" ARRAY, MAP< INTEGER, INTEGER > >)\n" + + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; + sql(query).ok(expected).done(); } /** Test case for * [CALCITE-4674] * Excess quotes in generated SQL when STAR is a column alias. */ @Test void testAliasOnStarNoExcessQuotes() { - final String query = "select \"customer_id\" as \"*\" from \"customer\""; + final String query = "select \"customer_id\" as \"*\"\n" + + "from \"foodmart\".\"customer\""; final String expected = "SELECT \"customer_id\" AS \"*\"\n" + "FROM \"foodmart\".\"customer\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testLiteral() { @@ -4835,7 +5035,7 @@ private void checkLiteral2(String expression, String expected) { String expectedHsqldb = "SELECT *\n" + "FROM (VALUES (" + expected + ")) AS t (EXPR$0)"; sql("VALUES " + expression) - .withHsqldb().ok(expectedHsqldb); + .withHsqldb().ok(expectedHsqldb).done(); } /** Test case for @@ -4843,10 +5043,11 @@ private void checkLiteral2(String expression, String expected) { * Removing Window Boundaries from SqlWindow of Aggregate Function which do * not allow Framing. */ @Test void testRowNumberFunctionForPrintingOfFrameBoundary() { - String query = "SELECT row_number() over (order by \"hire_date\") FROM \"employee\""; + String query = "SELECT row_number() over (order by \"hire_date\")\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT ROW_NUMBER() OVER (ORDER BY \"hire_date\")\n" + "FROM \"foodmart\".\"employee\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -4858,7 +5059,11 @@ private void checkLiteral2(String expression, String expected) { final String expected0 = "SELECT LEAD(\"employee_id\", 2) IGNORE NULLS OVER (ORDER BY " + "\"hire_date\")\n" + "FROM \"foodmart\".\"employee\""; - sql(query0).ok(expected0); + sql(query0) + // Execution disabled due to error: + // "IllegalStateException: Unable to implement EnumerableCalc" + .withPhase(PARSE) + .ok(expected0).done(); final String query1 = "SELECT " + "LAG(\"employee_id\", 1) IGNORE NULLS OVER (ORDER BY \"hire_date\")," @@ -4872,113 +5077,128 @@ private void checkLiteral2(String expression, String expected) { + "LAST_VALUE(\"employee_id\") IGNORE NULLS OVER (ORDER BY \"hire_date\"" + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)\n" + "FROM \"foodmart\".\"employee\""; - sql(query1).ok(expected1); + sql(query1) + // Execution disabled due to error: + // "IllegalStateException: Unable to implement EnumerableCalc" + .withPhase(PARSE) + .ok(expected1).done(); } /** Test case for * [CALCITE-3112] * Support Window in RelToSqlConverter. */ @Test void testConvertWindowToSql() { - String query0 = "SELECT row_number() over (order by \"hire_date\") FROM \"employee\""; + String query0 = "SELECT row_number() over (order by \"hire_date\")\n" + + "FROM \"foodmart\".\"employee\""; String expected0 = "SELECT ROW_NUMBER() OVER (ORDER BY \"hire_date\") AS \"$0\"\n" - + "FROM \"foodmart\".\"employee\""; + + "FROM \"foodmart\".\"employee\""; - String query1 = "SELECT rank() over (order by \"hire_date\") FROM \"employee\""; + String query1 = "SELECT rank() over (order by \"hire_date\")\n" + + "FROM \"foodmart\".\"employee\""; String expected1 = "SELECT RANK() OVER (ORDER BY \"hire_date\") AS \"$0\"\n" - + "FROM \"foodmart\".\"employee\""; + + "FROM \"foodmart\".\"employee\""; String query2 = "SELECT lead(\"employee_id\",1,'NA') over " - + "(partition by \"hire_date\" order by \"employee_id\")\n" - + "FROM \"employee\""; + + "(partition by \"hire_date\" order by \"employee_id\")\n" + + "FROM \"foodmart\".\"employee\""; String expected2 = "SELECT LEAD(\"employee_id\", 1, 'NA') OVER " - + "(PARTITION BY \"hire_date\" " - + "ORDER BY \"employee_id\") AS \"$0\"\n" - + "FROM \"foodmart\".\"employee\""; + + "(PARTITION BY \"hire_date\" " + + "ORDER BY \"employee_id\") AS \"$0\"\n" + + "FROM \"foodmart\".\"employee\""; String query3 = "SELECT lag(\"employee_id\",1,'NA') over " - + "(partition by \"hire_date\" order by \"employee_id\")\n" - + "FROM \"employee\""; + + "(partition by \"hire_date\" order by \"employee_id\")\n" + + "FROM \"foodmart\".\"employee\""; String expected3 = "SELECT LAG(\"employee_id\", 1, 'NA') OVER " - + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\") AS \"$0\"\n" - + "FROM \"foodmart\".\"employee\""; - - String query4 = "SELECT lag(\"employee_id\",1,'NA') " - + "over (partition by \"hire_date\" order by \"employee_id\") as lag1, " - + "lag(\"employee_id\",1,'NA') " - + "over (partition by \"birth_date\" order by \"employee_id\") as lag2, " - + "count(*) over (partition by \"hire_date\" order by \"employee_id\") as count1, " - + "count(*) over (partition by \"birth_date\" order by \"employee_id\") as count2\n" - + "FROM \"employee\""; + + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\") AS \"$0\"\n" + + "FROM \"foodmart\".\"employee\""; + + String query4 = "SELECT " + + "lag(\"employee_id\",1,'NA') over (partition by \"hire_date\"" + + " order by \"employee_id\") as lag1, " + + "lag(\"employee_id\",1,'NA') over (partition by \"birth_date\"" + + " order by \"employee_id\") as lag2, " + + "count(*) over (partition by \"hire_date\"" + + " order by \"employee_id\") as count1, " + + "count(*) over (partition by \"birth_date\"" + + " order by \"employee_id\") as count2\n" + + "FROM \"foodmart\".\"employee\""; String expected4 = "SELECT LAG(\"employee_id\", 1, 'NA') OVER " - + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\") AS \"$0\", " - + "LAG(\"employee_id\", 1, 'NA') OVER " - + "(PARTITION BY \"birth_date\" ORDER BY \"employee_id\") AS \"$1\", " - + "COUNT(*) OVER (PARTITION BY \"hire_date\" ORDER BY \"employee_id\" " - + "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$2\", " - + "COUNT(*) OVER (PARTITION BY \"birth_date\" ORDER BY \"employee_id\" " - + "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$3\"\n" - + "FROM \"foodmart\".\"employee\""; - - String query5 = "SELECT lag(\"employee_id\",1,'NA') " - + "over (partition by \"hire_date\" order by \"employee_id\") as lag1, " - + "lag(\"employee_id\",1,'NA') " - + "over (partition by \"birth_date\" order by \"employee_id\") as lag2, " - + "max(sum(\"employee_id\")) over (partition by \"hire_date\" order by \"employee_id\") as count1, " - + "max(sum(\"employee_id\")) over (partition by \"birth_date\" order by \"employee_id\") as count2\n" - + "FROM \"employee\" group by \"employee_id\", \"hire_date\", \"birth_date\""; - String expected5 = "SELECT LAG(\"employee_id\", 1, 'NA') OVER " - + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\") AS \"$0\", " - + "LAG(\"employee_id\", 1, 'NA') OVER " - + "(PARTITION BY \"birth_date\" ORDER BY \"employee_id\") AS \"$1\", " - + "MAX(SUM(\"employee_id\")) OVER (PARTITION BY \"hire_date\" ORDER BY \"employee_id\" " - + "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$2\", " - + "MAX(SUM(\"employee_id\")) OVER (PARTITION BY \"birth_date\" ORDER BY \"employee_id\" " - + "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$3\"\n" - + "FROM \"foodmart\".\"employee\"\n" - + "GROUP BY \"employee_id\", \"hire_date\", \"birth_date\""; - - String query6 = "SELECT lag(\"employee_id\",1,'NA') over " - + "(partition by \"hire_date\" order by \"employee_id\"), \"hire_date\"\n" - + "FROM \"employee\"\n" - + "group by \"hire_date\", \"employee_id\""; - String expected6 = "SELECT LAG(\"employee_id\", 1, 'NA') " - + "OVER (PARTITION BY \"hire_date\" ORDER BY \"employee_id\"), \"hire_date\"\n" - + "FROM \"foodmart\".\"employee\"\n" - + "GROUP BY \"hire_date\", \"employee_id\""; + + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\") AS \"$0\", " + + "LAG(\"employee_id\", 1, 'NA') OVER " + + "(PARTITION BY \"birth_date\" ORDER BY \"employee_id\") AS \"$1\", " + + "COUNT(*) OVER (PARTITION BY \"hire_date\" ORDER BY \"employee_id\" " + + "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$2\", " + + "COUNT(*) OVER (PARTITION BY \"birth_date\" ORDER BY \"employee_id\" " + + "RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$3\"\n" + + "FROM \"foodmart\".\"employee\""; + + String query5 = "SELECT " + + "lag(\"employee_id\",1,'NA') over (partition by \"hire_date\"" + + " order by \"employee_id\") as lag1, " + + "lag(\"employee_id\",1,'NA') over (partition by \"birth_date\"" + + " order by \"employee_id\") as lag2, " + + "max(sum(\"employee_id\")) over (partition by \"hire_date\"" + + " order by \"employee_id\") as count1, " + + "max(sum(\"employee_id\")) over (partition by \"birth_date\"" + + " order by \"employee_id\") as count2\n" + + "FROM \"foodmart\".\"employee\"\n" + + "group by \"employee_id\", \"hire_date\", \"birth_date\""; + String expected5 = "SELECT " + + "LAG(\"employee_id\", 1, 'NA') OVER (PARTITION BY \"hire_date\"" + + " ORDER BY \"employee_id\") AS \"$0\", " + + "LAG(\"employee_id\", 1, 'NA') OVER (PARTITION BY \"birth_date\"" + + " ORDER BY \"employee_id\") AS \"$1\", " + + "MAX(SUM(\"employee_id\")) OVER (PARTITION BY \"hire_date\"" + + " ORDER BY \"employee_id\"" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$2\", " + + "MAX(SUM(\"employee_id\")) OVER (PARTITION BY \"birth_date\"" + + " ORDER BY \"employee_id\"" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$3\"\n" + + "FROM \"foodmart\".\"employee\"\n" + + "GROUP BY \"employee_id\", \"hire_date\", \"birth_date\""; + + String query6 = "SELECT " + + "lag(\"employee_id\",1,'NA') over (partition by \"hire_date\"" + + " order by \"employee_id\"),\n" + + " \"hire_date\"\n" + + "FROM \"foodmart\".\"employee\"\n" + + "group by \"hire_date\", \"employee_id\""; + String expected6 = "SELECT " + + "LAG(\"employee_id\", 1, 'NA') OVER (PARTITION BY \"hire_date\"" + + " ORDER BY \"employee_id\"), \"hire_date\"\n" + + "FROM \"foodmart\".\"employee\"\n" + + "GROUP BY \"hire_date\", \"employee_id\""; String query7 = "SELECT " - + "count(distinct \"employee_id\") over (order by \"hire_date\") FROM \"employee\""; + + "count(distinct \"employee_id\") over (order by \"hire_date\")\n" + + "FROM \"foodmart\".\"employee\""; String expected7 = "SELECT " + "COUNT(DISTINCT \"employee_id\") OVER (ORDER BY \"hire_date\"" + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS \"$0\"\n" + "FROM \"foodmart\".\"employee\""; String query8 = "SELECT " - + "sum(distinct \"position_id\") over (order by \"hire_date\") FROM \"employee\""; - String expected8 = - "SELECT CASE WHEN (COUNT(DISTINCT \"position_id\") OVER (ORDER BY \"hire_date\" " - + "RANGE" - + " BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)) > 0 THEN COALESCE(SUM(DISTINCT " - + "\"position_id\") OVER (ORDER BY \"hire_date\" RANGE BETWEEN UNBOUNDED " - + "PRECEDING AND CURRENT ROW), 0) ELSE NULL END\n" - + "FROM \"foodmart\".\"employee\""; - - HepProgramBuilder builder = new HepProgramBuilder(); - builder.addRuleClass(ProjectOverSumToSum0Rule.class); - builder.addRuleClass(ProjectToWindowRule.class); - HepPlanner hepPlanner = new HepPlanner(builder.build()); - RuleSet rules = - RuleSets.ofList(CoreRules.PROJECT_OVER_SUM_TO_SUM0_RULE, - CoreRules.PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW); + + "sum(distinct \"position_id\") over (order by \"hire_date\")\n" + + "FROM \"foodmart\".\"employee\""; + String expected8 = "SELECT " + + "CASE WHEN (COUNT(DISTINCT \"position_id\") " + + "OVER (ORDER BY \"hire_date\"" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)) > 0 " + + "THEN COALESCE(SUM(DISTINCT \"position_id\") " + + "OVER (ORDER BY \"hire_date\"" + + " RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW), 0) " + + "ELSE NULL END\n" + + "FROM \"foodmart\".\"employee\""; - sql(query0).optimize(rules, hepPlanner).ok(expected0); - sql(query1).optimize(rules, hepPlanner).ok(expected1); - sql(query2).optimize(rules, hepPlanner).ok(expected2); - sql(query3).optimize(rules, hepPlanner).ok(expected3); - sql(query4).optimize(rules, hepPlanner).ok(expected4); - sql(query5).optimize(rules, hepPlanner).ok(expected5); - sql(query6).optimize(rules, hepPlanner).ok(expected6); - sql(query7).optimize(rules, hepPlanner).ok(expected7); - sql(query8).optimize(rules, hepPlanner).ok(expected8); + sql(query0).optimizeOver().ok(expected0).done(); + sql(query1).optimizeOver().ok(expected1).done(); + sql(query2).optimizeOver().ok(expected2).done(); + sql(query3).optimizeOver().ok(expected3).done(); + sql(query4).optimizeOver().ok(expected4).done(); + sql(query5).optimizeOver().ok(expected5).done(); + sql(query6).optimizeOver().ok(expected6).done(); + sql(query7).optimizeOver().ok(expected7).done(); + sql(query8).optimizeOver().ok(expected8).done(); } /** Test case for @@ -4993,7 +5213,7 @@ private void checkLiteral2(String expression, String expected) { + "WHERE \"product_id\" IN (SELECT *\n" + "FROM (VALUES (12),\n" + "(NULL)) AS \"t\" (\"ROW_VALUE\"))"; - sql(query).withConfig(c -> c.withInSubQueryThreshold(1)).ok(expected); + sql(query).withConfig(c -> c.withInSubQueryThreshold(1)).ok(expected).done(); } @Test void convertInListToValues2() { @@ -5004,7 +5224,7 @@ private void checkLiteral2(String expression, String expected) { + "WHERE CAST(\"brand_name\" AS CHAR(1) CHARACTER SET \"ISO-8859-1\") IN (SELECT *\n" + "FROM (VALUES ('n'),\n" + "(NULL)) AS \"t\" (\"ROW_VALUE\"))"; - sql(query).withConfig(c -> c.withInSubQueryThreshold(1)).ok(expected); + sql(query).withConfig(c -> c.withInSubQueryThreshold(1)).ok(expected).done(); } @Test void convertInListToValues3() { @@ -5015,7 +5235,7 @@ private void checkLiteral2(String expression, String expected) { + "WHERE (\"brand_name\" = \"product_name\") IN (SELECT *\n" + "FROM (VALUES (FALSE),\n" + "(NULL)) AS \"t\" (\"ROW_VALUE\"))"; - sql(query).withConfig(c -> c.withInSubQueryThreshold(1)).ok(expected); + sql(query).withConfig(c -> c.withInSubQueryThreshold(1)).ok(expected).done(); } /** Test case for @@ -5023,19 +5243,23 @@ private void checkLiteral2(String expression, String expected) { * "numeric field overflow" when running the generated SQL in * PostgreSQL. */ @Test void testSumReturnType() { - String query = - "select sum(e1.\"store_sales\"), sum(e2.\"store_sales\") from \"sales_fact_dec_1998\" as " - + "e1 , \"sales_fact_dec_1998\" as e2 where e1.\"product_id\" = e2.\"product_id\""; - - String expect = "SELECT SUM(CAST(\"t\".\"EXPR$0\" * \"t0\".\"$f1\" AS DECIMAL" - + "(19, 4))), SUM(CAST(\"t\".\"$f2\" * \"t0\".\"EXPR$1\" AS DECIMAL(19, 4)))\n" - + "FROM (SELECT \"product_id\", SUM(\"store_sales\") AS \"EXPR$0\", COUNT(*) AS \"$f2\"\n" + String query = "select sum(e1.\"store_sales\"), sum(e2.\"store_sales\")\n" + + "from \"foodmart\".\"sales_fact_dec_1998\" as e1,\n" + + " \"foodmart\".\"sales_fact_dec_1998\" as e2\n" + + "where e1.\"product_id\" = e2.\"product_id\""; + + String expect = "SELECT SUM(CAST(\"t\".\"EXPR$0\" * \"t0\".\"$f1\" AS DECIMAL(19, 4)))," + + " SUM(CAST(\"t\".\"$f2\" * \"t0\".\"EXPR$1\" AS DECIMAL(19, 4)))\n" + + "FROM (SELECT \"product_id\"," + + " SUM(\"store_sales\") AS \"EXPR$0\", COUNT(*) AS \"$f2\"\n" + "FROM \"foodmart\".\"sales_fact_dec_1998\"\n" + "GROUP BY \"product_id\") AS \"t\"\n" + "INNER JOIN " - + "(SELECT \"product_id\", COUNT(*) AS \"$f1\", SUM(\"store_sales\") AS \"EXPR$1\"\n" + + "(SELECT \"product_id\", COUNT(*) AS \"$f1\"," + + " SUM(\"store_sales\") AS \"EXPR$1\"\n" + "FROM \"foodmart\".\"sales_fact_dec_1998\"\n" - + "GROUP BY \"product_id\") AS \"t0\" ON \"t\".\"product_id\" = \"t0\".\"product_id\""; + + "GROUP BY \"product_id\") AS \"t0\"" + + " ON \"t\".\"product_id\" = \"t0\".\"product_id\""; HepProgramBuilder builder = new HepProgramBuilder(); builder.addRuleClass(FilterJoinRule.class); @@ -5047,13 +5271,14 @@ private void checkLiteral2(String expression, String expected) { CoreRules.JOIN_CONDITION_PUSH, CoreRules.AGGREGATE_PROJECT_MERGE, CoreRules.AGGREGATE_JOIN_TRANSPOSE_EXTENDED); - sql(query).withPostgresql().optimize(rules, hepPlanner).ok(expect); + sql(query).withPostgresql().optimize(rules, hepPlanner).ok(expect).done(); } @Test void testMultiplicationNotAliasedToStar() { - final String sql = "select s.\"customer_id\", sum(s.\"store_sales\" * s.\"store_cost\")" - + "from \"sales_fact_1997\" as s\n" - + "join \"customer\" as c\n" + final String sql = "select s.\"customer_id\",\n" + + " sum(s.\"store_sales\" * s.\"store_cost\")\n" + + "from \"foodmart\".\"sales_fact_1997\" as s\n" + + "join \"foodmart\".\"customer\" as c\n" + " on s.\"customer_id\" = c.\"customer_id\"\n" + "group by s.\"customer_id\""; final String expected = "SELECT \"t\".\"customer_id\", SUM(\"t\".\"$f1\")\n" @@ -5063,13 +5288,13 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"customer\") AS \"t0\" ON \"t\".\"customer_id\" = \"t0\".\"customer_id\"\n" + "GROUP BY \"t\".\"customer_id\""; RuleSet rules = RuleSets.ofList(CoreRules.PROJECT_JOIN_TRANSPOSE); - sql(sql).optimize(rules, null).ok(expected); + sql(sql).optimize(rules, null).ok(expected).done(); } @Test void testMultiplicationRetainsExplicitAlias() { final String sql = "select s.\"customer_id\", s.\"store_sales\" * s.\"store_cost\" as \"total\"" - + "from \"sales_fact_1997\" as s\n" - + "join \"customer\" as c\n" + + "from \"foodmart\".\"sales_fact_1997\" as s\n" + + "join \"foodmart\".\"customer\" as c\n" + " on s.\"customer_id\" = c.\"customer_id\"\n"; final String expected = "SELECT \"t\".\"customer_id\", \"t\".\"total\"\n" + "FROM (SELECT \"customer_id\", \"store_sales\" * \"store_cost\" AS \"total\"\n" @@ -5078,32 +5303,35 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"customer\") AS \"t0\" ON \"t\".\"customer_id\" = \"t0\"" + ".\"customer_id\""; RuleSet rules = RuleSets.ofList(CoreRules.PROJECT_JOIN_TRANSPOSE); - sql(sql).optimize(rules, null).ok(expected); + sql(sql).optimize(rules, null).ok(expected).done(); } @Test void testRankFunctionForPrintingOfFrameBoundary() { - String query = "SELECT rank() over (order by \"hire_date\") FROM \"employee\""; + String query = "SELECT rank() over (order by \"hire_date\")\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT RANK() OVER (ORDER BY \"hire_date\")\n" + "FROM \"foodmart\".\"employee\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testLeadFunctionForPrintingOfFrameBoundary() { String query = "SELECT lead(\"employee_id\",1,'NA') over " - + "(partition by \"hire_date\" order by \"employee_id\") FROM \"employee\""; + + "(partition by \"hire_date\" order by \"employee_id\")\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT LEAD(\"employee_id\", 1, 'NA') OVER " + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\")\n" + "FROM \"foodmart\".\"employee\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testLagFunctionForPrintingOfFrameBoundary() { String query = "SELECT lag(\"employee_id\",1,'NA') over " - + "(partition by \"hire_date\" order by \"employee_id\") FROM \"employee\""; + + "(partition by \"hire_date\" order by \"employee_id\")\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT LAG(\"employee_id\", 1, 'NA') OVER " + "(PARTITION BY \"hire_date\" ORDER BY \"employee_id\")\n" + "FROM \"foodmart\".\"employee\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -5124,7 +5352,7 @@ private void checkLiteral2(String expression, String expected) { + " ROW_NUMBER() OVER (ORDER BY \"product_id\") AS \"RN\"\n" + "FROM \"foodmart\".\"product\") AS \"t\""; sql(query) - .withPostgresql().ok(expected); + .withPostgresql().ok(expected).done(); } /** Test case for @@ -5146,8 +5374,8 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"product\") AS \"t\""; RuleSet rules = RuleSets.ofList(CoreRules.PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW); // PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW rule will remove alias - sql(query0).optimize(rules, null).ok(expected00); - sql(query0).ok(expected01); + sql(query0).optimize(rules, null).ok(expected00).done(); + sql(query0).ok(expected01).done(); String query1 = " SELECT \"product_id\"," + "RANK() OVER (ORDER BY \"product_name\" DESC) AS \"rank1\" " @@ -5158,15 +5386,16 @@ private void checkLiteral2(String expression, String expected) { String expected11 = "SELECT \"product_id\"," + " RANK() OVER (ORDER BY \"product_name\" DESC) AS \"rank1\"\n" + "FROM \"foodmart\".\"product\""; - sql(query1).optimize(rules, null).ok(expected10); - sql(query1).ok(expected11); + sql(query1).optimize(rules, null).ok(expected10).done(); + sql(query1).ok(expected11).done(); } /** Test case for * [CALCITE-1798] * Generate dialect-specific SQL for FLOOR operator. */ @Test void testFloor() { - String query = "SELECT floor(\"hire_date\" TO MINUTE) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO MINUTE)\n" + + "FROM \"foodmart\".\"employee\""; String expectedClickHouse = "SELECT toStartOfMinute(`hire_date`)\n" + "FROM `foodmart`.`employee`"; String expectedHsqldb = "SELECT TRUNC(hire_date, 'MI')\n" @@ -5187,7 +5416,8 @@ private void checkLiteral2(String expression, String expected) { .withOracle().ok(expectedOracle) .withPostgresql().ok(expectedPostgresql) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testFetchMssql() { @@ -5195,11 +5425,11 @@ private void checkLiteral2(String expression, String expected) { String expected = "SELECT TOP (1) *\n" + "FROM [foodmart].[employee]"; sql(query) - .withMssql().ok(expected); + .withMssql().ok(expected).done(); } @Test void testFetchOffset() { - final String query = "SELECT * FROM \"employee\" LIMIT 1 OFFSET 1"; + String query = "SELECT * FROM \"foodmart\".\"employee\" LIMIT 1 OFFSET 1"; final String expectedMssql = "SELECT *\n" + "FROM [foodmart].[employee]\n" + "OFFSET 1 ROWS\n" @@ -5218,27 +5448,31 @@ private void checkLiteral2(String expression, String expected) { .withMssql().ok(expectedMssql) .withSybase().ok(expectedSybase) .withPresto().ok(expectedPresto) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testFloorMssqlMonth() { - String query = "SELECT floor(\"hire_date\" TO MONTH) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO MONTH)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT CONVERT(DATETIME, CONVERT(VARCHAR(7), [hire_date] , 126)+'-01')\n" + "FROM [foodmart].[employee]"; sql(query) - .withMssql().ok(expected); + .withMssql().ok(expected).done(); } @Test void testFloorMysqlMonth() { - String query = "SELECT floor(\"hire_date\" TO MONTH) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO MONTH)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT DATE_FORMAT(`hire_date`, '%Y-%m-01')\n" + "FROM `foodmart`.`employee`"; sql(query) - .withMysql().ok(expected); + .withMysql().ok(expected).done(); } @Test void testFloorWeek() { - final String query = "SELECT floor(\"hire_date\" TO WEEK) FROM \"employee\""; + final String query = "SELECT floor(\"hire_date\" TO WEEK)\n" + + "FROM \"foodmart\".\"employee\""; final String expectedClickHouse = "SELECT toMonday(`hire_date`)\n" + "FROM `foodmart`.`employee`"; final String expectedMssql = "SELECT CONVERT(DATETIME, CONVERT(VARCHAR(10), " @@ -5252,11 +5486,12 @@ private void checkLiteral2(String expression, String expected) { sql(query) .withClickHouse().ok(expectedClickHouse) .withMssql().ok(expectedMssql) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql).done(); } @Test void testUnparseSqlIntervalQualifierDb2() { - String queryDatePlus = "select * from \"employee\" where \"hire_date\" + " + String queryDatePlus = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" + " + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; String expectedDatePlus = "SELECT *\n" + "FROM foodmart.employee AS employee\n" @@ -5264,74 +5499,83 @@ private void checkLiteral2(String expression, String expected) { + " > TIMESTAMP '2005-10-17 00:00:00'"; sql(queryDatePlus) - .withDb2().ok(expectedDatePlus); + .withDb2().ok(expectedDatePlus).done(); - String queryDateMinus = "select * from \"employee\" where \"hire_date\" - " - + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; + String queryDateMinus = "select *\n" + + "from \"foodmart\".\"employee\"\n" + + "where \"hire_date\" - INTERVAL '19800' SECOND(5)\n" + + " > TIMESTAMP '2005-10-17 00:00:00' "; String expectedDateMinus = "SELECT *\n" + "FROM foodmart.employee AS employee\n" + "WHERE (employee.hire_date - 19800 SECOND)" + " > TIMESTAMP '2005-10-17 00:00:00'"; sql(queryDateMinus) - .withDb2().ok(expectedDateMinus); + .withDb2().ok(expectedDateMinus).done(); } @Test void testUnparseSqlIntervalQualifierMySql() { - final String sql0 = "select * from \"employee\" where \"hire_date\" - " + final String sql0 = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" - " + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; final String expect0 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` - INTERVAL '19800' SECOND)" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql0).withMysql().ok(expect0); + sql(sql0).withMysql().ok(expect0).done(); - final String sql1 = "select * from \"employee\" where \"hire_date\" + " + final String sql1 = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" + " + "INTERVAL '10' HOUR > TIMESTAMP '2005-10-17 00:00:00' "; final String expect1 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` + INTERVAL '10' HOUR)" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql1).withMysql().ok(expect1); + sql(sql1).withMysql().ok(expect1).done(); - final String sql2 = "select * from \"employee\" where \"hire_date\" + " + final String sql2 = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" + " + "INTERVAL '1-2' year to month > TIMESTAMP '2005-10-17 00:00:00' "; final String expect2 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` + INTERVAL '1-2' YEAR_MONTH)" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql2).withMysql().ok(expect2); + sql(sql2).withMysql().ok(expect2).done(); - final String sql3 = "select * from \"employee\" " + final String sql3 = "select *\n" + + "from \"foodmart\".\"employee\" " + "where \"hire_date\" + INTERVAL '39:12' MINUTE TO SECOND" + " > TIMESTAMP '2005-10-17 00:00:00' "; final String expect3 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` + INTERVAL '39:12' MINUTE_SECOND)" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql3).withMysql().ok(expect3); + sql(sql3).withMysql().ok(expect3).done(); } - @Test void testUnparseSqlIntervalQualifierMsSql() { - String queryDatePlus = "select * from \"employee\" where \"hire_date\" +" + @Test void testUnparseSqlIntervalQualifierMssql() { + String queryDatePlus = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" +" + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; String expectedDatePlus = "SELECT *\n" + "FROM [foodmart].[employee]\n" + "WHERE DATEADD(SECOND, 19800, [hire_date]) > '2005-10-17 00:00:00'"; sql(queryDatePlus) - .withMssql().ok(expectedDatePlus); + .withMssql().ok(expectedDatePlus).done(); - String queryDateMinus = "select * from \"employee\" where \"hire_date\" -" + String queryDateMinus = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" -" + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; String expectedDateMinus = "SELECT *\n" + "FROM [foodmart].[employee]\n" + "WHERE DATEADD(SECOND, -19800, [hire_date]) > '2005-10-17 00:00:00'"; sql(queryDateMinus) - .withMssql().ok(expectedDateMinus); + .withMssql().ok(expectedDateMinus).done(); - String queryDateMinusNegate = "select * from \"employee\" " + String queryDateMinusNegate = "select *\n" + + "from \"foodmart\".\"employee\" " + "where \"hire_date\" -INTERVAL '-19800' SECOND(5)" + " > TIMESTAMP '2005-10-17 00:00:00' "; String expectedDateMinusNegate = "SELECT *\n" @@ -5339,63 +5583,78 @@ private void checkLiteral2(String expression, String expected) { + "WHERE DATEADD(SECOND, 19800, [hire_date]) > '2005-10-17 00:00:00'"; sql(queryDateMinusNegate) - .withMssql().ok(expectedDateMinusNegate); + .withMssql().ok(expectedDateMinusNegate).done(); } @Test void testUnparseSqlIntervalQualifierBigQuery() { - final String sql0 = "select * from \"employee\" where \"hire_date\" - " - + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; + final String sql0 = "select *\n" + + "from \"foodmart\".\"employee\"\n" + + "where \"hire_date\" - INTERVAL '19800' SECOND(5)\n" + + " > TIMESTAMP '2005-10-17 00:00:00'"; final String expect0 = "SELECT *\n" + "FROM foodmart.employee\n" + "WHERE (hire_date - INTERVAL 19800 SECOND)" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql0).withBigQuery().ok(expect0); + sql(sql0).withBigQuery().ok(expect0).done(); - final String sql1 = "select * from \"employee\" where \"hire_date\" + " - + "INTERVAL '10' HOUR > TIMESTAMP '2005-10-17 00:00:00' "; + final String sql1 = "select *\n" + + "from \"foodmart\".\"employee\"\n" + + "where \"hire_date\" + INTERVAL '10' HOUR\n" + + " > TIMESTAMP '2005-10-17 00:00:00' "; final String expect1 = "SELECT *\n" + "FROM foodmart.employee\n" + "WHERE (hire_date + INTERVAL 10 HOUR)" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql1).withBigQuery().ok(expect1); + sql(sql1).withBigQuery().ok(expect1).done(); - final String sql2 = "select * from \"employee\" where \"hire_date\" + " - + "INTERVAL '1 2:34:56.78' DAY TO SECOND > TIMESTAMP '2005-10-17 00:00:00' "; - sql(sql2).withBigQuery().throws_("Only INT64 is supported as the interval value for BigQuery."); + final String sql2 = "select *\n" + + "from \"foodmart\".\"employee\"\n" + + "where \"hire_date\" + INTERVAL '1 2:34:56.78' DAY TO SECOND\n" + + " > TIMESTAMP '2005-10-17 00:00:00' "; + sql(sql2).withBigQuery() + .throws_("Only INT64 is supported as the interval value for BigQuery.") + .done(); } @Test void testUnparseSqlIntervalQualifierFirebolt() { - final String sql0 = "select * from \"employee\" where \"hire_date\" - " + final String sql0 = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" - " + "INTERVAL '19800' SECOND(5) > TIMESTAMP '2005-10-17 00:00:00' "; final String expect0 = "SELECT *\n" + "FROM \"foodmart\".\"employee\"\n" + "WHERE (\"hire_date\" - INTERVAL '19800 SECOND ')" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql0).withFirebolt().ok(expect0); + sql(sql0).withFirebolt().ok(expect0).done(); - final String sql1 = "select * from \"employee\" where \"hire_date\" + " + final String sql1 = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" + " + "INTERVAL '10' HOUR > TIMESTAMP '2005-10-17 00:00:00' "; final String expect1 = "SELECT *\n" + "FROM \"foodmart\".\"employee\"\n" + "WHERE (\"hire_date\" + INTERVAL '10 HOUR ')" + " > TIMESTAMP '2005-10-17 00:00:00'"; - sql(sql1).withFirebolt().ok(expect1); + sql(sql1).withFirebolt().ok(expect1).done(); - final String sql2 = "select * from \"employee\" where \"hire_date\" + " + final String sql2 = "select *\n" + + "from \"foodmart\".\"employee\" where \"hire_date\" + " + "INTERVAL '1 2:34:56.78' DAY TO SECOND > TIMESTAMP '2005-10-17 00:00:00' "; - sql(sql2).withFirebolt().throws_("Only INT64 is supported as the interval value for Firebolt."); + sql(sql2).withFirebolt() + .throws_("Only INT64 is supported as the interval value for Firebolt.") + .done(); } @Test void testFloorMysqlWeek() { - String query = "SELECT floor(\"hire_date\" TO WEEK) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO WEEK)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT STR_TO_DATE(DATE_FORMAT(`hire_date` , '%x%v-1'), '%x%v-%w')\n" + "FROM `foodmart`.`employee`"; sql(query) - .withMysql().ok(expected); + .withMysql().ok(expected).done(); } @Test void testFloorMonth() { - final String query = "SELECT floor(\"hire_date\" TO MONTH) FROM \"employee\""; + final String query = "SELECT floor(\"hire_date\" TO MONTH)\n" + + "FROM \"foodmart\".\"employee\""; final String expectedClickHouse = "SELECT toStartOfMonth(`hire_date`)\n" + "FROM `foodmart`.`employee`"; final String expectedMssql = "SELECT CONVERT(DATETIME, CONVERT(VARCHAR(7), [hire_date] , " @@ -5406,31 +5665,34 @@ private void checkLiteral2(String expression, String expected) { sql(query) .withClickHouse().ok(expectedClickHouse) .withMssql().ok(expectedMssql) - .withMysql().ok(expectedMysql); + .withMysql().ok(expectedMysql).done(); } @Test void testFloorMysqlHour() { - String query = "SELECT floor(\"hire_date\" TO HOUR) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO HOUR)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT DATE_FORMAT(`hire_date`, '%Y-%m-%d %H:00:00')\n" + "FROM `foodmart`.`employee`"; sql(query) - .withMysql().ok(expected); + .withMysql().ok(expected).done(); } @Test void testFloorMysqlMinute() { - String query = "SELECT floor(\"hire_date\" TO MINUTE) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO MINUTE)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT DATE_FORMAT(`hire_date`, '%Y-%m-%d %H:%i:00')\n" + "FROM `foodmart`.`employee`"; sql(query) - .withMysql().ok(expected); + .withMysql().ok(expected).done(); } @Test void testFloorMysqlSecond() { - String query = "SELECT floor(\"hire_date\" TO SECOND) FROM \"employee\""; + String query = "SELECT floor(\"hire_date\" TO SECOND)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT DATE_FORMAT(`hire_date`, '%Y-%m-%d %H:%i:%s')\n" + "FROM `foodmart`.`employee`"; sql(query) - .withMysql().ok(expected); + .withMysql().ok(expected).done(); } /** Test case for @@ -5438,7 +5700,7 @@ private void checkLiteral2(String expression, String expected) { * JDBC dialect-specific FLOOR fails when in GROUP BY. */ @Test void testFloorWithGroupBy() { final String query = "SELECT floor(\"hire_date\" TO MINUTE)\n" - + "FROM \"employee\"\n" + + "FROM \"foodmart\".\"employee\"\n" + "GROUP BY floor(\"hire_date\" TO MINUTE)"; final String expected = "SELECT TRUNC(hire_date, 'MI')\n" + "FROM foodmart.employee\n" @@ -5463,12 +5725,12 @@ private void checkLiteral2(String expression, String expected) { .withHsqldb().ok(expected) .withMysql().ok(expectedMysql) .withOracle().ok(expectedOracle) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql).done(); } @Test void testSubstring() { final String query = "select substring(\"brand_name\" from 2) " - + "from \"product\"\n"; + + "from \"foodmart\".\"product\"\n"; final String expectedBigQuery = "SELECT SUBSTRING(brand_name, 2)\n" + "FROM foodmart.product"; final String expectedClickHouse = "SELECT SUBSTRING(`brand_name`, 2)\n" @@ -5499,12 +5761,13 @@ private void checkLiteral2(String expression, String expected) { .withPresto().ok(expectedPresto) .withRedshift().ok(expectedRedshift) .withSnowflake().ok(expectedSnowflake) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testSubstringWithFor() { final String query = "select substring(\"brand_name\" from 2 for 3) " - + "from \"product\"\n"; + + "from \"foodmart\".\"product\"\n"; final String expectedBigQuery = "SELECT SUBSTRING(brand_name, 2, 3)\n" + "FROM foodmart.product"; final String expectedClickHouse = "SELECT SUBSTRING(`brand_name`, 2, 3)\n" @@ -5535,71 +5798,77 @@ private void checkLiteral2(String expression, String expected) { .withPresto().ok(expectedPresto) .withRedshift().ok(expectedRedshift) .withSnowflake().ok(expectedSnowflake) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for * [CALCITE-1849] * Support sub-queries (RexSubQuery) in RelToSqlConverter. */ @Test void testExistsWithExpand() { - String query = "select \"product_name\" from \"product\" a " - + "where exists (select count(*) " - + "from \"sales_fact_1997\"b " - + "where b.\"product_id\" = a.\"product_id\")"; + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a\n" + + "where exists (select count(*)\n" + + " from \"foodmart\".\"sales_fact_1997\" b\n" + + " where b.\"product_id\" = a.\"product_id\")"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE EXISTS (SELECT COUNT(*)\n" + "FROM \"foodmart\".\"sales_fact_1997\"\n" + "WHERE \"product_id\" = \"product\".\"product_id\")"; - sql(query).withConfig(c -> c.withExpand(false)).ok(expected); + sql(query).withConfig(c -> c.withExpand(false)).ok(expected).done(); } @Test void testNotExistsWithExpand() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a " + "where not exists (select count(*) " - + "from \"sales_fact_1997\"b " + + "from \"foodmart\".\"sales_fact_1997\"b " + "where b.\"product_id\" = a.\"product_id\")"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE NOT EXISTS (SELECT COUNT(*)\n" + "FROM \"foodmart\".\"sales_fact_1997\"\n" + "WHERE \"product_id\" = \"product\".\"product_id\")"; - sql(query).withConfig(c -> c.withExpand(false)).ok(expected); + sql(query).withConfig(c -> c.withExpand(false)).ok(expected).done(); } @Test void testSubQueryInWithExpand() { - String query = "select \"product_name\" from \"product\" a " - + "where \"product_id\" in (select \"product_id\" " - + "from \"sales_fact_1997\"b " - + "where b.\"product_id\" = a.\"product_id\")"; + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a\n" + + "where \"product_id\" in (select \"product_id\"\n" + + " from \"foodmart\".\"sales_fact_1997\" b\n" + + " where b.\"product_id\" = a.\"product_id\")"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" IN (SELECT \"product_id\"\n" + "FROM \"foodmart\".\"sales_fact_1997\"\n" + "WHERE \"product_id\" = \"product\".\"product_id\")"; - sql(query).withConfig(c -> c.withExpand(false)).ok(expected); + sql(query).withConfig(c -> c.withExpand(false)).ok(expected).done(); } @Test void testSubQueryInWithExpand2() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a " + "where \"product_id\" in (1, 2)"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" = 1 OR \"product_id\" = 2"; - sql(query).withConfig(c -> c.withExpand(false)).ok(expected); + sql(query).withConfig(c -> c.withExpand(false)).ok(expected).done(); } @Test void testSubQueryNotInWithExpand() { - String query = "select \"product_name\" from \"product\" a " - + "where \"product_id\" not in (select \"product_id\" " - + "from \"sales_fact_1997\"b " - + "where b.\"product_id\" = a.\"product_id\")"; + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a\n" + + "where \"product_id\" not in (select \"product_id\"\n" + + " from \"foodmart\".\"sales_fact_1997\"b\n" + + " where b.\"product_id\" = a.\"product_id\")"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" NOT IN (SELECT \"product_id\"\n" + "FROM \"foodmart\".\"sales_fact_1997\"\n" + "WHERE \"product_id\" = \"product\".\"product_id\")"; - sql(query).withConfig(c -> c.withExpand(false)).ok(expected); + sql(query).withConfig(c -> c.withExpand(false)).ok(expected).done(); } /** Test case for @@ -5634,38 +5903,43 @@ private void checkLiteral2(String expression, String expected) { .withConfig(c -> c.withExpand(true)) .withMysql().ok(expectedMysql) .withPostgresql().ok(expectedPostgresql) - .withHsqldb().ok(expectedHsqldb); + .withHsqldb().ok(expectedHsqldb) + .done(); } @Test void testLike() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a " + "where \"product_name\" like 'abc'"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_name\" LIKE 'abc'"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testNotLike() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a " + "where \"product_name\" not like 'abc'"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_name\" NOT LIKE 'abc'"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testIlike() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a " + "where \"product_name\" ilike 'abC'"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_name\" ILIKE 'abC'"; - sql(query).withLibrary(SqlLibrary.POSTGRESQL).ok(expected); + sql(query).withLibrary(SqlLibrary.POSTGRESQL).ok(expected).done(); } @Test void testRlike() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a " + "where \"product_name\" rlike '.+@.+\\\\..+'"; String expectedSpark = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" @@ -5679,11 +5953,13 @@ private void checkLiteral2(String expression, String expected) { sql(query) .withLibrary(SqlLibrary.SPARK).ok(expectedSpark) .withLibrary(SqlLibrary.HIVE).ok(expectedHive) - .withLibrary(SqlLibrary.MYSQL).ok(expectedMysql); + .withLibrary(SqlLibrary.MYSQL).ok(expectedMysql) + .done(); } @Test void testNotRlike() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a\n" + "where \"product_name\" not rlike '.+@.+\\\\..+'"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" @@ -5697,21 +5973,23 @@ private void checkLiteral2(String expression, String expected) { sql(query) .withLibrary(SqlLibrary.SPARK).ok(expected) .withLibrary(SqlLibrary.HIVE).ok(expectedHive) - .withLibrary(SqlLibrary.MYSQL).ok(expectedMysql); + .withLibrary(SqlLibrary.MYSQL).ok(expectedMysql) + .done(); } @Test void testNotIlike() { - String query = "select \"product_name\" from \"product\" a " + String query = "select \"product_name\"\n" + + "from \"foodmart\".\"product\" a\n" + "where \"product_name\" not ilike 'abC'"; String expected = "SELECT \"product_name\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_name\" NOT ILIKE 'abC'"; - sql(query).withLibrary(SqlLibrary.POSTGRESQL).ok(expected); + sql(query).withLibrary(SqlLibrary.POSTGRESQL).ok(expected).done(); } @Test void testMatchRecognizePatternExpression() { String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " partition by \"product_class_id\", \"brand_name\"\n" + " order by \"product_class_id\" asc, \"brand_name\" desc\n" @@ -5733,7 +6011,9 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } /** Test case for @@ -5744,8 +6024,7 @@ private void checkLiteral2(String expression, String expected) { final String sql = "SELECT MOD(CAST(2 AS DECIMAL(39, 20)), 2)"; final String expected = "SELECT MOD(2.00000000000000000000, 2)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql).withPostgresqlModifiedDecimalTypeSystem() - .ok(expected); + sql(sql).withPostgresqlModifiedDecimalTypeSystem().ok(expected).done(); } /** Test case for @@ -5760,13 +6039,12 @@ private void checkLiteral2(String expression, String expected) { + "FROM (VALUES ('4.2')) AS " + "\"t\" (\"EXPR$0\"),\n" + "(VALUES ('4.2')) AS \"t0\" (\"EXPR$0\")"; - sql(sql).withPostgresqlModifiedDecimalTypeSystem() - .ok(expected); + sql(sql).withPostgresqlModifiedDecimalTypeSystem().ok(expected).done(); } @Test void testMatchRecognizePatternExpression2() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down+ up+$)\n" + " define\n" @@ -5784,12 +6062,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression3() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (^strt down+ up+)\n" + " define\n" @@ -5807,12 +6087,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression4() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (^strt down+ up+$)\n" + " define\n" @@ -5830,12 +6112,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression5() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down* up?)\n" + " define\n" @@ -5853,12 +6137,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression6() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt {-down-} up?)\n" + " define\n" @@ -5876,12 +6162,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression7() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down{2} up{3,})\n" + " define\n" @@ -5899,12 +6187,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression8() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down{,2} up{3,5})\n" + " define\n" @@ -5922,12 +6212,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression9() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt {-down+-} {-up*-})\n" + " define\n" @@ -5945,12 +6237,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression10() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (A B C | A C B | B A C | B C A | C A B | C B A)\n" + " define\n" @@ -5970,12 +6264,14 @@ private void checkLiteral2(String expression, String expected) { + "\"A\" AS PREV(\"A\".\"net_weight\", 0) < PREV(\"A\".\"net_weight\", 1), " + "\"B\" AS PREV(\"B\".\"net_weight\", 0) > PREV(\"B\".\"net_weight\", 1), " + "\"C\" AS PREV(\"C\".\"net_weight\", 0) < PREV(\"C\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression11() { final String sql = "select *\n" - + " from (select * from \"product\") match_recognize\n" + + " from (select * from \"foodmart\".\"product\") match_recognize\n" + " (\n" + " pattern (strt down+ up+)\n" + " define\n" @@ -5993,18 +6289,21 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression12() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down+ up+)\n" + " define\n" + " down as down.\"net_weight\" < PREV(down.\"net_weight\"),\n" + " up as up.\"net_weight\" > prev(up.\"net_weight\")\n" - + " ) mr order by MR.\"net_weight\""; + + " ) mr\n" + + "order by MR.\"net_weight\""; final String expected = "SELECT *\n" + "FROM (SELECT *\n" + "FROM \"foodmart\".\"product\") MATCH_RECOGNIZE(\n" @@ -6017,19 +6316,21 @@ private void checkLiteral2(String expression, String expected) { + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))\n" + "ORDER BY \"net_weight\""; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternExpression13() { final String sql = "select *\n" + " from (\n" + "select *\n" - + "from \"sales_fact_1997\" as s\n" - + "join \"customer\" as c\n" + + "from \"foodmart\".\"sales_fact_1997\" as s\n" + + "join \"foodmart\".\"customer\" as c\n" + " on s.\"customer_id\" = c.\"customer_id\"\n" - + "join \"product\" as p\n" + + "join \"foodmart\".\"product\" as p\n" + " on s.\"product_id\" = p.\"product_id\"\n" - + "join \"product_class\" as pc\n" + + "join \"foodmart\".\"product_class\" as pc\n" + " on p.\"product_class_id\" = pc.\"product_class_id\"\n" + "where c.\"city\" = 'San Francisco'\n" + "and pc.\"product_department\" = 'Snacks'" @@ -6039,7 +6340,8 @@ private void checkLiteral2(String expression, String expected) { + " define\n" + " down as down.\"net_weight\" < PREV(down.\"net_weight\"),\n" + " up as up.\"net_weight\" > prev(up.\"net_weight\")\n" - + " ) mr order by MR.\"net_weight\""; + + " ) mr\n" + + "order by MR.\"net_weight\""; final String expected = "SELECT *\n" + "FROM (SELECT " + "\"sales_fact_1997\".\"product_id\" AS \"product_id\", " @@ -6118,12 +6420,14 @@ private void checkLiteral2(String expression, String expected) { + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))\n" + "ORDER BY \"net_weight\""; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeDefineClause() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down+ up+)\n" + " define\n" @@ -6141,12 +6445,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeDefineClause2() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down+ up+)\n" + " define\n" @@ -6164,12 +6470,14 @@ private void checkLiteral2(String expression, String expected) { + "FIRST(\"DOWN\".\"net_weight\", 0), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "LAST(\"UP\".\"net_weight\", 0))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeDefineClause3() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down+ up+)\n" + " define\n" @@ -6187,12 +6495,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "LAST(\"UP\".\"net_weight\", 0) + LAST(\"UP\".\"gross_weight\", 0))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeDefineClause4() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " pattern (strt down+ up+)\n" + " define\n" @@ -6212,12 +6522,14 @@ private void checkLiteral2(String expression, String expected) { + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(LAST(\"UP\".\"net_weight\", 0) + " + "LAST(\"UP\".\"gross_weight\", 0), 3))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures1() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures MATCH_NUMBER() as match_num, " + " CLASSIFIER() as var_match, " @@ -6248,12 +6560,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures2() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " FINAL LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6280,12 +6594,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures3() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " RUNNING LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6312,12 +6628,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures4() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " FINAL COUNT(up.\"net_weight\") as up_cnt," @@ -6345,12 +6663,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures5() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures " + " FIRST(STRT.\"net_weight\") as start_nw," @@ -6379,12 +6699,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures6() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures " + " FIRST(STRT.\"net_weight\") as start_nw," @@ -6412,12 +6734,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeMeasures7() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures " + " FIRST(STRT.\"net_weight\") as start_nw," @@ -6427,7 +6751,8 @@ private void checkLiteral2(String expression, String expected) { + " define\n" + " down as down.\"net_weight\" < PREV(down.\"net_weight\"),\n" + " up as up.\"net_weight\" > prev(up.\"net_weight\")\n" - + " ) mr order by start_nw, up_cnt"; + + " ) mr\n" + + "order by start_nw, up_cnt"; final String expected = "SELECT *\n" + "FROM (SELECT *\n" @@ -6446,12 +6771,14 @@ private void checkLiteral2(String expression, String expected) { + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))\n" + "ORDER BY \"START_NW\", \"UP_CNT\""; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternSkip1() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " after match skip to next row\n" + " pattern (strt down+ up+)\n" @@ -6470,12 +6797,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternSkip2() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " after match skip past last row\n" + " pattern (strt down+ up+)\n" @@ -6494,12 +6823,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternSkip3() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " after match skip to FIRST down\n" + " pattern (strt down+ up+)\n" @@ -6517,12 +6848,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternSkip4() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " after match skip to last down\n" + " pattern (strt down+ up+)\n" @@ -6541,12 +6874,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizePatternSkip5() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " after match skip to down\n" + " pattern (strt down+ up+)\n" @@ -6565,12 +6900,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeSubset1() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " after match skip to down\n" + " pattern (strt down+ up+)\n" @@ -6591,12 +6928,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "NEXT(PREV(\"UP\".\"net_weight\", 0), 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeSubset2() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6626,12 +6965,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeSubset3() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6660,12 +7001,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeSubset4() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6694,12 +7037,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeRowsPerMatch1() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6729,12 +7074,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeRowsPerMatch2() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " measures STRT.\"net_weight\" as start_nw," + " LAST(DOWN.\"net_weight\") as bottom_nw," @@ -6764,12 +7111,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"net_weight\", 1), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeWithin() { final String sql = "select *\n" - + " from \"employee\" match_recognize\n" + + " from \"foodmart\".\"employee\" match_recognize\n" + " (\n" + " order by \"hire_date\"\n" + " ALL ROWS PER MATCH\n" @@ -6792,12 +7141,14 @@ private void checkLiteral2(String expression, String expected) { + "PREV(\"DOWN\".\"salary\", 1), " + "\"UP\" AS PREV(\"UP\".\"salary\", 0) > " + "PREV(\"UP\".\"salary\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } @Test void testMatchRecognizeIn() { final String sql = "select *\n" - + " from \"product\" match_recognize\n" + + " from \"foodmart\".\"product\" match_recognize\n" + " (\n" + " partition by \"product_class_id\", \"brand_name\"\n" + " order by \"product_class_id\" asc, \"brand_name\" desc\n" @@ -6820,7 +7171,9 @@ private void checkLiteral2(String expression, String expected) { + "CAST(0 AS DOUBLE) OR PREV(\"DOWN\".\"net_weight\", 0) = CAST(1 AS DOUBLE), " + "\"UP\" AS PREV(\"UP\".\"net_weight\", 0) > " + "PREV(\"UP\".\"net_weight\", 1))"; - sql(sql).ok(expected); + sql(sql) + .withPhase(PARSE) // only PARSE; Calcite cannot implement yet + .ok(expected).done(); } /** Test case for @@ -6874,7 +7227,7 @@ private void checkLiteral2(String expression, String expected) { .withOracle().ok(expectedOracle) .withPostgresql().ok(expectedPostgresql) .withRedshift().ok(expectedRedshift) - .withSnowflake().ok(expectedSnowflake); + .withSnowflake().ok(expectedSnowflake).done(); } /** Test case for @@ -6883,13 +7236,13 @@ private void checkLiteral2(String expression, String expected) { * when SqlDialect#supportsAliasedValues is false. */ @Test void testThreeValues() { final String sql = "select * from (values (1), (2), (3)) as t(\"a\")\n"; - sql(sql) - .withRedshift().ok("SELECT *\n" - + "FROM (SELECT 1 AS \"a\"\n" - + "UNION ALL\n" - + "SELECT 2 AS \"a\"\n" - + "UNION ALL\n" - + "SELECT 3 AS \"a\")"); + final String expected = "SELECT *\n" + + "FROM (SELECT 1 AS \"a\"\n" + + "UNION ALL\n" + + "SELECT 2 AS \"a\"\n" + + "UNION ALL\n" + + "SELECT 3 AS \"a\")"; + sql(sql).withRedshift().ok(expected).done(); } @Test void testValuesEmpty() { @@ -6913,7 +7266,7 @@ private void checkLiteral2(String expression, String expected) { .withClickHouse().ok(expectedClickHouse) .withMysql().ok(expectedMysql) .withOracle().ok(expectedOracle) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql).done(); } /** Tests SELECT without FROM clause; effectively the same as a VALUES @@ -6936,7 +7289,7 @@ private void checkLiteral2(String expression, String expected) { .withClickHouse().ok(expectedClickHouse) .withHive().ok(expectedHive) .withMysql().ok(expectedMysql) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql).done(); } @Test void testSelectOne() { @@ -6952,7 +7305,7 @@ private void checkLiteral2(String expression, String expected) { .withClickHouse().ok(expectedClickHouse) .withHive().ok(expectedHive) .withMysql().ok(expectedMysql) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql).done(); } /** As {@link #testValuesEmpty()} but with extra {@code SUBSTRING}. Before @@ -6969,7 +7322,7 @@ private void checkLiteral2(String expression, String expected) { final String expected = "SELECT SUBSTRING(`Y`, 1, 1)\n" + "FROM (SELECT NULL AS `X`, NULL AS `Y`) AS `t`\n" + "WHERE 1 = 0"; - sql(sql).optimize(rules, null).withMysql().ok(expected); + sql(sql).optimize(rules, null).withMysql().ok(expected).done(); } /** Test case for @@ -6977,19 +7330,17 @@ private void checkLiteral2(String expression, String expected) { * Re-aliasing of VALUES that has column aliases produces wrong SQL in the * JDBC adapter. */ @Test void testValuesReAlias() { - final RelBuilder builder = relBuilder(); - final RelNode root = builder - .values(new String[]{ "a", "b" }, 1, "x ", 2, "yy") - .values(new String[]{ "a", "b" }, 1, "x ", 2, "yy") - .join(JoinRelType.FULL) - .project(builder.field("a")) - .build(); + final Function relFn = b -> + b.values(new String[]{ "a", "b" }, 1, "x ", 2, "yy") + .values(new String[]{ "a", "b" }, 1, "x ", 2, "yy") + .join(JoinRelType.FULL) + .project(b.field("a")) + .build(); final String expectedSql = "SELECT \"t\".\"a\"\n" + "FROM (VALUES (1, 'x '),\n" + "(2, 'yy')) AS \"t\" (\"a\", \"b\")\n" + "FULL JOIN (VALUES (1, 'x '),\n" + "(2, 'yy')) AS \"t0\" (\"a\", \"b\") ON TRUE"; - assertThat(toSql(root), isLinux(expectedSql)); // Now with indentation. final String expectedSql2 = "SELECT \"t\".\"a\"\n" @@ -6997,36 +7348,39 @@ private void checkLiteral2(String expression, String expected) { + " (2, 'yy')) AS \"t\" (\"a\", \"b\")\n" + " FULL JOIN (VALUES (1, 'x '),\n" + " (2, 'yy')) AS \"t0\" (\"a\", \"b\") ON TRUE"; - assertThat( - toSql(root, DatabaseProduct.CALCITE.getDialect(), - c -> c.withIndentation(2)), - isLinux(expectedSql2)); + relFn(relFn) + .ok(expectedSql) + .withWriterConfig(c -> c.withIndentation(2)).ok(expectedSql2) + .done(); } @Test void testTableScanHints() { - final RelBuilder builder = relBuilder(); - builder.getCluster().setHintStrategies(HintStrategyTable.builder() - .hintStrategy("PLACEHOLDERS", HintPredicates.TABLE_SCAN) - .build()); - final RelNode root = builder - .scan("orders") - .hints(RelHint.builder("PLACEHOLDERS") - .hintOption("a", "b") - .build()) - .project(builder.field("PRODUCT")) - .build(); + final UnaryOperator placeholders = b -> { + final HintStrategyTable hintStrategyTable = + HintStrategyTable.builder() + .hintStrategy("PLACEHOLDERS", HintPredicates.TABLE_SCAN) + .build(); + b.getCluster().setHintStrategies(hintStrategyTable); + return b; + }; + final Function relFn = b -> + b.let(placeholders) + .scan("scott", "orders") // in the "SCOTT_WITH_TEMPORAL" schema + .hints(RelHint.builder("PLACEHOLDERS") + .hintOption("a", "b") + .build()) + .project(b.field("PRODUCT")) + .build(); final String expectedSql = "SELECT \"PRODUCT\"\n" + "FROM \"scott\".\"orders\""; - assertThat( - toSql(root, DatabaseProduct.CALCITE.getDialect()), - isLinux(expectedSql)); final String expectedSql2 = "SELECT PRODUCT\n" + "FROM scott.orders\n" + "/*+ PLACEHOLDERS(a = 'b') */"; - assertThat( - toSql(root, new AnsiSqlDialect(SqlDialect.EMPTY_CONTEXT)), - isLinux(expectedSql2)); + relFn(relFn) + .dialect(CALCITE).ok(expectedSql) + .dialect(ANSI).ok(expectedSql2) + .done(); } /** Test case for @@ -7035,33 +7389,33 @@ private void checkLiteral2(String expression, String expected) { @Test void testPreserveAlias() { final String sql = "select \"warehouse_class_id\" as \"id\",\n" + " \"description\"\n" - + "from \"warehouse_class\""; + + "from \"foodmart\".\"warehouse_class\""; final String expected = "" + "SELECT \"warehouse_class_id\" AS \"id\", \"description\"\n" + "FROM \"foodmart\".\"warehouse_class\""; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); final String sql2 = "select \"warehouse_class_id\", \"description\"\n" - + "from \"warehouse_class\""; + + "from \"foodmart\".\"warehouse_class\""; final String expected2 = "SELECT *\n" + "FROM \"foodmart\".\"warehouse_class\""; - sql(sql2).ok(expected2); + sql(sql2).ok(expected2).done(); } @Test void testPreservePermutation() { final String sql = "select \"description\", \"warehouse_class_id\"\n" - + "from \"warehouse_class\""; + + "from \"foodmart\".\"warehouse_class\""; final String expected = "SELECT \"description\", \"warehouse_class_id\"\n" + "FROM \"foodmart\".\"warehouse_class\""; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); } @Test void testFieldNamesWithAggregateSubQuery() { final String query = "select mytable.\"city\",\n" + " sum(mytable.\"store_sales\") as \"my-alias\"\n" + "from (select c.\"city\", s.\"store_sales\"\n" - + " from \"sales_fact_1997\" as s\n" - + " join \"customer\" as c using (\"customer_id\")\n" + + " from \"foodmart\".\"sales_fact_1997\" as s\n" + + " join \"foodmart\".\"customer\" as c using (\"customer_id\")\n" + " group by c.\"city\", s.\"store_sales\") AS mytable\n" + "group by mytable.\"city\""; @@ -7076,33 +7430,26 @@ private void checkLiteral2(String expression, String expected) { + "GROUP BY \"customer\".\"city\"," + " \"sales_fact_1997\".\"store_sales\") AS \"t0\"\n" + "GROUP BY \"t0\".\"city\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testUnparseSelectMustUseDialect() { - final String query = "select * from \"product\""; + final String query = "select * from \"foodmart\".\"product\""; final String expected = "SELECT *\n" + "FROM foodmart.product"; - final boolean[] callsUnparseCallOnSqlSelect = {false}; - final SqlDialect dialect = new SqlDialect(SqlDialect.EMPTY_CONTEXT) { - @Override public void unparseCall(SqlWriter writer, SqlCall call, - int leftPrec, int rightPrec) { - if (call instanceof SqlSelect) { - callsUnparseCallOnSqlSelect[0] = true; - } - super.unparseCall(writer, call, leftPrec, rightPrec); - } - }; - sql(query).dialect(dialect).ok(expected); + final int originalCount = + MockSqlDialect.THREAD_UNPARSE_SELECT_COUNT.get().get(); + sql(query).dialect(MOCK).ok(expected).done(); assertThat("Dialect must be able to customize unparseCall() for SqlSelect", - callsUnparseCallOnSqlSelect[0], is(true)); + MockSqlDialect.THREAD_UNPARSE_SELECT_COUNT.get().get(), + is(originalCount + 1)); } @Test void testCorrelate() { final String sql = "select d.\"department_id\", d_plusOne " - + "from \"department\" as d, " + + "from \"foodmart\".\"department\" as d, " + " lateral (select d.\"department_id\" + 1 as d_plusOne" + " from (values(true)))"; @@ -7111,7 +7458,7 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"department\") AS \"$cor0\",\n" + "LATERAL (SELECT \"$cor0\".\"$f2\" AS \"D_PLUSONE\"\n" + "FROM (VALUES (TRUE)) AS \"t\" (\"EXPR$0\")) AS \"t1\""; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); } /** Test case for @@ -7119,35 +7466,36 @@ private void checkLiteral2(String expression, String expected) { * NullPointerException when convert relational algebra that correlates * TableFunctionScan. */ @Test void testLateralCorrelate() { - final String query = "select * from \"product\",\n" + final String query = "select *\n" + + "from \"foodmart\".\"product\",\n" + "lateral table(RAMP(\"product\".\"product_id\"))"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\" AS \"$cor0\",\n" + "LATERAL (SELECT *\n" + "FROM TABLE(RAMP(\"$cor0\".\"product_id\"))) AS \"t\""; - sql(query).ok(expected); + sql(query).withPhase(PARSE).ok(expected).done(); } @Test void testUncollectExplicitAlias() { final String sql = "select did + 1\n" - + "from unnest(select collect(\"department_id\") as deptid" - + " from \"department\") as t(did)"; + + "from unnest(select collect(\"department_id\") as deptid\n" + + " from \"foodmart\".\"department\") as t(did)"; final String expected = "SELECT \"DEPTID\" + 1\n" + "FROM UNNEST((SELECT COLLECT(\"department_id\") AS \"DEPTID\"\n" + "FROM \"foodmart\".\"department\")) AS \"t0\" (\"DEPTID\")"; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); } @Test void testUncollectImplicitAlias() { final String sql = "select did + 1\n" - + "from unnest(select collect(\"department_id\") " - + " from \"department\") as t(did)"; + + "from unnest(select collect(\"department_id\")\n" + + " from \"foodmart\".\"department\") as t(did)"; final String expected = "SELECT \"col_0\" + 1\n" + "FROM UNNEST((SELECT COLLECT(\"department_id\")\n" + "FROM \"foodmart\".\"department\")) AS \"t0\" (\"col_0\")"; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); } /** Test case for @@ -7156,12 +7504,24 @@ private void checkLiteral2(String expression, String expected) { * {@code "UNNEST ... WITH ORDINALITY}. */ @Test void testUncollectExplicitAliasWithOrd() { final String sql = "select did + 1\n" - + "from unnest(select collect(\"department_id\") as deptid \n" + + "from unnest(select collect(\"department_id\") as deptid\n" + "from \"department\") with ordinality as t(did, pos)"; final String expected = "SELECT \"DEPTID\" + 1\n" + "FROM UNNEST((SELECT COLLECT(\"department_id\") AS \"DEPTID\"\n" - + "FROM \"foodmart\".\"department\")) WITH ORDINALITY AS \"t0\" (\"DEPTID\", \"ORDINALITY\")"; - sql(sql).ok(expected); + + "FROM \"foodmart\".\"department\")) WITH ORDINALITY" + + " AS \"t0\" (\"DEPTID\", \"ORDINALITY\")"; + sql(sql).ok(expected).done(); + } + + @Test void testUncollectImplicitAliasWithOrd() { + final String sql = "select did + 1\n" + + "from unnest(select collect(\"department_id\")\n" + + "from \"department\") with ordinality as t(did, pos)"; + final String expected = "SELECT \"col_0\" + 1\n" + + "FROM UNNEST((SELECT COLLECT(\"department_id\")\n" + + "FROM \"foodmart\".\"department\")) WITH ORDINALITY" + + " AS \"t0\" (\"col_0\", \"ORDINALITY\")"; + sql(sql).ok(expected).done(); } @Test void testUnnestArray() { @@ -7175,127 +7535,146 @@ private void checkLiteral2(String expression, String expected) { final String expectedHsqldb = "SELECT *\n" + "FROM UNNEST((SELECT ARRAY[1, 2, 3]\n" + "FROM (VALUES (0)) AS t (ZERO))) AS t0 (col_0)"; - sql(sql).ok(expected). - withPostgresql().ok(expectedPostgresql). - withHsqldb().ok(expectedHsqldb); + sql(sql) + .ok(expected) + .withPostgresql().ok(expectedPostgresql) + .withHsqldb().ok(expectedHsqldb) + .done(); } @Test void testWithinGroup1() { - final String query = "select \"product_class_id\", collect(\"net_weight\") " - + "within group (order by \"net_weight\" desc) " - + "from \"product\" group by \"product_class_id\""; + final String query = "select \"product_class_id\",\n" + + " collect(\"net_weight\")\n" + + " within group (order by \"net_weight\" desc)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\", COLLECT(\"net_weight\") " + "WITHIN GROUP (ORDER BY \"net_weight\" DESC)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testWithinGroup2() { - final String query = "select \"product_class_id\", collect(\"net_weight\") " - + "within group (order by \"low_fat\", \"net_weight\" desc nulls last) " - + "from \"product\" group by \"product_class_id\""; + final String query = "select \"product_class_id\",\n" + + " collect(\"net_weight\") within group (order by\n" + + " \"low_fat\", \"net_weight\" desc nulls last)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\", COLLECT(\"net_weight\") " + "WITHIN GROUP (ORDER BY \"low_fat\", \"net_weight\" DESC NULLS LAST)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testWithinGroup3() { - final String query = "select \"product_class_id\", collect(\"net_weight\") " - + "within group (order by \"net_weight\" desc), " - + "min(\"low_fat\")" - + "from \"product\" group by \"product_class_id\""; + final String query = "select \"product_class_id\",\n" + + " collect(\"net_weight\") within group (order by \"net_weight\" desc),\n" + + " min(\"low_fat\")\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\", COLLECT(\"net_weight\") " + "WITHIN GROUP (ORDER BY \"net_weight\" DESC), MIN(\"low_fat\")\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testWithinGroup4() { - final String query = "select \"product_class_id\", collect(\"net_weight\") " - + "within group (order by \"net_weight\" desc) filter (where \"net_weight\" > 0)" - + "from \"product\" group by \"product_class_id\""; + final String query = "select \"product_class_id\",\n" + + " collect(\"net_weight\")\n" + + " within group (order by \"net_weight\" desc)\n" + + " filter (where \"net_weight\" > 0)\n" + + "from \"foodmart\".\"product\"\n" + + "group by \"product_class_id\""; final String expected = "SELECT \"product_class_id\", COLLECT(\"net_weight\") " + "FILTER (WHERE \"net_weight\" > 0E0 IS TRUE) " + "WITHIN GROUP (ORDER BY \"net_weight\" DESC)\n" + "FROM \"foodmart\".\"product\"\n" + "GROUP BY \"product_class_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonValueExpressionOperator() { String query = "select \"product_name\" format json, " + "\"product_name\" format json encoding utf8, " + "\"product_name\" format json encoding utf16, " - + "\"product_name\" format json encoding utf32 from \"product\""; + + "\"product_name\" format json encoding utf32\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT \"product_name\" FORMAT JSON, " + "\"product_name\" FORMAT JSON, " + "\"product_name\" FORMAT JSON, " + "\"product_name\" FORMAT JSON\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonExists() { - String query = "select json_exists(\"product_name\", 'lax $') from \"product\""; + String query = "select json_exists(\"product_name\", 'lax $')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_EXISTS(\"product_name\", 'lax $')\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonPretty() { - String query = "select json_pretty(\"product_name\") from \"product\""; + String query = "select json_pretty(\"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_PRETTY(\"product_name\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonValue() { - String query = "select json_value(\"product_name\", 'lax $') from \"product\""; + String query = "select json_value(\"product_name\", 'lax $')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_VALUE(\"product_name\", 'lax $')\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonQuery() { - String query = "select json_query(\"product_name\", 'lax $') from \"product\""; + String query = "select json_query(\"product_name\", 'lax $')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_QUERY(\"product_name\", 'lax $' " + "WITHOUT ARRAY WRAPPER NULL ON EMPTY NULL ON ERROR)\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonArray() { - String query = "select json_array(\"product_name\", \"product_name\") from \"product\""; + String query = "select json_array(\"product_name\", \"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_ARRAY(\"product_name\", \"product_name\" ABSENT ON NULL)\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonArrayAgg() { - String query = "select json_arrayagg(\"product_name\") from \"product\""; + String query = "select json_arrayagg(\"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_ARRAYAGG(\"product_name\" ABSENT ON NULL)\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonObject() { - String query = "select json_object(\"product_name\": \"product_id\") from \"product\""; + String query = "select json_object(\"product_name\": \"product_id\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT " + "JSON_OBJECT(KEY \"product_name\" VALUE \"product_id\" NULL ON NULL)\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonObjectAgg() { - String query = "select json_objectagg(\"product_name\": \"product_id\") from \"product\""; + String query = "select json_objectagg(\"product_name\": \"product_id\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT " + "JSON_OBJECTAGG(KEY \"product_name\" VALUE \"product_id\" NULL ON NULL)\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonPredicate() { @@ -7310,7 +7689,7 @@ private void checkLiteral2(String expression, String expected) { + "\"product_name\" is not json object, " + "\"product_name\" is not json array, " + "\"product_name\" is not json scalar " - + "from \"product\""; + + "from \"foodmart\".\"product\""; final String expected = "SELECT " + "\"product_name\" IS JSON VALUE, " + "\"product_name\" IS JSON VALUE, " @@ -7323,7 +7702,7 @@ private void checkLiteral2(String expression, String expected) { + "\"product_name\" IS NOT JSON ARRAY, " + "\"product_name\" IS NOT JSON SCALAR\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -7375,7 +7754,7 @@ private void checkLiteral2(String expression, String expected) { + "LEFT JOIN \"tpch\".\"part\" ON \"t\".\"nation_name\" = \"part\".\"p_brand\""; relFn(relFn) .schema(CalciteAssert.SchemaSpec.TPCH) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql).done(); } /** A cartesian product is unparsed as a CROSS JOIN on Spark, @@ -7393,10 +7772,16 @@ private void checkLiteral2(String expression, String expected) { Consumer fn = sql -> sql(sql) .withSpark().ok(expectedSpark) - .withMysql().ok(expectedMysql); - fn.accept("select * from \"employee\", \"department\""); - fn.accept("select * from \"employee\" cross join \"department\""); - fn.accept("select * from \"employee\" join \"department\" on true"); + .withMysql().ok(expectedMysql).done(); + fn.accept("select *\n" + + "from \"foodmart\".\"employee\",\n" + + " \"foodmart\".\"department\""); + fn.accept("select *\n" + + "from \"foodmart\".\"employee\"\n" + + "cross join \"foodmart\".\"department\""); + fn.accept("select *\n" + + "from \"foodmart\".\"employee\"\n" + + "join \"foodmart\".\"department\" on true"); } /** Similar to {@link #testCommaCrossJoin()} (but uses SQL) @@ -7405,9 +7790,9 @@ private void checkLiteral2(String expression, String expected) { * {@code INNER JOIN ... ON TRUE}, and if we're not on Spark. */ @Test void testCommaCrossJoin3way() { String sql = "select *\n" - + "from \"store\" as s\n" - + "inner join \"employee\" as e on true\n" - + "cross join \"department\" as d"; + + "from \"foodmart\".\"store\" as s\n" + + "inner join \"foodmart\".\"employee\" as e on true\n" + + "cross join \"foodmart\".\"department\" as d"; final String expectedMysql = "SELECT *\n" + "FROM `foodmart`.`store`,\n" + "`foodmart`.`employee`,\n" @@ -7423,50 +7808,52 @@ private void checkLiteral2(String expression, String expected) { sql(sql) .withMysql().ok(expectedMysql) .withSpark().ok(expectedSpark) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } /** As {@link #testCommaCrossJoin3way()}, but shows that if there is a * {@code LEFT JOIN} in the FROM clause, we can't use comma-join. */ @Test void testLeftJoinPreventsCommaJoin() { String sql = "select *\n" - + "from \"store\" as s\n" + + "from \"foodmart\".\"store\" as s\n" + "left join \"employee\" as e on true\n" + "cross join \"department\" as d"; final String expectedMysql = "SELECT *\n" + "FROM `foodmart`.`store`\n" + "LEFT JOIN `foodmart`.`employee` ON TRUE\n" + "CROSS JOIN `foodmart`.`department`"; - sql(sql).withMysql().ok(expectedMysql); + sql(sql).withMysql().ok(expectedMysql).done(); } /** As {@link #testLeftJoinPreventsCommaJoin()}, but the non-cross-join * occurs later in the FROM clause. */ @Test void testRightJoinPreventsCommaJoin() { String sql = "select *\n" - + "from \"store\" as s\n" + + "from \"foodmart\".\"store\" as s\n" + "cross join \"employee\" as e\n" + "right join \"department\" as d on true"; final String expectedMysql = "SELECT *\n" + "FROM `foodmart`.`store`\n" + "CROSS JOIN `foodmart`.`employee`\n" + "RIGHT JOIN `foodmart`.`department` ON TRUE"; - sql(sql).withMysql().ok(expectedMysql); + sql(sql).withMysql().ok(expectedMysql).done(); } /** As {@link #testLeftJoinPreventsCommaJoin()}, but the impediment is a * {@code JOIN} whose condition is not {@code TRUE}. */ @Test void testOnConditionPreventsCommaJoin() { String sql = "select *\n" - + "from \"store\" as s\n" - + "join \"employee\" as e on s.\"store_id\" = e.\"store_id\"\n" - + "cross join \"department\" as d"; + + "from \"foodmart\".\"store\" as s\n" + + "join \"foodmart\".\"employee\" as e\n" + + " on s.\"store_id\" = e.\"store_id\"\n" + + "cross join \"foodmart\".\"department\" as d"; final String expectedMysql = "SELECT *\n" + "FROM `foodmart`.`store`\n" + "INNER JOIN `foodmart`.`employee`" + " ON `store`.`store_id` = `employee`.`store_id`\n" + "CROSS JOIN `foodmart`.`department`"; - sql(sql).withMysql().ok(expectedMysql); + sql(sql).withMysql().ok(expectedMysql).done(); } /** Test case for @@ -7477,7 +7864,9 @@ private void checkLiteral2(String expression, String expected) { + "from \"product\""; final String expectedSnowflake = "SELECT BITAND_AGG(\"product_id\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withSnowflake().ok(expectedSnowflake); + sql(query).withLibrary(SqlLibrary.SNOWFLAKE) + .withSnowflake().ok(expectedSnowflake) + .done(); } /** Test case for @@ -7488,7 +7877,9 @@ private void checkLiteral2(String expression, String expected) { + "from \"product\""; final String expectedSnowflake = "SELECT BITOR_AGG(\"product_id\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withSnowflake().ok(expectedSnowflake); + sql(query).withLibrary(SqlLibrary.SNOWFLAKE) + .withSnowflake().ok(expectedSnowflake) + .done(); } /** Test case for @@ -7520,11 +7911,12 @@ private void checkLiteral2(String expression, String expected) { + "MIN(\"brand_name\")\n" + "FROM \"foodmart\".\"product\""; sql(query) - .ok(expected) - .withBigQuery().ok(expectedBigQuery) - .withPostgresql().ok(expectedPostgres) - .withSnowflake().ok(expectedSnowflake) - .withRedshift().ok(expectedPostgres); + .ok(expected) + .withBigQuery().ok(expectedBigQuery) + .withPostgresql().ok(expectedPostgres) + .withSnowflake().ok(expectedSnowflake) + .withRedshift().ok(expectedRedshift) + .done(); } /** Test case for @@ -7540,9 +7932,11 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"product\""; final String expectedSnowflake = "SELECT STARTSWITH(\"brand_name\", 'a')\n" + "FROM \"foodmart\".\"product\""; - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withBigQuery().ok(expectedBigQuery); - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withPostgresql().ok(expectedPostgres); - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withSnowflake().ok(expectedSnowflake); + sql(query).withLibrary(SqlLibrary.SNOWFLAKE) + .withBigQuery().ok(expectedBigQuery) + .withPostgresql().ok(expectedPostgres) + .withSnowflake().ok(expectedSnowflake) + .done(); } /** Test case for @@ -7558,9 +7952,11 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"product\""; final String expectedSnowflake = "SELECT ENDSWITH(\"brand_name\", 'a')\n" + "FROM \"foodmart\".\"product\""; - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withBigQuery().ok(expectedBigQuery); - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withPostgresql().ok(expectedPostgres); - sql(query).withLibrary(SqlLibrary.SNOWFLAKE).withSnowflake().ok(expectedSnowflake); + sql(query).withLibrary(SqlLibrary.SNOWFLAKE) + .withBigQuery().ok(expectedBigQuery) + .withPostgresql().ok(expectedPostgres) + .withSnowflake().ok(expectedSnowflake) + .done(); } /** Test case for @@ -7575,9 +7971,10 @@ private void checkLiteral2(String expression, String expected) { // since it seems to be used across more dialects. final String expectedSnowflake = "SELECT LENGTH(\"brand_name\")\n" + "FROM \"foodmart\".\"product\""; - Sql sql = sql(query).withLibrary(SqlLibrary.BIG_QUERY); - sql.withBigQuery().ok(expectedBigQuery); - sql.withSnowflake().ok(expectedSnowflake); + sql(query) + .withLibrary(SqlLibrary.BIG_QUERY).withBigQuery().ok(expectedBigQuery) + .withSnowflake().ok(expectedSnowflake) + .done(); } /** Test case for @@ -7589,51 +7986,52 @@ private void checkLiteral2(String expression, String expected) { + "from \"product\""; final String expected = "SELECT LENGTH(\"brand_name\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).withPresto().ok(expected); + sql(query).withPresto().ok(expected).done(); } @Test void testSubstringInSpark() { - final String query = "select substring(\"brand_name\" from 2) " - + "from \"product\"\n"; + final String query = "select substring(\"brand_name\" from 2)\n" + + "from \"foodmart\".\"product\"\n"; final String expected = "SELECT SUBSTRING(`brand_name`, 2)\n" + "FROM `foodmart`.`product`"; - sql(query).withSpark().ok(expected); + sql(query).withSpark().ok(expected).done(); } @Test void testSubstringWithForInSpark() { - final String query = "select substring(\"brand_name\" from 2 for 3) " - + "from \"product\"\n"; + final String query = "select substring(\"brand_name\" from 2 for 3)\n" + + "from \"foodmart\".\"product\"\n"; final String expected = "SELECT SUBSTRING(`brand_name`, 2, 3)\n" + "FROM `foodmart`.`product`"; - sql(query).withSpark().ok(expected); + sql(query).withSpark().ok(expected).done(); } @Test void testFloorInSpark() { - final String query = "select floor(\"hire_date\" TO MINUTE) " - + "from \"employee\""; + final String query = "select floor(\"hire_date\" TO MINUTE)\n" + + "from \"foodmart\".\"employee\""; final String expected = "SELECT DATE_TRUNC('MINUTE', `hire_date`)\n" + "FROM `foodmart`.`employee`"; - sql(query).withSpark().ok(expected); + sql(query).withSpark().ok(expected).done(); } @Test void testNumericFloorInSpark() { - final String query = "select floor(\"salary\") " - + "from \"employee\""; + final String query = "select floor(\"salary\")\n" + + "from \"foodmart\".\"employee\""; final String expected = "SELECT FLOOR(`salary`)\n" + "FROM `foodmart`.`employee`"; - sql(query).withSpark().ok(expected); + sql(query).withSpark().ok(expected).done(); } @Test void testJsonStorageSize() { - String query = "select json_storage_size(\"product_name\") from \"product\""; + String query = "select json_storage_size(\"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_STORAGE_SIZE(\"product_name\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testCubeWithGroupBy() { - final String query = "select count(*) " - + "from \"foodmart\".\"product\" " + final String query = "select count(*)\n" + + "from \"foodmart\".\"product\"\n" + "group by cube(\"product_id\",\"product_class_id\")"; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" @@ -7644,12 +8042,13 @@ private void checkLiteral2(String expression, String expected) { sql(query) .ok(expected) .withPresto().ok(expected) - .withSpark().ok(expectedSpark); + .withSpark().ok(expectedSpark) + .done(); } @Test void testRollupWithGroupBy() { - final String query = "select count(*) " - + "from \"foodmart\".\"product\" " + final String query = "select count(*)\n" + + "from \"foodmart\".\"product\"\n" + "group by rollup(\"product_id\",\"product_class_id\")"; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\"\n" @@ -7664,50 +8063,57 @@ private void checkLiteral2(String expression, String expected) { .ok(expected) .withPresto().ok(expected) .withSpark().ok(expectedSpark) - .withStarRocks().ok(expectedStarRocks); + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testJsonType() { - String query = "select json_type(\"product_name\") from \"product\""; + String query = "select json_type(\"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT " + "JSON_TYPE(\"product_name\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonDepth() { - String query = "select json_depth(\"product_name\") from \"product\""; + String query = "select json_depth(\"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT " + "JSON_DEPTH(\"product_name\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonLength() { - String query = "select json_length(\"product_name\", 'lax $'), " - + "json_length(\"product_name\") from \"product\""; + String query = "select json_length(\"product_name\", 'lax $'),\n" + + " json_length(\"product_name\")\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_LENGTH(\"product_name\", 'lax $'), " + "JSON_LENGTH(\"product_name\")\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonKeys() { - String query = "select json_keys(\"product_name\", 'lax $') from \"product\""; + String query = "select json_keys(\"product_name\", 'lax $')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_KEYS(\"product_name\", 'lax $')\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testJsonRemove() { - String query = "select json_remove(\"product_name\", '$[0]') from \"product\""; + String query = "select json_remove(\"product_name\", '$[0]')\n" + + "from \"foodmart\".\"product\""; final String expected = "SELECT JSON_REMOVE(\"product_name\", '$[0]')\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test public void testJsonInsert() { - String query0 = "select json_insert(\"product_name\", '$', 10) from \"product\""; + String query0 = "select json_insert(\"product_name\", '$', 10)\n" + + "from \"product\""; String query1 = "select " + "json_insert(cast(null as varchar), '$', 10,\n" + " '$', null, '$', '\n" @@ -7721,12 +8127,13 @@ private void checkLiteral2(String expression, String expected) { + "\t\n" + "')\n" + "FROM \"foodmart\".\"product\""; - sql(query0).ok(expected0); - sql(query1).ok(expected1); + sql(query0).ok(expected0).done(); + sql(query1).ok(expected1).done(); } @Test public void testJsonReplace() { - String query = "select json_replace(\"product_name\", '$', 10) from \"product\""; + String query = "select json_replace(\"product_name\", '$', 10)\n" + + "from \"product\""; String query1 = "select " + "json_replace(cast(null as varchar), '$', 10, '$', null, '$', '\n" + "\t\n" @@ -7738,12 +8145,13 @@ private void checkLiteral2(String expression, String expected) { + "\t\n" + "')\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); - sql(query1).ok(expected1); + sql(query).ok(expected).done(); + sql(query1).ok(expected1).done(); } @Test public void testJsonSet() { - String query = "select json_set(\"product_name\", '$', 10) from \"product\""; + String query = "select json_set(\"product_name\", '$', 10)\n" + + "from \"product\""; String query1 = "select " + "json_set(cast(null as varchar), '$', 10, '$', null, '$', '\n" + "\t\n" @@ -7755,14 +8163,17 @@ private void checkLiteral2(String expression, String expected) { + "\t\n" + "')\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); - sql(query1).ok(expected1); + sql(query).ok(expected).done(); + sql(query1).ok(expected1).done(); } @Test void testUnionAll() { - String query = "select A.\"department_id\" " - + "from \"foodmart\".\"employee\" A " - + " where A.\"department_id\" = ( select min( A.\"department_id\") from \"foodmart\".\"department\" B where 1=2 )"; + String query = "select A.\"department_id\"\n" + + "from \"foodmart\".\"employee\" A\n" + + "where A.\"department_id\" = (\n" + + " select min( A.\"department_id\")\n" + + " from \"foodmart\".\"department\" B\n" + + " where 1=2 )"; final String expectedOracle = "SELECT \"employee\".\"department_id\"\n" + "FROM \"foodmart\".\"employee\"\n" + "INNER JOIN (SELECT \"t1\".\"department_id\" \"department_id0\", MIN(\"t1\".\"department_id\") \"EXPR$0\"\n" @@ -7793,47 +8204,52 @@ private void checkLiteral2(String expression, String expected) { sql(query) .ok(expectedNoExpand) .withConfig(c -> c.withExpand(true)).ok(expected) - .withOracle().ok(expectedOracle); + .withOracle().ok(expectedOracle).done(); } @Test void testSmallintOracle() { - String query = "SELECT CAST(\"department_id\" AS SMALLINT) FROM \"employee\""; + String query = "SELECT CAST(\"department_id\" AS SMALLINT)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT CAST(\"department_id\" AS NUMBER(5))\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withOracle().ok(expected); + .withOracle().ok(expected).done(); } @Test void testBigintOracle() { - String query = "SELECT CAST(\"department_id\" AS BIGINT) FROM \"employee\""; + String query = "SELECT CAST(\"department_id\" AS BIGINT)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT CAST(\"department_id\" AS NUMBER(19))\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withOracle().ok(expected); + .withOracle().ok(expected).done(); } @Test void testDoubleOracle() { - String query = "SELECT CAST(\"department_id\" AS DOUBLE) FROM \"employee\""; + String query = "SELECT CAST(\"department_id\" AS DOUBLE)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT CAST(\"department_id\" AS DOUBLE PRECISION)\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withOracle().ok(expected); + .withOracle().ok(expected).done(); } @Test void testRedshiftCastToTinyint() { - String query = "SELECT CAST(\"department_id\" AS tinyint) FROM \"employee\""; + String query = "SELECT CAST(\"department_id\" AS tinyint)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT CAST(\"department_id\" AS \"int2\")\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withRedshift().ok(expected); + .withRedshift().ok(expected).done(); } @Test void testRedshiftCastToDouble() { - String query = "SELECT CAST(\"department_id\" AS double) FROM \"employee\""; + String query = "SELECT CAST(\"department_id\" AS double)\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT CAST(\"department_id\" AS \"float8\")\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withRedshift().ok(expected); + .withRedshift().ok(expected).done(); } @Test void testIndexOperatorsBigQuery() { @@ -7841,7 +8257,7 @@ private void checkLiteral2(String expression, String expected) { String query = "SELECT SPLIT('h,e,l,l,o')[" + operator + "(1)] FROM \"employee\""; String expected = "SELECT SPLIT('h,e,l,l,o')[" + operator + "(1)]\n" + "FROM foodmart.employee"; - sql(query).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expected); + sql(query).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).ok(expected).done(); }; consumer.accept("OFFSET"); consumer.accept("ORDINAL"); @@ -7852,32 +8268,37 @@ private void checkLiteral2(String expression, String expected) { @Test void testIndexWithoutOperatorBigQuery() { String query = "SELECT SPLIT('h,e,l,l,o')[1] FROM \"employee\""; String error = "BigQuery requires an array subscript operator to index an array"; - sql(query).withBigQuery().withLibrary(SqlLibrary.BIG_QUERY).throws_(error); + sql(query).withLibrary(SqlLibrary.BIG_QUERY) + .withBigQuery().throws_(error) + .done(); } @Test void testDateLiteralOracle() { - String query = "SELECT DATE '1978-05-02' FROM \"employee\""; + String query = "SELECT DATE '1978-05-02'\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT TO_DATE('1978-05-02', 'YYYY-MM-DD')\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withOracle().ok(expected); + .withOracle().ok(expected).done(); } @Test void testTimestampLiteralOracle() { - String query = "SELECT TIMESTAMP '1978-05-02 12:34:56.78' FROM \"employee\""; + String query = "SELECT TIMESTAMP '1978-05-02 12:34:56.78'\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT TO_TIMESTAMP('1978-05-02 12:34:56.78'," + " 'YYYY-MM-DD HH24:MI:SS.FF')\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withOracle().ok(expected); + .withOracle().ok(expected).done(); } @Test void testTimeLiteralOracle() { - String query = "SELECT TIME '12:34:56.78' FROM \"employee\""; + String query = "SELECT TIME '12:34:56.78'\n" + + "FROM \"foodmart\".\"employee\""; String expected = "SELECT TO_TIME('12:34:56.78', 'HH24:MI:SS.FF')\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withOracle().ok(expected); + .withOracle().ok(expected).done(); } @Test void testSupportsDataType() { @@ -7885,10 +8306,11 @@ private void checkLiteral2(String expression, String expected) { new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT); final RelDataType booleanDataType = typeFactory.createSqlType(SqlTypeName.BOOLEAN); final RelDataType integerDataType = typeFactory.createSqlType(SqlTypeName.INTEGER); - final SqlDialect oracleDialect = DatabaseProduct.ORACLE.getDialect(); + final DialectTestConfig testConfig = CONFIG_SUPPLIER.get(); + final SqlDialect oracleDialect = testConfig.get(ORACLE_12).sqlDialect; assertFalse(oracleDialect.supportsDataType(booleanDataType)); assertTrue(oracleDialect.supportsDataType(integerDataType)); - final SqlDialect postgresqlDialect = DatabaseProduct.POSTGRESQL.getDialect(); + final SqlDialect postgresqlDialect = testConfig.get(POSTGRESQL).sqlDialect; assertTrue(postgresqlDialect.supportsDataType(booleanDataType)); assertTrue(postgresqlDialect.supportsDataType(integerDataType)); } @@ -7898,35 +8320,35 @@ private void checkLiteral2(String expression, String expected) { * JDBC adapter throws UnsupportedOperationException when generating SQL * for untyped NULL literal. */ @Test void testSelectRawNull() { - final String query = "SELECT NULL FROM \"product\""; + final String query = "SELECT NULL FROM \"foodmart\".\"product\""; final String expected = "SELECT NULL\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectRawNullWithAlias() { - final String query = "SELECT NULL AS DUMMY FROM \"product\""; + final String query = "SELECT NULL AS DUMMY FROM \"foodmart\".\"product\""; final String expected = "SELECT NULL AS \"DUMMY\"\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectNullWithCast() { final String query = "SELECT CAST(NULL AS INT)"; final String expected = "SELECT *\n" + "FROM (VALUES (NULL)) AS \"t\" (\"EXPR$0\")"; - sql(query).ok(expected); + sql(query).ok(expected).done(); // validate - sql(expected).exec(); + sql(expected).done().exec(); } @Test void testSelectNullWithCount() { final String query = "SELECT COUNT(CAST(NULL AS INT))"; final String expected = "SELECT COUNT(\"$f0\")\n" + "FROM (VALUES (NULL)) AS \"t\" (\"$f0\")"; - sql(query).ok(expected); + sql(query).ok(expected).done(); // validate - sql(expected).exec(); + sql(expected).done().exec(); } @Test void testSelectNullWithGroupByNull() { @@ -7936,26 +8358,26 @@ private void checkLiteral2(String expression, String expected) { final String expected = "SELECT COUNT(\"$f1\")\n" + "FROM (VALUES (NULL, NULL)) AS \"t\" (\"$f0\", \"$f1\")\n" + "GROUP BY \"$f0\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); // validate - sql(expected).exec(); + sql(expected).done().exec(); } @Test void testSelectNullWithGroupByVar() { final String query = "SELECT COUNT(CAST(NULL AS INT))\n" - + "FROM \"account\" AS \"t\"\n" + + "FROM \"foodmart\".\"account\" AS \"t\"\n" + "GROUP BY \"account_type\""; final String expected = "SELECT COUNT(CAST(NULL AS INTEGER))\n" + "FROM \"foodmart\".\"account\"\n" + "GROUP BY \"account_type\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); // validate - sql(expected).exec(); + sql(expected).done().exec(); } @Test void testSelectNullWithInsert() { - final String query = "insert into\n" - + "\"account\"(\"account_id\",\"account_parent\",\"account_type\",\"account_rollup\")\n" + final String query = "insert into \"foodmart\".\"account\"\n" + + "(\"account_id\",\"account_parent\",\"account_type\",\"account_rollup\")\n" + "select 1, cast(NULL AS INT), cast(123 as varchar), cast(123 as varchar)"; final String expected = "INSERT INTO \"foodmart\".\"account\" (" + "\"account_id\", \"account_parent\", \"account_description\", " @@ -7970,21 +8392,21 @@ private void checkLiteral2(String expression, String expected) { + "AS \"Custom_Members\"\n" + "FROM (VALUES (1, NULL, '123', '123')) " + "AS \"t\" (\"EXPR$0\", \"EXPR$1\", \"EXPR$2\", \"EXPR$3\")"; - sql(query).ok(expected); + sql(query).ok(expected).done(); // validate - sql(expected).exec(); + sql(expected).done().exec(); } @Test void testSelectNullWithInsertFromJoin() { final String query = "insert into\n" - + "\"account\"(\"account_id\",\"account_parent\",\n" + + "\"foodmart\".\"account\"(\"account_id\",\"account_parent\",\n" + "\"account_type\",\"account_rollup\")\n" + "select \"product\".\"product_id\",\n" + "cast(NULL AS INT),\n" + "cast(\"product\".\"product_id\" as varchar),\n" + "cast(\"sales_fact_1997\".\"store_id\" as varchar)\n" - + "from \"product\"\n" - + "inner join \"sales_fact_1997\"\n" + + "from \"foodmart\".\"product\"\n" + + "inner join \"foodmart\".\"sales_fact_1997\"\n" + "on \"product\".\"product_id\" = \"sales_fact_1997\".\"product_id\""; final String expected = "INSERT INTO \"foodmart\".\"account\" " + "(\"account_id\", \"account_parent\", \"account_description\", " @@ -8000,24 +8422,26 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"product\"\n" + "INNER JOIN \"foodmart\".\"sales_fact_1997\" " + "ON \"product\".\"product_id\" = \"sales_fact_1997\".\"product_id\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); // validate - sql(expected).exec(); + sql(expected).done().exec(); } @Test void testCastDecimalOverflow() { - final String query = - "SELECT CAST('11111111111111111111111111111111.111111' AS DECIMAL(38,6)) AS \"num\" from \"product\""; + final String query = "SELECT\n" + + " CAST('11111111111111111111111111111111.111111' AS DECIMAL(38,6))\n" + + " AS \"num\"\n" + + "FROM \"foodmart\".\"product\""; final String expected = "SELECT CAST('11111111111111111111111111111111.111111' AS DECIMAL(19, 6)) AS \"num\"\n" + "FROM \"foodmart\".\"product\""; - sql(query).ok(expected); + sql(query).ok(expected).done(); - final String query2 = - "SELECT CAST(1111111 AS DECIMAL(5,2)) AS \"num\" from \"product\""; + final String query2 = "SELECT CAST(1111111 AS DECIMAL(5,2)) AS \"num\"\n" + + "FROM \"foodmart\".\"product\""; final String expected2 = "SELECT CAST(1111111 AS DECIMAL(5, 2)) AS \"num\"\n" + "FROM \"foodmart\".\"product\""; - sql(query2).ok(expected2); + sql(query2).ok(expected2).done(); } @Test void testCastInStringIntegerComparison() { @@ -8035,7 +8459,7 @@ private void checkLiteral2(String expression, String expected) { + "CAST('1996-01-01 ' || '00:00:00' AS TIMESTAMP)"; sql(query) .ok(expected) - .withBigQuery().ok(expectedBiqquery); + .withBigQuery().ok(expectedBiqquery).done(); } /** Test case for @@ -8061,22 +8485,25 @@ private void checkLiteral2(String expression, String expected) { + "CAST(\"full_name\" AS TIMESTAMP(0)) = \"hire_date\" OR " + "CAST('10' AS INTEGER) = 1"; sql(query) - .withPostgresql().ok(expectedPostgresql); + .withPostgresql().ok(expectedPostgresql) + .done(); } /** Test case for * [CALCITE-6149] * Unparse for CAST Nullable with ClickHouseSqlDialect. */ @Test void testCastToNullableInClickhouse() { - final String query = "" - + "SELECT CASE WHEN \"product_id\" IS NULL " - + "THEN CAST(\"product_id\" AS TINYINT) END, CAST(\"product_id\" AS TINYINT)\n" + final String query = "SELECT\n" + + " CASE WHEN \"product_id\" IS NULL\n" + + " THEN CAST(\"product_id\" AS TINYINT) END,\n" + + " CAST(\"product_id\" AS TINYINT)\n" + "FROM \"foodmart\".\"product\""; - final String expectedSql = "" - + "SELECT CAST(NULL AS `Nullable(Int8)`), CAST(`product_id` AS `Int8`)\n" + final String expected = "SELECT CAST(NULL AS `Nullable(Int8)`)," + + " CAST(`product_id` AS `Int8`)\n" + "FROM `foodmart`.`product`"; - - sql(query).withClickHouse().ok(expectedSql); + sql(query) + .withClickHouse().ok(expected) + .done(); } /** Test case for @@ -8097,19 +8524,21 @@ private void checkLiteral2(String expression, String expected) { + "\"hire_date\" = CAST('1996-01-01 ' || '00:00:00' AS TIMESTAMP)"; sql(query) .ok(expected) - .withPresto().ok(expectedPresto); + .withPresto().ok(expectedPresto) + .done(); } @Test void testDialectQuoteStringLiteral() { - dialects().forEach((dialect, databaseProduct) -> { + dialects().forEach(d -> { + final SqlDialect dialect = d.sqlDialect; assertThat(dialect.quoteStringLiteral(""), is("''")); assertThat(dialect.quoteStringLiteral("can't run"), - databaseProduct == DatabaseProduct.BIG_QUERY + d.code == BIG_QUERY ? is("'can\\'t run'") : is("'can''t run'")); assertThat(dialect.unquoteStringLiteral("''"), is("")); - if (databaseProduct == DatabaseProduct.BIG_QUERY) { + if (d.code == BIG_QUERY) { assertThat(dialect.unquoteStringLiteral("'can\\'t run'"), is("can't run")); } else { @@ -8120,15 +8549,15 @@ private void checkLiteral2(String expression, String expected) { } @Test void testSelectCountStar() { - final String query = "select count(*) from \"product\""; + final String query = "select count(*) from \"foodmart\".\"product\""; final String expected = "SELECT COUNT(*)\n" + "FROM \"foodmart\".\"product\""; - Sql sql = sql(query); - sql.ok(expected); + sql(query).ok(expected).done(); } @Test void testSelectApproxCountDistinct() { - final String query = "select approx_count_distinct(\"product_id\") from \"product\""; + final String query = "select approx_count_distinct(\"product_id\")\n" + + "from \"foodmart\".\"product\""; final String expectedExact = "SELECT COUNT(DISTINCT \"product_id\")\n" + "FROM \"foodmart\".\"product\""; final String expectedApprox = "SELECT APPROX_COUNT_DISTINCT(`product_id`)\n" @@ -8148,11 +8577,12 @@ private void checkLiteral2(String expression, String expected) { .withOracle().ok(expectedApproxQuota) .withSnowflake().ok(expectedApproxQuota) .withPresto().ok(expectedPrestoSql) - .withStarRocks().ok(expectedStarRocksSql); + .withStarRocks().ok(expectedStarRocksSql) + .done(); } @Test void testRowValueExpression() { - String sql = "insert into \"DEPT\"\n" + String sql = "insert into dept\n" + "values ROW(1,'Fred', 'San Francisco'),\n" + " ROW(2, 'Eric', 'Washington')"; final String expectedDefault = "INSERT INTO \"SCOTT\".\"DEPT\"" @@ -8216,6 +8646,8 @@ private void checkLiteral2(String expression, String expected) { + "SELECT 2, 'Eric', 'Washington'\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; sql(sql) + // only PARSE; execution hits 'There are not enough rules' + .withPhase(PARSE) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) .ok(expectedDefault) .withHive().ok(expectedHive) @@ -8231,7 +8663,7 @@ private void checkLiteral2(String expression, String expected) { .withMysql().ok(expectedMysqlX) .withOracle().ok(expectedOracleX) .withMssql().ok(expectedMssqlX) - .withCalcite().ok(expectedCalciteX); + .withCalcite().ok(expectedCalciteX).done(); } /** Test case for @@ -8244,20 +8676,22 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"employee\"\n" + "OFFSET 1 ROWS\n" + "FETCH NEXT 2 ROWS ONLY"; + final String message = + "Lower Oracle version(<12) doesn't support offset/fetch syntax!"; sql(query) .withOracle().ok(expected) .withOracle(19).ok(expected) - .withOracle(11).throws_("Lower Oracle version(<12) doesn't support offset/fetch syntax!"); + .withOracle(11).throws_(message) + .done(); } /** Test case for * [CALCITE-6482] * Oracle dialect convert boolean literal when version < 23. */ @Test void testBoolLiteralOracle() { - String query = "SELECT \"e1\".\"department_id\" " - + "FROM \"employee\" \"e1\"" - + "LEFT JOIN \"employee\" \"e2\"" - + "ON TRUE"; + String query = "SELECT \"e1\".\"department_id\"\n" + + "FROM \"foodmart\".\"employee\" \"e1\"\n" + + "LEFT JOIN \"employee\" \"e2\" ON TRUE"; String expectedVersionLow = "SELECT \"employee\".\"department_id\"\n" + "FROM \"foodmart\".\"employee\"\n" + "LEFT JOIN \"foodmart\".\"employee\" \"employee0\" " @@ -8268,28 +8702,37 @@ private void checkLiteral2(String expression, String expected) { + "ON TRUE"; sql(query) .withOracle(23).ok(expectedVersionHigh) - .withOracle(11).ok(expectedVersionLow); + .withOracle(11).ok(expectedVersionLow) + .done(); } /** Test case for * [CALCITE-6819] * MSSQL doesn't support TRUE/FALSE keywords in its Join predicate. */ - @Test void testJoinBoolLiteralMSSQL() { - final String queryTrue = "SELECT \"hire_date\", \"department_description\" FROM \"employee\" " + @Test void testJoinBoolLiteralMssql() { + final String queryTrue = "SELECT\n" + + " \"hire_date\", \"department_description\"\n" + + "FROM \"employee\"\n" + "LEFT JOIN \"department\" ON TRUE"; final String mssqlExpected1 = "SELECT [employee].[hire_date]," - + " [department].[department_description]\nFROM [foodmart].[employee]\nLEFT JOIN" - + " [foodmart].[department] ON (1 = 1)"; + + " [department].[department_description]\n" + + "FROM [foodmart].[employee]\n" + + "LEFT JOIN [foodmart].[department] ON (1 = 1)"; sql(queryTrue) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected1); + .withMssql().ok(mssqlExpected1) + .done(); - final String queryFalse = "SELECT \"hire_date\", \"department_description\" FROM \"employee\" " + final String queryFalse = "SELECT\n" + + " \"hire_date\", \"department_description\"\n" + + "FROM \"employee\"\n" + "LEFT JOIN \"department\" ON False"; final String mssqlExpected2 = "SELECT [employee].[hire_date]," - + " [department].[department_description]\nFROM [foodmart].[employee]\nLEFT JOIN" - + " [foodmart].[department] ON (1 = 0)"; + + " [department].[department_description]\n" + + "FROM [foodmart].[employee]\n" + + "LEFT JOIN [foodmart].[department] ON (1 = 0)"; sql(queryFalse) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected2); + .withMssql().ok(mssqlExpected2) + .done(); } /** Test case for @@ -8297,8 +8740,8 @@ private void checkLiteral2(String expression, String expected) { * OracleDialect does not support CASE WHEN returning boolean. */ @Test void testBooleanCaseWhenOracle() { String query0 = "SELECT \"e1\".\"department_id\" " - + "FROM \"employee\" \"e1\"" - + "LEFT JOIN \"employee\" \"e2\"" + + "FROM \"foodmart\".\"employee\" \"e1\"" + + "LEFT JOIN \"foodmart\".\"employee\" \"e2\"" + "ON CASE WHEN \"e2\".\"employee_id\" = 'a' " + "THEN \"e1\".\"department_id\" > 10 " + "WHEN \"e2\".\"employee_id\" = 'b' " @@ -8323,7 +8766,7 @@ private void checkLiteral2(String expression, String expected) { + " ELSE \"employee0\".\"employee_id\" = 'c' END"; String query1 = "SELECT \"department_id\" " - + "FROM \"employee\"" + + "FROM \"foodmart\".\"employee\"" + "WHERE CASE \"employee_id\" " + "WHEN 'a' THEN \"department_id\" > 10 " + "WHEN 'b' THEN \"department_id\" > 20 " @@ -8340,12 +8783,17 @@ private void checkLiteral2(String expression, String expected) { + "ELSE TRUE END"; sql(query0) + .schema(CalciteAssert.SchemaSpec.JDBC_FOODMART) + .withPhase(DialectTestConfig.Phase.PREPARE) .withOracle(23).ok(expectedVersionHigh0) - .withOracle(11).ok(expectedVersionLow0); + .withOracle(11).ok(expectedVersionLow0) + .done(); sql(query1) + .schema(CalciteAssert.SchemaSpec.JDBC_FOODMART) .withOracle(23).ok(expectedVersionHigh1) - .withOracle(11).ok(expectedVersionLow1); + .withOracle(11).ok(expectedVersionLow1) + .done(); } /** Test case for @@ -8353,14 +8801,16 @@ private void checkLiteral2(String expression, String expected) { * JDBC adapter sometimes adds unnecessary parentheses around SELECT in * INSERT. */ @Test void testInsertSelect() { - final String sql = "insert into \"DEPT\" select * from \"DEPT\""; + final String sql = "insert into dept select * from \"DEPT\""; final String expected = "" + "INSERT INTO \"SCOTT\".\"DEPT\" (\"DEPTNO\", \"DNAME\", \"LOC\")\n" + "SELECT *\n" + "FROM \"SCOTT\".\"DEPT\""; sql(sql) + // only PARSE; execution hits 'There are not enough rules' + .withPhase(PARSE) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } @Test void testMerge() { @@ -8382,7 +8832,8 @@ private void checkLiteral2(String expression, String expected) { + "UPPER(\"DEPT\".\"LOC\")"; sql(sql1) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected1); + .ok(expected1) + .done(); // without insert columns final String sql2 = "merge into \"DEPT\" as \"t\"\n" @@ -8395,7 +8846,8 @@ private void checkLiteral2(String expression, String expected) { final String expected2 = expected1; sql(sql2) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected2); + .ok(expected2) + .done(); // reorder insert columns final String sql3 = "merge into \"DEPT\" as \"t\"\n" @@ -8416,7 +8868,8 @@ private void checkLiteral2(String expression, String expected) { + "CAST(LOWER(\"DEPT\".\"DNAME\") AS VARCHAR(13) CHARACTER SET \"ISO-8859-1\")"; sql(sql3) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected3); + .ok(expected3) + .done(); // without WHEN NOT MATCHED THEN final String sql4 = "merge into \"DEPT\" as \"t\"\n" @@ -8430,7 +8883,8 @@ private void checkLiteral2(String expression, String expected) { + "WHEN MATCHED THEN UPDATE SET \"DNAME\" = \"DEPT\".\"DNAME\""; sql(sql4) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected4); + .ok(expected4) + .done(); // without WHEN MATCHED THEN final String sql5 = "merge into \"DEPT\" as \"t\"\n" @@ -8448,7 +8902,8 @@ private void checkLiteral2(String expression, String expected) { + "UPPER(\"DEPT\".\"LOC\")"; sql(sql5) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected5); + .ok(expected5) + .done(); // using query final String sql6 = "merge into \"DEPT\" as \"t\"\n" @@ -8468,7 +8923,8 @@ private void checkLiteral2(String expression, String expected) { + "UPPER(\"t0\".\"LOC\")"; sql(sql6) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected6); + .ok(expected6) + .done(); final String sql7 = "merge into \"DEPT\" as \"t\"\n" + "using (select * from (values (1, 'name', 'loc'))) as \"s\"(\"a\", \"b\", \"c\")\n" @@ -8490,7 +8946,8 @@ private void checkLiteral2(String expression, String expected) { + "CAST(UPPER(\"t0\".\"EXPR$2\") AS VARCHAR(13) CHARACTER SET \"ISO-8859-1\")"; sql(sql7) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected7); + .ok(expected7) + .done(); } /** Test case for @@ -8501,26 +8958,34 @@ private void checkLiteral2(String expression, String expected) { final String expected1 = "SELECT HIGHER_ORDER_FUNCTION(" + "1, (\"X\", \"Y\") -> CHAR_LENGTH(\"X\") + 1)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql1).ok(expected1); + sql(sql1) + .withPhase(PARSE) // no validation: higher_order_function not defined + .ok(expected1).done(); final String sql2 = "select higher_order_function2(1, () -> abs(-1))"; final String expected2 = "SELECT HIGHER_ORDER_FUNCTION2(" + "1, () -> ABS(-1))\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql2).ok(expected2); + sql(sql2) + .withPhase(PARSE) // no validation: higher_order_function not defined + .ok(expected2).done(); final String sql3 = "select \"department_id\", " + "higher_order_function(1, (department_id, y) -> department_id + 1) from \"employee\""; final String expected3 = "SELECT \"department_id\", HIGHER_ORDER_FUNCTION(1, " + "(\"DEPARTMENT_ID\", \"Y\") -> CAST(\"DEPARTMENT_ID\" AS INTEGER) + 1)\n" + "FROM \"foodmart\".\"employee\""; - sql(sql3).ok(expected3); + sql(sql3) + .withPhase(PARSE) // no validation: higher_order_function not defined + .ok(expected3).done(); final String sql4 = "select higher_order_function2(1, () -> cast(null as integer))"; final String expected4 = "SELECT HIGHER_ORDER_FUNCTION2(" + "1, () -> NULL)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql4).ok(expected4); + sql(sql4) + .withPhase(PARSE) // no validation: higher_order_function not defined + .ok(expected4).done(); final String sql5 = "select \"employee_id\", " + "higher_order_function(" @@ -8530,13 +8995,17 @@ private void checkLiteral2(String expression, String expected) { + "\"employee_id\", (\"PRODUCT_ID\", \"EMPLOYEE_ID\") -> " + "CHAR_LENGTH(\"PRODUCT_ID\") + \"EMPLOYEE_ID\")\n" + "FROM \"foodmart\".\"employee\""; - sql(sql5).ok(expected5); + sql(sql5) + .withPhase(PARSE) // no validation: higher_order_function not defined + .ok(expected5).done(); final String sql6 = "select higher_order_function(1, (y, x) -> x + char_length(y) + 1)"; final String expected6 = "SELECT HIGHER_ORDER_FUNCTION(" + "1, (\"Y\", \"X\") -> \"X\" + CHAR_LENGTH(\"Y\") + 1)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql6).ok(expected6); + sql(sql6) + .withPhase(PARSE) // no validation: higher_order_function not defined + .ok(expected6).done(); } /** Test case for @@ -8546,24 +9015,35 @@ private void checkLiteral2(String expression, String expected) { final String sql = "select \"EXISTS\"(array[1,2,3], x -> x > 2)"; final String expected = "SELECT EXISTS(ARRAY[1, 2, 3], \"X\" -> \"X\" > 2)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql) - .withLibrary(SqlLibrary.SPARK) - .ok(expected); + final String expectedSpark = "SELECT" + + " EXISTS(ARRAY (1, 2, 3), `X` -> `X` > 2)\n" + + "FROM (VALUES (0)) `t` (`ZERO`)"; + sql(sql).withLibrary(SqlLibrary.SPARK) + .ok(expected) + .withSpark().ok(expectedSpark) + .done(); final String sql2 = "select \"EXISTS\"(array[1,2,3], (x) -> false)"; final String expected2 = "SELECT EXISTS(ARRAY[1, 2, 3], \"X\" -> FALSE)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql2) - .withLibrary(SqlLibrary.SPARK) - .ok(expected2); + final String expectedSpark2 = "SELECT" + + " EXISTS(ARRAY (1, 2, 3), `X` -> FALSE)\n" + + "FROM (VALUES (0)) `t` (`ZERO`)"; + sql(sql2).withLibrary(SqlLibrary.SPARK) + .ok(expected2) + .withSpark().ok(expectedSpark2) + .done(); // empty array final String sql3 = "select \"EXISTS\"(array(), (x) -> false)"; final String expected3 = "SELECT EXISTS(ARRAY(), \"X\" -> FALSE)\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; - sql(sql3) - .withLibrary(SqlLibrary.SPARK) - .ok(expected3); + final String expectedSpark3 = "SELECT EXISTS(ARRAY(), `X` -> FALSE)\n" + + "FROM (VALUES (0)) `t` (`ZERO`)"; + sql(sql3).withLibrary(SqlLibrary.SPARK) + .ok(expected3) + .withSpark().ok(expectedSpark3) + .done(); final String sql4 = "select \"EXISTS\"('string', (x) -> false)"; final String error4 = "org.apache.calcite.runtime.CalciteContextException: " @@ -8571,9 +9051,10 @@ private void checkLiteral2(String expression, String expected) { + "Cannot apply 'EXISTS' to arguments of type " + "'EXISTS(, BOOLEAN>)'. " + "Supported form(s): EXISTS(, BOOLEAN>)"; - sql(sql4) - .withLibrary(SqlLibrary.SPARK) - .throws_(error4); + sql(sql4).withLibrary(SqlLibrary.SPARK) + .withPhase(PARSE) + .withSpark().throws_(error4) + .done(); } /** Test case for @@ -8583,8 +9064,11 @@ private void checkLiteral2(String expression, String expected) { @Test void testInsertUnionThenIntersect() { final String sql = "" + "insert into \"DEPT\"\n" - + "(select * from \"DEPT\" union select * from \"DEPT\")\n" - + "intersect select * from \"DEPT\""; + + "(select * from \"DEPT\"\n" + + " union\n" + + " select * from \"DEPT\")\n" + + "intersect\n" + + "select * from \"DEPT\""; final String expected = "" + "INSERT INTO \"SCOTT\".\"DEPT\" (\"DEPTNO\", \"DNAME\", \"LOC\")\n" + "SELECT *\n" @@ -8597,12 +9081,15 @@ private void checkLiteral2(String expression, String expected) { + "SELECT *\n" + "FROM \"SCOTT\".\"DEPT\""; sql(sql) + // only PARSE; execution hits 'There are not enough rules' + .withPhase(PARSE) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } @Test void testInsertValuesWithDynamicParams() { - final String sql = "insert into \"DEPT\" values (?,?,?), (?,?,?)"; + final String sql = "insert into dept\n" + + " values (?,?,?), (?,?,?)"; final String expected = "" + "INSERT INTO \"SCOTT\".\"DEPT\" (\"DEPTNO\", \"DNAME\", \"LOC\")\n" + "SELECT ?, ?, ?\n" @@ -8612,12 +9099,11 @@ private void checkLiteral2(String expression, String expected) { + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; sql(sql) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } @Test void testInsertValuesWithExplicitColumnsAndDynamicParams() { - final String sql = "" - + "insert into \"DEPT\" (\"DEPTNO\", \"DNAME\", \"LOC\")\n" + final String sql = "insert into dept (deptno, dname, loc)\n" + "values (?,?,?), (?,?,?)"; final String expected = "" + "INSERT INTO \"SCOTT\".\"DEPT\" (\"DEPTNO\", \"DNAME\", \"LOC\")\n" @@ -8627,35 +9113,47 @@ private void checkLiteral2(String expression, String expected) { + "SELECT ?, ?, ?\n" + "FROM (VALUES (0)) AS \"t\" (\"ZERO\")"; sql(sql) + // only PARSE; execution hits 'There are not enough rules' + .withPhase(PARSE) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .ok(expected); + .ok(expected).done(); } @Test void testTableFunctionScan() { final String query = "SELECT *\n" + "FROM TABLE(DEDUP(CURSOR(select \"product_id\", \"product_name\"\n" - + "from \"product\"), CURSOR(select \"employee_id\", \"full_name\"\n" - + "from \"employee\"), 'NAME'))"; + + "from \"foodmart\".\"product\"), CURSOR(select \"employee_id\", \"full_name\"\n" + + "from \"foodmart\".\"employee\"), 'NAME'))"; final String expected = "SELECT *\n" + "FROM TABLE(DEDUP(CURSOR ((SELECT \"product_id\", \"product_name\"\n" + "FROM \"foodmart\".\"product\")), CURSOR ((SELECT \"employee_id\", \"full_name\"\n" + "FROM \"foodmart\".\"employee\")), 'NAME'))"; - sql(query).ok(expected); + sql(query) + .withPhase(PARSE) // only PARSE; DEDUP/RAMP not available at runtime + .ok(expected).done(); final String query2 = "select * from table(ramp(3))"; - sql(query2).ok("SELECT *\n" - + "FROM TABLE(RAMP(3))"); + final String expected2 = "SELECT *\n" + + "FROM TABLE(RAMP(3))"; + sql(query2) + .withPhase(PARSE) // only PARSE; DEDUP/RAMP not available at runtime + .ok(expected2).done(); } @Test void testTableFunctionScanWithComplexQuery() { final String query = "SELECT *\n" - + "FROM TABLE(DEDUP(CURSOR(select \"product_id\", \"product_name\"\n" - + "from \"product\"\n" - + "where \"net_weight\" > 100 and \"product_name\" = 'Hello World')\n" - + ",CURSOR(select \"employee_id\", \"full_name\"\n" - + "from \"employee\"\n" - + "group by \"employee_id\", \"full_name\"), 'NAME'))"; + + "FROM TABLE(\n" + + " DEDUP(\n" + + " CURSOR(\n" + + " select \"product_id\", \"product_name\"\n" + + " from \"product\"\n" + + " where \"net_weight\" > 100\n" + + " and \"product_name\" = 'Hello World'),\n" + + " CURSOR(\n" + + " select \"employee_id\", \"full_name\"\n" + + " from \"employee\"\n" + + " group by \"employee_id\", \"full_name\"), 'NAME'))"; final String expected = "SELECT *\n" + "FROM TABLE(DEDUP(CURSOR ((SELECT \"product_id\", \"product_name\"\n" @@ -8664,7 +9162,9 @@ private void checkLiteral2(String expression, String expected) { + "CURSOR ((SELECT \"employee_id\", \"full_name\"\n" + "FROM \"foodmart\".\"employee\"\n" + "GROUP BY \"employee_id\", \"full_name\")), 'NAME'))"; - sql(query).ok(expected); + sql(query) + .withPhase(PARSE) // only PARSE; DEDUP/RAMP not available at runtime + .ok(expected).done(); } /** Test case for @@ -8672,20 +9172,24 @@ private void checkLiteral2(String expression, String expected) { * Add support for SqlWindowTableFunction in RelToSql Converter. */ @Test void testWindowTableFunctionScan() { final String query = "SELECT *\n" - + "FROM TABLE(TUMBLE(TABLE \"employee\", DESCRIPTOR(\"hire_date\"), INTERVAL '1' MINUTE))"; + + "FROM TABLE(TUMBLE(TABLE \"employee\", DESCRIPTOR(\"hire_date\"),\n" + + " INTERVAL '1' MINUTE))"; final String expected = "SELECT *\n" - + "FROM TABLE(TUMBLE((SELECT *\n" - + "FROM \"foodmart\".\"employee\"), DESCRIPTOR(\"hire_date\"), INTERVAL '1' MINUTE))"; - sql(query).ok(expected); + + "FROM TABLE(TUMBLE((SELECT *\n" + + "FROM \"foodmart\".\"employee\"), DESCRIPTOR(\"hire_date\")," + + " INTERVAL '1' MINUTE))"; + sql(query).ok(expected).done(); } @Test void testWindowTableFunctionScanWithSubQuery() { final String query = "SELECT * \n" - + "FROM TABLE(TUMBLE((SELECT \"employee_id\", \"hire_date\" FROM \"employee\"), DESCRIPTOR(\"hire_date\"), INTERVAL '1' MINUTE))"; + + "FROM TABLE(TUMBLE(\n" + + " (SELECT \"employee_id\", \"hire_date\" FROM \"employee\"),\n" + + " DESCRIPTOR(\"hire_date\"), INTERVAL '1' MINUTE))"; final String expected = "SELECT *\n" + "FROM TABLE(TUMBLE((SELECT \"employee_id\", \"hire_date\"\n" + "FROM \"foodmart\".\"employee\"), DESCRIPTOR(\"hire_date\"), INTERVAL '1' MINUTE))"; - sql(query).ok(expected); + sql(query).ok(expected).done(); } /** Test case for @@ -8693,43 +9197,38 @@ private void checkLiteral2(String expression, String expected) { * RelToSqlConverter changes target of ambiguous HAVING clause with a Project * on Filter on Aggregate. */ @Test void testBigQueryHaving() { - final String sql = "" - + "SELECT \"DEPTNO\" - 10 \"DEPTNO\"\n" - + "FROM \"EMP\"\n" - + "GROUP BY \"DEPTNO\"\n" - + "HAVING \"DEPTNO\" > 0"; - final String expected = "" - + "SELECT DEPTNO - 10 AS DEPTNO\n" + final String sql = "SELECT deptno - 10 deptno\n" + + "FROM emp\n" + + "GROUP BY deptno\n" + + "HAVING deptno > 0"; + final String expected = "SELECT DEPTNO - 10 AS DEPTNO\n" + "FROM (SELECT DEPTNO\n" + "FROM SCOTT.EMP\n" + "GROUP BY DEPTNO\n" + "HAVING CAST(DEPTNO AS INT64) > 0) AS t1"; - // Parse the input SQL with PostgreSQL dialect, - // in which "isHavingAlias" is false. + // Parse the input SQL with "isHavingAlias" false (similar to Postgres). final SqlParser.Config parserConfig = - PostgresqlSqlDialect.DEFAULT.configureParser(SqlParser.config()); + SqlParser.config().withConformance(SqlConformanceEnum.PRAGMATIC_2003); // Convert rel node to SQL with BigQuery dialect, // in which "isHavingAlias" is true. sql(sql) .parserConfig(parserConfig) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .withBigQuery().ok(expected); + .withBigQuery().ok(expected).done(); } /** Test case for * [CALCITE-4740] * JDBC adapter generates incorrect HAVING clause in BigQuery dialect. */ @Test void testBigQueryHavingWithoutGeneratedAlias() { - final String sql = "" - + "SELECT \"DEPTNO\", COUNT(DISTINCT \"EMPNO\")\n" - + "FROM \"EMP\"\n" - + "GROUP BY \"DEPTNO\"\n" - + "HAVING COUNT(DISTINCT \"EMPNO\") > 0\n" - + "ORDER BY COUNT(DISTINCT \"EMPNO\") DESC"; - final String expected = "" - + "SELECT DEPTNO, COUNT(DISTINCT EMPNO)\n" + final String sql = "SELECT deptno, COUNT(DISTINCT empno)\n" + + "FROM emp\n" + + "GROUP BY deptno\n" + + "HAVING COUNT(DISTINCT empno) > 0\n" + + "ORDER BY COUNT(DISTINCT empno) DESC"; + final String expected = "SELECT DEPTNO, COUNT(DISTINCT EMPNO)\n" + "FROM SCOTT.EMP\n" + "GROUP BY DEPTNO\n" + "HAVING COUNT(DISTINCT EMPNO) > 0\n" @@ -8739,7 +9238,7 @@ private void checkLiteral2(String expression, String expected) { // in which "isHavingAlias" is true. sql(sql) .schema(CalciteAssert.SchemaSpec.JDBC_SCOTT) - .withBigQuery().ok(expected); + .withBigQuery().ok(expected).done(); } /** Test case for @@ -8762,8 +9261,9 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY CASE WHEN GROUPING([brand_name]) IS NULL THEN 1 ELSE 0 END, 3," + " CASE WHEN [brand_name] IS NULL THEN 1 ELSE 0 END, [brand_name]," + " CASE WHEN [product_class_id] IS NULL THEN 1 ELSE 0 END, [product_class_id]"; - - sql(query).withMssql().ok(expectedMssql); + sql(query) + .withMssql().ok(expectedMssql) + .done(); } /** Test case for @@ -8776,15 +9276,19 @@ private void checkLiteral2(String expression, String expected) { final String query = "select soundex('Miller') from \"product\"\n"; final String expectedSql = "SELECT SOUNDEX('Miller')\n" + "FROM `foodmart`.`product`"; - - sql(query).withSpark().withLibrary(SqlLibrary.SPARK).ok(expectedSql); + sql(query) + .withLibrary(SqlLibrary.SPARK) + .withSpark().ok(expectedSql) + .done(); } /** Test case for * [CALCITE-6213] * The default behavior of NullCollation in Presto is LAST. */ @Test void testNullCollation() { - final String query = "select * from \"product\" order by \"brand_name\""; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\""; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\""; @@ -8793,11 +9297,14 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name` NULLS LAST"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); + .withSpark().ok(sparkExpected) + .done(); } @Test void testNullCollationAsc() { - final String query = "select * from \"product\" order by \"brand_name\" asc"; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\" asc"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\""; @@ -8806,11 +9313,14 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name` NULLS LAST"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); + .withSpark().ok(sparkExpected) + .done(); } @Test void testNullCollationAscNullLast() { - final String query = "select * from \"product\" order by \"brand_name\" asc nulls last"; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\" asc nulls last"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\""; @@ -8819,33 +9329,14 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name` NULLS LAST"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); - } - - /** Test case for - * [CALCITE-6748] - * RelToSqlConverter returns the wrong result when Aggregate is on Sort. */ - @Test void testAggregateOnSort() { - final String query0 = "select max(\"product_class_id\") " - + "from (select * from \"product\" order by \"brand_name\" asc limit 10) t"; - final String expected0 = "SELECT MAX(\"product_class_id\")\n" - + "FROM (SELECT \"product_class_id\"\n" - + "FROM \"foodmart\".\"product\"\n" - + "ORDER BY \"brand_name\"\n" - + "FETCH NEXT 10 ROWS ONLY) AS \"t1\""; - sql(query0).ok(expected0); - - final String query1 = "select max(\"product_class_id\") " - + "from (select * from \"product\" offset 10 ) t"; - final String expected1 = "SELECT MAX(\"product_class_id\")\n" - + "FROM (SELECT \"product_class_id\"\n" - + "FROM \"foodmart\".\"product\"\n" - + "OFFSET 10 ROWS) AS \"t1\""; - sql(query1).ok(expected1); + .withSpark().ok(sparkExpected) + .done(); } @Test void testNullCollationAscNullFirst() { - final String query = "select * from \"product\" order by \"brand_name\" asc nulls first"; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\" asc nulls first"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\" IS NULL DESC, \"brand_name\""; @@ -8854,11 +9345,14 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name`"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); + .withSpark().ok(sparkExpected) + .done(); } @Test void testNullCollationDesc() { - final String query = "select * from \"product\" order by \"brand_name\" desc"; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\" desc"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\" IS NULL DESC, \"brand_name\" DESC"; @@ -8867,11 +9361,14 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name` DESC NULLS FIRST"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); + .withSpark().ok(sparkExpected) + .done(); } @Test void testNullCollationDescLast() { - final String query = "select * from \"product\" order by \"brand_name\" desc nulls last"; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\" desc nulls last"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\" DESC"; @@ -8880,11 +9377,14 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name` DESC"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); + .withSpark().ok(sparkExpected) + .done(); } @Test void testNullCollationDescFirst() { - final String query = "select * from \"product\" order by \"brand_name\" desc nulls first"; + final String query = "select *\n" + + "from \"product\"\n" + + "order by \"brand_name\" desc nulls first"; final String expected = "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "ORDER BY \"brand_name\" IS NULL DESC, \"brand_name\" DESC"; @@ -8893,7 +9393,32 @@ private void checkLiteral2(String expression, String expected) { + "ORDER BY `brand_name` DESC NULLS FIRST"; sql(query) .withPresto().ok(expected) - .withSpark().ok(sparkExpected); + .withSpark().ok(sparkExpected) + .done(); + } + + /** Test case for + * [CALCITE-6748] + * RelToSqlConverter returns the wrong result when Aggregate is on Sort. */ + @Test void testAggregateOnSort() { + final String query0 = "select max(\"product_class_id\")\n" + + "from (select *\n" + + " from \"product\"\n" + + " order by \"brand_name\" asc limit 10) t"; + final String expected0 = "SELECT MAX(\"product_class_id\")\n" + + "FROM (SELECT \"product_class_id\"\n" + + "FROM \"foodmart\".\"product\"\n" + + "ORDER BY \"brand_name\"\n" + + "FETCH NEXT 10 ROWS ONLY) AS \"t1\""; + sql(query0).ok(expected0).done(); + + final String query1 = "select max(\"product_class_id\") " + + "from (select * from \"product\" offset 10 ) t"; + final String expected1 = "SELECT MAX(\"product_class_id\")\n" + + "FROM (SELECT \"product_class_id\"\n" + + "FROM \"foodmart\".\"product\"\n" + + "OFFSET 10 ROWS) AS \"t1\""; + sql(query1).ok(expected1).done(); } /** Test case for @@ -8911,7 +9436,8 @@ private void checkLiteral2(String expression, String expected) { .withPresto().ok(expectedPresto) .withStarRocks().ok(expectedStarRocks) .withSpark().ok(expectedSpark) - .withHive().ok(expectedHive); + .withHive().ok(expectedHive) + .done(); } @Test void testMapValueConstructorWithArray() { @@ -8922,7 +9448,8 @@ private void checkLiteral2(String expression, String expected) { + "FROM (VALUES (0)) `t` (`ZERO`)"; sql(query) .withPresto().ok(expectedPresto) - .withSpark().ok(expectedSpark); + .withSpark().ok(expectedSpark) + .done(); } /** Test case for @@ -8938,7 +9465,7 @@ private void checkLiteral2(String expression, String expected) { @Test void testHiveMapValueConstructorWithArray() { final String query = "SELECT MAP[1, ARRAY['v1', 'v2']]"; final String expectedHive = "SELECT MAP (1, ARRAY ('v1', 'v2'))"; - sql(query).withHive().ok(expectedHive); + sql(query).withHive().ok(expectedHive).done(); } /** Test case for @@ -8946,12 +9473,17 @@ private void checkLiteral2(String expression, String expected) { * StarRocks dialect implementation. */ @Test void testCastToTimestamp() { - final String query = "select * from \"employee\" where \"hire_date\" - " - + "INTERVAL '19800' SECOND(5) > cast(\"hire_date\" as TIMESTAMP) "; + final String query = "select *\n" + + "from \"employee\"\n" + + "where \"hire_date\" - INTERVAL '19800' SECOND(5)\n" + + " > cast(\"hire_date\" as TIMESTAMP) "; final String expectedStarRocks = "SELECT *\n" + "FROM `foodmart`.`employee`\n" - + "WHERE (`hire_date` - INTERVAL '19800' SECOND) > CAST(`hire_date` AS DATETIME)"; - sql(query).withStarRocks().ok(expectedStarRocks); + + "WHERE (`hire_date` - INTERVAL '19800' SECOND)" + + " > CAST(`hire_date` AS DATETIME)"; + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } /** Test case for @@ -8965,16 +9497,17 @@ private void checkLiteral2(String expression, String expected) { final String expectedHive = "SELECT ARRAY (1, 2, 3)"; sql(query).withStarRocks().ok(expectedStarRocks) .withSpark().ok(expectedSpark) - .withHive().ok(expectedHive); + .withHive().ok(expectedHive).done(); } @Test void testTrimWithBothSpecialCharacter() { final String query = "SELECT TRIM(BOTH '$@*A' from '$@*AABC$@*AADCAA$@*A')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT REGEXP_REPLACE('$@*AABC$@*AADCAA$@*A'," + " '^(\\$\\@\\*A)*|(\\$\\@\\*A)*$', '')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks).done(); } @Test void testUnparseSqlIntervalQualifier() { @@ -8983,21 +9516,21 @@ private void checkLiteral2(String expression, String expected) { final String expect0 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` - INTERVAL '19800' SECOND) > DATETIME '2005-10-17 00:00:00'"; - sql(sql0).withStarRocks().ok(expect0); + sql(sql0).withStarRocks().ok(expect0).done(); final String sql1 = "select * from \"employee\" where \"hire_date\" + " + "INTERVAL '10' HOUR > TIMESTAMP '2005-10-17 00:00:00' "; final String expect1 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` + INTERVAL '10' HOUR) > DATETIME '2005-10-17 00:00:00'"; - sql(sql1).withStarRocks().ok(expect1); + sql(sql1).withStarRocks().ok(expect1).done(); final String sql2 = "select * from \"employee\" where \"hire_date\" + " + "INTERVAL '1' YEAR > TIMESTAMP '2005-10-17 00:00:00' "; final String expect2 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` + INTERVAL '1' YEAR) > DATETIME '2005-10-17 00:00:00'"; - sql(sql2).withStarRocks().ok(expect2); + sql(sql2).withStarRocks().ok(expect2).done(); final String sql3 = "select * from \"employee\" " + "where \"hire_date\" + INTERVAL '39' MINUTE" @@ -9005,63 +9538,75 @@ private void checkLiteral2(String expression, String expected) { final String expect3 = "SELECT *\n" + "FROM `foodmart`.`employee`\n" + "WHERE (`hire_date` + INTERVAL '39' MINUTE) > DATETIME '2005-10-17 00:00:00'"; - sql(sql3).withStarRocks().ok(expect3); + sql(sql3).withStarRocks().ok(expect3).done(); } @Test void testTrim() { final String query = "SELECT TRIM(' str ')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT TRIM(' str ')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testTrimWithBoth() { final String query = "SELECT TRIM(both ' ' from ' str ')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT TRIM(' str ')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testTrimWithLeading() { final String query = "SELECT TRIM(LEADING ' ' from ' str ')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT LTRIM(' str ')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testTrimWithTailing() { final String query = "SELECT TRIM(TRAILING ' ' from ' str ')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT RTRIM(' str ')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testTrimWithBothChar() { final String query = "SELECT TRIM(both 'a' from 'abcda')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT REGEXP_REPLACE('abcda', '^(a)*|(a)*$', '')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testTrimWithTailingChar() { final String query = "SELECT TRIM(TRAILING 'a' from 'abcd')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT REGEXP_REPLACE('abcd', '(a)*$', '')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query) + .withStarRocks().ok(expectedStarRocks) + .done(); } @Test void testTrimWithLeadingChar() { final String query = "SELECT TRIM(LEADING 'a' from 'abcd')\n" - + "from \"foodmart\".\"reserve_employee\""; + + "from \"reserve_employee\""; final String expectedStarRocks = "SELECT REGEXP_REPLACE('abcd', '^(a)*', '')\n" + "FROM `foodmart`.`reserve_employee`"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query).withStarRocks().ok(expectedStarRocks).done(); } @Test void testSelectQueryWithRollup() { @@ -9070,22 +9615,23 @@ private void checkLiteral2(String expression, String expected) { final String expectedStarRocks = "SELECT `product_class_id`, `product_id`, COUNT(*)\n" + "FROM `foodmart`.`product`\n" + "GROUP BY ROLLUP(`product_class_id`, `product_id`)"; - sql(query).withStarRocks().ok(expectedStarRocks); + sql(query).withStarRocks().ok(expectedStarRocks).done(); } /** Test case for * [CALCITE-6370] - * AS operator problems with USING clause. - */ + * AS operator problems with USING clause. */ @Test void testUsingClauseWithAsInProjection() { - String query = "select \"product_id\" AS \"x\" from \"foodmart\".\"product\" p0 join " - + " \"foodmart\".\"product\" p1 using (\"product_id\")"; + String query = "select \"product_id\" AS \"x\"\n" + + "from \"product\" p0\n" + + "join \"product\" p1 using (\"product_id\")"; String expectedQuery = "SELECT \"product\".\"product_id\" AS \"x\"\n" + "FROM \"foodmart\".\"product\"\n" + "INNER JOIN \"foodmart\".\"product\" AS \"product0\" ON " + "\"product\".\"product_id\" = \"product0\".\"product_id\""; sql(query) - .withPostgresql().ok(expectedQuery); + .withPostgresql().ok(expectedQuery) + .done(); } /** Test case for @@ -9121,7 +9667,9 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"employee\"\n" + "INNER JOIN \"foodmart\".\"employee\" AS \"employee0\" ON " + "\"employee\".\"employee_id\" = \"employee0\".\"employee_id\""; - sql(query).withPostgresql().ok(expectedQuery); + sql(query) + .withPostgresql().ok(expectedQuery) + .done(); } @Test void testUsingClauseWithStarAndAsInProjection() { @@ -9153,37 +9701,34 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"employee\"\n" + "INNER JOIN \"foodmart\".\"employee\" AS \"employee0\" ON " + "\"employee\".\"employee_id\" = \"employee0\".\"employee_id\""; - sql(query).withPostgresql().ok(expectedQuery); + sql(query) + .withPostgresql().ok(expectedQuery) + .done(); } - /** Test case for + /** Test case for * [CALCITE-6633] - * MSSQL Dialect does not generate CEILING function. - */ - @Test void testMSSQLCeiling() { + * MSSQL Dialect does not generate CEILING function. */ + @Test void testMssqlCeiling() { final String query = "select 1.24, FLOOR(1.24), CEILING(1.24)"; final String mssqlExpected = "SELECT 1.24, FLOOR(1.24), CEILING(1.24)\n" + "FROM (VALUES (0)) AS [t] ([ZERO])"; - sql(query) - .dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected); + sql(query).withMssql().ok(mssqlExpected).done(); } - /** Test case for + /** Test case for * [CALCITE-6726] - * Add translation for MOD operator in MSSQL. - */ - @Test public void testModFunctionEmulationForMSSQL() { + * Add translation for MOD operator in MSSQL. */ + @Test public void testModFunctionEmulationForMssql() { final String query = "select mod(11,3)"; final String mssqlExpected = "SELECT 11 % 3\n" + "FROM (VALUES (0)) AS [t] ([ZERO])"; - sql(query).dialect(MssqlSqlDialect.DEFAULT).ok(mssqlExpected); + sql(query).withMssql().ok(mssqlExpected).done(); } - /** Test case for * [CALCITE-6655] - * Aggregation of deeply nested window not detected when unparsing. - */ + * Aggregation of deeply nested window not detected when unparsing. */ @Test void testAggregatedDeeplyNested() { // The CASE statement makes the inner sum deep enough to test we're // recursively looking for it @@ -9209,7 +9754,6 @@ private void checkLiteral2(String expression, String expected) { + "ELSE 0.0000 END `INNER_SUM`\n" + "FROM `foodmart`.`employee`" + ") `t`"; - sql(query).withSpark().ok(spark); // Oracle does support nested aggregations String oracle = @@ -9219,7 +9763,10 @@ private void checkLiteral2(String expression, String expected) { + "(PARTITION BY \"first_name\" RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) " + "ELSE 0.0000 END)\n" + "FROM \"foodmart\".\"employee\""; - sql(query).withOracle().ok(oracle); + sql(query) + .withSpark().ok(spark) + .withOracle().ok(oracle) + .done(); } /** Test case for @@ -9227,11 +9774,11 @@ private void checkLiteral2(String expression, String expected) { * Convert Type from BINARY to VARBINARY in PrestoDialect. */ @Test void testPrestoBinaryCast() { String query = "SELECT cast(cast(\"employee_id\" as varchar) as binary)\n" - + "from \"foodmart\".\"reserve_employee\" "; + + "from \"reserve_employee\""; String expected = "SELECT " + "CAST(CAST(\"employee_id\" AS VARCHAR) AS VARBINARY)\n" + "FROM \"foodmart\".\"reserve_employee\""; - sql(query).withPresto().ok(expected); + sql(query).withPresto().ok(expected).done(); } /** Test case for @@ -9246,7 +9793,7 @@ private void checkLiteral2(String expression, String expected) { + "CAST(\"department_id\" AS REAL)\n" + "FROM \"foodmart\".\"employee\""; sql(query) - .withPresto().ok(expected); + .withPresto().ok(expected).done(); } /** Test case for @@ -9272,7 +9819,7 @@ private void checkLiteral2(String expression, String expected) { + "SELECT *\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"t\".\"product_id\" = \"product_id\")"; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); } @Test void testAntiJoinWithComplexInput2() { @@ -9297,7 +9844,7 @@ private void checkLiteral2(String expression, String expected) { + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" = \"t\".\"product_id\" " + "AND \"product_id\" > 10)"; - sql(sql).ok(expected); + sql(sql).ok(expected).done(); } @Test void testFilterWithSubQuery() { @@ -9320,316 +9867,6 @@ private void checkLiteral2(String expression, String expected) { + "SELECT \"gross_weight\"\n" + "FROM \"foodmart\".\"product\"\n" + "WHERE \"product_id\" = \"t\".\"product_id\" AND \"product_id\" > 10)"; - sql(sql).ok(expected); - } - - - /** Fluid interface to run tests. */ - static class Sql { - private final CalciteAssert.SchemaSpec schemaSpec; - private final String sql; - private final SqlDialect dialect; - private final Set librarySet; - private final @Nullable Function relFn; - private final List> transforms; - private final SqlParser.Config parserConfig; - private final UnaryOperator config; - - Sql(CalciteAssert.SchemaSpec schemaSpec, String sql, SqlDialect dialect, - SqlParser.Config parserConfig, Set librarySet, - UnaryOperator config, - @Nullable Function relFn, - List> transforms) { - this.schemaSpec = schemaSpec; - this.sql = sql; - this.dialect = dialect; - this.librarySet = librarySet; - this.relFn = relFn; - this.transforms = ImmutableList.copyOf(transforms); - this.parserConfig = parserConfig; - this.config = config; - } - - Sql withSql(String sql) { - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } - - Sql dialect(SqlDialect dialect) { - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } - - Sql relFn(Function relFn) { - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } - - Sql withCalcite() { - return dialect(DatabaseProduct.CALCITE.getDialect()); - } - - Sql withClickHouse() { - return dialect(DatabaseProduct.CLICKHOUSE.getDialect()); - } - - Sql withDb2() { - return dialect(DatabaseProduct.DB2.getDialect()); - } - - Sql withExasol() { - return dialect(DatabaseProduct.EXASOL.getDialect()); - } - - Sql withFirebolt() { - return dialect(DatabaseProduct.FIREBOLT.getDialect()); - } - - Sql withHive() { - return dialect(DatabaseProduct.HIVE.getDialect()); - } - - Sql withHsqldb() { - return dialect(DatabaseProduct.HSQLDB.getDialect()); - } - - Sql withMssql() { - return withMssql(14); // MSSQL 2008 = 10.0, 2012 = 11.0, 2017 = 14.0 - } - - Sql withMssql(int majorVersion) { - final SqlDialect mssqlDialect = DatabaseProduct.MSSQL.getDialect(); - return dialect( - new MssqlSqlDialect(MssqlSqlDialect.DEFAULT_CONTEXT - .withDatabaseMajorVersion(majorVersion) - .withIdentifierQuoteString(mssqlDialect.quoteIdentifier("") - .substring(0, 1)) - .withNullCollation(mssqlDialect.getNullCollation()))); - } - - Sql withMysql() { - return dialect(DatabaseProduct.MYSQL.getDialect()); - } - - Sql withMysql8() { - final SqlDialect mysqlDialect = DatabaseProduct.MYSQL.getDialect(); - return dialect( - new SqlDialect(MysqlSqlDialect.DEFAULT_CONTEXT - .withDatabaseMajorVersion(8) - .withIdentifierQuoteString(mysqlDialect.quoteIdentifier("") - .substring(0, 1)) - .withNullCollation(mysqlDialect.getNullCollation()))); - } - - Sql withOracle() { - return withOracle(12); - } - - Sql withOracle(int majorVersion) { - final SqlDialect oracleDialect = DatabaseProduct.ORACLE.getDialect(); - return dialect( - new OracleSqlDialect(OracleSqlDialect.DEFAULT_CONTEXT - .withDatabaseProduct(DatabaseProduct.ORACLE) - .withDatabaseMajorVersion(majorVersion) - .withIdentifierQuoteString(oracleDialect.quoteIdentifier("") - .substring(0, 1)) - .withNullCollation(oracleDialect.getNullCollation()))); - } - - Sql withPostgresql() { - return dialect(DatabaseProduct.POSTGRESQL.getDialect()); - } - - Sql withPresto() { - return dialect(DatabaseProduct.PRESTO.getDialect()); - } - - Sql withRedshift() { - return dialect(DatabaseProduct.REDSHIFT.getDialect()); - } - - Sql withInformix() { - return dialect(DatabaseProduct.INFORMIX.getDialect()); - } - - Sql withSnowflake() { - return dialect(DatabaseProduct.SNOWFLAKE.getDialect()); - } - - Sql withSybase() { - return dialect(DatabaseProduct.SYBASE.getDialect()); - } - - Sql withVertica() { - return dialect(DatabaseProduct.VERTICA.getDialect()); - } - - Sql withBigQuery() { - return dialect(DatabaseProduct.BIG_QUERY.getDialect()); - } - - Sql withSpark() { - return dialect(DatabaseProduct.SPARK.getDialect()); - } - - Sql withStarRocks() { - return dialect(DatabaseProduct.STARROCKS.getDialect()); - } - - Sql withPostgresqlModifiedTypeSystem() { - // Postgresql dialect with max length for varchar set to 256 - final PostgresqlSqlDialect postgresqlSqlDialect = - new PostgresqlSqlDialect(PostgresqlSqlDialect.DEFAULT_CONTEXT - .withDataTypeSystem(new RelDataTypeSystemImpl() { - @Override public int getMaxPrecision(SqlTypeName typeName) { - switch (typeName) { - case VARCHAR: - return 256; - default: - return super.getMaxPrecision(typeName); - } - } - })); - return dialect(postgresqlSqlDialect); - } - - Sql withPostgresqlModifiedDecimalTypeSystem() { - final PostgresqlSqlDialect postgresqlSqlDialect = - new PostgresqlSqlDialect(PostgresqlSqlDialect.DEFAULT_CONTEXT - .withDataTypeSystem( - new RelDataTypeSystemImpl() { - @Override public int getMaxNumericScale() { - return getMaxScale(SqlTypeName.DECIMAL); - } - - @Override public int getMaxScale(SqlTypeName typeName) { - switch (typeName) { - case DECIMAL: - return 10; - default: - return super.getMaxScale(typeName); - } - } - - @Override public int getMaxNumericPrecision() { - return getMaxPrecision(SqlTypeName.DECIMAL); - } - - @Override public int getMaxPrecision(SqlTypeName typeName) { - switch (typeName) { - case DECIMAL: - return 39; - default: - return super.getMaxPrecision(typeName); - } - } - })); - return dialect(postgresqlSqlDialect); - } - - Sql withOracleModifiedTypeSystem() { - // Oracle dialect with max length for varchar set to 512 - final OracleSqlDialect oracleSqlDialect = - new OracleSqlDialect(OracleSqlDialect.DEFAULT_CONTEXT - .withDataTypeSystem(new RelDataTypeSystemImpl() { - @Override public int getMaxPrecision(SqlTypeName typeName) { - switch (typeName) { - case VARCHAR: - return 512; - default: - return super.getMaxPrecision(typeName); - } - } - })); - return dialect(oracleSqlDialect); - } - - Sql parserConfig(SqlParser.Config parserConfig) { - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } - - Sql withConfig(UnaryOperator config) { - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } - - final Sql withLibrary(SqlLibrary library) { - return withLibrarySet(ImmutableSet.of(library)); - } - - Sql withLibrarySet(Iterable librarySet) { - return new Sql(schemaSpec, sql, dialect, parserConfig, - ImmutableSet.copyOf(librarySet), config, relFn, transforms); - } - - Sql optimize(final RuleSet ruleSet, - final @Nullable RelOptPlanner relOptPlanner) { - final List> transforms = - FlatLists.append(this.transforms, r -> { - Program program = Programs.of(ruleSet); - final RelOptPlanner p = - Util.first(relOptPlanner, - new HepPlanner( - new HepProgramBuilder().addRuleClass(RelOptRule.class) - .build())); - return program.run(p, r, r.getTraitSet(), - ImmutableList.of(), ImmutableList.of()); - }); - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } - - Sql ok(String expectedQuery) { - assertThat(exec(), isLinux(expectedQuery)); - return this; - } - - Sql throws_(String errorMessage) { - try { - final String s = exec(); - throw new AssertionError("Expected exception with message `" - + errorMessage + "` but nothing was thrown; got " + s); - } catch (Exception e) { - assertThat(e.getMessage(), is(errorMessage)); - return this; - } - } - - String exec() { - try { - final SchemaPlus rootSchema = Frameworks.createRootSchema(true); - final SchemaPlus defaultSchema = - CalciteAssert.addSchema(rootSchema, schemaSpec); - RelNode rel; - if (relFn != null) { - final FrameworkConfig frameworkConfig = RelBuilderTest.config() - .defaultSchema(defaultSchema) - .build(); - final RelBuilder relBuilder = RelBuilder.create(frameworkConfig); - rel = relFn.apply(relBuilder); - } else { - final SqlToRelConverter.Config config = this.config.apply(SqlToRelConverter.config() - .withTrimUnusedFields(false)); - RelDataTypeSystem typeSystem = dialect.getTypeSystem(); - final Planner planner = - getPlanner(null, parserConfig, defaultSchema, config, librarySet, typeSystem); - SqlNode parse = planner.parse(sql); - SqlNode validate = planner.validate(parse); - rel = planner.rel(validate).project(); - } - for (Function transform : transforms) { - rel = transform.apply(rel); - } - return toSql(rel, dialect); - } catch (Exception e) { - throw TestUtil.rethrow(e); - } - } - - public Sql schema(CalciteAssert.SchemaSpec schemaSpec) { - return new Sql(schemaSpec, sql, dialect, parserConfig, librarySet, config, - relFn, transforms); - } + sql(sql).ok(expected).done(); } } diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlFixture.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlFixture.java new file mode 100644 index 000000000000..d894aff366a6 --- /dev/null +++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlFixture.java @@ -0,0 +1,590 @@ +/* + * 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.rel.rel2sql; + +import org.apache.calcite.plan.RelOptPlanner; +import org.apache.calcite.plan.RelOptRule; +import org.apache.calcite.plan.RelTraitDef; +import org.apache.calcite.plan.hep.HepPlanner; +import org.apache.calcite.plan.hep.HepProgramBuilder; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.rules.CoreRules; +import org.apache.calcite.rel.rules.ProjectOverSumToSum0Rule; +import org.apache.calcite.rel.rules.ProjectToWindowRule; +import org.apache.calcite.rel.type.RelDataTypeSystem; +import org.apache.calcite.runtime.FlatLists; +import org.apache.calcite.schema.SchemaPlus; +import org.apache.calcite.sql.SqlDialect; +import org.apache.calcite.sql.SqlNode; +import org.apache.calcite.sql.SqlWriterConfig; +import org.apache.calcite.sql.fun.SqlLibrary; +import org.apache.calcite.sql.parser.SqlParser; +import org.apache.calcite.sql2rel.SqlToRelConverter; +import org.apache.calcite.test.CalciteAssert; +import org.apache.calcite.test.MockSqlOperatorTable; +import org.apache.calcite.test.RelBuilderTest; +import org.apache.calcite.tools.FrameworkConfig; +import org.apache.calcite.tools.Frameworks; +import org.apache.calcite.tools.Planner; +import org.apache.calcite.tools.Program; +import org.apache.calcite.tools.Programs; +import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.tools.RuleSet; +import org.apache.calcite.tools.RuleSets; +import org.apache.calcite.util.TestUtil; +import org.apache.calcite.util.Token; +import org.apache.calcite.util.Util; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import org.checkerframework.checker.nullness.qual.Nullable; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.EnumSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; +import java.util.function.UnaryOperator; + +import static org.apache.calcite.rel.rel2sql.DialectCode.BIG_QUERY; +import static org.apache.calcite.rel.rel2sql.DialectCode.CALCITE; +import static org.apache.calcite.rel.rel2sql.DialectCode.CLICKHOUSE; +import static org.apache.calcite.rel.rel2sql.DialectCode.DB2; +import static org.apache.calcite.rel.rel2sql.DialectCode.EXASOL; +import static org.apache.calcite.rel.rel2sql.DialectCode.FIREBOLT; +import static org.apache.calcite.rel.rel2sql.DialectCode.HIVE; +import static org.apache.calcite.rel.rel2sql.DialectCode.HSQLDB; +import static org.apache.calcite.rel.rel2sql.DialectCode.INFORMIX; +import static org.apache.calcite.rel.rel2sql.DialectCode.JETHRO; +import static org.apache.calcite.rel.rel2sql.DialectCode.MSSQL_2017; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL_8; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL_FIRST; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL_HIGH; +import static org.apache.calcite.rel.rel2sql.DialectCode.MYSQL_LAST; +import static org.apache.calcite.rel.rel2sql.DialectCode.ORACLE_11; +import static org.apache.calcite.rel.rel2sql.DialectCode.ORACLE_12; +import static org.apache.calcite.rel.rel2sql.DialectCode.ORACLE_19; +import static org.apache.calcite.rel.rel2sql.DialectCode.ORACLE_23; +import static org.apache.calcite.rel.rel2sql.DialectCode.ORACLE_MODIFIED; +import static org.apache.calcite.rel.rel2sql.DialectCode.POSTGRESQL; +import static org.apache.calcite.rel.rel2sql.DialectCode.POSTGRESQL_MODIFIED; +import static org.apache.calcite.rel.rel2sql.DialectCode.POSTGRESQL_MODIFIED_DECIMAL; +import static org.apache.calcite.rel.rel2sql.DialectCode.PRESTO; +import static org.apache.calcite.rel.rel2sql.DialectCode.REDSHIFT; +import static org.apache.calcite.rel.rel2sql.DialectCode.SNOWFLAKE; +import static org.apache.calcite.rel.rel2sql.DialectCode.SPARK; +import static org.apache.calcite.rel.rel2sql.DialectCode.STARROCKS; +import static org.apache.calcite.rel.rel2sql.DialectCode.SYBASE; +import static org.apache.calcite.rel.rel2sql.DialectCode.VERTICA; +import static org.apache.calcite.test.Matchers.returnsUnordered; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +import static java.util.Objects.requireNonNull; + +/** Fluid interface to run tests. */ +class RelToSqlFixture { + /** A pool of tokens, used to identify fixtures that forgot to call + * {@link #done()}. */ + static final Token.Pool POOL = Token.pool(); + + private final Token token; + private final CalciteAssert.SchemaSpec schemaSpec; + private final String sql; + private final DialectTestConfig.Dialect dialect; + private final DialectTestConfig.Phase phase; + private final Set librarySet; + private final @Nullable Function relFn; + private final List> relTransforms; + private final SqlParser.Config parserConfig; + private final UnaryOperator configTransform; + private final DialectTestConfig testConfig; + private final UnaryOperator writerTransform; + + RelToSqlFixture(Token token, CalciteAssert.SchemaSpec schemaSpec, String sql, + DialectTestConfig.Dialect dialect, DialectTestConfig.Phase phase, + SqlParser.Config parserConfig, Set librarySet, + UnaryOperator configTransform, + @Nullable Function relFn, + List> relTransforms, + DialectTestConfig testConfig, + UnaryOperator writerTransform) { + this.token = requireNonNull(token, "token"); + this.schemaSpec = schemaSpec; + this.sql = sql; + this.dialect = dialect; + this.phase = requireNonNull(phase, "phase"); + this.librarySet = librarySet; + this.relFn = relFn; + this.relTransforms = ImmutableList.copyOf(relTransforms); + this.parserConfig = parserConfig; + this.configTransform = configTransform; + this.testConfig = requireNonNull(testConfig, "testConfig"); + this.writerTransform = requireNonNull(writerTransform, "writerTransform"); + } + + /** Default writer configuration. */ + static SqlWriterConfig transformWriter(SqlWriterConfig c) { + return c.withAlwaysUseParentheses(false) + .withSelectListItemsOnSeparateLines(false) + .withUpdateSetListNewline(false) + .withIndentation(0); + } + + public RelToSqlFixture schema(CalciteAssert.SchemaSpec schemaSpec) { + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture withSql(String sql) { + if (sql.equals(this.sql)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture dialect(DialectCode dialectCode) { + DialectTestConfig.Dialect dialect = testConfig.get(dialectCode); + return withDialect(dialect); + } + + public RelToSqlFixture withDialect(DialectTestConfig.Dialect dialect) { + if (dialect.equals(this.dialect)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture parserConfig(SqlParser.Config parserConfig) { + if (parserConfig.equals(this.parserConfig)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public final RelToSqlFixture withLibrary(SqlLibrary library) { + return withLibrarySet(ImmutableSet.of(library)); + } + + public RelToSqlFixture withLibrarySet( + Iterable librarySet) { + final ImmutableSet librarySet1 = + ImmutableSet.copyOf(librarySet); + if (librarySet1.equals(this.librarySet)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet1, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture withPhase(DialectTestConfig.Phase phase) { + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture withConfig( + UnaryOperator configTransform) { + if (configTransform.equals(this.configTransform)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture relFn(Function relFn) { + if (relFn.equals(this.relFn)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture withExtraTransform( + Function relTransform) { + final List> relTransforms2 = + FlatLists.append(relTransforms, relTransform); + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms2, + testConfig, writerTransform); + } + + public RelToSqlFixture withTestConfig( + UnaryOperator transform) { + DialectTestConfig testConfig = transform.apply(this.testConfig); + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + public RelToSqlFixture withWriterConfig( + UnaryOperator writerTransform) { + if (writerTransform.equals(this.writerTransform)) { + return this; + } + return new RelToSqlFixture(token, schemaSpec, sql, dialect, phase, + parserConfig, librarySet, configTransform, relFn, relTransforms, + testConfig, writerTransform); + } + + RelToSqlFixture withBigQuery() { + return dialect(BIG_QUERY); + } + + RelToSqlFixture withCalcite() { + return dialect(CALCITE); + } + + RelToSqlFixture withClickHouse() { + return dialect(CLICKHOUSE); + } + + RelToSqlFixture withDb2() { + return dialect(DB2); + } + + RelToSqlFixture withExasol() { + return dialect(EXASOL); + } + + RelToSqlFixture withFirebolt() { + return dialect(FIREBOLT); + } + + RelToSqlFixture withHive() { + return dialect(HIVE); + } + + RelToSqlFixture withHsqldb() { + return dialect(HSQLDB); + } + + RelToSqlFixture withInformix() { + return dialect(INFORMIX); + } + + RelToSqlFixture withJethro() { + return dialect(JETHRO); + } + + RelToSqlFixture withMssql() { + return dialect(MSSQL_2017); // MSSQL 2008 = 10.0, 2012 = 11.0, 2017 = 14.0 + } + + RelToSqlFixture withMysql() { + return dialect(MYSQL); + } + + RelToSqlFixture withMysqlHigh() { + return dialect(MYSQL_HIGH); + } + + RelToSqlFixture withMysqlFirst() { + return dialect(MYSQL_FIRST); + } + + RelToSqlFixture withMysqlLast() { + return dialect(MYSQL_LAST); + } + + RelToSqlFixture withMysql8() { + return dialect(MYSQL_8); + } + + RelToSqlFixture withOracle() { + return withOracle(12); + } + + RelToSqlFixture withOracle(int majorVersion) { + switch (majorVersion) { + case 11: + return dialect(ORACLE_11); + case 12: + return dialect(ORACLE_12); + case 19: + return dialect(ORACLE_19); + case 23: + return dialect(ORACLE_23); + default: + throw new IllegalArgumentException("Oracle version is not supported: " + + majorVersion); + } + } + + RelToSqlFixture withOracleModifiedTypeSystem() { + return dialect(ORACLE_MODIFIED); + } + + RelToSqlFixture withPostgresql() { + return dialect(POSTGRESQL); + } + + RelToSqlFixture withPostgresqlModifiedDecimalTypeSystem() { + return dialect(POSTGRESQL_MODIFIED_DECIMAL); + } + + RelToSqlFixture withPostgresqlModifiedTypeSystem() { + return dialect(POSTGRESQL_MODIFIED); + } + + RelToSqlFixture withPresto() { + return dialect(PRESTO); + } + + RelToSqlFixture withRedshift() { + return dialect(REDSHIFT); + } + + RelToSqlFixture withSnowflake() { + return dialect(SNOWFLAKE); + } + + RelToSqlFixture withSpark() { + return dialect(SPARK); + } + + RelToSqlFixture withStarRocks() { + return dialect(STARROCKS); + } + + RelToSqlFixture withSybase() { + return dialect(SYBASE); + } + + RelToSqlFixture withVertica() { + return dialect(VERTICA); + } + + /** Disables this test for a given list of dialects. */ + RelToSqlFixture withDisable(DialectCode code0, DialectCode... codes) { + final Set dialectCodes = EnumSet.of(code0, codes); + return withTestConfig(c -> + c.withDialects(d -> + dialectCodes.contains(d.code) ? d.withEnabled(false) : d)); + } + + RelToSqlFixture optimize(final RuleSet ruleSet, + final @Nullable RelOptPlanner relOptPlanner) { + final Function relTransform = r -> { + Program program = Programs.of(ruleSet); + final RelOptPlanner p = + Util.first(relOptPlanner, + new HepPlanner( + new HepProgramBuilder().addRuleClass(RelOptRule.class) + .build())); + return program.run(p, r, r.getTraitSet(), ImmutableList.of(), + ImmutableList.of()); + }; + return withExtraTransform(relTransform); + } + + RelToSqlFixture optimizeOver() { + HepProgramBuilder builder = new HepProgramBuilder(); + builder.addRuleClass(ProjectOverSumToSum0Rule.class); + builder.addRuleClass(ProjectToWindowRule.class); + HepPlanner hepPlanner = new HepPlanner(builder.build()); + RuleSet rules = + RuleSets.ofList(CoreRules.PROJECT_OVER_SUM_TO_SUM0_RULE, + CoreRules.PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW); + return optimize(rules, hepPlanner); + } + + RelToSqlFixture ok(String expectedQuery) { + return withTestConfig(c -> + c.withDialect(dialect.code, + d -> d.withExpectedQuery(expectedQuery).withEnabled(true))); + } + + RelToSqlFixture throws_(String errorMessage) { + return withTestConfig(c -> + c.withDialect(dialect.code, + d -> d.withExpectedError(errorMessage).withEnabled(true))); + } + + String exec() { + try { + final SchemaPlus rootSchema = Frameworks.createRootSchema(true); + final SchemaPlus defaultSchema = + CalciteAssert.addSchema(rootSchema, schemaSpec); + RelNode rel; + if (relFn != null) { + final FrameworkConfig frameworkConfig = + RelBuilderTest.config().defaultSchema(defaultSchema).build(); + final RelBuilder relBuilder = RelBuilder.create(frameworkConfig); + rel = relFn.apply(relBuilder); + } else { + final SqlToRelConverter.Config config = + this.configTransform.apply(SqlToRelConverter.config() + .withTrimUnusedFields(false)); + final RelDataTypeSystem typeSystem = dialect.sqlDialect.getTypeSystem(); + final Planner planner = + getPlanner(null, parserConfig, defaultSchema, config, librarySet, + typeSystem); + SqlNode parse = planner.parse(sql); + SqlNode validate = planner.validate(parse); + rel = planner.rel(validate).project(); + } + for (Function transform : relTransforms) { + rel = transform.apply(rel); + } + return toSql(rel, dialect.code); + } catch (Exception e) { + throw TestUtil.rethrow(e); + } + } + + public RelToSqlFixture done() { + token.close(); + + final AtomicReference> referenceResultSet = new AtomicReference<>(); + final AtomicReference> referenceValid = new AtomicReference<>(); + if (testConfig.refDialectCode != null) { + DialectTestConfig.Dialect referenceDialect = + testConfig.get(testConfig.refDialectCode); + final String referenceSql = + testConfig.refDialectCode == CALCITE && relFn == null + ? sql + : dialect(testConfig.refDialectCode).exec(); + switch (phase) { + case PREPARE: + referenceDialect.withConnection(schemaSpec, c -> { + try (PreparedStatement ps = c.prepareStatement(referenceSql)) { + List columnNames = new ArrayList<>(); + final ResultSetMetaData metaData = ps.getMetaData(); + for (int i = 0; i < metaData.getColumnCount(); i++) { + columnNames.add(metaData.getColumnName(i + 1)); + } + referenceValid.set(ImmutableList.copyOf(columnNames)); + } catch (SQLException e) { + throw new RuntimeException("while preparing [" + referenceSql + "]", e); + } + }); + break; + + case EXECUTE: + referenceDialect.withStatement(schemaSpec, statement -> { + try (ResultSet resultSet = statement.executeQuery(referenceSql)) { + final List rows = new ArrayList<>(); + CalciteAssert.toStringList(resultSet, rows); + referenceResultSet.set(ImmutableList.copyOf(rows)); + } catch (SQLException e) { + throw new RuntimeException(e); + } + }); + } + } + + // Generate the query in all enabled dialects, and check results if there + // is a reference dialect. + testConfig.dialectMap.forEach((dialectName, dialect) -> { + if (dialect.enabled) { + final String sql; + if (dialect.expectedError != null) { + try { + sql = dialect(dialect.code).exec(); + throw new AssertionError("Expected exception with message `" + + dialect.expectedError + "` but nothing was thrown; got " + + sql); + } catch (Exception e) { + assertThat(e.getMessage(), is(dialect.expectedError)); + return; + } + } else { + sql = dialect(dialect.code).exec(); + } + + if (dialect.expectedQuery != null) { + assertThat(sql, is(dialect.expectedQuery)); + } + + if (dialect.execute) { + dialect.withStatement(schemaSpec, statement -> { + try { + final ResultSet resultSet = statement.executeQuery(sql); + if (referenceResultSet.get() != null) { + assertThat(resultSet, returnsUnordered(referenceResultSet.get())); + } + } catch (SQLException e) { + throw new RuntimeException(e); + } + }); + } + } + }); + return this; + } + + private static Planner getPlanner(@Nullable List traitDefs, + SqlParser.Config parserConfig, SchemaPlus schema, + SqlToRelConverter.Config sqlToRelConf, Collection librarySet, + RelDataTypeSystem typeSystem, Program... programs) { + final FrameworkConfig config = Frameworks.newConfigBuilder() + .parserConfig(parserConfig) + .defaultSchema(schema) + .traitDefs(traitDefs) + .sqlToRelConverterConfig(sqlToRelConf) + .programs(programs) + .operatorTable(MockSqlOperatorTable.standard() + .plus(librarySet) + .extend()) + .typeSystem(typeSystem) + .build(); + return Frameworks.getPlanner(config); + } + + /** Converts a relational expression to SQL in a given dialect. */ + private String toSql(RelNode root, + DialectCode dialectCode) { + return toSql(root, dialectCode, writerTransform); + } + + /** Converts a relational expression to SQL in a given dialect + * and with a particular writer configuration. */ + private String toSql(RelNode root, DialectCode dialectCode, + UnaryOperator transform) { + final DialectTestConfig.Dialect dialect = testConfig.get(dialectCode); + final SqlDialect sqlDialect = dialect.sqlDialect; + final RelToSqlConverter converter = new RelToSqlConverter(sqlDialect); + final SqlNode sqlNode = converter.visitRoot(root).asStatement(); + return sqlNode.toSqlString(c -> transform.apply(c.withDialect(sqlDialect))) + .getSql(); + } + + /** Returns the dialect. */ + public SqlDialect sqlDialect() { + return dialect.sqlDialect; + } +} diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java index 29fc5ed40b37..4c88b4ebcf77 100644 --- a/core/src/test/java/org/apache/calcite/util/UtilTest.java +++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java @@ -1952,6 +1952,25 @@ private void checkResourceMethodNames(Object resource) { } } + @SuppressWarnings("resource") + @Test void testToken() { + final Token.Pool pool = Token.pool(); + final Token token1 = pool.token(); + final Token token2 = pool.token(); + final Token token3 = pool.token(); + final Token token4 = pool.token(); + token2.close(); + token4.close(); + // Token 2 closed twice + assertThrows(RuntimeException.class, token2::close); + token1.close(); + // Pool is not empty + assertThrows(RuntimeException.class, pool::assertEmpty); + token3.close(); + // Pool is now empty + pool.assertEmpty(); + } + /** Tests that sorted sets behave the way we expect. */ @Test void testSortedSet() { final TreeSet treeSet = new TreeSet(); diff --git a/testkit/src/main/java/org/apache/calcite/test/CalciteAssert.java b/testkit/src/main/java/org/apache/calcite/test/CalciteAssert.java index 7e9f675306d0..f028ccd739e2 100644 --- a/testkit/src/main/java/org/apache/calcite/test/CalciteAssert.java +++ b/testkit/src/main/java/org/apache/calcite/test/CalciteAssert.java @@ -747,12 +747,12 @@ static int countRows(ResultSet resultSet) throws SQLException { return n; } - static Collection toStringList(ResultSet resultSet, + public static Collection toStringList(ResultSet resultSet, Collection list) throws SQLException { return new ResultSetFormatter().toStringList(resultSet, list); } - static List toList(ResultSet resultSet) throws SQLException { + public static List toList(ResultSet resultSet) throws SQLException { return (List) toStringList(resultSet, new ArrayList()); } diff --git a/testkit/src/main/java/org/apache/calcite/test/Matchers.java b/testkit/src/main/java/org/apache/calcite/test/Matchers.java index 2779a8ecef60..ce8941fc1e4e 100644 --- a/testkit/src/main/java/org/apache/calcite/test/Matchers.java +++ b/testkit/src/main/java/org/apache/calcite/test/Matchers.java @@ -25,6 +25,7 @@ import org.apache.calcite.util.TestUtil; import org.apache.calcite.util.Util; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.RangeSet; @@ -84,10 +85,15 @@ private Matchers() {} * returnsUnordered("empno=1234", "empno=100")); */ public static Matcher returnsUnordered(String... lines) { + return returnsUnordered(ImmutableList.copyOf(lines)); + } + + /** As {@link #returnsUnordered(String...)}, but argument is a list. */ + public static Matcher returnsUnordered(List lines) { final List expectedList = Lists.newArrayList(lines); Collections.sort(expectedList); - return new CustomTypeSafeMatcher(Arrays.toString(lines)) { + return new CustomTypeSafeMatcher(lines.toString()) { @Override protected void describeMismatchSafely(ResultSet item, Description description) { final Object value = THREAD_ACTUAL.get();