diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseTestOracleDistributedQueries.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java similarity index 56% rename from plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseTestOracleDistributedQueries.java rename to plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java index 5ac5a6ce3fd6..6f58ed88c135 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseTestOracleDistributedQueries.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleConnectorTest.java @@ -13,9 +13,13 @@ */ package io.trino.plugin.oracle; +import com.google.common.collect.ImmutableList; import io.trino.Session; import io.trino.execution.QueryInfo; -import io.trino.testing.AbstractTestDistributedQueries; +import io.trino.sql.planner.plan.AggregationNode; +import io.trino.sql.planner.plan.FilterNode; +import io.trino.sql.planner.plan.ProjectNode; +import io.trino.testing.BaseConnectorTest; import io.trino.testing.MaterializedResult; import io.trino.testing.ResultWithQueryId; import io.trino.testing.sql.SqlExecutor; @@ -24,15 +28,18 @@ import java.util.Optional; +import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; import static io.trino.spi.type.VarcharType.VARCHAR; import static io.trino.testing.MaterializedResult.resultBuilder; +import static io.trino.testing.assertions.Assert.assertEquals; import static io.trino.testing.sql.TestTable.randomTableSuffix; import static java.lang.String.format; -import static org.testng.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; +import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertTrue; -public abstract class BaseTestOracleDistributedQueries - extends AbstractTestDistributedQueries +public abstract class BaseOracleConnectorTest + extends BaseConnectorTest { @Override protected boolean supportsDelete() @@ -90,7 +97,7 @@ protected Optional filterDataMappingSmokeTestData(DataMapp protected TestTable createTableWithDefaultColumns() { return new TestTable( - createJdbcSqlExecutor(), + onOracle(), "test_default_columns", "(col_required decimal(20,0) NOT NULL," + "col_nullable decimal(20,0)," + @@ -288,5 +295,169 @@ protected Optional filterColumnNameTestData(String columnName) return Optional.empty(); } - protected abstract SqlExecutor createJdbcSqlExecutor(); + @Test + @Override + public void testDescribeTable() + { + MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "decimal(19,0)", "", "") + .row("custkey", "decimal(19,0)", "", "") + .row("orderstatus", "varchar(1)", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "timestamp(3)", "", "") + .row("orderpriority", "varchar(15)", "", "") + .row("clerk", "varchar(15)", "", "") + .row("shippriority", "decimal(10,0)", "", "") + .row("comment", "varchar(79)", "", "") + .build(); + MaterializedResult actualColumns = computeActual("DESCRIBE orders"); + assertEquals(actualColumns, expectedColumns); + } + + @Test + @Override + public void testShowCreateTable() + { + assertThat((String) computeActual("SHOW CREATE TABLE orders").getOnlyValue()) + // If the connector reports additional column properties, the expected value needs to be adjusted in the test subclass + .matches("CREATE TABLE \\w+\\.\\w+\\.orders \\Q(\n" + + " orderkey decimal(19, 0),\n" + + " custkey decimal(19, 0),\n" + + " orderstatus varchar(1),\n" + + " totalprice double,\n" + + " orderdate timestamp(3),\n" + + " orderpriority varchar(15),\n" + + " clerk varchar(15),\n" + + " shippriority decimal(10, 0),\n" + + " comment varchar(79)\n" + + ")"); + } + + @Test + public void testDropTable() + { + String tableName = "test_drop" + randomTableSuffix(); + assertUpdate(format("CREATE TABLE %s AS SELECT 1 test_drop", tableName), 1); + assertTrue(getQueryRunner().tableExists(getSession(), tableName)); + + assertUpdate("DROP TABLE " + tableName); + assertFalse(getQueryRunner().tableExists(getSession(), tableName)); + } + + @Test + public void testViews() + { + try (TestView view = new TestView(onOracle(), getUser() + ".test_view", "AS SELECT 'O' as status FROM dual")) { + assertQuery("SELECT status FROM " + view.getName(), "SELECT 'O'"); + } + } + + @Test + public void testSynonyms() + { + try (TestSynonym synonym = new TestSynonym(onOracle(), getUser() + ".test_synonym", "FOR ORDERS")) { + assertQueryFails("SELECT orderkey FROM " + synonym.getName(), "line 1:22: Table 'oracle.*' does not exist"); + } + } + + @Test + public void testPredicatePushdown() + { + // varchar equality + assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name = 'ROMANIA'")) + .matches("VALUES (CAST(3 AS DECIMAL(19,0)), CAST(19 AS DECIMAL(19,0)), CAST('ROMANIA' AS varchar(25)))") + .isFullyPushedDown(); + + // varchar range + assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name BETWEEN 'POLAND' AND 'RPA'")) + .matches("VALUES (CAST(3 AS DECIMAL(19,0)), CAST(19 AS DECIMAL(19,0)), CAST('ROMANIA' AS varchar(25)))") + .isFullyPushedDown(); + + // varchar different case + assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name = 'romania'")) + .returnsEmptyResult() + .isFullyPushedDown(); + + // date equality + assertThat(query("SELECT orderkey FROM orders WHERE orderdate = DATE '1992-09-29'")) + .matches("VALUES CAST(1250 AS DECIMAL(19,0)), 34406, 38436, 57570") + .isFullyPushedDown(); + + // predicate over aggregation key (likely to be optimized before being pushed down into the connector) + assertThat(query("SELECT * FROM (SELECT regionkey, sum(nationkey) FROM nation GROUP BY regionkey) WHERE regionkey = 3")) + .matches("VALUES (CAST(3 AS decimal(19,0)), CAST(77 AS decimal(38,0)))") + .isNotFullyPushedDown(AggregationNode.class, ProjectNode.class); + + // predicate over aggregation result + assertThat(query("SELECT regionkey, sum(nationkey) FROM nation GROUP BY regionkey HAVING sum(nationkey) = 77")) + .matches("VALUES (CAST(3 AS decimal(19,0)), CAST(77 AS decimal(38,0)))") + .isNotFullyPushedDown(AggregationNode.class, ProjectNode.class); + } + + @Test + public void testPredicatePushdownForNumerics() + { + predicatePushdownTest("DECIMAL(9, 3)", "123.321", "<=", "124"); + predicatePushdownTest("DECIMAL(9, 3)", "123.321", "<=", "123.321"); + predicatePushdownTest("DECIMAL(9, 3)", "123.321", "=", "123.321"); + predicatePushdownTest("DECIMAL(30, 10)", "123456789.987654321", "<=", "123456790"); + predicatePushdownTest("DECIMAL(30, 10)", "123456789.987654321", "<=", "123456789.987654321"); + predicatePushdownTest("DECIMAL(30, 10)", "123456789.987654321", "=", "123456789.987654321"); + predicatePushdownTest("FLOAT(63)", "123456789.987654321", "<=", "CAST(123456789.99 AS REAL)"); + predicatePushdownTest("FLOAT(63)", "123456789.987654321", "<=", "CAST(123456789.99 AS DOUBLE)"); + predicatePushdownTest("FLOAT(126)", "123456789.987654321", "<=", "CAST(123456789.99 AS REAL)"); + predicatePushdownTest("FLOAT(126)", "123456789.987654321", "<=", "CAST(123456789.99 AS DOUBLE)"); + predicatePushdownTest("BINARY_FLOAT", "5.0f", "=", "CAST(5.0 AS REAL)"); + predicatePushdownTest("BINARY_DOUBLE", "20.233", "=", "CAST(20.233 AS DOUBLE)"); + predicatePushdownTest("NUMBER(5,3)", "5.0", "=", "CAST(5.0 AS DECIMAL(5,3))"); + } + + @Test + public void testPredicatePushdownForChars() + { + predicatePushdownTest("CHAR(1)", "'0'", "=", "'0'"); + predicatePushdownTest("CHAR(1)", "'0'", "<=", "'0'"); + predicatePushdownTest("CHAR(5)", "'0'", "=", "CHAR'0'"); + predicatePushdownTest("CHAR(7)", "'my_char'", "=", "CAST('my_char' AS CHAR(7))"); + predicatePushdownTest("NCHAR(7)", "'my_char'", "=", "CAST('my_char' AS CHAR(7))"); + predicatePushdownTest("VARCHAR2(7)", "'my_char'", "=", "CAST('my_char' AS VARCHAR(7))"); + predicatePushdownTest("NVARCHAR2(7)", "'my_char'", "=", "CAST('my_char' AS VARCHAR(7))"); + + try (TestTable table = new TestTable( + onOracle(), + getUser() + ".test_pdown_", + "(c_clob CLOB, c_nclob NCLOB)", + ImmutableList.of("'my_clob', 'my_nclob'"))) { + assertThat(query(format("SELECT c_clob FROM %s WHERE c_clob = cast('my_clob' as varchar)", table.getName()))).isNotFullyPushedDown(FilterNode.class); + assertThat(query(format("SELECT c_nclob FROM %s WHERE c_nclob = cast('my_nclob' as varchar)", table.getName()))).isNotFullyPushedDown(FilterNode.class); + } + } + + @Test + public void testTooLargeDomainCompactionThreshold() + { + assertQueryFails( + Session.builder(getSession()) + .setCatalogSessionProperty("oracle", "domain_compaction_threshold", "10000") + .build(), + "SELECT * from nation", "Domain compaction threshold \\(10000\\) cannot exceed 1000"); + } + + private void predicatePushdownTest(String oracleType, String oracleLiteral, String operator, String filterLiteral) + { + String tableName = "test_pdown_" + oracleType.replaceAll("[^a-zA-Z0-9]", ""); + try (TestTable table = new TestTable(onOracle(), getUser() + "." + tableName, format("(c %s)", oracleType))) { + onOracle().execute(format("INSERT INTO %s VALUES (%s)", table.getName(), oracleLiteral)); + + assertThat(query(format("SELECT * FROM %s WHERE c %s %s", table.getName(), operator, filterLiteral))) + .isFullyPushedDown(); + } + } + + protected String getUser() + { + return TEST_USER; + } + + protected abstract SqlExecutor onOracle(); } diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java deleted file mode 100644 index 5561081d3a20..000000000000 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/BaseOracleIntegrationSmokeTest.java +++ /dev/null @@ -1,205 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.plugin.oracle; - -import com.google.common.collect.ImmutableList; -import io.trino.Session; -import io.trino.sql.planner.plan.AggregationNode; -import io.trino.sql.planner.plan.FilterNode; -import io.trino.sql.planner.plan.ProjectNode; -import io.trino.testing.AbstractTestIntegrationSmokeTest; -import io.trino.testing.MaterializedResult; -import io.trino.testing.sql.SqlExecutor; -import io.trino.testing.sql.TestTable; -import org.testng.annotations.Test; - -import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; -import static io.trino.spi.type.VarcharType.VARCHAR; -import static io.trino.testing.assertions.Assert.assertEquals; -import static io.trino.testing.sql.TestTable.randomTableSuffix; -import static java.lang.String.format; -import static org.assertj.core.api.Assertions.assertThat; -import static org.testng.Assert.assertFalse; -import static org.testng.Assert.assertTrue; - -public abstract class BaseOracleIntegrationSmokeTest - // TODO extend BaseConnectorTest - extends AbstractTestIntegrationSmokeTest -{ - @Test - @Override - public void testDescribeTable() - { - MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) - .row("orderkey", "decimal(19,0)", "", "") - .row("custkey", "decimal(19,0)", "", "") - .row("orderstatus", "varchar(1)", "", "") - .row("totalprice", "double", "", "") - .row("orderdate", "timestamp(3)", "", "") - .row("orderpriority", "varchar(15)", "", "") - .row("clerk", "varchar(15)", "", "") - .row("shippriority", "decimal(10,0)", "", "") - .row("comment", "varchar(79)", "", "") - .build(); - MaterializedResult actualColumns = computeActual("DESCRIBE orders"); - assertEquals(actualColumns, expectedColumns); - } - - @Test - @Override - public void testShowCreateTable() - { - assertThat((String) computeActual("SHOW CREATE TABLE orders").getOnlyValue()) - // If the connector reports additional column properties, the expected value needs to be adjusted in the test subclass - .matches("CREATE TABLE \\w+\\.\\w+\\.orders \\Q(\n" + - " orderkey decimal(19, 0),\n" + - " custkey decimal(19, 0),\n" + - " orderstatus varchar(1),\n" + - " totalprice double,\n" + - " orderdate timestamp(3),\n" + - " orderpriority varchar(15),\n" + - " clerk varchar(15),\n" + - " shippriority decimal(10, 0),\n" + - " comment varchar(79)\n" + - ")"); - } - - @Test - public void testDropTable() - { - String tableName = "test_drop" + randomTableSuffix(); - assertUpdate(format("CREATE TABLE %s AS SELECT 1 test_drop", tableName), 1); - assertTrue(getQueryRunner().tableExists(getSession(), tableName)); - - assertUpdate("DROP TABLE " + tableName); - assertFalse(getQueryRunner().tableExists(getSession(), tableName)); - } - - @Test - public void testViews() - { - try (TestView view = new TestView(onOracle(), getUser() + ".test_view", "AS SELECT 'O' as status FROM dual")) { - assertQuery("SELECT status FROM " + view.getName(), "SELECT 'O'"); - } - } - - @Test - public void testSynonyms() - { - try (TestSynonym synonym = new TestSynonym(onOracle(), getUser() + ".test_synonym", "FOR ORDERS")) { - assertQueryFails("SELECT orderkey FROM " + synonym.getName(), "line 1:22: Table 'oracle.*' does not exist"); - } - } - - @Test - public void testPredicatePushdown() - { - // varchar equality - assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name = 'ROMANIA'")) - .matches("VALUES (CAST(3 AS DECIMAL(19,0)), CAST(19 AS DECIMAL(19,0)), CAST('ROMANIA' AS varchar(25)))") - .isFullyPushedDown(); - - // varchar range - assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name BETWEEN 'POLAND' AND 'RPA'")) - .matches("VALUES (CAST(3 AS DECIMAL(19,0)), CAST(19 AS DECIMAL(19,0)), CAST('ROMANIA' AS varchar(25)))") - .isFullyPushedDown(); - - // varchar different case - assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name = 'romania'")) - .returnsEmptyResult() - .isFullyPushedDown(); - - // date equality - assertThat(query("SELECT orderkey FROM orders WHERE orderdate = DATE '1992-09-29'")) - .matches("VALUES CAST(1250 AS DECIMAL(19,0)), 34406, 38436, 57570") - .isFullyPushedDown(); - - // predicate over aggregation key (likely to be optimized before being pushed down into the connector) - assertThat(query("SELECT * FROM (SELECT regionkey, sum(nationkey) FROM nation GROUP BY regionkey) WHERE regionkey = 3")) - .matches("VALUES (CAST(3 AS decimal(19,0)), CAST(77 AS decimal(38,0)))") - .isNotFullyPushedDown(AggregationNode.class, ProjectNode.class); - - // predicate over aggregation result - assertThat(query("SELECT regionkey, sum(nationkey) FROM nation GROUP BY regionkey HAVING sum(nationkey) = 77")) - .matches("VALUES (CAST(3 AS decimal(19,0)), CAST(77 AS decimal(38,0)))") - .isNotFullyPushedDown(AggregationNode.class, ProjectNode.class); - } - - @Test - public void testPredicatePushdownForNumerics() - { - predicatePushdownTest("DECIMAL(9, 3)", "123.321", "<=", "124"); - predicatePushdownTest("DECIMAL(9, 3)", "123.321", "<=", "123.321"); - predicatePushdownTest("DECIMAL(9, 3)", "123.321", "=", "123.321"); - predicatePushdownTest("DECIMAL(30, 10)", "123456789.987654321", "<=", "123456790"); - predicatePushdownTest("DECIMAL(30, 10)", "123456789.987654321", "<=", "123456789.987654321"); - predicatePushdownTest("DECIMAL(30, 10)", "123456789.987654321", "=", "123456789.987654321"); - predicatePushdownTest("FLOAT(63)", "123456789.987654321", "<=", "CAST(123456789.99 AS REAL)"); - predicatePushdownTest("FLOAT(63)", "123456789.987654321", "<=", "CAST(123456789.99 AS DOUBLE)"); - predicatePushdownTest("FLOAT(126)", "123456789.987654321", "<=", "CAST(123456789.99 AS REAL)"); - predicatePushdownTest("FLOAT(126)", "123456789.987654321", "<=", "CAST(123456789.99 AS DOUBLE)"); - predicatePushdownTest("BINARY_FLOAT", "5.0f", "=", "CAST(5.0 AS REAL)"); - predicatePushdownTest("BINARY_DOUBLE", "20.233", "=", "CAST(20.233 AS DOUBLE)"); - predicatePushdownTest("NUMBER(5,3)", "5.0", "=", "CAST(5.0 AS DECIMAL(5,3))"); - } - - @Test - public void testPredicatePushdownForChars() - { - predicatePushdownTest("CHAR(1)", "'0'", "=", "'0'"); - predicatePushdownTest("CHAR(1)", "'0'", "<=", "'0'"); - predicatePushdownTest("CHAR(5)", "'0'", "=", "CHAR'0'"); - predicatePushdownTest("CHAR(7)", "'my_char'", "=", "CAST('my_char' AS CHAR(7))"); - predicatePushdownTest("NCHAR(7)", "'my_char'", "=", "CAST('my_char' AS CHAR(7))"); - predicatePushdownTest("VARCHAR2(7)", "'my_char'", "=", "CAST('my_char' AS VARCHAR(7))"); - predicatePushdownTest("NVARCHAR2(7)", "'my_char'", "=", "CAST('my_char' AS VARCHAR(7))"); - - try (TestTable table = new TestTable( - onOracle(), - getUser() + ".test_pdown_", - "(c_clob CLOB, c_nclob NCLOB)", - ImmutableList.of("'my_clob', 'my_nclob'"))) { - assertThat(query(format("SELECT c_clob FROM %s WHERE c_clob = cast('my_clob' as varchar)", table.getName()))).isNotFullyPushedDown(FilterNode.class); - assertThat(query(format("SELECT c_nclob FROM %s WHERE c_nclob = cast('my_nclob' as varchar)", table.getName()))).isNotFullyPushedDown(FilterNode.class); - } - } - - @Test - public void testTooLargeDomainCompactionThreshold() - { - assertQueryFails( - Session.builder(getSession()) - .setCatalogSessionProperty("oracle", "domain_compaction_threshold", "10000") - .build(), - "SELECT * from nation", "Domain compaction threshold \\(10000\\) cannot exceed 1000"); - } - - private void predicatePushdownTest(String oracleType, String oracleLiteral, String operator, String filterLiteral) - { - String tableName = "test_pdown_" + oracleType.replaceAll("[^a-zA-Z0-9]", ""); - try (TestTable table = new TestTable(onOracle(), getUser() + "." + tableName, format("(c %s)", oracleType))) { - onOracle().execute(format("INSERT INTO %s VALUES (%s)", table.getName(), oracleLiteral)); - - assertThat(query(format("SELECT * FROM %s WHERE c %s %s", table.getName(), operator, filterLiteral))) - .isFullyPushedDown(); - } - } - - protected String getUser() - { - return TEST_USER; - } - - protected abstract SqlExecutor onOracle(); -} diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleIntegrationSmokeTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConnectorTest.java similarity index 79% rename from plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleIntegrationSmokeTest.java rename to plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConnectorTest.java index de0c1842c7df..72c38ebe5c1c 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleIntegrationSmokeTest.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleConnectorTest.java @@ -13,11 +13,11 @@ */ package io.trino.plugin.oracle; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.testing.Closeables; import io.trino.testing.QueryRunner; import io.trino.testing.sql.SqlExecutor; +import io.trino.tpch.TpchTable; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; @@ -26,16 +26,14 @@ import static io.trino.plugin.oracle.TestingOracleServer.TEST_PASS; import static io.trino.plugin.oracle.TestingOracleServer.TEST_SCHEMA; import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; -import static io.trino.tpch.TpchTable.CUSTOMER; -import static io.trino.tpch.TpchTable.NATION; -import static io.trino.tpch.TpchTable.ORDERS; -import static io.trino.tpch.TpchTable.REGION; +import static io.trino.testing.sql.TestTable.randomTableSuffix; import static java.lang.String.format; import static java.util.stream.Collectors.joining; import static java.util.stream.IntStream.range; +import static org.assertj.core.api.Assertions.assertThat; -public class TestOracleIntegrationSmokeTest - extends BaseOracleIntegrationSmokeTest +public class TestOracleConnectorTest + extends BaseOracleConnectorTest { private TestingOracleServer oracleServer; @@ -55,7 +53,7 @@ protected QueryRunner createQueryRunner() .put("oracle.connection-pool.enabled", "false") .put("oracle.remarks-reporting.enabled", "false") .build(), - ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION)); + TpchTable.getTables()); } @AfterClass(alwaysRun = true) @@ -86,6 +84,20 @@ private String getLongInClause(int start, int length) return "orderkey IN (" + longValues + ")"; } + @Test + @Override + public void testCommentColumn() + { + String tableName = "test_comment_column_" + randomTableSuffix(); + + assertUpdate("CREATE TABLE " + tableName + "(a integer)"); + + // comment set + assertUpdate("COMMENT ON COLUMN " + tableName + ".a IS 'new comment'"); + // without remarksReporting Oracle does not return comments set + assertThat((String) computeActual("SHOW CREATE TABLE " + tableName).getOnlyValue()).doesNotContain("COMMENT 'new comment'"); + } + @Override protected SqlExecutor onOracle() { diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleDistributedQueries.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOraclePoolConnectorTest.java similarity index 85% rename from plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleDistributedQueries.java rename to plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOraclePoolConnectorTest.java index ea3418609420..48fb5452898d 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOracleDistributedQueries.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOraclePoolConnectorTest.java @@ -14,18 +14,22 @@ package io.trino.plugin.oracle; import com.google.common.collect.ImmutableMap; +import io.airlift.testing.Closeables; import io.trino.testing.QueryRunner; import io.trino.testing.sql.SqlExecutor; +import io.trino.tpch.TpchTable; import org.testng.annotations.AfterClass; import org.testng.annotations.Test; +import java.io.IOException; + import static io.trino.plugin.oracle.TestingOracleServer.TEST_PASS; import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; import static io.trino.testing.sql.TestTable.randomTableSuffix; import static org.assertj.core.api.Assertions.assertThat; -public class TestOracleDistributedQueries - extends BaseTestOracleDistributedQueries +public class TestOraclePoolConnectorTest + extends BaseOracleConnectorTest { private TestingOracleServer oracleServer; @@ -33,7 +37,7 @@ public class TestOracleDistributedQueries protected QueryRunner createQueryRunner() throws Exception { - this.oracleServer = new TestingOracleServer(); + oracleServer = new TestingOracleServer(); return OracleQueryRunner.createOracleQueryRunner( oracleServer, ImmutableMap.of(), @@ -42,18 +46,18 @@ protected QueryRunner createQueryRunner() .put("connection-user", TEST_USER) .put("connection-password", TEST_PASS) .put("allow-drop-table", "true") - .put("oracle.connection-pool.enabled", "false") + .put("oracle.connection-pool.enabled", "true") .put("oracle.remarks-reporting.enabled", "false") .build(), - REQUIRED_TPCH_TABLES); + TpchTable.getTables()); } @AfterClass(alwaysRun = true) public final void destroy() + throws IOException { - if (oracleServer != null) { - oracleServer.close(); - } + Closeables.closeAll(oracleServer); + oracleServer = null; } @Test @@ -71,7 +75,7 @@ public void testCommentColumn() } @Override - protected SqlExecutor createJdbcSqlExecutor() + protected SqlExecutor onOracle() { return oracleServer::execute; } diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOracleDistributedQueries.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOracleDistributedQueries.java deleted file mode 100644 index c8af670053d6..000000000000 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOracleDistributedQueries.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.plugin.oracle; - -import com.google.common.collect.ImmutableMap; -import io.trino.testing.QueryRunner; -import io.trino.testing.sql.SqlExecutor; -import org.testng.annotations.AfterClass; - -import static io.trino.plugin.oracle.TestingOracleServer.TEST_PASS; -import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; - -public class TestRemarksReportingOracleDistributedQueries - extends BaseTestOracleDistributedQueries -{ - private TestingOracleServer oracleServer; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - this.oracleServer = new TestingOracleServer(); - return OracleQueryRunner.createOracleQueryRunner( - oracleServer, - ImmutableMap.of(), - ImmutableMap.builder() - .put("connection-url", oracleServer.getJdbcUrl()) - .put("connection-user", TEST_USER) - .put("connection-password", TEST_PASS) - .put("allow-drop-table", "true") - .put("oracle.connection-pool.enabled", "false") - .put("oracle.remarks-reporting.enabled", "true") - .build(), - REQUIRED_TPCH_TABLES); - } - - @AfterClass(alwaysRun = true) - public final void destroy() - { - if (oracleServer != null) { - oracleServer.close(); - } - } - - @Override - protected SqlExecutor createJdbcSqlExecutor() - { - return oracleServer::execute; - } -} diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOraclePoolIntegrationSmokeTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOraclePoolConnectorTest.java similarity index 83% rename from plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOraclePoolIntegrationSmokeTest.java rename to plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOraclePoolConnectorTest.java index 60f1c66aab50..24a21bbe6fde 100644 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestOraclePoolIntegrationSmokeTest.java +++ b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOraclePoolConnectorTest.java @@ -13,24 +13,20 @@ */ package io.trino.plugin.oracle; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import io.airlift.testing.Closeables; import io.trino.testing.QueryRunner; import io.trino.testing.sql.SqlExecutor; +import io.trino.tpch.TpchTable; import org.testng.annotations.AfterClass; import java.io.IOException; import static io.trino.plugin.oracle.TestingOracleServer.TEST_PASS; import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; -import static io.trino.tpch.TpchTable.CUSTOMER; -import static io.trino.tpch.TpchTable.NATION; -import static io.trino.tpch.TpchTable.ORDERS; -import static io.trino.tpch.TpchTable.REGION; -public class TestOraclePoolIntegrationSmokeTest - extends BaseOracleIntegrationSmokeTest +public class TestRemarksReportingOraclePoolConnectorTest + extends BaseOracleConnectorTest { private TestingOracleServer oracleServer; @@ -48,9 +44,9 @@ protected QueryRunner createQueryRunner() .put("connection-password", TEST_PASS) .put("allow-drop-table", "true") .put("oracle.connection-pool.enabled", "true") - .put("oracle.remarks-reporting.enabled", "false") + .put("oracle.remarks-reporting.enabled", "true") .build(), - ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION)); + TpchTable.getTables()); } @AfterClass(alwaysRun = true) diff --git a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOraclePoolIntegrationSmokeTest.java b/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOraclePoolIntegrationSmokeTest.java deleted file mode 100644 index 6c2b3cbd4eff..000000000000 --- a/plugin/trino-oracle/src/test/java/io/trino/plugin/oracle/TestRemarksReportingOraclePoolIntegrationSmokeTest.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package io.trino.plugin.oracle; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import io.airlift.testing.Closeables; -import io.trino.testing.QueryRunner; -import io.trino.testing.sql.SqlExecutor; -import org.testng.annotations.AfterClass; - -import java.io.IOException; - -import static io.trino.plugin.oracle.TestingOracleServer.TEST_PASS; -import static io.trino.plugin.oracle.TestingOracleServer.TEST_USER; -import static io.trino.tpch.TpchTable.CUSTOMER; -import static io.trino.tpch.TpchTable.NATION; -import static io.trino.tpch.TpchTable.ORDERS; -import static io.trino.tpch.TpchTable.REGION; - -public class TestRemarksReportingOraclePoolIntegrationSmokeTest - extends BaseOracleIntegrationSmokeTest -{ - private TestingOracleServer oracleServer; - - @Override - protected QueryRunner createQueryRunner() - throws Exception - { - oracleServer = new TestingOracleServer(); - return OracleQueryRunner.createOracleQueryRunner( - oracleServer, - ImmutableMap.of(), - ImmutableMap.builder() - .put("connection-url", oracleServer.getJdbcUrl()) - .put("connection-user", TEST_USER) - .put("connection-password", TEST_PASS) - .put("allow-drop-table", "true") - .put("oracle.connection-pool.enabled", "true") - .put("oracle.remarks-reporting.enabled", "true") - .build(), - ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION)); - } - - @AfterClass(alwaysRun = true) - public final void destroy() - throws IOException - { - Closeables.closeAll(oracleServer); - oracleServer = null; - } - - @Override - protected SqlExecutor onOracle() - { - return oracleServer::execute; - } -}