Skip to content

Commit

Permalink
Migrate to BaseConnectorTest in BigQuery
Browse files Browse the repository at this point in the history
  • Loading branch information
ebyhr committed Feb 25, 2022
1 parent dfeef89 commit 2b67dc9
Showing 1 changed file with 203 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -13,10 +13,12 @@
*/
package io.trino.plugin.bigquery;

import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import io.trino.testing.AbstractTestIntegrationSmokeTest;
import io.trino.testing.BaseConnectorTest;
import io.trino.testing.MaterializedResult;
import io.trino.testing.QueryRunner;
import io.trino.testing.TestingConnectorBehavior;
import io.trino.testing.sql.TestTable;
import io.trino.testing.sql.TestView;
import org.testng.annotations.BeforeClass;
Expand All @@ -25,19 +27,21 @@

import java.util.List;

import static com.google.common.base.Strings.nullToEmpty;
import static io.trino.plugin.bigquery.BigQueryQueryRunner.BigQuerySqlExecutor;
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 java.util.Locale.ENGLISH;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;

public class TestBigQueryConnectorTest
// TODO extend BaseConnectorTest
extends AbstractTestIntegrationSmokeTest
extends BaseConnectorTest
{
protected BigQuerySqlExecutor bigQuerySqlExecutor;

Expand All @@ -56,7 +60,30 @@ protected QueryRunner createQueryRunner()
ImmutableMap.of());
}

@Override
protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
{
switch (connectorBehavior) {
case SUPPORTS_TOPN_PUSHDOWN:
case SUPPORTS_RENAME_SCHEMA:
case SUPPORTS_RENAME_TABLE:
case SUPPORTS_NOT_NULL_CONSTRAINT:
case SUPPORTS_CREATE_TABLE_WITH_DATA:
case SUPPORTS_DELETE:
case SUPPORTS_INSERT:
case SUPPORTS_ADD_COLUMN:
case SUPPORTS_DROP_COLUMN:
case SUPPORTS_RENAME_COLUMN:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
return false;
default:
return super.hasBehavior(connectorBehavior);
}
}

@Test
@Override
public void testCreateSchema()
{
String schemaName = "test_schema_create_" + randomTableSuffix();
Expand All @@ -82,6 +109,28 @@ public void testCreateSchema()
assertUpdate("DROP SCHEMA IF EXISTS " + schemaName);
}

@Test
@Override
public void testShowColumns()
{
// shippriority column is bigint (not integer) in BigQuery connector
MaterializedResult actual = computeActual("SHOW COLUMNS FROM orders");

MaterializedResult expectedParametrizedVarchar = resultBuilder(getSession(), VARCHAR, VARCHAR, VARCHAR, VARCHAR)
.row("orderkey", "bigint", "", "")
.row("custkey", "bigint", "", "")
.row("orderstatus", "varchar", "", "")
.row("totalprice", "double", "", "")
.row("orderdate", "date", "", "")
.row("orderpriority", "varchar", "", "")
.row("clerk", "varchar", "", "")
.row("shippriority", "bigint", "", "")
.row("comment", "varchar", "", "")
.build();

assertEquals(actual, expectedParametrizedVarchar);
}

@Override
public void testDescribeTable()
{
Expand Down Expand Up @@ -181,6 +230,22 @@ public void testCreateTableIfNotExists()
}
}

@Test
@Override
public void testCreateTableAsSelect()
{
assertThatThrownBy(super::testCreateTableAsSelect)
.hasStackTraceContaining("This connector does not support creating tables with data");
}

@Test
@Override
public void testCreateTableAsSelectWithUnicode()
{
assertThatThrownBy(super::testCreateTableAsSelectWithUnicode)
.hasStackTraceContaining("This connector does not support creating tables with data");
}

@Test
public void testDropTable()
{
Expand All @@ -192,6 +257,80 @@ public void testDropTable()
assertFalse(getQueryRunner().tableExists(getSession(), tableName));
}

@Test
@Override
public void testRenameTable()
{
// Use CREATE TABLE instead of CREATE TABLE AS statement
String tableName = "test_rename_" + randomTableSuffix();
assertUpdate("CREATE TABLE " + tableName + " (x int)");

String renamedTable = "test_rename_new_" + randomTableSuffix();
assertQueryFails("ALTER TABLE " + tableName + " RENAME TO " + renamedTable, "This connector does not support renaming tables");
}

@Test(dataProvider = "testDataMappingSmokeTestDataProvider")
@Override
public void testDataMappingSmokeTest(DataMappingTestSetup dataMappingTestSetup)
{
assertThatThrownBy(() -> super.testDataMappingSmokeTest(dataMappingTestSetup))
.hasMessageContaining("This connector does not support creating tables with data");
}

@Test(dataProvider = "testCaseSensitiveDataMappingProvider")
@Override
public void testCaseSensitiveDataMapping(DataMappingTestSetup dataMappingTestSetup)
{
assertThatThrownBy(() -> super.testCaseSensitiveDataMapping(dataMappingTestSetup))
.hasMessageContaining("This connector does not support creating tables with data");
}

@Override
protected void testColumnName(String columnName, boolean delimited)
{
// Override because BigQuery connector doesn't support INSERT statement
String nameInSql = columnName;
if (delimited) {
nameInSql = "\"" + columnName.replace("\"", "\"\"") + "\"";
}
String tableName = "test.tcn_" + nameInSql.toLowerCase(ENGLISH).replaceAll("[^a-z0-9]", "") + randomTableSuffix();

try {
// TODO test with both CTAS *and* CREATE TABLE + INSERT, since they use different connector API methods.
assertUpdate("CREATE TABLE " + tableName + "(key varchar(50), " + nameInSql + " varchar(50))");
}
catch (RuntimeException e) {
if (isColumnNameRejected(e, columnName, delimited)) {
// It is OK if give column name is not allowed and is clearly rejected by the connector.
return;
}
throw e;
}
try {
// Execute INSERT statement in BigQuery
onBigQuery("INSERT INTO " + tableName + " VALUES ('null value', NULL), ('sample value', 'abc'), ('other value', 'xyz')");

// SELECT *
assertQuery("SELECT * FROM " + tableName, "VALUES ('null value', NULL), ('sample value', 'abc'), ('other value', 'xyz')");

// projection
assertQuery("SELECT " + nameInSql + " FROM " + tableName, "VALUES (NULL), ('abc'), ('xyz')");

// predicate
assertQuery("SELECT key FROM " + tableName + " WHERE " + nameInSql + " IS NULL", "VALUES ('null value')");
assertQuery("SELECT key FROM " + tableName + " WHERE " + nameInSql + " = 'abc'", "VALUES ('sample value')");
}
finally {
assertUpdate("DROP TABLE " + tableName);
}
}

@Override
protected boolean isColumnNameRejected(Exception exception, String columnName, boolean delimited)
{
return nullToEmpty(exception.getMessage()).matches(".*(Fields must contain only letters, numbers, and underscores, start with a letter or underscore, and be at most 300 characters long).*");
}

@Test(enabled = false)
public void testSelectFromHourlyPartitionedTable()
{
Expand Down Expand Up @@ -323,6 +462,46 @@ public void testShowCreateTable()
")");
}

@Test
@Override
public void testCharVarcharComparison()
{
// BigQuery doesn't have char type
assertThatThrownBy(super::testCharVarcharComparison)
.hasMessage("This connector does not support creating tables with data");
}

@Test
@Override
public void testVarcharCharComparison()
{
// Use BigQuery SQL executor because the connector doesn't support INSERT statement
try (TestTable table = new TestTable(
bigQuerySqlExecutor,
"test.test_varchar_char",
"(k int, v string(3))",
ImmutableList.of(
"-1, NULL",
"0, ''",
"1, ' '",
"2, ' '",
"3, ' '",
"4, 'x'",
"5, 'x '",
"6, 'x '"))) {
assertQuery(
"SELECT k, v FROM " + table.getName() + " WHERE v = CAST(' ' AS char(2))",
// The 3-spaces value is included because both sides of the comparison are coerced to char(3)
"VALUES (0, ''), (1, ' '), (2, ' '), (3, ' ')");

// value that's not all-spaces
assertQuery(
"SELECT k, v FROM " + table.getName() + " WHERE v = CAST('x ' AS char(2))",
// The 3-spaces value is included because both sides of the comparison are coerced to char(3)
"VALUES (4, 'x'), (5, 'x '), (6, 'x ')");
}
}

@Test
public void testSkipUnsupportedType()
{
Expand All @@ -340,6 +519,27 @@ public void testSkipUnsupportedType()
}
}

@Test
@Override
public void testDateYearOfEraPredicate()
{
// Override because the connector throws an exception instead of an empty result when the value is out of supported range
assertQuery("SELECT orderdate FROM orders WHERE orderdate = DATE '1997-09-14'", "VALUES DATE '1997-09-14'");
assertThatThrownBy(() -> query("SELECT * FROM orders WHERE orderdate = DATE '-1996-09-14'"))
.hasMessageMatching(".*Row filter for .* is invalid\\. Filter is '\\(`orderdate` = '-1996-09-14'\\)'");
}

@Test
@Override
public void testSymbolAliasing()
{
// Create table in BigQuery because the connector doesn't support CREATE TABLE AS SELECT statement
String tableName = "test.test_symbol_aliasing" + randomTableSuffix();
onBigQuery("CREATE TABLE " + tableName + " AS SELECT 1 foo_1, 2 foo_2_4");
assertQuery("SELECT foo_1, foo_2_4 FROM " + tableName, "SELECT 1, 2");
assertUpdate("DROP TABLE " + tableName);
}

private void onBigQuery(String sql)
{
bigQuerySqlExecutor.execute(sql);
Expand Down

0 comments on commit 2b67dc9

Please sign in to comment.