Skip to content

Commit

Permalink
Migrate connector smoke tests to JUnit
Browse files Browse the repository at this point in the history
  • Loading branch information
martint committed Oct 10, 2023
1 parent 5600cfd commit 24803a4
Show file tree
Hide file tree
Showing 35 changed files with 271 additions and 157 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@

import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;
import org.testng.annotations.Test;
import org.junit.jupiter.api.Test;

import java.util.Map;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import io.trino.testing.BaseConnectorSmokeTest;
import io.trino.testing.TestingConnectorBehavior;
import io.trino.testing.sql.TestTable;
import org.testng.annotations.Test;
import org.junit.jupiter.api.Test;

import java.time.ZoneId;
import java.time.ZonedDateTime;
Expand Down Expand Up @@ -50,6 +50,7 @@ protected boolean hasBehavior(TestingConnectorBehavior connectorBehavior)
};
}

@Test
@Override
public void testDeleteAllDataFromTable()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@
import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;
import io.trino.testing.sql.TestTable;
import org.testng.annotations.Test;
import org.junit.jupiter.api.Test;

import java.sql.Timestamp;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@

import io.trino.plugin.jdbc.BaseJdbcConnectorSmokeTest;
import io.trino.testing.TestingConnectorBehavior;
import org.testng.annotations.Test;
import org.junit.jupiter.api.Test;

import static org.assertj.core.api.Assertions.assertThat;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@

import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;
import org.junit.jupiter.api.Test;

import static io.trino.plugin.clickhouse.ClickHouseQueryRunner.createClickHouseQueryRunner;
import static io.trino.plugin.clickhouse.TestingClickHouseServer.ALTINITY_DEFAULT_IMAGE;
Expand All @@ -34,6 +35,7 @@ protected QueryRunner createQueryRunner()
REQUIRED_TPCH_TABLES);
}

@Test
@Override
public void testRenameSchema()
{
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,16 +16,21 @@
import io.trino.plugin.hive.containers.HiveHadoop;
import io.trino.plugin.hive.containers.HiveMinioDataLake;
import io.trino.testing.QueryRunner;
import org.testng.annotations.AfterClass;
import io.trino.testng.services.ManageTestResources;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.TestInstance;

import java.util.List;

import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.lang.String.format;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;

@TestInstance(PER_CLASS)
public abstract class BaseDeltaLakeAwsConnectorSmokeTest
extends BaseDeltaLakeConnectorSmokeTest
{
@ManageTestResources.Suppress(because = "Not a TestNG test class")
protected HiveMinioDataLake hiveMinioDataLake;

@Override
Expand All @@ -37,7 +42,7 @@ protected HiveHadoop createHiveHadoop()
}

@Override
@AfterClass(alwaysRun = true)
@AfterAll
public void cleanUp()
{
hiveMinioDataLake = null; // closed by closeAfterClass
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,10 +40,10 @@
import io.trino.testing.sql.TestTable;
import io.trino.tpch.TpchTable;
import org.intellij.lang.annotations.Language;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInstance;
import org.testng.SkipException;
import org.testng.annotations.AfterClass;
import org.testng.annotations.DataProvider;
import org.testng.annotations.Test;

import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -84,10 +84,12 @@
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.assertj.core.api.Assertions.assertThat;
import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertTrue;

@TestInstance(PER_CLASS)
public abstract class BaseDeltaLakeConnectorSmokeTest
extends BaseConnectorSmokeTest
{
Expand Down Expand Up @@ -228,7 +230,7 @@ private DistributedQueryRunner createDeltaLakeQueryRunner()
queryRunner -> {});
}

@AfterClass(alwaysRun = true)
@AfterAll
public void cleanUp()
{
hiveHadoop = null; // closed by closeAfterClass
Expand Down Expand Up @@ -729,6 +731,7 @@ private void validatePath(String schemaLocation, String schemaName, String table
assertThat((String) materializedRows.get(0).getField(0)).matches(format("%s/%s.*", schemaLocation, tableName));
}

@Test
@Override
public void testRenameTable()
{
Expand Down Expand Up @@ -763,6 +766,7 @@ public void testRenameExternalTable()
assertUpdate("DROP TABLE " + newTable);
}

@Test
@Override
public void testRenameTableAcrossSchemas()
{
Expand Down Expand Up @@ -1331,8 +1335,28 @@ public void testCheckpointing()
assertUpdate("DROP TABLE " + tableName);
}

@Test(dataProvider = "testCheckpointWriteStatsAsStructDataProvider")
public void testCheckpointWriteStatsAsStruct(String type, String sampleValue, String highValue, String nullsFraction, String minValue, String maxValue)
@Test
public void testCheckpointWriteStatsAsStruct()
{
testCheckpointWriteStatsAsStruct("boolean", "true", "false", "0.0", "null", "null");
testCheckpointWriteStatsAsStruct("integer", "1", "2147483647", "0.0", "1", "2147483647");
testCheckpointWriteStatsAsStruct("tinyint", "2", "127", "0.0", "2", "127");
testCheckpointWriteStatsAsStruct("smallint", "3", "32767", "0.0", "3", "32767");
testCheckpointWriteStatsAsStruct("bigint", "1000", "9223372036854775807", "0.0", "1000", "9223372036854775807");
testCheckpointWriteStatsAsStruct("real", "0.1", "999999.999", "0.0", "0.1", "1000000.0");
testCheckpointWriteStatsAsStruct("double", "1.0", "9999999999999.999", "0.0", "1.0", "'1.0E13'");
testCheckpointWriteStatsAsStruct("decimal(3,2)", "3.14", "9.99", "0.0", "3.14", "9.99");
testCheckpointWriteStatsAsStruct("decimal(30,1)", "12345", "99999999999999999999999999999.9", "0.0", "12345.0", "'1.0E29'");
testCheckpointWriteStatsAsStruct("varchar", "'test'", "'ŻŻŻŻŻŻŻŻŻŻ'", "0.0", "null", "null");
testCheckpointWriteStatsAsStruct("varbinary", "X'65683F'", "X'ffffffffffffffffffff'", "0.0", "null", "null");
testCheckpointWriteStatsAsStruct("date", "date '2021-02-03'", "date '9999-12-31'", "0.0", "'2021-02-03'", "'9999-12-31'");
testCheckpointWriteStatsAsStruct("timestamp(3) with time zone", "timestamp '2001-08-22 03:04:05.321 -08:00'", "timestamp '9999-12-31 23:59:59.999 +12:00'", "0.0", "'2001-08-22 11:04:05.321 UTC'", "'9999-12-31 11:59:59.999 UTC'");
testCheckpointWriteStatsAsStruct("array(int)", "array[1]", "array[2147483647]", "null", "null", "null");
testCheckpointWriteStatsAsStruct("map(varchar,int)", "map(array['foo', 'bar'], array[1, 2])", "map(array['foo', 'bar'], array[-2147483648, 2147483647])", "null", "null", "null");
testCheckpointWriteStatsAsStruct("row(x bigint)", "cast(row(1) as row(x bigint))", "cast(row(9223372036854775807) as row(x bigint))", "null", "null", "null");
}

private void testCheckpointWriteStatsAsStruct(String type, String sampleValue, String highValue, String nullsFraction, String minValue, String maxValue)
{
String tableName = "test_checkpoint_write_stats_as_struct_" + randomNameSuffix();

Expand All @@ -1359,30 +1383,6 @@ public void testCheckpointWriteStatsAsStruct(String type, String sampleValue, St
assertUpdate("DROP TABLE " + tableName);
}

@DataProvider
public Object[][] testCheckpointWriteStatsAsStructDataProvider()
{
// type, sampleValue, highValue, nullsFraction, minValue, maxValue
return new Object[][] {
{"boolean", "true", "false", "0.0", "null", "null"},
{"integer", "1", "2147483647", "0.0", "1", "2147483647"},
{"tinyint", "2", "127", "0.0", "2", "127"},
{"smallint", "3", "32767", "0.0", "3", "32767"},
{"bigint", "1000", "9223372036854775807", "0.0", "1000", "9223372036854775807"},
{"real", "0.1", "999999.999", "0.0", "0.1", "1000000.0"},
{"double", "1.0", "9999999999999.999", "0.0", "1.0", "'1.0E13'"},
{"decimal(3,2)", "3.14", "9.99", "0.0", "3.14", "9.99"},
{"decimal(30,1)", "12345", "99999999999999999999999999999.9", "0.0", "12345.0", "'1.0E29'"},
{"varchar", "'test'", "'ŻŻŻŻŻŻŻŻŻŻ'", "0.0", "null", "null"},
{"varbinary", "X'65683F'", "X'ffffffffffffffffffff'", "0.0", "null", "null"},
{"date", "date '2021-02-03'", "date '9999-12-31'", "0.0", "'2021-02-03'", "'9999-12-31'"},
{"timestamp(3) with time zone", "timestamp '2001-08-22 03:04:05.321 -08:00'", "timestamp '9999-12-31 23:59:59.999 +12:00'", "0.0", "'2001-08-22 11:04:05.321 UTC'", "'9999-12-31 11:59:59.999 UTC'"},
{"array(int)", "array[1]", "array[2147483647]", "null", "null", "null"},
{"map(varchar,int)", "map(array['foo', 'bar'], array[1, 2])", "map(array['foo', 'bar'], array[-2147483648, 2147483647])", "null", "null", "null"},
{"row(x bigint)", "cast(row(1) as row(x bigint))", "cast(row(9223372036854775807) as row(x bigint))", "null", "null", "null"},
};
}

@Test
public void testCheckpointWriteStatsAsStructWithPartiallyUnsupportedColumnStats()
{
Expand Down Expand Up @@ -1419,21 +1419,13 @@ public void testDeltaLakeTableLocationChangedSameVersionNumber()
testDeltaLakeTableLocationChanged(false, false, false);
}

@Test(dataProvider = "testDeltaLakeTableLocationChangedPartitionedDataProvider")
public void testDeltaLakeTableLocationChangedPartitioned(boolean firstPartitioned, boolean secondPartitioned)
@Test
public void testDeltaLakeTableLocationChangedPartitioned()
throws Exception
{
testDeltaLakeTableLocationChanged(true, firstPartitioned, secondPartitioned);
}

@DataProvider
public Object[][] testDeltaLakeTableLocationChangedPartitionedDataProvider()
{
return new Object[][] {
{true, false},
{false, true},
{true, true},
};
testDeltaLakeTableLocationChanged(true, true, false);
testDeltaLakeTableLocationChanged(true, false, true);
testDeltaLakeTableLocationChanged(true, true, true);
}

private void testDeltaLakeTableLocationChanged(boolean fewerEntries, boolean firstPartitioned, boolean secondPartitioned)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,8 @@
import com.google.common.reflect.ClassPath;
import io.trino.plugin.hive.containers.HiveHadoop;
import io.trino.testing.QueryRunner;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Parameters;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.TestInstance;

import java.io.IOException;
import java.io.UncheckedIOException;
Expand All @@ -49,8 +49,10 @@
import static java.util.Objects.requireNonNull;
import static java.util.regex.Matcher.quoteReplacement;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
import static org.testcontainers.containers.Network.newNetwork;

@TestInstance(PER_CLASS)
public class TestDeltaLakeAdlsConnectorSmokeTest
extends BaseDeltaLakeConnectorSmokeTest
{
Expand All @@ -60,15 +62,11 @@ public class TestDeltaLakeAdlsConnectorSmokeTest
private final BlobContainerClient azureContainerClient;
private final String adlsDirectory;

@Parameters({
"hive.hadoop2.azure-abfs-container",
"hive.hadoop2.azure-abfs-account",
"hive.hadoop2.azure-abfs-access-key"})
public TestDeltaLakeAdlsConnectorSmokeTest(String container, String account, String accessKey)
public TestDeltaLakeAdlsConnectorSmokeTest()
{
this.container = requireNonNull(container, "container is null");
this.account = requireNonNull(account, "account is null");
this.accessKey = requireNonNull(accessKey, "accessKey is null");
this.container = requireNonNull(System.getProperty("hive.hadoop2.azure-abfs-container"), "container is null");
this.account = requireNonNull(System.getProperty("hive.hadoop2.azure-abfs-account"), "account is null");
this.accessKey = requireNonNull(System.getProperty("hive.hadoop2.azure-abfs-access-key"), "accessKey is null");

String connectionString = format("DefaultEndpointsProtocol=https;AccountName=%s;AccountKey=%s;EndpointSuffix=core.windows.net", account, accessKey);
BlobServiceClient blobServiceClient = new BlobServiceClientBuilder().connectionString(connectionString).buildClient();
Expand Down Expand Up @@ -113,7 +111,7 @@ protected Map<String, String> deltaStorageConfiguration()
return hiveStorageConfiguration();
}

@AfterClass(alwaysRun = true)
@AfterAll
public void removeTestData()
{
if (adlsDirectory != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,8 @@
import io.trino.plugin.hive.containers.HiveHadoop;
import io.trino.testing.QueryRunner;
import org.apache.hadoop.conf.Configuration;
import org.testng.annotations.AfterClass;
import org.testng.annotations.Parameters;
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.TestInstance;

import java.io.FileNotFoundException;
import java.io.IOException;
Expand All @@ -53,6 +53,7 @@
import static java.nio.charset.StandardCharsets.UTF_8;
import static java.util.Objects.requireNonNull;
import static java.util.regex.Matcher.quoteReplacement;
import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS;
import static org.testcontainers.containers.Network.newNetwork;

/**
Expand All @@ -61,6 +62,7 @@
* - gcp-credentials-key: A base64 encoded copy of the JSON authentication file for the service account used to connect to GCP.
* For example, `cat service-account-key.json | base64`
*/
@TestInstance(PER_CLASS)
public class TestDeltaLakeGcsConnectorSmokeTest
extends BaseDeltaLakeConnectorSmokeTest
{
Expand All @@ -74,11 +76,10 @@ public class TestDeltaLakeGcsConnectorSmokeTest
private String gcpCredentials;
private TrinoFileSystem fileSystem;

@Parameters({"testing.gcp-storage-bucket", "testing.gcp-credentials-key"})
public TestDeltaLakeGcsConnectorSmokeTest(String gcpStorageBucket, String gcpCredentialKey)
public TestDeltaLakeGcsConnectorSmokeTest()
{
this.gcpStorageBucket = requireNonNull(gcpStorageBucket, "gcpStorageBucket is null");
this.gcpCredentialKey = requireNonNull(gcpCredentialKey, "gcpCredentialKey is null");
this.gcpStorageBucket = requireNonNull(System.getProperty("testing.gcp-storage-bucket"), "GCP storage bucket is null");
this.gcpCredentialKey = requireNonNull(System.getProperty("testing.gcp-credentials-key"), "GCP credential key is null");
}

@Override
Expand All @@ -99,7 +100,7 @@ protected void environmentSetup()
}
}

@AfterClass(alwaysRun = true)
@AfterAll
public void removeTestData()
{
if (fileSystem != null) {
Expand Down
Loading

0 comments on commit 24803a4

Please sign in to comment.