Skip to content

Commit

Permalink
Convert Kudu standard schema connector tests to smoke tests
Browse files Browse the repository at this point in the history
Additional non-default configurations should be tested with BaseConnectorSmokeTest to avoid longer
CI without loss of test coverage. This is already followed in other connectors.
  • Loading branch information
chen-ni authored and hashhar committed Jun 30, 2022
1 parent f77c0d3 commit 48c2638
Show file tree
Hide file tree
Showing 5 changed files with 105 additions and 29 deletions.
@@ -0,0 +1,90 @@
/*
* 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.kudu;

import io.trino.testing.BaseConnectorSmokeTest;
import io.trino.testing.QueryRunner;
import io.trino.testing.TestingConnectorBehavior;

import java.util.Optional;

import static io.trino.plugin.kudu.KuduQueryRunnerFactory.createKuduQueryRunnerTpch;
import static org.assertj.core.api.Assertions.assertThat;

public abstract class BaseKuduConnectorSmokeTest
extends BaseConnectorSmokeTest
{
private TestingKuduServer kuduServer;

protected abstract String getKuduServerVersion();

protected abstract Optional<String> getKuduSchemaEmulationPrefix();

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
kuduServer = new TestingKuduServer(getKuduServerVersion());
return createKuduQueryRunnerTpch(kuduServer, getKuduSchemaEmulationPrefix(), REQUIRED_TPCH_TABLES);
}

@Override
protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
{
switch (connectorBehavior) {
case SUPPORTS_DELETE:
return true;
case SUPPORTS_RENAME_SCHEMA:
case SUPPORTS_CREATE_TABLE_WITH_TABLE_COMMENT:
case SUPPORTS_COMMENT_ON_TABLE:
case SUPPORTS_COMMENT_ON_COLUMN:
case SUPPORTS_ARRAY:
case SUPPORTS_NOT_NULL_CONSTRAINT:
case SUPPORTS_TOPN_PUSHDOWN:
case SUPPORTS_NEGATIVE_DATE:
return false;

case SUPPORTS_ROW_TYPE:
return false;

default:
return super.hasBehavior(connectorBehavior);
}
}

@Override
protected String getCreateTableDefaultDefinition()
{
return "(a bigint WITH (primary_key=true), b double) " +
"WITH (partition_by_hash_columns = ARRAY['a'], partition_by_hash_buckets = 2)";
}

@Override
public void testShowCreateTable()
{
assertThat((String) computeScalar("SHOW CREATE TABLE region"))
.isEqualTo("CREATE TABLE kudu.tpch.region (\n" +
" regionkey bigint COMMENT '' WITH ( nullable = true ),\n" +
" name varchar COMMENT '' WITH ( nullable = true ),\n" +
" comment varchar COMMENT '' WITH ( nullable = true )\n" +
")\n" +
"WITH (\n" +
" number_of_replicas = 3,\n" +
" partition_by_hash_buckets = 2,\n" +
" partition_by_hash_columns = ARRAY['row_uuid'],\n" +
" partition_by_range_columns = ARRAY['row_uuid'],\n" +
" range_partitions = '[{\"lower\":null,\"upper\":null}]'\n" +
")");
}
}
Expand Up @@ -17,11 +17,10 @@

import java.util.Optional;

import static io.trino.testing.sql.TestTable.randomTableSuffix;
import static org.testng.Assert.assertEquals;

public abstract class AbstractKuduWithStandardInferSchemaConnectorTest
extends AbstractKuduConnectorTest
public abstract class BaseKuduWithStandardInferSchemaConnectorSmokeTest
extends BaseKuduConnectorSmokeTest
{
@Override
protected Optional<String> getKuduSchemaEmulationPrefix()
Expand All @@ -36,22 +35,4 @@ public void testListingOfTableForDefaultSchema()
// Depending on test ordering, this table may or may not be created when this test runs, so filter it out
assertEquals(computeActual("SHOW TABLES FROM default LIKE '%$schemas'").getRowCount(), 0);
}

@Test
@Override
public void testDropNonEmptySchemaWithTable()
{
// Set column and table properties in CREATE TABLE statement
String schemaName = "test_drop_non_empty_schema_" + randomTableSuffix();

try {
assertUpdate("CREATE SCHEMA " + schemaName);
assertUpdate("CREATE TABLE " + schemaName + ".t(x int WITH (primary_key=true)) WITH (partition_by_hash_columns=ARRAY['x'], partition_by_hash_buckets=2)");
assertQueryFails("DROP SCHEMA " + schemaName, ".*Cannot drop non-empty schema '\\Q" + schemaName + "\\E'");
}
finally {
assertUpdate("DROP TABLE IF EXISTS " + schemaName + ".t");
assertUpdate("DROP SCHEMA IF EXISTS " + schemaName);
}
}
}
Expand Up @@ -15,8 +15,8 @@

import static io.trino.plugin.kudu.TestingKuduServer.LATEST_TAG;

public class TestKuduLatestWithStandardInferSchemaConnectorTest
extends AbstractKuduWithStandardInferSchemaConnectorTest
public class TestKuduLatestWithStandardInferSchemaConnectorSmokeTest
extends BaseKuduWithStandardInferSchemaConnectorSmokeTest
{
@Override
protected String getKuduServerVersion()
Expand Down
Expand Up @@ -15,8 +15,8 @@

import static io.trino.plugin.kudu.TestingKuduServer.EARLIEST_TAG;

public class TestKuduWithStandardInferSchemaConnectorTest
extends AbstractKuduWithStandardInferSchemaConnectorTest
public class TestKuduWithStandardInferSchemaConnectorSmokeTest
extends BaseKuduWithStandardInferSchemaConnectorSmokeTest
{
@Override
protected String getKuduServerVersion()
Expand Down
Expand Up @@ -133,12 +133,17 @@ public void testCreateTable()
}

String tableName = "test_create_" + randomTableSuffix();
assertUpdate("CREATE TABLE " + tableName + " (a bigint, b double)");
assertUpdate("CREATE TABLE " + tableName + " " + getCreateTableDefaultDefinition());
assertThat(query("SELECT a, b FROM " + tableName))
.returnsEmptyResult();
assertUpdate("DROP TABLE " + tableName);
}

protected String getCreateTableDefaultDefinition()
{
return "(a bigint, b double)";
}

@Test
public void testCreateTableAsSelect()
{
Expand Down Expand Up @@ -166,7 +171,7 @@ public void testInsert()
throw new AssertionError("Cannot test INSERT without CREATE TABLE, the test needs to be implemented in a connector-specific way");
}

try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", "(a bigint, b double)")) {
try (TestTable table = new TestTable(getQueryRunner()::execute, "test_insert_", getCreateTableDefaultDefinition())) {
assertUpdate("INSERT INTO " + table.getName() + " (a, b) VALUES (42, -38.5)", 1);
assertThat(query("SELECT CAST(a AS bigint), b FROM " + table.getName()))
.matches("VALUES (BIGINT '42', -385e-1)");
Expand Down Expand Up @@ -301,7 +306,7 @@ public void testRenameTable()
}

String oldTable = "test_rename_old_" + randomTableSuffix();
assertUpdate("CREATE TABLE " + oldTable + " (a bigint, b double)");
assertUpdate("CREATE TABLE " + oldTable + " " + getCreateTableDefaultDefinition());

String newTable = "test_rename_new_" + randomTableSuffix();
try {
Expand Down Expand Up @@ -350,7 +355,7 @@ public void testRenameTableAcrossSchemas()
}

String oldTable = "test_rename_old_" + randomTableSuffix();
assertUpdate("CREATE TABLE " + oldTable + " (a bigint, b double)");
assertUpdate("CREATE TABLE " + oldTable + " " + getCreateTableDefaultDefinition());

String schemaName = "test_schema_" + randomTableSuffix();
assertUpdate(createSchemaSql(schemaName));
Expand Down

0 comments on commit 48c2638

Please sign in to comment.