Skip to content

Commit

Permalink
Refactor and improve coverage for Oracle integration tests
Browse files Browse the repository at this point in the history
  • Loading branch information
aalbu authored and losipiuk committed Sep 18, 2020
1 parent 2fe7727 commit 0893543
Show file tree
Hide file tree
Showing 6 changed files with 236 additions and 63 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -14,44 +14,25 @@
package io.prestosql.plugin.oracle;

import com.google.common.collect.ImmutableList;
import io.prestosql.sql.planner.plan.FilterNode;
import io.prestosql.testing.AbstractTestIntegrationSmokeTest;
import io.prestosql.testing.MaterializedResult;
import io.prestosql.testing.QueryRunner;
import io.prestosql.tpch.TpchTable;
import org.testng.annotations.AfterClass;
import io.prestosql.testing.sql.SqlExecutor;
import io.prestosql.testing.sql.TestTable;
import org.testng.annotations.Test;

import static io.prestosql.plugin.oracle.TestingOracleServer.TEST_USER;
import static io.prestosql.spi.type.VarcharType.VARCHAR;
import static io.prestosql.testing.assertions.Assert.assertEquals;
import static io.prestosql.tpch.TpchTable.CUSTOMER;
import static io.prestosql.tpch.TpchTable.NATION;
import static io.prestosql.tpch.TpchTable.ORDERS;
import static io.prestosql.tpch.TpchTable.REGION;
import static io.prestosql.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;

abstract class BaseOracleIntegrationSmokeTest
public abstract class BaseOracleIntegrationSmokeTest
extends AbstractTestIntegrationSmokeTest
{
private TestingOracleServer oracleServer;

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
oracleServer = new TestingOracleServer();
return createOracleQueryRunner(oracleServer, ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION));
}

protected abstract QueryRunner createOracleQueryRunner(TestingOracleServer server, Iterable<TpchTable<?>> tables)
throws Exception;

@AfterClass(alwaysRun = true)
public final void destroy()
{
oracleServer.close();
}

@Test
@Override
public void testDescribeTable()
Expand Down Expand Up @@ -90,9 +71,35 @@ public void testShowCreateTable()
")");
}

@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()
throws Exception
{
// varchar equality
assertThat(query("SELECT regionkey, nationkey, name FROM nation WHERE name = 'ROMANIA'"))
Expand All @@ -113,39 +120,62 @@ public void testPredicatePushdown()
assertThat(query("SELECT orderkey FROM orders WHERE orderdate = DATE '1992-09-29'"))
.matches("VALUES CAST(1250 AS DECIMAL(19,0)), 34406, 38436, 57570")
.isCorrectlyPushedDown();
}

predicatePushdownTest("decimal(9, 3)", "123.321", "<=", "124", "CAST(123.321 AS decimal(9, 3))");
predicatePushdownTest("decimal(9, 3)", "123.321", "<=", "123.321", "CAST(123.321 AS decimal(9, 3))");
predicatePushdownTest("decimal(9, 3)", "123.321", "=", "123.321", "CAST(123.321 AS decimal(9, 3))");
predicatePushdownTest("decimal(30, 10)", "123456789.987654321", "<=", "123456790", "CAST(123456789.987654321 AS decimal(30, 10))");
predicatePushdownTest("decimal(30, 10)", "123456789.987654321", "<=", "123456789.987654321", "CAST(123456789.987654321 AS decimal(30, 10))");
predicatePushdownTest("decimal(30, 10)", "123456789.987654321", "=", "123456789.987654321", "CAST(123456789.987654321 AS decimal(30, 10))");
predicatePushdownTest("float(63)", "123456789.987654321", "<=", "CAST(123456789.99 AS REAL)", "CAST(123456789.987654321 AS DOUBLE)");
predicatePushdownTest("float(63)", "123456789.987654321", "<=", "CAST(123456789.99 AS DOUBLE)", "CAST(123456789.987654321 AS DOUBLE)");
predicatePushdownTest("float(126)", "123456789.987654321", "<=", "CAST(123456789.99 AS REAL)", "CAST(123456789.987654321 AS DOUBLE)");
predicatePushdownTest("float(126)", "123456789.987654321", "<=", "CAST(123456789.99 AS DOUBLE)", "CAST(123456789.987654321 AS DOUBLE)");
predicatePushdownTest("CHAR(1)", "'0'", "=", "'0'", "CHAR'0'");
predicatePushdownTest("CHAR(1)", "'0'", "<=", "'0'", "CHAR'0'");
predicatePushdownTest("CHAR(5)", "'0'", "=", "CHAR'0'", "CHAR'0 '");
@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);
}
}

private void predicatePushdownTest(String oracleType, String oracleLiteral, String operator, String filterLiteral, String compareLiteral)
throws Exception
private void predicatePushdownTest(String oracleType, String oracleLiteral, String operator, String filterLiteral)
{
String tableName = "test_pushdown_" + oracleType.replaceAll("[^a-zA-Z0-9]", "");
try (AutoCloseable ignored = withTable(tableName, format("(c %s)", oracleType))) {
oracleServer.execute(format("INSERT INTO %s VALUES (%s)", tableName, oracleLiteral));
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", tableName, operator, filterLiteral)))
.matches(format("VALUES (%s)", compareLiteral))
assertThat(query(format("SELECT * FROM %s WHERE c %s %s", table.getName(), operator, filterLiteral)))
.isCorrectlyPushedDown();
}
}

private AutoCloseable withTable(String tableName, String tableDefinition)
throws Exception
protected String getUser()
{
oracleServer.execute(format("CREATE TABLE %s%s", tableName, tableDefinition));
return () -> oracleServer.execute(format("DROP TABLE %s", tableName));
return TEST_USER;
}

protected abstract SqlExecutor onOracle();
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,15 +13,43 @@
*/
package io.prestosql.plugin.oracle;

import com.google.common.collect.ImmutableList;
import io.airlift.testing.Closeables;
import io.prestosql.testing.QueryRunner;
import io.prestosql.tpch.TpchTable;
import io.prestosql.testing.sql.SqlExecutor;
import org.testng.annotations.AfterClass;

import java.io.IOException;

import static io.prestosql.tpch.TpchTable.CUSTOMER;
import static io.prestosql.tpch.TpchTable.NATION;
import static io.prestosql.tpch.TpchTable.ORDERS;
import static io.prestosql.tpch.TpchTable.REGION;

public class TestOracleIntegrationSmokeTest
extends BaseOracleIntegrationSmokeTest
{
private TestingOracleServer oracleServer;

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
oracleServer = new TestingOracleServer();
return OracleQueryRunner.createOracleQueryRunner(oracleServer, ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION));
}

@AfterClass(alwaysRun = true)
public final void destroy()
throws IOException
{
Closeables.closeAll(oracleServer);
oracleServer = null;
}

@Override
protected QueryRunner createOracleQueryRunner(TestingOracleServer server, Iterable<TpchTable<?>> tables) throws Exception
protected SqlExecutor onOracle()
{
return OracleQueryRunner.createOracleQueryRunner(server, tables);
return oracleServer::execute;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -13,15 +13,43 @@
*/
package io.prestosql.plugin.oracle;

import com.google.common.collect.ImmutableList;
import io.airlift.testing.Closeables;
import io.prestosql.testing.QueryRunner;
import io.prestosql.tpch.TpchTable;
import io.prestosql.testing.sql.SqlExecutor;
import org.testng.annotations.AfterClass;

import java.io.IOException;

import static io.prestosql.tpch.TpchTable.CUSTOMER;
import static io.prestosql.tpch.TpchTable.NATION;
import static io.prestosql.tpch.TpchTable.ORDERS;
import static io.prestosql.tpch.TpchTable.REGION;

public class TestOraclePoolIntegrationSmokeTest
extends BaseOracleIntegrationSmokeTest
{
private TestingOracleServer oracleServer;

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
oracleServer = new TestingOracleServer();
return OracleQueryRunner.createOraclePoolQueryRunner(oracleServer, ImmutableList.of(CUSTOMER, NATION, ORDERS, REGION));
}

@AfterClass(alwaysRun = true)
public final void destroy()
throws IOException
{
Closeables.closeAll(oracleServer);
oracleServer = null;
}

@Override
protected QueryRunner createOracleQueryRunner(TestingOracleServer server, Iterable<TpchTable<?>> tables) throws Exception
protected SqlExecutor onOracle()
{
return OracleQueryRunner.createOraclePoolQueryRunner(server, tables);
return oracleServer::execute;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.prestosql.plugin.oracle;

import io.prestosql.testing.sql.SqlExecutor;

import java.io.Closeable;

import static io.prestosql.testing.sql.TestTable.randomTableSuffix;
import static java.lang.String.format;

public class TestSynonym
implements Closeable
{
private final SqlExecutor sqlExecutor;
private final String name;

public TestSynonym(SqlExecutor sqlExecutor, String namePrefix, String definition)
{
this.sqlExecutor = sqlExecutor;
this.name = namePrefix + "_" + randomTableSuffix();
sqlExecutor.execute(format("CREATE SYNONYM %s %s", name, definition));
}

public String getName()
{
return name;
}

@Override
public void close()
{
sqlExecutor.execute("DROP SYNONYM " + name);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* 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.prestosql.plugin.oracle;

import io.prestosql.testing.sql.SqlExecutor;

import java.io.Closeable;

import static io.prestosql.testing.sql.TestTable.randomTableSuffix;
import static java.lang.String.format;

public class TestView
implements Closeable
{
private final SqlExecutor sqlExecutor;
private final String name;

public TestView(SqlExecutor sqlExecutor, String namePrefix, String definition)
{
this.sqlExecutor = sqlExecutor;
this.name = namePrefix + "_" + randomTableSuffix();
sqlExecutor.execute(format("CREATE VIEW %s %s", name, definition));
}

public String getName()
{
return name;
}

@Override
public void close()
{
sqlExecutor.execute("DROP VIEW " + name);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import java.sql.Connection;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Properties;

import static io.prestosql.testing.TestingConnectorSession.SESSION;
import static java.lang.String.format;
Expand All @@ -52,10 +51,6 @@ public TestingOracleServer()

start();

Properties properties = new Properties();
properties.put("user", getUsername());
properties.put("password", getPassword());

try (Connection connection = getConnectionFactory().openConnection(IDENTITY);
Statement statement = connection.createStatement()) {
// this is added to allow more processes on database, otherwise the tests end up giving
Expand Down

0 comments on commit 0893543

Please sign in to comment.