From b629cdb5664f9e2ae2205ade6135198fa6fc76a1 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Thu, 23 Mar 2023 22:12:48 -0400 Subject: [PATCH 01/27] Add support for setting column type in Clickhouse. --- .../java/io/trino/plugin/clickhouse/ClickHouseClient.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java index 34301bbaa6b75..8cbe95dd07340 100644 --- a/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java +++ b/plugin/trino-clickhouse/src/main/java/io/trino/plugin/clickhouse/ClickHouseClient.java @@ -460,7 +460,12 @@ public void setColumnComment(ConnectorSession session, JdbcTableHandle handle, J @Override public void setColumnType(ConnectorSession session, JdbcTableHandle handle, JdbcColumnHandle column, Type type) { - throw new TrinoException(NOT_SUPPORTED, "This connector does not support setting column types"); + String sql = format( + "ALTER TABLE %s MODIFY COLUMN %s %s", + quoted(handle.asPlainTable().getRemoteTableName()), + quoted(column.getColumnName()), + toWriteMapping(session, type).getDataType()); + execute(session, sql); } private static String clickhouseVarcharLiteral(String value) From 272419ce2463977ae926ecb9ad8cc3406f7c20ef Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 25 Mar 2023 17:03:37 -0400 Subject: [PATCH 02/27] Added test to validate MODIFY COLUMN DATA TYPE does not affect modifier. --- .../io/trino/testing/BaseConnectorTest.java | 106 ++++-------------- 1 file changed, 24 insertions(+), 82 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 8192b0a098f55..35b34fb641285 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -52,22 +52,8 @@ import java.time.Instant; import java.time.ZonedDateTime; import java.time.temporal.ChronoUnit; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Deque; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.OptionalInt; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.CyclicBarrier; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.*; +import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -90,55 +76,16 @@ import static io.trino.connector.informationschema.InformationSchemaTable.INFORMATION_SCHEMA; import static io.trino.server.testing.TestingSystemSessionProperties.TESTING_SESSION_TIME; import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE; -import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.FRESH; -import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.STALE; -import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.UNKNOWN; +import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.*; import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; -import static io.trino.sql.planner.assertions.PlanMatchPattern.node; -import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; +import static io.trino.sql.planner.assertions.PlanMatchPattern.*; import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static io.trino.sql.planner.planprinter.PlanPrinter.textLogicalPlan; import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.QueryAssertions.assertContains; import static io.trino.testing.QueryAssertions.getTrinoExceptionCause; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN_WITH_COMMENT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ARRAY; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_TABLE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_VIEW; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_VIEW_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_FEDERATED_MATERIALIZED_VIEW; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_MATERIALIZED_VIEW; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_MATERIALIZED_VIEW_GRACE_PERIOD; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_SCHEMA; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_VIEW; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DELETE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DROP_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DROP_FIELD; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_MERGE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_MULTI_STATEMENT_WRITES; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NEGATIVE_DATE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_MATERIALIZED_VIEW; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_MATERIALIZED_VIEW_ACROSS_SCHEMAS; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_SCHEMA; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_TABLE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ROW_LEVEL_DELETE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ROW_TYPE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TOPN_PUSHDOWN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TRUNCATE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_UPDATE; +import static io.trino.testing.TestingConnectorBehavior.*; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.assertions.Assert.assertEventually; import static io.trino.testing.assertions.TestUtil.verifyResultOrFailure; @@ -150,20 +97,13 @@ import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newFixedThreadPool; -import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static java.util.concurrent.TimeUnit.MINUTES; -import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.concurrent.TimeUnit.*; import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.InstanceOfAssertFactories.ZONED_DATE_TIME; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertNull; -import static org.testng.Assert.assertTrue; -import static org.testng.Assert.fail; +import static org.testng.Assert.*; /** * Generic test for connectors. @@ -2625,23 +2565,25 @@ public void testRenameColumn() } @Test - public void testSetColumnType() - { - if (!hasBehavior(SUPPORTS_SET_COLUMN_TYPE)) { - assertQueryFails("ALTER TABLE nation ALTER COLUMN nationkey SET DATA TYPE bigint", "This connector does not support setting column types"); - return; - } - + public void testSetColumnType() { skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", "AS SELECT CAST(123 AS integer) AS col")) { - assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE bigint"); - - assertEquals(getColumnType(table.getName(), "col"), "bigint"); - assertThat(query("SELECT * FROM " + table.getName())) - .skippingTypesCheck() - .matches("VALUES bigint '123'"); - } + String tableName = "test_set_column_type"; + assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double NOT NULL, c varchar(50)) WITH (order_by=ARRAY['b'], engine = 'MergeTree')"); + + assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN a SET DATA TYPE varchar(50)"); + + assertEquals(getColumnType(tableName, "a"), "varchar"); + assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE clickhouse.tpch.test_set_column_type (\n" + + " a varchar NOT NULL,\n" + + " b double NOT NULL,\n" + + " c varchar\n" + + ")\n" + + "WITH (\n" + + " engine = 'MERGETREE',\n" + + " order_by = ARRAY['b'],\n" + + " primary_key = ARRAY['b']\n" + + ")"); } @Test(dataProvider = "setColumnTypesDataProvider") From 94a643c0a1dbfb450a271ce09c76d5149207afa9 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 25 Mar 2023 17:16:59 -0400 Subject: [PATCH 03/27] Fixed wildcard imports. --- .../io/trino/testing/BaseConnectorTest.java | 75 +++++++++++++++++-- 1 file changed, 67 insertions(+), 8 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 35b34fb641285..3c65eeac2b086 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -52,8 +52,22 @@ import java.time.Instant; import java.time.ZonedDateTime; import java.time.temporal.ChronoUnit; -import java.util.*; -import java.util.concurrent.*; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Deque; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.OptionalInt; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -76,16 +90,55 @@ import static io.trino.connector.informationschema.InformationSchemaTable.INFORMATION_SCHEMA; import static io.trino.server.testing.TestingSystemSessionProperties.TESTING_SESSION_TIME; import static io.trino.spi.connector.ConnectorMetadata.MODIFYING_ROWS_MESSAGE; -import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.*; +import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.FRESH; +import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.STALE; +import static io.trino.spi.connector.MaterializedViewFreshness.Freshness.UNKNOWN; import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.sql.planner.assertions.PlanMatchPattern.*; +import static io.trino.sql.planner.assertions.PlanMatchPattern.anyTree; +import static io.trino.sql.planner.assertions.PlanMatchPattern.node; +import static io.trino.sql.planner.assertions.PlanMatchPattern.tableScan; import static io.trino.sql.planner.optimizations.PlanNodeSearcher.searchFrom; import static io.trino.sql.planner.planprinter.PlanPrinter.textLogicalPlan; import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.QueryAssertions.assertContains; import static io.trino.testing.QueryAssertions.getTrinoExceptionCause; -import static io.trino.testing.TestingConnectorBehavior.*; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN_WITH_COMMENT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ARRAY; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_COLUMN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_TABLE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_VIEW; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_COMMENT_ON_VIEW_COLUMN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_FEDERATED_MATERIALIZED_VIEW; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_MATERIALIZED_VIEW; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_MATERIALIZED_VIEW_GRACE_PERIOD; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_SCHEMA; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_VIEW; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DELETE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DROP_COLUMN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_DROP_FIELD; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_MERGE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_MULTI_STATEMENT_WRITES; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NEGATIVE_DATE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_COLUMN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_MATERIALIZED_VIEW; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_MATERIALIZED_VIEW_ACROSS_SCHEMAS; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_SCHEMA; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_TABLE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_RENAME_TABLE_ACROSS_SCHEMAS; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ROW_LEVEL_DELETE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ROW_TYPE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TOPN_PUSHDOWN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_TRUNCATE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_UPDATE; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.assertions.Assert.assertEventually; import static io.trino.testing.assertions.TestUtil.verifyResultOrFailure; @@ -97,14 +150,20 @@ import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newFixedThreadPool; -import static java.util.concurrent.TimeUnit.*; +import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.MINUTES; +import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.joining; import static java.util.stream.Collectors.toList; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.InstanceOfAssertFactories.ZONED_DATE_TIME; -import static org.testng.Assert.*; - +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import static org.testng.Assert.fail; /** * Generic test for connectors. */ From 170546a9ae421a71a034a489ede818d857337b0e Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 29 Mar 2023 19:38:25 -0400 Subject: [PATCH 04/27] Removed flag to disble column data type --- .../java/io/trino/testing/BaseConnectorTest.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 3c65eeac2b086..39422bb56595b 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -2813,10 +2813,14 @@ public void testSetColumnOutOfRangeType() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_invalid_range_", "AS SELECT CAST(9223372036854775807 AS bigint) AS col")) { - assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) - .satisfies(this::verifySetColumnTypeFailurePermissible); - } + String tableName = "test_set_column_type"; + assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); + assertUpdate("insert into " + tableName + " values(9223372036854775807, 22)", "SELECT count(*) FROM " + tableName); +//ass("insert into " + tableName + " values(44, 22)"); + +// assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) +// .satisfies(this::verifySetColumnTypeFailurePermissible); +// } } protected void verifySetColumnTypeFailurePermissible(Throwable e) From 0d6e541aeddbb833a1b7751f50242c9816069208 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 29 Mar 2023 19:46:44 -0400 Subject: [PATCH 05/27] Reverted back changes to BaseConnectorTest --- .../io/trino/testing/BaseConnectorTest.java | 53 +++++++++---------- 1 file changed, 24 insertions(+), 29 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 39422bb56595b..8aa6fc14e950c 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -164,6 +164,7 @@ import static org.testng.Assert.assertNull; import static org.testng.Assert.assertTrue; import static org.testng.Assert.fail; + /** * Generic test for connectors. */ @@ -187,8 +188,8 @@ protected MockConnectorPlugin buildMockConnectorPlugin() MockConnectorFactory connectorFactory = MockConnectorFactory.builder() .withListSchemaNames(session -> ImmutableList.copyOf(mockTableListings.keySet())) .withListTables((session, schemaName) -> - verifyNotNull(mockTableListings.get(schemaName), "No listing function registered for [%s]", schemaName) - .apply(session)) + verifyNotNull(mockTableListings.get(schemaName), "No listing function registered for [%s]", schemaName) + .apply(session)) .build(); return new MockConnectorPlugin(connectorFactory); } @@ -1640,9 +1641,9 @@ public void testViewMetadata(String securityClauseInCreate, String securityClaus // test SHOW COLUMNS assertThat(query("SHOW COLUMNS FROM " + viewName)) .matches(resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("x", "bigint", "", "") - .row("y", "varchar(3)", "", "") - .build()); + .row("x", "bigint", "", "") + .row("y", "varchar(3)", "", "") + .build()); // test SHOW CREATE VIEW String expectedSql = formatSqlText(format( @@ -2624,25 +2625,23 @@ public void testRenameColumn() } @Test - public void testSetColumnType() { - skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); + public void testSetColumnType() + { + if (!hasBehavior(SUPPORTS_SET_COLUMN_TYPE)) { + assertQueryFails("ALTER TABLE nation ALTER COLUMN nationkey SET DATA TYPE bigint", "This connector does not support setting column types"); + return; + } - String tableName = "test_set_column_type"; - assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double NOT NULL, c varchar(50)) WITH (order_by=ARRAY['b'], engine = 'MergeTree')"); + skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN a SET DATA TYPE varchar(50)"); + try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", "AS SELECT CAST(123 AS integer) AS col")) { + assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE bigint"); - assertEquals(getColumnType(tableName, "a"), "varchar"); - assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE clickhouse.tpch.test_set_column_type (\n" + - " a varchar NOT NULL,\n" + - " b double NOT NULL,\n" + - " c varchar\n" + - ")\n" + - "WITH (\n" + - " engine = 'MERGETREE',\n" + - " order_by = ARRAY['b'],\n" + - " primary_key = ARRAY['b']\n" + - ")"); + assertEquals(getColumnType(table.getName(), "col"), "bigint"); + assertThat(query("SELECT * FROM " + table.getName())) + .skippingTypesCheck() + .matches("VALUES bigint '123'"); + } } @Test(dataProvider = "setColumnTypesDataProvider") @@ -2813,14 +2812,10 @@ public void testSetColumnOutOfRangeType() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - String tableName = "test_set_column_type"; - assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); - assertUpdate("insert into " + tableName + " values(9223372036854775807, 22)", "SELECT count(*) FROM " + tableName); -//ass("insert into " + tableName + " values(44, 22)"); - -// assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) -// .satisfies(this::verifySetColumnTypeFailurePermissible); -// } + try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_invalid_range_", "AS SELECT CAST(9223372036854775807 AS bigint) AS col")) { + assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) + .satisfies(this::verifySetColumnTypeFailurePermissible); + } } protected void verifySetColumnTypeFailurePermissible(Throwable e) From c36a5cb3bd669135393812e484cb9534e8f11039 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 29 Mar 2023 19:49:26 -0400 Subject: [PATCH 06/27] Reverted back changes to BaseConnectorTest --- .../main/java/io/trino/testing/BaseConnectorTest.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 8aa6fc14e950c..8192b0a098f55 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -188,8 +188,8 @@ protected MockConnectorPlugin buildMockConnectorPlugin() MockConnectorFactory connectorFactory = MockConnectorFactory.builder() .withListSchemaNames(session -> ImmutableList.copyOf(mockTableListings.keySet())) .withListTables((session, schemaName) -> - verifyNotNull(mockTableListings.get(schemaName), "No listing function registered for [%s]", schemaName) - .apply(session)) + verifyNotNull(mockTableListings.get(schemaName), "No listing function registered for [%s]", schemaName) + .apply(session)) .build(); return new MockConnectorPlugin(connectorFactory); } @@ -1641,9 +1641,9 @@ public void testViewMetadata(String securityClauseInCreate, String securityClaus // test SHOW COLUMNS assertThat(query("SHOW COLUMNS FROM " + viewName)) .matches(resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("x", "bigint", "", "") - .row("y", "varchar(3)", "", "") - .build()); + .row("x", "bigint", "", "") + .row("y", "varchar(3)", "", "") + .build()); // test SHOW CREATE VIEW String expectedSql = formatSqlText(format( From 21c325d86f6bd0682d0310fa7d41f6fe2f4c93ff Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 8 Apr 2023 11:56:43 -0400 Subject: [PATCH 07/27] Override tests in ClickHouseConnector for MODIFY DATA TYPE --- .../BaseClickHouseConnectorTest.java | 134 +++++++++++++++++- .../io/trino/testing/BaseConnectorTest.java | 4 +- 2 files changed, 132 insertions(+), 6 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java index 5a8284ec9c57f..9ac5fa2158787 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java @@ -43,6 +43,7 @@ import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingConnectorBehavior.*; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -57,6 +58,7 @@ public abstract class BaseClickHouseConnectorTest { protected TestingClickHouseServer clickhouseServer; + @SuppressWarnings("DuplicateBranchesInSwitch") @Override protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) @@ -67,9 +69,6 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) case SUPPORTS_TOPN_PUSHDOWN: return false; - case SUPPORTS_SET_COLUMN_TYPE: - return false; - case SUPPORTS_DELETE: return false; @@ -284,7 +283,7 @@ protected TestTable createTableWithDefaultColumns() "col_nullable Nullable(Int64)," + "col_default Nullable(Int64) DEFAULT 43," + "col_nonnull_default Int64 DEFAULT 42," + - "col_required2 Int64) ENGINE=Log"); + "col_required2 Int64) ENGINE=MergeTree() order by col_required2"); } @Override @@ -885,4 +884,131 @@ private Map getTableProperties(String schemaName, String tableNa return properties.buildOrThrow(); } } + + @Test + @Override + public void testSetColumnType() + { + skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); + String tableName = "test_set_column_type"; + assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double NOT NULL, c varchar(50)) WITH (order_by=ARRAY['b'], engine = 'MergeTree')"); + assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN a SET DATA TYPE varchar(50)"); + assertEquals(getColumnType(tableName, "a"), "varchar"); + assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE clickhouse.tpch.test_set_column_type (\n" + + " a varchar NOT NULL,\n" + + " b double NOT NULL,\n" + + " c varchar\n" + + ")\n" + + "WITH (\n" + + " engine = 'MERGETREE',\n" + + " order_by = ARRAY['b'],\n" + + " primary_key = ARRAY['b']\n" + + ")"); + } + @Test + @Override + public void testSetColumnIncompatibleType() + { + skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); + + String tableName = "test_set_column_incompatible_type"; + + assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); + assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer")) + .satisfies(this::verifySetColumnTypeFailurePermissible); + } + + @Test + @Override + public void testSetColumnOutOfRangeType() + { + skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); + + String tableName = "test_set_column_out_of_range"; + assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); + assertUpdate("insert into " + tableName + " values(9223372036854775807, 22)", "SELECT count(*) FROM " + tableName); + +// try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_invalid_range_", "AS SELECT CAST(9223372036854775807 AS bigint) AS col")) { +// assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) +// .satisfies(this::verifySetColumnTypeFailurePermissible); +// } + } + + @Test + @Override + public void testSetColumnTypeWithComment() + { + skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); + String tableName = "test_set_column_with_comment"; + + assertUpdate("CREATE TABLE " + tableName + " (col bigint COMMENT 'test comment', col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); + assertEquals(getColumnComment(tableName, "col"), "test comment"); + + assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE bigint"); + assertEquals(getColumnComment(tableName, "col"), "test comment"); + } + + @Test + @Override + public void testSetColumnTypeWithDefaultColumn() + { + skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_INSERT)); + try (TestTable table = createTableWithDefaultColumns()) { + // col_default column inserts 43 by default + assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col_default SET DATA TYPE bigint"); + assertUpdate("INSERT INTO " + table.getName() + " (col_required, col_required2) VALUES (1, 10)", 1); + assertQuery("SELECT col_default FROM " + table.getName(), "VALUES 43"); + } + } + + @Test + @Override + public void testSetColumnTypeWithNotNull() + { + skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); + + String tableName = "test_set_column_with_not_null"; + + assertUpdate("CREATE TABLE " + tableName + " (col bigint not null, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); + assertFalse(columnIsNullable(tableName, "col")); + + assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE bigint"); + assertFalse(columnIsNullable(tableName, "col")); + } + + @Test(dataProvider = "setColumnTypesDataProvider") + @Override + public void testSetColumnTypes(SetColumnTypeSetup setup) + { + skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); + + //String tableName = "test_set_column_type_with_data_provider"; + + // String tableDefinition = "CREATE TABLE " + tableName + "(AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col , col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"; + + // assertUpdate(tableDefinition); + + TestTable table; + try { + table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", " AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col"); + } + catch (Exception e) { + verifyUnsupportedTypeException(e, setup.sourceColumnType()); + throw new SkipException("Unsupported column type: " + setup.sourceColumnType()); + } + try (table) { + Runnable setColumnType = () -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE " + setup.newColumnType()); + if (setup.unsupportedType()) { + assertThatThrownBy(setColumnType::run) + .satisfies(this::verifySetColumnTypeFailurePermissible); + return; + } + setColumnType.run(); + + assertEquals(getColumnType(table.getName(), "col"), setup.newColumnType()); + assertThat(query("SELECT * FROM " + table.getName())) + .skippingTypesCheck() + .matches("SELECT " + setup.newValueLiteral()); + } + } } diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 8192b0a098f55..e5675c3838776 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -2823,7 +2823,7 @@ protected void verifySetColumnTypeFailurePermissible(Throwable e) throw new AssertionError("Unexpected set column type failure", e); } - private String getColumnType(String tableName, String columnName) + protected String getColumnType(String tableName, String columnName) { return (String) computeScalar(format("SELECT data_type FROM information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = '%s' AND column_name = '%s'", tableName, @@ -5725,7 +5725,7 @@ public void testMergeAllColumnsReversed() assertUpdate("DROP TABLE " + targetTable); } - private void verifyUnsupportedTypeException(Throwable exception, String trinoTypeName) + protected void verifyUnsupportedTypeException(Throwable exception, String trinoTypeName) { String typeNameBase = trinoTypeName.replaceFirst("\\(.*", ""); String expectedMessagePart = format("(%1$s.*not (yet )?supported)|((?i)unsupported.*%1$s)|((?i)not supported.*%1$s)", Pattern.quote(typeNameBase)); From e7dd6527fa3f0b8b168ab80916795649c0545b56 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 8 Apr 2023 16:39:01 -0400 Subject: [PATCH 08/27] Fix Clickhouse MODIFY DATA TYPE tests --- .../clickhouse/BaseClickHouseConnectorTest.java | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java index 9ac5fa2158787..8ddb5934d9a3b 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java @@ -914,8 +914,8 @@ public void testSetColumnIncompatibleType() String tableName = "test_set_column_incompatible_type"; assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); - assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer")) - .satisfies(this::verifySetColumnTypeFailurePermissible); + assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer"); + assertEquals(getColumnType(tableName, "col"), "integer"); } @Test @@ -926,12 +926,11 @@ public void testSetColumnOutOfRangeType() String tableName = "test_set_column_out_of_range"; assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); - assertUpdate("insert into " + tableName + " values(9223372036854775807, 22)", "SELECT count(*) FROM " + tableName); + query("insert into " + tableName + " values(9223372036854775807, 22)"); + assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer"); -// try (TestTable table = new TestTable(getQueryRunner()::execute, "test_set_column_type_invalid_range_", "AS SELECT CAST(9223372036854775807 AS bigint) AS col")) { -// assertThatThrownBy(() -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE integer")) -// .satisfies(this::verifySetColumnTypeFailurePermissible); -// } + assertEquals(getColumnType(tableName, "col"), "integer"); + assertQuery("SELECT col FROM " + tableName, "VALUES -1"); } @Test From a6d518be73821d20df82cfd70b092f84930c8e10 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 8 Apr 2023 16:40:21 -0400 Subject: [PATCH 09/27] Fix Clickhouse MODIFY DATA TYPE tests --- .../plugin/clickhouse/BaseClickHouseConnectorTest.java | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java index 8ddb5934d9a3b..f081ad81d0bfa 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/BaseClickHouseConnectorTest.java @@ -980,13 +980,7 @@ public void testSetColumnTypeWithNotNull() public void testSetColumnTypes(SetColumnTypeSetup setup) { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - - //String tableName = "test_set_column_type_with_data_provider"; - - // String tableDefinition = "CREATE TABLE " + tableName + "(AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col , col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"; - - // assertUpdate(tableDefinition); - + TestTable table; try { table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", " AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col"); From b44596aae2f8be809336a9c975d0497b623105bf Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 8 Apr 2023 17:19:51 -0400 Subject: [PATCH 10/27] Fixed missing imports --- .../io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java | 1 + 1 file changed, 1 insertion(+) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 448f85bc8d240..76f73d489ee71 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -46,6 +46,7 @@ import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingConnectorBehavior.*; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; From 10300f32bf1bbcb0e6764825beeee76dcb824c87 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 8 Apr 2023 20:40:48 -0400 Subject: [PATCH 11/27] Fix checkstyle errors --- .../trino/plugin/clickhouse/TestClickHouseConnectorTest.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 76f73d489ee71..1974344913eea 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -46,7 +46,8 @@ import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; -import static io.trino.testing.TestingConnectorBehavior.*; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -899,7 +900,6 @@ private Map getTableProperties(String schemaName, String tableNa } } - @Test @Override public void testSetColumnType() @@ -920,6 +920,7 @@ public void testSetColumnType() " primary_key = ARRAY['b']\n" + ")"); } + @Test @Override public void testSetColumnIncompatibleType() From 967601aa169bf1bc86c357aaeec9a967304f3e0a Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 8 Apr 2023 21:23:44 -0400 Subject: [PATCH 12/27] Fix checkstyle imports. --- .../plugin/clickhouse/TestClickHouseConnectorTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 1974344913eea..3b2b6ffb54723 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -21,6 +21,11 @@ import io.trino.testing.MaterializedResult; import io.trino.testing.QueryRunner; import io.trino.testing.TestingConnectorBehavior; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; import io.trino.testing.sql.SqlExecutor; import io.trino.testing.sql.TestTable; import org.testng.SkipException; @@ -46,8 +51,6 @@ import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; From a5e6bcf935ff50c6ca489fd8b9f702b8953c52f4 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sun, 9 Apr 2023 17:39:40 -0400 Subject: [PATCH 13/27] Fix order of imports. --- .../plugin/clickhouse/TestClickHouseConnectorTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 3b2b6ffb54723..f78d963177f4b 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -20,14 +20,14 @@ import io.trino.sql.planner.plan.AggregationNode; import io.trino.testing.MaterializedResult; import io.trino.testing.QueryRunner; +import io.trino.testing.sql.SqlExecutor; +import io.trino.testing.sql.TestTable; import io.trino.testing.TestingConnectorBehavior; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; -import io.trino.testing.sql.SqlExecutor; -import io.trino.testing.sql.TestTable; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; import org.testng.SkipException; import org.testng.annotations.Test; From bd81ef710cb41efa207d3bf6ab18d061070442a8 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Thu, 13 Apr 2023 18:43:25 -0400 Subject: [PATCH 14/27] Fix order of imports. --- .../clickhouse/TestClickHouseConnectorTest.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index f78d963177f4b..12642988c38de 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -20,14 +20,9 @@ import io.trino.sql.planner.plan.AggregationNode; import io.trino.testing.MaterializedResult; import io.trino.testing.QueryRunner; +import io.trino.testing.TestingConnectorBehavior; import io.trino.testing.sql.SqlExecutor; import io.trino.testing.sql.TestTable; -import io.trino.testing.TestingConnectorBehavior; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; import org.testng.SkipException; import org.testng.annotations.Test; @@ -51,6 +46,11 @@ import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; +import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; import static org.assertj.core.api.Assertions.assertThat; @@ -1002,7 +1002,7 @@ public void testSetColumnTypes(SetColumnTypeSetup setup) TestTable table; try { - table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", " AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col"); + table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", "WITH (engine='mergetree',order_by=ARRAY['col2']) AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col, CAST(123 as int) AS col2"); } catch (Exception e) { verifyUnsupportedTypeException(e, setup.sourceColumnType()); From 12f56562613d2007839e366763e4740cf2294e04 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Tue, 25 Apr 2023 17:04:03 -0400 Subject: [PATCH 15/27] Override testSetColumnTypes function to test alter modify column for different data types. --- .../TestClickHouseConnectorTest.java | 33 ++++++++----------- 1 file changed, 13 insertions(+), 20 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 12642988c38de..394006e3582ba 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -1000,27 +1000,20 @@ public void testSetColumnTypes(SetColumnTypeSetup setup) { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - TestTable table; - try { - table = new TestTable(getQueryRunner()::execute, "test_set_column_type_", "WITH (engine='mergetree',order_by=ARRAY['col2']) AS SELECT CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + ") AS col, CAST(123 as int) AS col2"); + String tableName = "test_set_column_type_" + System.currentTimeMillis(); + assertUpdate("CREATE TABLE " + tableName + "(col " + setup.sourceColumnType() + ", col2 int not null) WITH (engine='mergetree', order_by=ARRAY['col2'])"); + query("insert into " + tableName + "(col, col2) values(CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + "), 2)"); + Runnable setColumnType = () -> assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE " + setup.newColumnType()); + if (setup.unsupportedType()) { + assertThatThrownBy(setColumnType::run) + .satisfies(this::verifySetColumnTypeFailurePermissible); + return; } - catch (Exception e) { - verifyUnsupportedTypeException(e, setup.sourceColumnType()); - throw new SkipException("Unsupported column type: " + setup.sourceColumnType()); - } - try (table) { - Runnable setColumnType = () -> assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col SET DATA TYPE " + setup.newColumnType()); - if (setup.unsupportedType()) { - assertThatThrownBy(setColumnType::run) - .satisfies(this::verifySetColumnTypeFailurePermissible); - return; - } - setColumnType.run(); + setColumnType.run(); - assertEquals(getColumnType(table.getName(), "col"), setup.newColumnType()); - assertThat(query("SELECT * FROM " + table.getName())) - .skippingTypesCheck() - .matches("SELECT " + setup.newValueLiteral()); - } + assertEquals(getColumnType(tableName, "col"), setup.newColumnType()); + assertThat(query("SELECT col FROM " + tableName)) + .skippingTypesCheck() + .matches("SELECT " + setup.newValueLiteral()); } } From f29f6c324cb63ed3864de80495632055fb3c014f Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 26 Apr 2023 19:41:59 -0400 Subject: [PATCH 16/27] Fixed checkstyle errors. --- .../plugin/clickhouse/TestClickHouseConnectorTest.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 927d105111938..1066913a052ff 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -1015,6 +1015,7 @@ public void testSetColumnTypeWithNotNull() } @DataProvider + @Override public Object[][] setColumnTypesDataProvider() { return setColumnTypeSetupData().stream() @@ -1055,8 +1056,8 @@ public void testSetColumnTypes(SetColumnTypeSetup setup) { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - String tableName = "test_set_column_type_" + System.currentTimeMillis(); - assertUpdate("CREATE TABLE " + tableName + "(col " + setup.sourceColumnType() + ", col2 int not null) WITH (engine='mergetree', order_by=ARRAY['col2'])"); + String tableName = "test_set_column_type_" + System.currentTimeMillis(); + assertUpdate("CREATE TABLE " + tableName + "(col " + setup.sourceColumnType() + ", col2 int not null) WITH (engine='mergetree', order_by=ARRAY['col2'])"); query("insert into " + tableName + "(col, col2) values(CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + "), 2)"); Runnable setColumnType = () -> assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE " + setup.newColumnType()); if (setup.unsupportedType()) { @@ -1078,7 +1079,7 @@ public void testAddNotNullColumnToEmptyTable() { skipTestUnless(hasBehavior(SUPPORTS_ADD_COLUMN)); - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_to_empty", "(a_varchar varchar, b varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['b'])")) { + try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_to_empty", "(a_varchar varchar, b varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['b'])")) { String tableName = table.getName(); String addNonNullColumn = "ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"; From d0682ea9523539a6ac025fdfca919aff69307e32 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 26 Apr 2023 21:13:12 -0400 Subject: [PATCH 17/27] Reverted back commented out tests. --- .../TestClickHouseConnectorTest.java | 104 ++++++------------ 1 file changed, 33 insertions(+), 71 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 1066913a052ff..e5535011fb7c1 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -92,9 +92,6 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior) case SUPPORTS_AGGREGATION_PUSHDOWN_COUNT_DISTINCT: return false; - case SUPPORTS_SET_COLUMN_TYPE: - return true; - case SUPPORTS_ARRAY: case SUPPORTS_ROW_TYPE: case SUPPORTS_NEGATIVE_DATE: @@ -225,24 +222,38 @@ protected String tableDefinitionForAddColumn() return "(x VARCHAR NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['x'])"; } -// @Override -// public void testAddNotNullColumnToNonEmptyTable() -// { -// // Override because the default storage type doesn't support adding columns -// try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_notnull_col", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { -// String tableName = table.getName(); -// -// assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"); -// assertFalse(columnIsNullable(tableName, "b_varchar")); -// -// assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 'b')", 1); -// -// // ClickHouse set an empty character as the default value -// assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN c_varchar varchar NOT NULL"); -// assertFalse(columnIsNullable(tableName, "c_varchar")); -// assertQuery("SELECT c_varchar FROM " + tableName, "VALUES ''"); -// } -// } + @Override // Overridden because the default storage type doesn't support adding columns + public void testAddNotNullColumnToEmptyTable() + { + try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_notnull_col_to_empty", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { + String tableName = table.getName(); + + assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"); + assertFalse(columnIsNullable(tableName, "b_varchar")); + assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 'b')", 1); + assertThat(query("TABLE " + tableName)) + .skippingTypesCheck() + .matches("VALUES ('a', 'b')"); + } + } + + @Override // Overridden because (a) the default storage type doesn't support adding columns and (b) ClickHouse has implicit default value for new NON NULL column + public void testAddNotNullColumn() + { + try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_notnull_col", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { + String tableName = table.getName(); + + assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"); + assertFalse(columnIsNullable(tableName, "b_varchar")); + + assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 'b')", 1); + + // ClickHouse set an empty character as the default value + assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN c_varchar varchar NOT NULL"); + assertFalse(columnIsNullable(tableName, "c_varchar")); + assertQuery("SELECT c_varchar FROM " + tableName, "VALUES ''"); + } + } @Test @Override @@ -321,7 +332,7 @@ protected TestTable createTableWithDefaultColumns() "col_nullable Nullable(Int64)," + "col_default Nullable(Int64) DEFAULT 43," + "col_nonnull_default Int64 DEFAULT 42," + - "col_required2 Int64) ENGINE=MergeTree() order by col_required2"); + "col_required2 Int64) ENGINE=MergeTree() ORDER BY col_required2"); } @Override @@ -927,7 +938,6 @@ private Map getTableProperties(String schemaName, String tableNa @Override public void testSetColumnType() { - skipTestUnless(hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); String tableName = "test_set_column_type"; assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double NOT NULL, c varchar(50)) WITH (order_by=ARRAY['b'], engine = 'MergeTree')"); assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN a SET DATA TYPE varchar(50)"); @@ -1072,52 +1082,4 @@ public void testSetColumnTypes(SetColumnTypeSetup setup) .skippingTypesCheck() .matches("SELECT " + setup.newValueLiteral()); } - - @Test - @Override - public void testAddNotNullColumnToEmptyTable() - { - skipTestUnless(hasBehavior(SUPPORTS_ADD_COLUMN)); - - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_nn_to_empty", "(a_varchar varchar, b varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['b'])")) { - String tableName = table.getName(); - String addNonNullColumn = "ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"; - - if (!hasBehavior(SUPPORTS_ADD_COLUMN_NOT_NULL_CONSTRAINT)) { - assertQueryFails( - addNonNullColumn, - hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT) - ? "This connector does not support adding not null columns" - : ".* Catalog '.*' does not support NOT NULL for column '.*'"); - return; - } - - assertUpdate(addNonNullColumn); - assertFalse(columnIsNullable(tableName, "b_varchar")); - assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 'b', 'c')", 1); - assertThat(query("TABLE " + tableName)) - .skippingTypesCheck() - .matches("VALUES ('a', 'b', 'c')"); - } - } - - @Test - @Override - public void testAddNotNullColumn() - { - // Override because the default storage type doesn't support adding columns - try (TestTable table = new TestTable(getQueryRunner()::execute, "test_add_notnull_col", "(a_varchar varchar NOT NULL) WITH (engine = 'MergeTree', order_by = ARRAY['a_varchar'])")) { - String tableName = table.getName(); - - assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN b_varchar varchar NOT NULL"); - assertFalse(columnIsNullable(tableName, "b_varchar")); - - assertUpdate("INSERT INTO " + tableName + " VALUES ('a', 'b')", 1); - - // ClickHouse set an empty character as the default value - assertUpdate("ALTER TABLE " + tableName + " ADD COLUMN c_varchar varchar NOT NULL"); - assertFalse(columnIsNullable(tableName, "c_varchar")); - assertQuery("SELECT c_varchar FROM " + tableName, "VALUES ''"); - } - } } From fd18ba2455b907fc77288b427cffce1692d5688f Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Wed, 26 Apr 2023 21:19:22 -0400 Subject: [PATCH 18/27] Reverted back changes. --- .../TestClickHouseConnectorTest.java | 109 ------------------ 1 file changed, 109 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index e5535011fb7c1..b78bf03ea640f 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -693,115 +693,6 @@ public void testCharTrailingSpace() throw new SkipException("Implement test for ClickHouse"); } - @Override - public void testNativeQuerySimple() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertQueryFails("SELECT * FROM TABLE(system.query(query => 'SELECT 1'))", "line 1:21: Table function system.query not registered"); - } - - @Override - public void testNativeQueryParameters() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - Session session = Session.builder(getSession()) - .addPreparedStatement("my_query_simple", "SELECT * FROM TABLE(system.query(query => ?))") - .addPreparedStatement("my_query", "SELECT * FROM TABLE(system.query(query => format('SELECT %s FROM %s', ?, ?)))") - .build(); - assertQueryFails(session, "EXECUTE my_query_simple USING 'SELECT 1 a'", "line 1:21: Table function system.query not registered"); - assertQueryFails(session, "EXECUTE my_query USING 'a', '(SELECT 2 a) t'", "line 1:21: Table function system.query not registered"); - } - - @Override - public void testNativeQuerySelectFromNation() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertQueryFails( - format("SELECT * FROM TABLE(system.query(query => 'SELECT name FROM %s.nation WHERE nationkey = 0'))", getSession().getSchema().orElseThrow()), - "line 1:21: Table function system.query not registered"); - } - - @Override - public void testNativeQuerySelectFromTestTable() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - try (TestTable testTable = simpleTable()) { - assertQueryFails( - format("SELECT * FROM TABLE(system.query(query => 'SELECT * FROM %s'))", testTable.getName()), - "line 1:21: Table function system.query not registered"); - } - } - - @Override - public void testNativeQueryColumnAlias() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertQueryFails( - "SELECT * FROM TABLE(system.query(query => 'SELECT name AS region_name FROM tpch.region WHERE regionkey = 0'))", - ".* Table function system.query not registered"); - } - - @Override - public void testNativeQueryColumnAliasNotFound() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertQueryFails( - "SELECT name FROM TABLE(system.query(query => 'SELECT name AS region_name FROM tpch.region'))", - ".* Table function system.query not registered"); - } - - @Override - public void testNativeQuerySelectUnsupportedType() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - try (TestTable testTable = createTableWithUnsupportedColumn()) { - String unqualifiedTableName = testTable.getName().replaceAll("^\\w+\\.", ""); - // Check that column 'two' is not supported. - assertQuery("SELECT column_name FROM information_schema.columns WHERE table_name = '" + unqualifiedTableName + "'", "VALUES 'one', 'three'"); - assertUpdate("INSERT INTO " + testTable.getName() + " (one, three) VALUES (123, 'test')", 1); - assertThatThrownBy(() -> query(format("SELECT * FROM TABLE(system.query(query => 'SELECT * FROM %s'))", testTable.getName()))) - .hasMessage("line 1:21: Table function system.query not registered"); - } - } - - @Override - public void testNativeQueryCreateStatement() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertFalse(getQueryRunner().tableExists(getSession(), "numbers")); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'CREATE TABLE numbers(n INTEGER)'))")) - .hasMessage("line 1:21: Table function system.query not registered"); - assertFalse(getQueryRunner().tableExists(getSession(), "numbers")); - } - - @Override - public void testNativeQueryInsertStatementTableDoesNotExist() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertFalse(getQueryRunner().tableExists(getSession(), "non_existent_table")); - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'INSERT INTO non_existent_table VALUES (1)'))")) - .hasMessage("line 1:21: Table function system.query not registered"); - } - - @Override - public void testNativeQueryInsertStatementTableExists() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - try (TestTable testTable = simpleTable()) { - assertThatThrownBy(() -> query(format("SELECT * FROM TABLE(system.query(query => 'INSERT INTO %s VALUES (3)'))", testTable.getName()))) - .hasMessage("line 1:21: Table function system.query not registered"); - assertQuery("SELECT * FROM " + testTable.getName(), "VALUES 1, 2"); - } - } - - @Override - public void testNativeQueryIncorrectSyntax() - { - // table function disabled for ClickHouse, because it doesn't provide ResultSetMetaData, so the result relation type cannot be determined - assertThatThrownBy(() -> query("SELECT * FROM TABLE(system.query(query => 'some wrong syntax'))")) - .hasMessage("line 1:21: Table function system.query not registered"); - } - @Override protected TestTable simpleTable() { From 42ac5a6b990eba6e82c362a49791f31126d4103e Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 29 Apr 2023 17:30:01 -0400 Subject: [PATCH 19/27] Add randomNameSuffix to table name --- .../clickhouse/TestClickHouseConnectorTest.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index b78bf03ea640f..4b0bf23e5ce64 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -15,7 +15,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import io.trino.Session; import io.trino.plugin.jdbc.BaseJdbcConnectorTest; import io.trino.sql.planner.plan.AggregationNode; import io.trino.testing.MaterializedResult; @@ -48,8 +47,6 @@ import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.testing.MaterializedResult.resultBuilder; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_ADD_COLUMN_NOT_NULL_CONSTRAINT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; @@ -829,11 +826,11 @@ private Map getTableProperties(String schemaName, String tableNa @Override public void testSetColumnType() { - String tableName = "test_set_column_type"; + String tableName = "test_set_column_type" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double NOT NULL, c varchar(50)) WITH (order_by=ARRAY['b'], engine = 'MergeTree')"); assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN a SET DATA TYPE varchar(50)"); assertEquals(getColumnType(tableName, "a"), "varchar"); - assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE clickhouse.tpch.test_set_column_type (\n" + + assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE " + "clickhouse.tpch." + tableName +" (\n" + " a varchar NOT NULL,\n" + " b double NOT NULL,\n" + " c varchar\n" + @@ -851,7 +848,7 @@ public void testSetColumnIncompatibleType() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - String tableName = "test_set_column_incompatible_type"; + String tableName = "test_set_column_incompatible_type" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer"); @@ -864,7 +861,7 @@ public void testSetColumnOutOfRangeType() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - String tableName = "test_set_column_out_of_range"; + String tableName = "test_set_column_out_of_range" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); query("insert into " + tableName + " values(9223372036854775807, 22)"); assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer"); @@ -878,7 +875,7 @@ public void testSetColumnOutOfRangeType() public void testSetColumnTypeWithComment() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); - String tableName = "test_set_column_with_comment"; + String tableName = "test_set_column_with_comment" + randomNameSuffix();; assertUpdate("CREATE TABLE " + tableName + " (col bigint COMMENT 'test comment', col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); assertEquals(getColumnComment(tableName, "col"), "test comment"); @@ -906,7 +903,7 @@ public void testSetColumnTypeWithNotNull() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); - String tableName = "test_set_column_with_not_null"; + String tableName = "test_set_column_with_not_null" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (col bigint not null, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); assertFalse(columnIsNullable(tableName, "col")); @@ -925,6 +922,7 @@ public Object[][] setColumnTypesDataProvider() .collect(toDataProvider()); } + @Override protected Optional filterSetColumnTypesDataProvider(SetColumnTypeSetup setup) { return Optional.of(setup); From 9c18655984d2ca7cf0fdea948dcb4b880282e019 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 29 Apr 2023 18:25:07 -0400 Subject: [PATCH 20/27] ClickHouseConnectorTest - Move logic from setColumnTypesDataProvider to filterSetColumnTypesDataProvider --- .../TestClickHouseConnectorTest.java | 65 ++++++++++--------- 1 file changed, 33 insertions(+), 32 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 4b0bf23e5ce64..a879561013a52 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -912,41 +912,42 @@ public void testSetColumnTypeWithNotNull() assertFalse(columnIsNullable(tableName, "col")); } - @DataProvider - @Override - public Object[][] setColumnTypesDataProvider() - { - return setColumnTypeSetupData().stream() - .map(this::filterSetColumnTypesDataProvider) - .flatMap(Optional::stream) - .collect(toDataProvider()); - } - @Override protected Optional filterSetColumnTypesDataProvider(SetColumnTypeSetup setup) { - return Optional.of(setup); - } - - private List setColumnTypeSetupData() - { - return ImmutableList.builder() - .add(new SetColumnTypeSetup("tinyint", "TINYINT '127'", "smallint")) - .add(new SetColumnTypeSetup("smallint", "SMALLINT '32767'", "integer")) - .add(new SetColumnTypeSetup("integer", "2147483647", "bigint")) - .add(new SetColumnTypeSetup("bigint", "BIGINT '-2147483648'", "integer")) - .add(new SetColumnTypeSetup("real", "REAL '10.3'", "double")) - .add(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")) - .add(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(10,3)")) // short decimal -> short decimal - .add(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")) // long decimal -> long decimal - .add(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(38,3)")) // short decimal -> long decimal - .add(new SetColumnTypeSetup("decimal(5,3)", "12.340", "decimal(5,2)")) - .add(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")) - .add(new SetColumnTypeSetup("varchar(100)", "'shorten-varchar'", "varchar")) - .add(new SetColumnTypeSetup("char(25)", "'shorten-char'", "varchar")) - .add(new SetColumnTypeSetup("char(20)", "'char-to-varchar'", "varchar")) - .add(new SetColumnTypeSetup("varchar", "'varchar-to-char'", "varchar")) - .build(); + if (setup.sourceColumnType().equals("tinyint")) { + return Optional.of(new SetColumnTypeSetup("tinyint", "TINYINT '127'", "smallint")); + } else if(setup.sourceColumnType().equals("smallint")) { + return Optional.of(new SetColumnTypeSetup("smallint", "SMALLINT '32767'", "integer")); + } else if(setup.sourceColumnType().equals("integer")) { + return Optional.of(new SetColumnTypeSetup("integer", "2147483647", "bigint")); + } else if(setup.sourceColumnType().equals("bigint")) { + return Optional.of(new SetColumnTypeSetup("bigint", "BIGINT '-2147483648'", "integer")); + } else if(setup.sourceColumnType().equals("real")) { + return Optional.of(new SetColumnTypeSetup("real", "REAL '10.3'", "double")); + } else if(setup.sourceColumnType().equals("real")) { + return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); + } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(10,3)")); + } else if(setup.sourceColumnType().equals("decimal(28,3)")) { + return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); + } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(38,3)")); + } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.340", "decimal(5,2)")); + } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); + } else if(setup.sourceColumnType().equals("varchar(100)")) { + return Optional.of(new SetColumnTypeSetup("varchar(100)", "'shorten-varchar'", "varchar")); + } else if(setup.sourceColumnType().equals("char(25)")) { + return Optional.of(new SetColumnTypeSetup("char(25)", "'shorten-char'", "varchar")); + } else if(setup.sourceColumnType().equals("char(20)")) { + return Optional.of(new SetColumnTypeSetup("char(20)", "'char-to-varchar'", "varchar")); + } else if(setup.sourceColumnType().equals("varchar")) { + return Optional.of(new SetColumnTypeSetup("varchar", "'varchar-to-char'", "varchar")); + } + + return Optional.empty(); } @Test(dataProvider = "setColumnTypesDataProvider") From 90338751c04e62a0b5bec2f3c14aea925584fc55 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Mon, 1 May 2023 18:20:39 -0400 Subject: [PATCH 21/27] Fix checkstyle errors --- .../TestClickHouseConnectorTest.java | 51 +++++++++++-------- 1 file changed, 31 insertions(+), 20 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index a879561013a52..8c92758c50a2d 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -23,7 +23,6 @@ import io.trino.testing.sql.SqlExecutor; import io.trino.testing.sql.TestTable; import org.testng.SkipException; -import org.testng.annotations.DataProvider; import org.testng.annotations.Test; import java.sql.Connection; @@ -45,7 +44,6 @@ import static io.trino.plugin.clickhouse.TestingClickHouseServer.CLICKHOUSE_LATEST_IMAGE; import static io.trino.plugin.jdbc.JdbcMetadataSessionProperties.DOMAIN_COMPACTION_THRESHOLD; import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.testing.DataProviders.toDataProvider; import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; @@ -830,7 +828,7 @@ public void testSetColumnType() assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double NOT NULL, c varchar(50)) WITH (order_by=ARRAY['b'], engine = 'MergeTree')"); assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN a SET DATA TYPE varchar(50)"); assertEquals(getColumnType(tableName, "a"), "varchar"); - assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE " + "clickhouse.tpch." + tableName +" (\n" + + assertThat((String) computeScalar("show create table " + tableName)).contains("CREATE TABLE " + "clickhouse.tpch." + tableName + " (\n" + " a varchar NOT NULL,\n" + " b double NOT NULL,\n" + " c varchar\n" + @@ -865,7 +863,6 @@ public void testSetColumnOutOfRangeType() assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); query("insert into " + tableName + " values(9223372036854775807, 22)"); assertUpdate("ALTER TABLE " + tableName + " ALTER COLUMN col SET DATA TYPE integer"); - assertEquals(getColumnType(tableName, "col"), "integer"); assertQuery("SELECT col FROM " + tableName, "VALUES -1"); } @@ -875,7 +872,7 @@ public void testSetColumnOutOfRangeType() public void testSetColumnTypeWithComment() { skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT)); - String tableName = "test_set_column_with_comment" + randomNameSuffix();; + String tableName = "test_set_column_with_comment" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (col bigint COMMENT 'test comment', col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); assertEquals(getColumnComment(tableName, "col"), "test comment"); @@ -917,37 +914,51 @@ protected Optional filterSetColumnTypesDataProvider(SetColum { if (setup.sourceColumnType().equals("tinyint")) { return Optional.of(new SetColumnTypeSetup("tinyint", "TINYINT '127'", "smallint")); - } else if(setup.sourceColumnType().equals("smallint")) { + } + else if (setup.sourceColumnType().equals("smallint")) { return Optional.of(new SetColumnTypeSetup("smallint", "SMALLINT '32767'", "integer")); - } else if(setup.sourceColumnType().equals("integer")) { + } + else if (setup.sourceColumnType().equals("integer")) { return Optional.of(new SetColumnTypeSetup("integer", "2147483647", "bigint")); - } else if(setup.sourceColumnType().equals("bigint")) { + } + else if (setup.sourceColumnType().equals("bigint")) { return Optional.of(new SetColumnTypeSetup("bigint", "BIGINT '-2147483648'", "integer")); - } else if(setup.sourceColumnType().equals("real")) { + } + else if (setup.sourceColumnType().equals("real")) { return Optional.of(new SetColumnTypeSetup("real", "REAL '10.3'", "double")); - } else if(setup.sourceColumnType().equals("real")) { + } + else if (setup.sourceColumnType().equals("real")) { return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); - } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + } + else if (setup.sourceColumnType().equals("decimal(5,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(10,3)")); - } else if(setup.sourceColumnType().equals("decimal(28,3)")) { + } + else if (setup.sourceColumnType().equals("decimal(28,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); - } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + } + else if (setup.sourceColumnType().equals("decimal(5,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(38,3)")); - } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + } + else if (setup.sourceColumnType().equals("decimal(5,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.340", "decimal(5,2)")); - } else if(setup.sourceColumnType().equals("decimal(5,3)")) { + } + else if (setup.sourceColumnType().equals("decimal(5,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); - } else if(setup.sourceColumnType().equals("varchar(100)")) { + } + else if (setup.sourceColumnType().equals("varchar(100)")) { return Optional.of(new SetColumnTypeSetup("varchar(100)", "'shorten-varchar'", "varchar")); - } else if(setup.sourceColumnType().equals("char(25)")) { + } + else if (setup.sourceColumnType().equals("char(25)")) { return Optional.of(new SetColumnTypeSetup("char(25)", "'shorten-char'", "varchar")); - } else if(setup.sourceColumnType().equals("char(20)")) { + } + else if (setup.sourceColumnType().equals("char(20)")) { return Optional.of(new SetColumnTypeSetup("char(20)", "'char-to-varchar'", "varchar")); - } else if(setup.sourceColumnType().equals("varchar")) { + } + else if (setup.sourceColumnType().equals("varchar")) { return Optional.of(new SetColumnTypeSetup("varchar", "'varchar-to-char'", "varchar")); } - return Optional.empty(); + return Optional.empty(); } @Test(dataProvider = "setColumnTypesDataProvider") From 3fe83b40268c104adaa6747c3cb4ec6b129cf57a Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 27 May 2023 20:28:54 -0400 Subject: [PATCH 22/27] Changed ClickHouseConnectorTest for data types to switch/case --- .../TestClickHouseConnectorTest.java | 90 ++++++++----------- .../io/trino/testing/BaseConnectorTest.java | 2 +- 2 files changed, 36 insertions(+), 56 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index 8c92758c50a2d..adf48b307025d 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -47,8 +47,6 @@ import static io.trino.testing.MaterializedResult.resultBuilder; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_COLUMN_COMMENT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_CREATE_TABLE_WITH_DATA; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_INSERT; -import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_NOT_NULL_CONSTRAINT; import static io.trino.testing.TestingConnectorBehavior.SUPPORTS_SET_COLUMN_TYPE; import static io.trino.testing.TestingNames.randomNameSuffix; import static java.lang.String.format; @@ -857,8 +855,6 @@ public void testSetColumnIncompatibleType() @Override public void testSetColumnOutOfRangeType() { - skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - String tableName = "test_set_column_out_of_range" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (col bigint, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); query("insert into " + tableName + " values(9223372036854775807, 22)"); @@ -885,7 +881,6 @@ public void testSetColumnTypeWithComment() @Override public void testSetColumnTypeWithDefaultColumn() { - skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_INSERT)); try (TestTable table = createTableWithDefaultColumns()) { // col_default column inserts 43 by default assertUpdate("ALTER TABLE " + table.getName() + " ALTER COLUMN col_default SET DATA TYPE bigint"); @@ -898,8 +893,6 @@ public void testSetColumnTypeWithDefaultColumn() @Override public void testSetColumnTypeWithNotNull() { - skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_NOT_NULL_CONSTRAINT)); - String tableName = "test_set_column_with_not_null" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (col bigint not null, col2 int not null) WITH (order_by=ARRAY['col2'], engine = 'MergeTree')"); @@ -912,61 +905,48 @@ public void testSetColumnTypeWithNotNull() @Override protected Optional filterSetColumnTypesDataProvider(SetColumnTypeSetup setup) { - if (setup.sourceColumnType().equals("tinyint")) { - return Optional.of(new SetColumnTypeSetup("tinyint", "TINYINT '127'", "smallint")); - } - else if (setup.sourceColumnType().equals("smallint")) { - return Optional.of(new SetColumnTypeSetup("smallint", "SMALLINT '32767'", "integer")); - } - else if (setup.sourceColumnType().equals("integer")) { - return Optional.of(new SetColumnTypeSetup("integer", "2147483647", "bigint")); - } - else if (setup.sourceColumnType().equals("bigint")) { - return Optional.of(new SetColumnTypeSetup("bigint", "BIGINT '-2147483648'", "integer")); - } - else if (setup.sourceColumnType().equals("real")) { - return Optional.of(new SetColumnTypeSetup("real", "REAL '10.3'", "double")); - } - else if (setup.sourceColumnType().equals("real")) { - return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); - } - else if (setup.sourceColumnType().equals("decimal(5,3)")) { - return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(10,3)")); - } - else if (setup.sourceColumnType().equals("decimal(28,3)")) { - return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); - } - else if (setup.sourceColumnType().equals("decimal(5,3)")) { - return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(38,3)")); - } - else if (setup.sourceColumnType().equals("decimal(5,3)")) { - return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.340", "decimal(5,2)")); - } - else if (setup.sourceColumnType().equals("decimal(5,3)")) { - return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); - } - else if (setup.sourceColumnType().equals("varchar(100)")) { - return Optional.of(new SetColumnTypeSetup("varchar(100)", "'shorten-varchar'", "varchar")); - } - else if (setup.sourceColumnType().equals("char(25)")) { - return Optional.of(new SetColumnTypeSetup("char(25)", "'shorten-char'", "varchar")); - } - else if (setup.sourceColumnType().equals("char(20)")) { - return Optional.of(new SetColumnTypeSetup("char(20)", "'char-to-varchar'", "varchar")); - } - else if (setup.sourceColumnType().equals("varchar")) { - return Optional.of(new SetColumnTypeSetup("varchar", "'varchar-to-char'", "varchar")); + switch (setup.sourceColumnType()) { + case "tinyint": + return Optional.of(new SetColumnTypeSetup("tinyint", "TINYINT '127'", "smallint")); + case "smallint": + return Optional.of(new SetColumnTypeSetup("smallint", "SMALLINT '32767'", "integer")); + case "integer": + return Optional.of(new SetColumnTypeSetup("integer", "2147483647", "bigint")); + case "bigint": + return Optional.of(new SetColumnTypeSetup("bigint", "BIGINT '-2147483648'", "integer")); + case "real": + if (setup.sourceValueLiteral() == "REAL '10.3'") + return Optional.of(new SetColumnTypeSetup("real", "REAL '10.3'", "double")); + else if (setup.sourceValueLiteral() == "REAL 'NaN'") + return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); + case "decimal(5,3)": + if (setup.sourceValueLiteral() == "12.345" && setup.newColumnType() == "decimal(10,3)") + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(10,3)")); + else if (setup.sourceValueLiteral() == "12.345" && setup.newColumnType() == "decimal(38,3)") + return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); + else if (setup.sourceValueLiteral() == "12.340") + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.340", "decimal(5,2)")); + else if (setup.sourceValueLiteral() == "12.35") + return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); + case "decimal(28,3)": + return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); + case "varchar(100)": + return Optional.of(new SetColumnTypeSetup("varchar(100)", "'shorten-varchar'", "varchar")); + case "char(25)": + return Optional.of(new SetColumnTypeSetup("char(25)", "'shorten-char'", "varchar")); + case "char(20)": + return Optional.of(new SetColumnTypeSetup("char(20)", "'char-to-varchar'", "varchar")); + case "varchar": + return Optional.of(new SetColumnTypeSetup("varchar", "'varchar-to-char'", "varchar")); + default: + return Optional.empty(); } - - return Optional.empty(); } @Test(dataProvider = "setColumnTypesDataProvider") @Override public void testSetColumnTypes(SetColumnTypeSetup setup) { - skipTestUnless(hasBehavior(SUPPORTS_SET_COLUMN_TYPE) && hasBehavior(SUPPORTS_CREATE_TABLE_WITH_DATA)); - String tableName = "test_set_column_type_" + System.currentTimeMillis(); assertUpdate("CREATE TABLE " + tableName + "(col " + setup.sourceColumnType() + ", col2 int not null) WITH (engine='mergetree', order_by=ARRAY['col2'])"); query("insert into " + tableName + "(col, col2) values(CAST(" + setup.sourceValueLiteral() + " AS " + setup.sourceColumnType() + "), 2)"); diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 8c0d613f50084..37cc7fa3ec8f4 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -5864,7 +5864,7 @@ public void testMergeAllColumnsReversed() assertUpdate("DROP TABLE " + targetTable); } - protected void verifyUnsupportedTypeException(Throwable exception, String trinoTypeName) + private void verifyUnsupportedTypeException(Throwable exception, String trinoTypeName) { String typeNameBase = trinoTypeName.replaceFirst("\\(.*", ""); String expectedMessagePart = format("(%1$s.*not (yet )?supported)|((?i)unsupported.*%1$s)|((?i)not supported.*%1$s)", Pattern.quote(typeNameBase)); From d7cce9096ee1eb349fcaa891a56a27bab04769d6 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sat, 27 May 2023 20:30:41 -0400 Subject: [PATCH 23/27] Reverted unrelated change --- .../src/main/java/io/trino/testing/BaseConnectorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 37cc7fa3ec8f4..8956a47cb880f 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -2890,7 +2890,7 @@ protected void verifySetColumnTypeFailurePermissible(Throwable e) throw new AssertionError("Unexpected set column type failure", e); } - protected String getColumnType(String tableName, String columnName) + private String getColumnType(String tableName, String columnName) { return (String) computeScalar(format("SELECT data_type FROM information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = '%s' AND column_name = '%s'", tableName, From 5cdb1cb8b47db8f1b246c9d9b9c5ec67bb5b7f41 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sun, 28 May 2023 09:23:48 -0400 Subject: [PATCH 24/27] Fixed checkstyle errors. --- .../TestClickHouseConnectorTest.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index adf48b307025d..f21b2e22f1aac 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -915,19 +915,25 @@ protected Optional filterSetColumnTypesDataProvider(SetColum case "bigint": return Optional.of(new SetColumnTypeSetup("bigint", "BIGINT '-2147483648'", "integer")); case "real": - if (setup.sourceValueLiteral() == "REAL '10.3'") + if (setup.sourceValueLiteral().equals("REAL '10.3'")) { return Optional.of(new SetColumnTypeSetup("real", "REAL '10.3'", "double")); - else if (setup.sourceValueLiteral() == "REAL 'NaN'") + } + else if (setup.sourceValueLiteral().equals("REAL 'NaN'")) { return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); + } case "decimal(5,3)": - if (setup.sourceValueLiteral() == "12.345" && setup.newColumnType() == "decimal(10,3)") + if (setup.sourceValueLiteral().equals("12.345") && setup.newColumnType().equals("decimal(10,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.345", "decimal(10,3)")); - else if (setup.sourceValueLiteral() == "12.345" && setup.newColumnType() == "decimal(38,3)") + } + else if (setup.sourceValueLiteral().equals("12.345") && setup.newColumnType().equals("decimal(38,3)")) { return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); - else if (setup.sourceValueLiteral() == "12.340") + } + else if (setup.sourceValueLiteral().equals("12.340")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.340", "decimal(5,2)")); - else if (setup.sourceValueLiteral() == "12.35") + } + else if (setup.sourceValueLiteral().equals("12.35")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); + } case "decimal(28,3)": return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); case "varchar(100)": From 907f738f156c62f6d59b666637ba2335f8096c71 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sun, 28 May 2023 10:07:20 -0400 Subject: [PATCH 25/27] Make getColumnType protected so it can be accessed. --- .../src/main/java/io/trino/testing/BaseConnectorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java index 8956a47cb880f..37cc7fa3ec8f4 100644 --- a/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java +++ b/testing/trino-testing/src/main/java/io/trino/testing/BaseConnectorTest.java @@ -2890,7 +2890,7 @@ protected void verifySetColumnTypeFailurePermissible(Throwable e) throw new AssertionError("Unexpected set column type failure", e); } - private String getColumnType(String tableName, String columnName) + protected String getColumnType(String tableName, String columnName) { return (String) computeScalar(format("SELECT data_type FROM information_schema.columns WHERE table_schema = CURRENT_SCHEMA AND table_name = '%s' AND column_name = '%s'", tableName, From cdb2b35215d68eecbed77e203c599cc5c86e8d2a Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sun, 28 May 2023 12:07:19 -0400 Subject: [PATCH 26/27] Fix fallthrough in switch/case of TestClickHouseConnectorTest --- .../trino/plugin/clickhouse/TestClickHouseConnectorTest.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index f21b2e22f1aac..d5fcb1df6a372 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -920,6 +920,8 @@ protected Optional filterSetColumnTypesDataProvider(SetColum } else if (setup.sourceValueLiteral().equals("REAL 'NaN'")) { return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); + } else { + return Optional.empty(); } case "decimal(5,3)": if (setup.sourceValueLiteral().equals("12.345") && setup.newColumnType().equals("decimal(10,3)")) { @@ -933,6 +935,8 @@ else if (setup.sourceValueLiteral().equals("12.340")) { } else if (setup.sourceValueLiteral().equals("12.35")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); + } else { + return Optional.empty(); } case "decimal(28,3)": return Optional.of(new SetColumnTypeSetup("decimal(28,3)", "12.345", "decimal(38,3)")); From a0aa99331024207176efe364902d746f70b6c686 Mon Sep 17 00:00:00 2001 From: Kanthi Subramanian Date: Sun, 28 May 2023 18:43:45 -0400 Subject: [PATCH 27/27] Fix checkstyle violations in TestClickHouseConnectorTest --- .../plugin/clickhouse/TestClickHouseConnectorTest.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java index d5fcb1df6a372..807b30748c58e 100644 --- a/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java +++ b/plugin/trino-clickhouse/src/test/java/io/trino/plugin/clickhouse/TestClickHouseConnectorTest.java @@ -920,7 +920,8 @@ protected Optional filterSetColumnTypesDataProvider(SetColum } else if (setup.sourceValueLiteral().equals("REAL 'NaN'")) { return Optional.of(new SetColumnTypeSetup("real", "REAL 'NaN'", "double")); - } else { + } + else { return Optional.empty(); } case "decimal(5,3)": @@ -935,7 +936,8 @@ else if (setup.sourceValueLiteral().equals("12.340")) { } else if (setup.sourceValueLiteral().equals("12.35")) { return Optional.of(new SetColumnTypeSetup("decimal(5,3)", "12.35", "decimal(5,2)")); - } else { + } + else { return Optional.empty(); } case "decimal(28,3)":