diff --git a/pom.xml b/pom.xml index 6b8b6d5d60498..f81001acb4aa5 100644 --- a/pom.xml +++ b/pom.xml @@ -52,6 +52,7 @@ 1.11.445 3.9.0 3.4.0 + 19.3.0.0 1.25 2.10.5 1.50 @@ -112,6 +113,7 @@ presto-base-jdbc presto-testing-docker presto-mysql + presto-oracle presto-postgresql presto-redshift presto-sqlserver @@ -351,6 +353,20 @@ ${project.version} + + com.facebook.presto + presto-oracle + ${project.version} + + + + org.testcontainers + testcontainers-bom + 1.14.3 + pom + import + + com.facebook.presto presto-geospatial diff --git a/presto-docs/src/main/sphinx/connector.rst b/presto-docs/src/main/sphinx/connector.rst index 4b6b3cd1b0dea..2227fbfefa94c 100644 --- a/presto-docs/src/main/sphinx/connector.rst +++ b/presto-docs/src/main/sphinx/connector.rst @@ -23,6 +23,7 @@ from different data sources. connector/memory connector/mongodb connector/mysql + connector/oracle connector/pinot connector/postgresql connector/redis diff --git a/presto-docs/src/main/sphinx/connector/oracle.rst b/presto-docs/src/main/sphinx/connector/oracle.rst new file mode 100644 index 0000000000000..fdba3c8f6a04e --- /dev/null +++ b/presto-docs/src/main/sphinx/connector/oracle.rst @@ -0,0 +1,71 @@ +================ +Oracle Connector +================ + +The Oracle connector allows querying and creating tables in an +external Oracle database. This can be used to join data between +different systems like Oracle and Hive, or between two different +Oracle instances. + +Configuration +------------- + +To configure the Oracle connector, create a catalog properties file +in ``etc/catalog`` named, for example, ``oracle.properties``, to +mount the Oracle connector as the ``oracle`` catalog. +Create the file with the following contents, replacing the +connection properties as appropriate for your setup: + +.. code-block:: none + + connector.name=oracle + connection-url=jdbc:oracle:thin:@example.net:1521/ORCLCDB + connection-user=root + connection-password=secret + +Multiple Oracle Servers +^^^^^^^^^^^^^^^^^^^^^^^ + +You can have as many catalogs as you need, so if you have additional +Oracle servers, simply add another properties file to ``etc/catalog`` +with a different name (making sure it ends in ``.properties``). For +example, if you name the property file ``sales.properties``, Presto +will create a catalog named ``sales`` using the configured connector. + +Querying Oracle +--------------- + +The Oracle connector provides a schema for every Oracle *database*. +You can see the available Oracle databases by running ``SHOW SCHEMAS``:: + + SHOW SCHEMAS FROM oracle; + +If you have a Oracle database named ``web``, you can view the tables +in this database by running ``SHOW TABLES``:: + + SHOW TABLES FROM oracle.web; + +You can see a list of the columns in the ``clicks`` table in the ``web`` database +using either of the following:: + + DESCRIBE oracle.web.clicks; + SHOW COLUMNS FROM oracle.web.clicks; + +Finally, you can access the ``clicks`` table in the ``web`` database:: + + SELECT * FROM oracle.web.clicks; + +If you used a different name for your catalog properties file, use +that catalog name instead of ``oracle`` in the above examples. + +Oracle Connector Limitations +---------------------------- + +The following SQL statements are not yet supported: + +* :doc:`/sql/delete` +* :doc:`/sql/grant` +* :doc:`/sql/revoke` +* :doc:`/sql/show-grants` +* :doc:`/sql/show-roles` +* :doc:`/sql/show-role-grants` diff --git a/presto-oracle/pom.xml b/presto-oracle/pom.xml new file mode 100644 index 0000000000000..1fc9c4adfbdc2 --- /dev/null +++ b/presto-oracle/pom.xml @@ -0,0 +1,223 @@ + + + 4.0.0 + + + com.facebook.presto + presto-root + 0.241-SNAPSHOT + + + presto-oracle + Presto - Oracle Connector + presto-plugin + + + ${project.parent.basedir} + + + + + com.oracle.ojdbc + ojdbc8 + ${dep.oracle.version} + + + + com.facebook.presto + presto-base-jdbc + + + + com.facebook.airlift + configuration + + + + com.facebook.airlift + log + + + + com.facebook.airlift + log-manager + runtime + + + + com.google.guava + guava + + + + com.google.inject + guice + + + + javax.inject + javax.inject + + + + javax.validation + validation-api + + + + + com.facebook.presto + presto-spi + provided + + + + com.facebook.presto + presto-common + provided + + + + io.airlift + slice + provided + + + + io.airlift + units + provided + + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + + org.openjdk.jol + jol-core + provided + + + + + org.testng + testng + test + + + + com.facebook.presto + presto-main + test + + + + com.facebook.presto + presto-tpch + test + + + + com.facebook.airlift + testing + test + ${dep.airlift.version} + + + + io.airlift.tpch + tpch + test + + + + org.assertj + assertj-core + test + + + + com.facebook.presto + presto-tests + test + + + + org.testcontainers + testcontainers + test + 1.14.3 + + + org.slf4j + slf4j-api + + + org.jetbrains + annotations + + + + + + org.testcontainers + oracle-xe + test + 1.14.3 + + + org.slf4j + slf4j-api + + + org.jetbrains + annotations + + + + + + + + ci + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + + + + + default + + true + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + **/TestOracleIntegrationSmokeTest.java + **/TestOracleDistributedQueries.java + + + + + + + + diff --git a/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleClient.java b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleClient.java new file mode 100644 index 0000000000000..86642b1fe8fd0 --- /dev/null +++ b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleClient.java @@ -0,0 +1,170 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.common.type.Decimals; +import com.facebook.presto.common.type.VarcharType; +import com.facebook.presto.plugin.jdbc.BaseJdbcClient; +import com.facebook.presto.plugin.jdbc.BaseJdbcConfig; +import com.facebook.presto.plugin.jdbc.ConnectionFactory; +import com.facebook.presto.plugin.jdbc.JdbcConnectorId; +import com.facebook.presto.plugin.jdbc.JdbcIdentity; +import com.facebook.presto.plugin.jdbc.JdbcTypeHandle; +import com.facebook.presto.plugin.jdbc.ReadMapping; +import com.facebook.presto.spi.ConnectorSession; +import com.facebook.presto.spi.PrestoException; +import com.facebook.presto.spi.SchemaTableName; + +import javax.inject.Inject; + +import java.sql.Connection; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Types; +import java.util.Optional; + +import static com.facebook.presto.common.type.DecimalType.createDecimalType; +import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; +import static com.facebook.presto.common.type.VarcharType.createVarcharType; +import static com.facebook.presto.plugin.jdbc.JdbcErrorCode.JDBC_ERROR; +import static com.facebook.presto.plugin.jdbc.StandardReadMappings.bigintReadMapping; +import static com.facebook.presto.plugin.jdbc.StandardReadMappings.decimalReadMapping; +import static com.facebook.presto.plugin.jdbc.StandardReadMappings.doubleReadMapping; +import static com.facebook.presto.plugin.jdbc.StandardReadMappings.realReadMapping; +import static com.facebook.presto.plugin.jdbc.StandardReadMappings.smallintReadMapping; +import static com.facebook.presto.plugin.jdbc.StandardReadMappings.varcharReadMapping; +import static com.facebook.presto.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; +import static java.util.Objects.requireNonNull; + +public class OracleClient + extends BaseJdbcClient +{ + private static final int FETCH_SIZE = 1000; + + private final boolean synonymsEnabled; + private final int numberDefaultScale; + + @Inject + public OracleClient( + JdbcConnectorId connectorId, + BaseJdbcConfig config, + OracleConfig oracleConfig, + ConnectionFactory connectionFactory) + { + super(connectorId, config, "\"", connectionFactory); + + requireNonNull(oracleConfig, "oracle config is null"); + this.synonymsEnabled = oracleConfig.isSynonymsEnabled(); + this.numberDefaultScale = oracleConfig.getNumberDefaultScale(); + } + + private String[] getTableTypes() + { + if (synonymsEnabled) { + return new String[] {"TABLE", "VIEW", "SYNONYM"}; + } + return new String[] {"TABLE", "VIEW"}; + } + + @Override + protected ResultSet getTables(Connection connection, Optional schemaName, Optional tableName) + throws SQLException + { + DatabaseMetaData metadata = connection.getMetaData(); + String escape = metadata.getSearchStringEscape(); + return metadata.getTables( + connection.getCatalog(), + escapeNamePattern(schemaName, Optional.of(escape)).orElse(null), + escapeNamePattern(tableName, Optional.of(escape)).orElse(null), + getTableTypes()); + } + @Override + public PreparedStatement getPreparedStatement(Connection connection, String sql) + throws SQLException + { + PreparedStatement statement = connection.prepareStatement(sql); + statement.setFetchSize(FETCH_SIZE); + return statement; + } + + @Override + protected String generateTemporaryTableName() + { + return "presto_tmp_" + System.nanoTime(); + } + + @Override + protected void renameTable(JdbcIdentity identity, String catalogName, SchemaTableName oldTable, SchemaTableName newTable) + { + if (!oldTable.getSchemaName().equalsIgnoreCase(newTable.getSchemaName())) { + throw new PrestoException(NOT_SUPPORTED, "Table rename across schemas is not supported in Oracle"); + } + + String newTableName = newTable.getTableName().toUpperCase(ENGLISH); + String oldTableName = oldTable.getTableName().toUpperCase(ENGLISH); + String sql = format( + "ALTER TABLE %s RENAME TO %s", + quoted(catalogName, oldTable.getSchemaName(), oldTableName), + quoted(newTableName)); + + try (Connection connection = connectionFactory.openConnection(identity)) { + execute(connection, sql); + } + catch (SQLException e) { + throw new PrestoException(JDBC_ERROR, e); + } + } + + @Override + public Optional toPrestoType(ConnectorSession session, JdbcTypeHandle typeHandle) + { + int columnSize = typeHandle.getColumnSize(); + + switch (typeHandle.getJdbcType()) { + case Types.CLOB: + return Optional.of(varcharReadMapping(createUnboundedVarcharType())); + case Types.SMALLINT: + return Optional.of(smallintReadMapping()); + case Types.FLOAT: + case Types.DOUBLE: + return Optional.of(doubleReadMapping()); + case Types.REAL: + return Optional.of(realReadMapping()); + case Types.NUMERIC: + int precision = columnSize == 0 ? Decimals.MAX_PRECISION : columnSize; + int scale = typeHandle.getDecimalDigits(); + + if (scale == 0) { + return Optional.of(bigintReadMapping()); + } + if (scale < 0 || scale > precision) { + return Optional.of(decimalReadMapping(createDecimalType(precision, numberDefaultScale))); + } + + return Optional.of(decimalReadMapping(createDecimalType(precision, scale))); + case Types.LONGVARCHAR: + if (columnSize > VarcharType.MAX_LENGTH || columnSize == 0) { + return Optional.of(varcharReadMapping(createUnboundedVarcharType())); + } + return Optional.of(varcharReadMapping(createVarcharType(columnSize))); + case Types.VARCHAR: + return Optional.of(varcharReadMapping(createVarcharType(columnSize))); + } + return super.toPrestoType(session, typeHandle); + } +} diff --git a/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleClientModule.java b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleClientModule.java new file mode 100644 index 0000000000000..b1e0ace06a71a --- /dev/null +++ b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleClientModule.java @@ -0,0 +1,58 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.plugin.jdbc.BaseJdbcConfig; +import com.facebook.presto.plugin.jdbc.ConnectionFactory; +import com.facebook.presto.plugin.jdbc.DriverConnectionFactory; +import com.facebook.presto.plugin.jdbc.JdbcClient; +import com.google.inject.Binder; +import com.google.inject.Module; +import com.google.inject.Provides; +import com.google.inject.Scopes; +import com.google.inject.Singleton; +import oracle.jdbc.OracleConnection; +import oracle.jdbc.OracleDriver; + +import java.sql.SQLException; +import java.util.Properties; + +import static com.facebook.airlift.configuration.ConfigBinder.configBinder; + +public class OracleClientModule + implements Module +{ + @Override + public void configure(Binder binder) + { + binder.bind(JdbcClient.class).to(OracleClient.class) + .in(Scopes.SINGLETON); + configBinder(binder).bindConfig(BaseJdbcConfig.class); + configBinder(binder).bindConfig(OracleConfig.class); + } + + @Provides + @Singleton + public static ConnectionFactory connectionFactory(BaseJdbcConfig config, OracleConfig oracleConfig) + throws SQLException + { + Properties connectionProperties = new Properties(); + connectionProperties.setProperty(OracleConnection.CONNECTION_PROPERTY_INCLUDE_SYNONYMS, String.valueOf(oracleConfig.isSynonymsEnabled())); + + return new DriverConnectionFactory( + new OracleDriver(), + config.getConnectionUrl(), + connectionProperties); + } +} diff --git a/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleConfig.java b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleConfig.java new file mode 100644 index 0000000000000..f651844a49dd3 --- /dev/null +++ b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OracleConfig.java @@ -0,0 +1,98 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.airlift.configuration.Config; + +import javax.validation.constraints.Max; +import javax.validation.constraints.Min; +import javax.validation.constraints.NotNull; + +import java.math.RoundingMode; + +public class OracleConfig +{ + private boolean synonymsEnabled; + private int varcharMaxSize = 4000; + private int timestampDefaultPrecision = 6; + private int numberDefaultScale = 10; + private RoundingMode numberRoundingMode = RoundingMode.HALF_UP; + + @NotNull + public boolean isSynonymsEnabled() + { + return synonymsEnabled; + } + + @Config("oracle.synonyms.enabled") + public OracleConfig setSynonymsEnabled(boolean enabled) + { + this.synonymsEnabled = enabled; + return this; + } + + @Min(0) + @Max(38) + public int getNumberDefaultScale() + { + return numberDefaultScale; + } + + @Config("oracle.number.default-scale") + public OracleConfig setNumberDefaultScale(int numberDefaultScale) + { + this.numberDefaultScale = numberDefaultScale; + return this; + } + + @NotNull + public RoundingMode getNumberRoundingMode() + { + return numberRoundingMode; + } + + @Config("oracle.number.rounding-mode") + public OracleConfig setNumberRoundingMode(RoundingMode numberRoundingMode) + { + this.numberRoundingMode = numberRoundingMode; + return this; + } + + @Min(4000) + public int getVarcharMaxSize() + { + return varcharMaxSize; + } + + @Config("oracle.varchar.max-size") + public OracleConfig setVarcharMaxSize(int varcharMaxSize) + { + this.varcharMaxSize = varcharMaxSize; + return this; + } + + @Min(0) + @Max(9) + public int getTimestampDefaultPrecision() + { + return timestampDefaultPrecision; + } + + @Config("oracle.timestamp.precision") + public OracleConfig setTimestampDefaultPrecision(int timestampDefaultPrecision) + { + this.timestampDefaultPrecision = timestampDefaultPrecision; + return this; + } +} diff --git a/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OraclePlugin.java b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OraclePlugin.java new file mode 100644 index 0000000000000..b81736c6cd07a --- /dev/null +++ b/presto-oracle/src/main/java/com/facebook/presto/plugin/oracle/OraclePlugin.java @@ -0,0 +1,30 @@ +/* + * 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 com.facebook.presto.plugin.oracle; +import com.facebook.presto.plugin.jdbc.JdbcPlugin; + +/** + * Initialize OraclePlugin class for prestoDB + */ +public class OraclePlugin + extends JdbcPlugin +{ + /** + * Oracle Plugin Constructor + */ + public OraclePlugin() + { + super("oracle", new OracleClientModule()); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/BaseOracleIntegrationSmokeTest.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/BaseOracleIntegrationSmokeTest.java new file mode 100644 index 0000000000000..c9a6848af0f34 --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/BaseOracleIntegrationSmokeTest.java @@ -0,0 +1,84 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; +import org.testcontainers.shaded.com.google.common.collect.ImmutableList; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.plugin.oracle.OracleQueryRunner.createOracleQueryRunner; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; +import static io.airlift.tpch.TpchTable.CUSTOMER; +import static io.airlift.tpch.TpchTable.NATION; +import static io.airlift.tpch.TpchTable.ORDERS; +import static io.airlift.tpch.TpchTable.REGION; +import static org.assertj.core.api.Assertions.assertThat; + +public abstract class BaseOracleIntegrationSmokeTest + extends AbstractTestIntegrationSmokeTest +{ + private final TestingOracleServer oracleServer; + + protected BaseOracleIntegrationSmokeTest(TestingOracleServer oracleServer) + { + super(() -> createOracleQueryRunner(oracleServer, ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION))); + this.oracleServer = new TestingOracleServer(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + oracleServer.close(); + } + + @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 + 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" + + ")"); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/OracleQueryRunner.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/OracleQueryRunner.java new file mode 100644 index 0000000000000..fdfb780c4270f --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/OracleQueryRunner.java @@ -0,0 +1,98 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.airlift.log.Logger; +import com.facebook.airlift.log.Logging; +import com.facebook.presto.Session; +import com.facebook.presto.tests.DistributedQueryRunner; +import com.facebook.presto.tpch.TpchPlugin; +import com.google.common.collect.ImmutableList; +import io.airlift.tpch.TpchTable; + +import java.util.HashMap; +import java.util.Map; + +import static com.facebook.airlift.testing.Closeables.closeAllSuppress; +import static com.facebook.presto.testing.TestingSession.testSessionBuilder; +import static com.facebook.presto.tests.QueryAssertions.copyTpchTables; +import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; + +public class OracleQueryRunner +{ + private OracleQueryRunner() {} + + public static DistributedQueryRunner createOracleQueryRunner(TestingOracleServer server) + throws Exception + { + return createOracleQueryRunner(server, ImmutableList.of()); + } + + public static DistributedQueryRunner createOracleQueryRunner(TestingOracleServer server, TpchTable... tables) + throws Exception + { + return createOracleQueryRunner(server, ImmutableList.copyOf(tables)); + } + + public static DistributedQueryRunner createOracleQueryRunner(TestingOracleServer server, Iterable> tables) + throws Exception + { + DistributedQueryRunner queryRunner = null; + try { + queryRunner = DistributedQueryRunner.builder(createSession()).build(); + + queryRunner.installPlugin(new TpchPlugin()); + queryRunner.createCatalog("tpch", "tpch"); + + Map connectorProperties = new HashMap<>(); + connectorProperties.putIfAbsent("connection-url", server.getJdbcUrl()); + connectorProperties.putIfAbsent("connection-user", TestingOracleServer.TEST_USER); + connectorProperties.putIfAbsent("connection-password", TestingOracleServer.TEST_PASS); + connectorProperties.putIfAbsent("allow-drop-table", "true"); + + queryRunner.installPlugin(new OraclePlugin()); + queryRunner.createCatalog("oracle", "oracle", connectorProperties); + + copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), tables); + + return queryRunner; + } + catch (Throwable e) { + closeAllSuppress(e, queryRunner, server); + throw e; + } + } + + public static Session createSession() + { + return testSessionBuilder() + .setCatalog("oracle") + .setSchema(TestingOracleServer.TEST_SCHEMA) + .build(); + } + + public static void main(String[] args) + throws Exception + { + Logging.initialize(); + + DistributedQueryRunner queryRunner = createOracleQueryRunner( + new TestingOracleServer(), + TpchTable.getTables()); + + Logger log = Logger.get(OracleQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleConfig.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleConfig.java new file mode 100644 index 0000000000000..e311d754e271e --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleConfig.java @@ -0,0 +1,59 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import java.math.RoundingMode; +import java.util.Map; + +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static com.facebook.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestOracleConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(OracleConfig.class) + .setSynonymsEnabled(false) + .setVarcharMaxSize(4000) + .setTimestampDefaultPrecision(6) + .setNumberDefaultScale(10) + .setNumberRoundingMode(RoundingMode.HALF_UP)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = new ImmutableMap.Builder() + .put("oracle.synonyms.enabled", "true") + .put("oracle.varchar.max-size", "10000") + .put("oracle.timestamp.precision", "3") + .put("oracle.number.default-scale", "2") + .put("oracle.number.rounding-mode", "CEILING") + .build(); + + OracleConfig expected = new OracleConfig() + .setSynonymsEnabled(true) + .setVarcharMaxSize(10000) + .setTimestampDefaultPrecision(3) + .setNumberDefaultScale(2) + .setNumberRoundingMode(RoundingMode.CEILING); + + assertFullMapping(properties, expected); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleDistributedQueries.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleDistributedQueries.java new file mode 100644 index 0000000000000..f7af17e86e33d --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleDistributedQueries.java @@ -0,0 +1,174 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.AbstractTestDistributedQueries; +import io.airlift.tpch.TpchTable; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.plugin.oracle.OracleQueryRunner.createOracleQueryRunner; +import static com.facebook.presto.testing.MaterializedResult.resultBuilder; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; +import static java.lang.String.format; +import static java.util.stream.Collectors.joining; +import static java.util.stream.IntStream.range; +import static org.testng.Assert.assertFalse; +import static org.testng.Assert.assertTrue; + +public class TestOracleDistributedQueries + extends AbstractTestDistributedQueries +{ + private final TestingOracleServer oracleServer; + + protected TestOracleDistributedQueries(TestingOracleServer oracleServer) + { + super(() -> createOracleQueryRunner(oracleServer, TpchTable.getTables())); + this.oracleServer = new TestingOracleServer(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + if (oracleServer != null) { + oracleServer.close(); + } + } + + @Override + protected boolean supportsViews() + { + return false; + } + + @Test + @Override + public void testLargeIn() + { + int numberOfElements = 1000; + String longValues = range(0, numberOfElements) + .mapToObj(Integer::toString) + .collect(joining(", ")); + assertQuery("SELECT orderkey FROM orders WHERE orderkey IN (" + longValues + ")"); + assertQuery("SELECT orderkey FROM orders WHERE orderkey NOT IN (" + longValues + ")"); + + assertQuery("SELECT orderkey FROM orders WHERE orderkey IN (mod(1000, orderkey), " + longValues + ")"); + assertQuery("SELECT orderkey FROM orders WHERE orderkey NOT IN (mod(1000, orderkey), " + longValues + ")"); + + String arrayValues = range(0, numberOfElements) + .mapToObj(i -> format("ARRAY[%s, %s, %s]", i, i + 1, i + 2)) + .collect(joining(", ")); + assertQuery("SELECT ARRAY[0, 0, 0] in (ARRAY[0, 0, 0], " + arrayValues + ")", "values true"); + assertQuery("SELECT ARRAY[0, 0, 0] in (" + arrayValues + ")", "values false"); + } + + @Test + @Override + public void testCreateTable() + { + assertUpdate("CREATE TABLE test_create (a bigint, b double, c varchar)"); + assertTrue(getQueryRunner().tableExists(getSession(), "test_create")); + assertTableColumnNames("test_create", "a", "b", "c"); + + assertUpdate("DROP TABLE test_create"); + assertFalse(getQueryRunner().tableExists(getSession(), "test_create")); + + assertQueryFails("CREATE TABLE test_create (a bad_type)", ".* Unknown type 'bad_type' for column 'a'"); + assertFalse(getQueryRunner().tableExists(getSession(), "test_create")); + + // Replace test_create_table_if_not_exists with test_create_table_if_not_exist to fetch max size naming on oracle + assertUpdate("CREATE TABLE test_create_table_if_not_exist (a bigint, b varchar, c double)"); + assertTrue(getQueryRunner().tableExists(getSession(), "test_create_table_if_not_exist")); + assertTableColumnNames("test_create_table_if_not_exist", "a", "b", "c"); + + assertUpdate("CREATE TABLE IF NOT EXISTS test_create_table_if_not_exist (d bigint, e varchar)"); + assertTrue(getQueryRunner().tableExists(getSession(), "test_create_table_if_not_exist")); + assertTableColumnNames("test_create_table_if_not_exist", "a", "b", "c"); + + assertUpdate("DROP TABLE test_create_table_if_not_exist"); + assertFalse(getQueryRunner().tableExists(getSession(), "test_create_table_if_not_exist")); + + // Test CREATE TABLE LIKE + assertUpdate("CREATE TABLE test_create_original (a bigint, b double, c varchar)"); + assertTrue(getQueryRunner().tableExists(getSession(), "test_create_original")); + assertTableColumnNames("test_create_original", "a", "b", "c"); + + assertUpdate("CREATE TABLE test_create_like (LIKE test_create_original, d boolean, e varchar)"); + assertTrue(getQueryRunner().tableExists(getSession(), "test_create_like")); + assertTableColumnNames("test_create_like", "a", "b", "c", "d", "e"); + + assertUpdate("DROP TABLE test_create_original"); + assertFalse(getQueryRunner().tableExists(getSession(), "test_create_original")); + + assertUpdate("DROP TABLE test_create_like"); + assertFalse(getQueryRunner().tableExists(getSession(), "test_create_like")); + } + + @Test + @Override + public void testSymbolAliasing() + { + // Replace tablename to less than 30chars, max size naming on oracle + String tableName = "symbol_aliasing" + System.currentTimeMillis(); + assertUpdate("CREATE TABLE " + tableName + " AS SELECT 1 foo_1, 2 foo_2_4", 1); + assertQuery("SELECT foo_1, foo_2_4 FROM " + tableName, "SELECT 1, 2"); + assertUpdate("DROP TABLE " + tableName); + } + + @Test + @Override + public void testRenameColumn() + { + // Replace tablename to less than 30chars, max size naming on oracle + String tableName = "test_renamecol_" + System.currentTimeMillis(); + assertUpdate("CREATE TABLE " + tableName + " AS SELECT 'some value' x", 1); + + assertUpdate("ALTER TABLE " + tableName + " RENAME COLUMN x TO y"); + assertQuery("SELECT y FROM " + tableName, "VALUES 'some value'"); + + assertUpdate("ALTER TABLE " + tableName + " RENAME COLUMN y TO Z"); // 'Z' is upper-case, not delimited + assertQuery( + "SELECT z FROM " + tableName, // 'z' is lower-case, not delimited + "VALUES 'some value'"); + + // There should be exactly one column + assertQuery("SELECT * FROM " + tableName, "VALUES 'some value'"); + + assertUpdate("DROP TABLE " + tableName); + } + + @Test + @Override + public void testShowColumns() + { + MaterializedResult actual = computeActual("SHOW COLUMNS FROM orders"); + + MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "", "") + .row("custkey", "bigint", "", "") + .row("orderstatus", "varchar(1)", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "timestamp", "", "") + .row("orderpriority", "varchar(15)", "", "") + .row("clerk", "varchar(15)", "", "") + .row("shippriority", "bigint", "", "") + .row("comment", "varchar(79)", "", "") + .build(); + + // Until we migrate all connectors to parametrized varchar we check two options + assertEquals(actual, expectedParametrizedVarchar, format("%s does not matches %s", actual, expectedParametrizedVarchar)); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleIntegrationSmokeTest.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleIntegrationSmokeTest.java new file mode 100644 index 0000000000000..5367276cf0353 --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleIntegrationSmokeTest.java @@ -0,0 +1,61 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.testing.MaterializedResult; +import com.facebook.presto.tests.AbstractTestIntegrationSmokeTest; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import static com.facebook.presto.common.type.VarcharType.VARCHAR; +import static com.facebook.presto.plugin.oracle.OracleQueryRunner.createOracleQueryRunner; +import static com.facebook.presto.testing.assertions.Assert.assertEquals; +import static io.airlift.tpch.TpchTable.ORDERS; + +public class TestOracleIntegrationSmokeTest + extends AbstractTestIntegrationSmokeTest +{ + private final TestingOracleServer oracleServer; + + protected TestOracleIntegrationSmokeTest(TestingOracleServer oracleServer) + { + super(() -> createOracleQueryRunner(oracleServer, ORDERS)); + this.oracleServer = new TestingOracleServer(); + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + oracleServer.close(); + } + + @Test + @Override + public void testDescribeTable() + { + MaterializedResult expectedColumns = MaterializedResult.resultBuilder(getQueryRunner().getDefaultSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR) + .row("orderkey", "bigint", "", "") + .row("custkey", "bigint", "", "") + .row("orderstatus", "varchar(1)", "", "") + .row("totalprice", "double", "", "") + .row("orderdate", "timestamp", "", "") + .row("orderpriority", "varchar(15)", "", "") + .row("clerk", "varchar(15)", "", "") + .row("shippriority", "bigint", "", "") + .row("comment", "varchar(79)", "", "") + .build(); + MaterializedResult actualColumns = computeActual("DESCRIBE orders"); + assertEquals(actualColumns, expectedColumns); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOraclePlugin.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOraclePlugin.java new file mode 100644 index 0000000000000..a690ec6f6fb21 --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOraclePlugin.java @@ -0,0 +1,33 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.spi.Plugin; +import com.facebook.presto.spi.connector.ConnectorFactory; +import com.facebook.presto.testing.TestingConnectorContext; +import com.google.common.collect.ImmutableMap; +import org.testng.annotations.Test; + +import static com.google.common.collect.Iterables.getOnlyElement; + +public class TestOraclePlugin +{ + @Test + public void testCreateConnector() + { + Plugin plugin = new OraclePlugin(); + ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories()); + factory.create("test", ImmutableMap.of("connection-url", "jdbc:oracle:thin//test"), new TestingConnectorContext()); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleTypes.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleTypes.java new file mode 100644 index 0000000000000..a92f7c6c57c34 --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestOracleTypes.java @@ -0,0 +1,127 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import com.facebook.presto.common.type.BigintType; +import com.facebook.presto.common.type.DecimalType; +import com.facebook.presto.common.type.Type; +import com.facebook.presto.tests.AbstractTestQueryFramework; +import com.facebook.presto.tests.datatype.CreateAsSelectDataSetup; +import com.facebook.presto.tests.datatype.DataSetup; +import com.facebook.presto.tests.datatype.DataType; +import com.facebook.presto.tests.datatype.DataTypeTest; +import com.facebook.presto.tests.sql.PrestoSqlExecutor; +import org.testcontainers.containers.OracleContainer; +import org.testng.annotations.AfterClass; +import org.testng.annotations.Test; + +import java.math.BigDecimal; +import java.util.function.Function; + +import static com.facebook.presto.common.type.VarcharType.createUnboundedVarcharType; +import static com.facebook.presto.common.type.VarcharType.createVarcharType; +import static com.facebook.presto.plugin.oracle.OracleQueryRunner.createOracleQueryRunner; +import static com.facebook.presto.tests.datatype.DataType.stringDataType; +import static com.facebook.presto.tests.datatype.DataType.varcharDataType; +import static java.lang.String.format; +import static java.math.RoundingMode.HALF_UP; + +public class TestOracleTypes + extends AbstractTestQueryFramework +{ + private final TestingOracleServer oracleServer; + + @Test + public void test() + { + OracleContainer oracle = new OracleContainer("wnameless/oracle-xe-11g-r2"); + oracle.start(); + } + + private TestOracleTypes(TestingOracleServer oracleServer) + { + super(() -> createOracleQueryRunner(oracleServer)); + this.oracleServer = oracleServer; + } + + @AfterClass(alwaysRun = true) + public final void destroy() + { + if (oracleServer != null) { + oracleServer.close(); + } + } + + private DataSetup prestoCreateAsSelect(String tableNamePrefix) + { + return new CreateAsSelectDataSetup(new PrestoSqlExecutor(getQueryRunner()), tableNamePrefix); + } + + @Test + public void testBooleanType() + { + DataTypeTest.create() + .addRoundTrip(booleanOracleType(), true) + .addRoundTrip(booleanOracleType(), false) + .execute(getQueryRunner(), prestoCreateAsSelect("boolean_types")); + } + + @Test + public void testSpecialNumberFormats() + { + oracleServer.execute("CREATE TABLE test (num1 number)"); + oracleServer.execute("INSERT INTO test VALUES (12345678901234567890.12345678901234567890123456789012345678)"); + assertQuery("SELECT * FROM test", "VALUES (12345678901234567890.1234567890)"); + } + + @Test + public void testVarcharType() + { + DataTypeTest.create() + .addRoundTrip(varcharDataType(10), "test") + .addRoundTrip(stringDataType("varchar", createVarcharType(4000)), "test") + .addRoundTrip(stringDataType("varchar(5000)", createUnboundedVarcharType()), "test") + .addRoundTrip(varcharDataType(3), String.valueOf('\u2603')) + .execute(getQueryRunner(), prestoCreateAsSelect("varchar_types")); + } + + private static DataType booleanOracleType() + { + return DataType.dataType( + "boolean", + BigintType.BIGINT, + value -> value ? "1" : "0", + value -> value ? 1L : 0L); + } + + private static DataType numberOracleType(DecimalType type) + { + String databaseType = format("decimal(%s, %s)", type.getPrecision(), type.getScale()); + return numberOracleType(databaseType, type); + } + + private static DataType numberOracleType(String inputType, Type resultType) + { + Function queryResult = (Function) value -> + (value instanceof BigDecimal && resultType instanceof DecimalType) + ? ((BigDecimal) value).setScale(((DecimalType) resultType).getScale(), HALF_UP) + : value; + + return DataType.dataType( + inputType, + resultType, + value -> format("CAST('%s' AS %s)", value, resultType), + queryResult); + } +} diff --git a/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestingOracleServer.java b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestingOracleServer.java new file mode 100644 index 0000000000000..f7ac933bd1b5b --- /dev/null +++ b/presto-oracle/src/test/java/com/facebook/presto/plugin/oracle/TestingOracleServer.java @@ -0,0 +1,100 @@ +/* + * 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 com.facebook.presto.plugin.oracle; + +import org.testcontainers.containers.OracleContainer; +import org.testcontainers.utility.MountableFile; + +import java.io.Closeable; +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; + +import static java.lang.String.format; + +public class TestingOracleServer + extends OracleContainer + implements Closeable +{ + private static final String TEST_TABLESPACE = "presto_test"; + + public static final String TEST_USER = "presto_test"; + public static final String TEST_SCHEMA = TEST_USER; // schema and user is the same thing in Oracle + public static final String TEST_PASS = "presto_test_password"; + + public TestingOracleServer() + { + super("wnameless/oracle-xe-11g-r2"); + + withCopyFileToContainer(MountableFile.forClasspathResource("init.sql"), "/docker-entrypoint-initdb.d/init.sql"); + + start(); + + try (Connection connection = DriverManager.getConnection(getJdbcUrl(), getUsername(), getPassword()); + Statement statement = connection.createStatement()) { + // this is added to allow more processes on database, otherwise the tests end up giving + // ORA-12519, TNS:no appropriate service handler found + // ORA-12505, TNS:listener does not currently know of SID given in connect descriptor + // to fix this we have to change the number of processes of SPFILE + statement.execute("ALTER SYSTEM SET processes=1000 SCOPE=SPFILE"); + statement.execute("ALTER SYSTEM SET disk_asynch_io = FALSE SCOPE = SPFILE"); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + + try { + execInContainer("/bin/bash", "/etc/init.d/oracle-xe", "restart"); + } + catch (InterruptedException | IOException e) { + throw new RuntimeException(e); + } + + waitUntilContainerStarted(); + try (Connection connection = DriverManager.getConnection(getJdbcUrl(), getUsername(), getPassword()); + Statement statement = connection.createStatement()) { + statement.execute(format("CREATE TABLESPACE %s DATAFILE 'test_db.dat' SIZE 100M ONLINE", TEST_TABLESPACE)); + statement.execute(format("CREATE USER %s IDENTIFIED BY %s DEFAULT TABLESPACE %s", TEST_USER, TEST_PASS, TEST_TABLESPACE)); + statement.execute(format("GRANT UNLIMITED TABLESPACE TO %s", TEST_USER)); + statement.execute(format("GRANT ALL PRIVILEGES TO %s", TEST_USER)); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + public void execute(String sql) + { + execute(sql, TEST_USER, TEST_PASS); + } + + public void execute(String sql, String user, String password) + { + try (Connection connection = DriverManager.getConnection(getJdbcUrl(), user, password); + Statement statement = connection.createStatement()) { + statement.execute(sql); + } + catch (SQLException e) { + throw new RuntimeException(e); + } + } + + @Override + public void close() + { + stop(); + } +} diff --git a/presto-oracle/src/test/resources/init.sql b/presto-oracle/src/test/resources/init.sql new file mode 100644 index 0000000000000..978629f80ffcc --- /dev/null +++ b/presto-oracle/src/test/resources/init.sql @@ -0,0 +1,2 @@ +/* Drop apex_040000 because it has 601 tables and leads to slow tests */ +DROP USER apex_040000 CASCADE; diff --git a/presto-server/pom.xml b/presto-server/pom.xml index a3d36506f2b91..24bd52349187f 100644 --- a/presto-server/pom.xml +++ b/presto-server/pom.xml @@ -180,6 +180,14 @@ provided + + com.facebook.presto + presto-oracle + ${project.version} + zip + provided + + com.facebook.presto presto-postgresql diff --git a/presto-server/src/main/assembly/presto.xml b/presto-server/src/main/assembly/presto.xml index c26cb2646b2fe..a54f5062db535 100644 --- a/presto-server/src/main/assembly/presto.xml +++ b/presto-server/src/main/assembly/presto.xml @@ -120,6 +120,10 @@ ${project.build.directory}/dependency/presto-mysql-${project.version} plugin/mysql + + ${project.build.directory}/dependency/presto-oracle-${project.version} + plugin/oracle + ${project.build.directory}/dependency/presto-postgresql-${project.version} plugin/postgresql