From 411184310080bf83a3f0abe10b022227c762c668 Mon Sep 17 00:00:00 2001 From: Martin Traverso Date: Wed, 15 Nov 2023 14:05:57 -0800 Subject: [PATCH] Migrate tests to JUnit --- lib/trino-hdfs/pom.xml | 18 ----- .../trino/hdfs/TestFSDataInputStreamTail.java | 21 +++-- .../io/trino/hdfs/TestFileSystemCache.java | 17 ++-- .../cache/TestCachingHiveMetastore.java | 25 ++++-- .../hive/parquet/TestTimestampMicros.java | 2 +- plugin/trino-kinesis/pom.xml | 19 ----- .../plugin/kinesis/TestKinesisPlugin.java | 13 ++-- .../TestKinesisTableDescriptionSupplier.java | 50 ++++++------ .../kinesis/TestMinimalFunctionality.java | 46 +++++------ .../plugin/kinesis/TestRecordAccess.java | 23 +++--- .../s3config/TestS3TableConfigClient.java | 78 ++++++++++--------- .../trino/plugin/kinesis/util/TestUtils.java | 4 +- 12 files changed, 149 insertions(+), 167 deletions(-) diff --git a/lib/trino-hdfs/pom.xml b/lib/trino-hdfs/pom.xml index 813b9c6f44f2..220906b25c2e 100644 --- a/lib/trino-hdfs/pom.xml +++ b/lib/trino-hdfs/pom.xml @@ -271,24 +271,6 @@ - - org.apache.maven.plugins - maven-surefire-plugin - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - - - ca.vanzyl.provisio.maven.plugins provisio-maven-plugin diff --git a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFSDataInputStreamTail.java b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFSDataInputStreamTail.java index d1bb545d5058..73a7b8832ede 100644 --- a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFSDataInputStreamTail.java +++ b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFSDataInputStreamTail.java @@ -21,10 +21,12 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RawLocalFileSystem; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.io.File; import java.io.IOException; @@ -33,18 +35,21 @@ import static io.airlift.testing.Closeables.closeAll; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertSame; import static org.testng.Assert.assertTrue; -@Test(singleThreaded = true) // e.g. test methods operate on shared, mutated tempFile +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) // e.g. test methods operate on shared, mutated tempFile public class TestFSDataInputStreamTail { private File tempRoot; private Path tempFile; private RawLocalFileSystem fs; - @BeforeClass + @BeforeAll public void setUp() throws Exception { @@ -54,7 +59,7 @@ public void setUp() tempFile = new Path(Files.createTempFile(tempRoot.toPath(), "tempfile", "txt").toUri()); } - @BeforeMethod + @BeforeEach public void truncateTempFile() throws Exception { @@ -62,7 +67,7 @@ public void truncateTempFile() fs.truncate(tempFile, 0); } - @AfterClass(alwaysRun = true) + @AfterAll public void tearDown() throws Exception { diff --git a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFileSystemCache.java b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFileSystemCache.java index 3abe7b2e72cd..83c4496735ce 100644 --- a/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFileSystemCache.java +++ b/lib/trino-hdfs/src/test/java/io/trino/hdfs/TestFileSystemCache.java @@ -22,9 +22,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.gaul.modernizer_maven_annotations.SuppressModernizer; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.io.IOException; import java.util.ArrayList; @@ -38,15 +40,18 @@ import static io.trino.plugin.base.security.UserNameProvider.SIMPLE_USER_NAME_PROVIDER; import static java.util.Objects.requireNonNull; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertNotSame; import static org.testng.Assert.assertSame; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestFileSystemCache { - @BeforeMethod(alwaysRun = true) - @AfterClass(alwaysRun = true) + @BeforeEach + @AfterAll public void cleanup() throws IOException { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java index 7ab32a611bcc..a4d65e13e014 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/metastore/cache/TestCachingHiveMetastore.java @@ -49,9 +49,13 @@ import io.trino.spi.predicate.TupleDomain; import io.trino.spi.predicate.ValueSet; import org.apache.thrift.TException; -import org.testng.annotations.AfterClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.RepeatedTest; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.parallel.Execution; import java.util.ArrayList; import java.util.HashMap; @@ -112,12 +116,15 @@ import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.function.Function.identity; import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; import static org.testng.Assert.assertEquals; import static org.testng.Assert.assertFalse; import static org.testng.Assert.assertNotNull; import static org.testng.Assert.assertTrue; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestCachingHiveMetastore { private static final Logger log = Logger.get(TestCachingHiveMetastore.class); @@ -136,7 +143,7 @@ public class TestCachingHiveMetastore private CachingHiveMetastore statsOnlyCacheMetastore; private ThriftMetastoreStats stats; - @BeforeMethod + @BeforeEach public void setUp() { mockClient = new MockThriftMetastoreClient(); @@ -149,7 +156,7 @@ public void setUp() stats = ((ThriftHiveMetastore) thriftHiveMetastore).getStats(); } - @AfterClass(alwaysRun = true) + @AfterAll public void tearDown() { executor.shutdownNow(); @@ -357,7 +364,8 @@ public void testGetPartitionThenGetPartitions() * here simulated with an explicit invalidation. */ // Repeat test with invocationCount for better test coverage, since the tested aspect is inherently non-deterministic. - @Test(timeOut = 60_000, invocationCount = 20) + @RepeatedTest(20) + @Timeout(60) public void testGetPartitionThenGetPartitionsRacingWithInvalidation() throws Exception { @@ -927,7 +935,8 @@ public void testCachingHiveMetastoreCreationViaMemoize() assertEquals(metastore.getDatabaseNamesStats().getRequestCount(), 0); } - @Test(timeOut = 60_000) + @Test + @Timeout(60) public void testLoadAfterInvalidate() throws Exception { diff --git a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestTimestampMicros.java b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestTimestampMicros.java index 4deaa9e2684f..b3002c501de9 100644 --- a/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestTimestampMicros.java +++ b/plugin/trino-hive/src/test/java/io/trino/plugin/hive/parquet/TestTimestampMicros.java @@ -20,7 +20,7 @@ import io.trino.spi.type.Type; import io.trino.testing.MaterializedResult; import io.trino.testing.MaterializedRow; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; import java.io.File; import java.time.LocalDateTime; diff --git a/plugin/trino-kinesis/pom.xml b/plugin/trino-kinesis/pom.xml index 1d28d1dbbf63..df6234712f55 100644 --- a/plugin/trino-kinesis/pom.xml +++ b/plugin/trino-kinesis/pom.xml @@ -196,12 +196,6 @@ junit-jupiter-engine test - - - org.testng - testng - test - @@ -221,19 +215,6 @@ s3://S3-LOC - - - - org.apache.maven.surefire - surefire-junit-platform - ${dep.plugin.surefire.version} - - - org.apache.maven.surefire - surefire-testng - ${dep.plugin.surefire.version} - - org.basepom.maven diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisPlugin.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisPlugin.java index 1ca455ff46c8..91ade692250a 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisPlugin.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisPlugin.java @@ -25,8 +25,7 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static io.trino.spi.transaction.IsolationLevel.READ_COMMITTED; import static io.trino.testing.TestingConnectorSession.SESSION; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; /** * Test that the plug in API is satisfied and all of the required objects can be created. @@ -51,16 +50,16 @@ public void testCreateConnector() .put("kinesis.secret-key", TestUtils.noneToBlank(secretKey)) .put("bootstrap.quiet", "true") .buildOrThrow(), new TestingConnectorContext()); - assertNotNull(c); + assertThat(c).isNotNull(); // Verify that the key objects have been created on the connector - assertNotNull(c.getRecordSetProvider()); - assertNotNull(c.getSplitManager()); + assertThat(c.getRecordSetProvider()).isNotNull(); + assertThat(c.getSplitManager()).isNotNull(); ConnectorMetadata md = c.getMetadata(SESSION, KinesisTransactionHandle.INSTANCE); - assertNotNull(md); + assertThat(md).isNotNull(); ConnectorTransactionHandle handle = c.beginTransaction(READ_COMMITTED, true, true); - assertTrue(handle instanceof KinesisTransactionHandle); + assertThat(handle instanceof KinesisTransactionHandle).isTrue(); c.shutdown(); } diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisTableDescriptionSupplier.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisTableDescriptionSupplier.java index 27a28bfe4a42..590b16bc537c 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisTableDescriptionSupplier.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestKinesisTableDescriptionSupplier.java @@ -31,11 +31,9 @@ import java.util.Map; import static io.trino.testing.TestingConnectorSession.SESSION; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; @TestInstance(PER_CLASS) @Execution(CONCURRENT) @@ -70,55 +68,55 @@ public void testTableDefinition() KinesisMetadata metadata = (KinesisMetadata) connector.getMetadata(SESSION, new ConnectorTransactionHandle() {}); SchemaTableName tblName = new SchemaTableName("prod", "test_table"); KinesisTableHandle tableHandle = metadata.getTableHandle(SESSION, tblName); - assertNotNull(metadata); + assertThat(metadata).isNotNull(); SchemaTableName tableSchemaName = tableHandle.toSchemaTableName(); - assertEquals(tableSchemaName.getSchemaName(), "prod"); - assertEquals(tableSchemaName.getTableName(), "test_table"); - assertEquals(tableHandle.getStreamName(), "test_kinesis_stream"); - assertEquals(tableHandle.getMessageDataFormat(), "json"); + assertThat(tableSchemaName.getSchemaName()).isEqualTo("prod"); + assertThat(tableSchemaName.getTableName()).isEqualTo("test_table"); + assertThat(tableHandle.getStreamName()).isEqualTo("test_kinesis_stream"); + assertThat(tableHandle.getMessageDataFormat()).isEqualTo("json"); Map columnHandles = metadata.getColumnHandles(SESSION, tableHandle); - assertEquals(columnHandles.size(), 14); - assertEquals(columnHandles.values().stream().filter(x -> ((KinesisColumnHandle) x).isInternal()).count(), 10); + assertThat(columnHandles.size()).isEqualTo(14); + assertThat(columnHandles.values().stream().filter(x -> ((KinesisColumnHandle) x).isInternal()).count()).isEqualTo(10); } @Test public void testRelatedObjects() { KinesisMetadata metadata = (KinesisMetadata) connector.getMetadata(SESSION, new ConnectorTransactionHandle() {}); - assertNotNull(metadata); + assertThat(metadata).isNotNull(); SchemaTableName tblName = new SchemaTableName("prod", "test_table"); List schemas = metadata.listSchemaNames(null); - assertEquals(schemas.size(), 1); - assertEquals(schemas.get(0), "prod"); + assertThat(schemas.size()).isEqualTo(1); + assertThat(schemas.get(0)).isEqualTo("prod"); KinesisTableHandle tblHandle = metadata.getTableHandle(null, tblName); - assertNotNull(tblHandle); - assertEquals(tblHandle.getSchemaName(), "prod"); - assertEquals(tblHandle.getTableName(), "test_table"); - assertEquals(tblHandle.getStreamName(), "test_kinesis_stream"); - assertEquals(tblHandle.getMessageDataFormat(), "json"); + assertThat(tblHandle).isNotNull(); + assertThat(tblHandle.getSchemaName()).isEqualTo("prod"); + assertThat(tblHandle.getTableName()).isEqualTo("test_table"); + assertThat(tblHandle.getStreamName()).isEqualTo("test_kinesis_stream"); + assertThat(tblHandle.getMessageDataFormat()).isEqualTo("json"); ConnectorTableMetadata tblMeta = metadata.getTableMetadata(null, tblHandle); - assertNotNull(tblMeta); - assertEquals(tblMeta.getTable().getSchemaName(), "prod"); - assertEquals(tblMeta.getTable().getTableName(), "test_table"); + assertThat(tblMeta).isNotNull(); + assertThat(tblMeta.getTable().getSchemaName()).isEqualTo("prod"); + assertThat(tblMeta.getTable().getTableName()).isEqualTo("test_table"); List columnList = tblMeta.getColumns(); - assertNotNull(columnList); + assertThat(columnList).isNotNull(); boolean foundServiceType = false; boolean foundPartitionKey = false; for (ColumnMetadata column : columnList) { if (column.getName().equals("service_type")) { foundServiceType = true; - assertEquals(column.getType().getDisplayName(), "varchar(20)"); + assertThat(column.getType().getDisplayName()).isEqualTo("varchar(20)"); } if (column.getName().equals("_partition_key")) { foundPartitionKey = true; - assertEquals(column.getType().getDisplayName(), "varchar"); + assertThat(column.getType().getDisplayName()).isEqualTo("varchar"); } } - assertTrue(foundServiceType); - assertTrue(foundPartitionKey); + assertThat(foundServiceType).isTrue(); + assertThat(foundPartitionKey).isTrue(); } } diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java index 554c0aa842f8..18a3297bf2c7 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestMinimalFunctionality.java @@ -26,12 +26,12 @@ import io.trino.spi.security.Identity; import io.trino.sql.query.QueryAssertions; import io.trino.testing.StandaloneQueryRunner; -import org.testng.annotations.AfterClass; -import org.testng.annotations.AfterMethod; -import org.testng.annotations.BeforeClass; -import org.testng.annotations.BeforeMethod; -import org.testng.annotations.Parameters; -import org.testng.annotations.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.io.File; import java.nio.ByteBuffer; @@ -49,7 +49,8 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.Locale.ENGLISH; import static org.assertj.core.api.Assertions.assertThat; -import static org.testng.Assert.assertTrue; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; /** * Note: this is an integration test that connects to AWS Kinesis. @@ -58,7 +59,8 @@ * You may incur AWS charges if you run this test. You probably want to setup an IAM * user for your CI server to use. */ -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestMinimalFunctionality { public static final Session SESSION = Session.builder(new SessionPropertyManager()) @@ -70,35 +72,29 @@ public class TestMinimalFunctionality .setLocale(ENGLISH) .setQueryId(new QueryId("dummy")) .build(); + private final String accessKey; + private final String secretKey; - private EmbeddedKinesisStream embeddedKinesisStream; + private final EmbeddedKinesisStream embeddedKinesisStream; private String streamName; private StandaloneQueryRunner queryRunner; private QueryAssertions assertions; - @Parameters({ - "kinesis.awsAccessKey", - "kinesis.awsSecretKey" - }) - @BeforeClass - public void start(String accessKey, String secretKey) + public TestMinimalFunctionality() { + accessKey = System.getProperty("kinesis.awsAccessKey"); + secretKey = System.getProperty("kinesis.awsSecretKey"); embeddedKinesisStream = new EmbeddedKinesisStream(TestUtils.noneToBlank(accessKey), TestUtils.noneToBlank(secretKey)); } - @AfterClass(alwaysRun = true) + @AfterAll public void stop() { embeddedKinesisStream.close(); - embeddedKinesisStream = null; } - @Parameters({ - "kinesis.awsAccessKey", - "kinesis.awsSecretKey" - }) - @BeforeMethod - public void spinUp(String accessKey, String secretKey) + @BeforeEach + public void spinUp() throws Exception { streamName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_"); @@ -146,7 +142,7 @@ public void testStreamExists() .singleStatement() .execute(SESSION, session -> { Optional handle = queryRunner.getServer().getMetadata().getTableHandle(session, name); - assertTrue(handle.isPresent()); + assertThat(handle.isPresent()).isTrue(); }); } @@ -163,7 +159,7 @@ public void testStreamHasData() .matches("VALUES %s".formatted(count)); } - @AfterMethod(alwaysRun = true) + @AfterEach public void tearDown() { embeddedKinesisStream.deleteStream(streamName); diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java index b00f2d35bee2..4ee7baffd22e 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/TestRecordAccess.java @@ -46,10 +46,9 @@ import static io.trino.transaction.TransactionBuilder.transaction; import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertTrue; /** * Test record access and querying along with all associated setup. @@ -159,7 +158,7 @@ public void testStreamExists() .singleStatement() .execute(SESSION, session -> { Optional handle = queryRunner.getServer().getMetadata().getTableHandle(session, name); - assertTrue(handle.isPresent()); + assertThat(handle.isPresent()).isTrue(); }); log.info("Completed first test (access table handle)"); } @@ -172,7 +171,7 @@ public void testStreamHasData() .row(0) .build(); - assertEquals(result.getRowCount(), expected.getRowCount()); + assertThat(result.getRowCount()).isEqualTo(expected.getRowCount()); int count = 500; createDummyMessages(dummyStreamName, count); @@ -183,7 +182,7 @@ public void testStreamHasData() .row(count) .build(); - assertEquals(result.getRowCount(), expected.getRowCount()); + assertThat(result.getRowCount()).isEqualTo(expected.getRowCount()); log.info("Completed second test (select counts)"); } @@ -205,19 +204,19 @@ private void testJsonStream(int uncompressedMessages, int compressedMessages, St createJsonMessages(streamName, compressedMessages, 100 + uncompressedMessages, true); } MaterializedResult result = queryRunner.execute("Select id, name, _shard_id, _message_length, _message from " + streamName + " where _message_length >= 1"); - assertEquals(result.getRowCount(), uncompressedMessages + compressedMessages); + assertThat(result.getRowCount()).isEqualTo(uncompressedMessages + compressedMessages); List types = result.getTypes(); - assertEquals(types.size(), 5); - assertEquals(types.get(0).toString(), "bigint"); - assertEquals(types.get(1).toString(), "varchar"); + assertThat(types.size()).isEqualTo(5); + assertThat(types.get(0).toString()).isEqualTo("bigint"); + assertThat(types.get(1).toString()).isEqualTo("varchar"); log.info("Types : %s", types); List rows = result.getMaterializedRows(); - assertEquals(rows.size(), uncompressedMessages + compressedMessages); + assertThat(rows.size()).isEqualTo(uncompressedMessages + compressedMessages); for (MaterializedRow row : rows) { - assertEquals(row.getFieldCount(), 5); - assertTrue((long) row.getFields().get(0) >= 100); + assertThat(row.getFieldCount()).isEqualTo(5); + assertThat((long) row.getFields().get(0) >= 100).isTrue(); log.info("ROW: %s", row); } } diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/s3config/TestS3TableConfigClient.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/s3config/TestS3TableConfigClient.java index 656d97238767..dd641207d86c 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/s3config/TestS3TableConfigClient.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/s3config/TestS3TableConfigClient.java @@ -24,65 +24,73 @@ import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorTransactionHandle; import io.trino.spi.connector.SchemaTableName; -import org.testng.annotations.Parameters; -import org.testng.annotations.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; import java.util.Map; import static io.trino.testing.TestingConnectorSession.SESSION; -import static org.testng.Assert.assertEquals; -import static org.testng.Assert.assertNotNull; -import static org.testng.Assert.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.SAME_THREAD; -@Test(singleThreaded = true) +@TestInstance(PER_CLASS) +@Execution(SAME_THREAD) public class TestS3TableConfigClient { private static final Logger log = Logger.get(TestS3TableConfigClient.class); + private final String tableDescriptionS3; + private final String accessKey; + private final String secretKey; + + public TestS3TableConfigClient() + { + tableDescriptionS3 = System.getProperty("kinesis.test-table-description-location"); + accessKey = System.getProperty("kinesis.awsAccessKey"); + secretKey = System.getProperty("kinesis.awsSecretKey"); + } + @Test public void testS3URIValues() { // Verify that S3URI values will work: AmazonS3URI uri1 = new AmazonS3URI("s3://our.data.warehouse/prod/client_actions"); - assertNotNull(uri1.getKey()); - assertNotNull(uri1.getBucket()); + assertThat(uri1.getKey()).isNotNull(); + assertThat(uri1.getBucket()).isNotNull(); - assertEquals(uri1.toString(), "s3://our.data.warehouse/prod/client_actions"); - assertEquals(uri1.getBucket(), "our.data.warehouse"); - assertEquals(uri1.getKey(), "prod/client_actions"); - assertTrue(uri1.getRegion() == null); + assertThat(uri1.toString()).isEqualTo("s3://our.data.warehouse/prod/client_actions"); + assertThat(uri1.getBucket()).isEqualTo("our.data.warehouse"); + assertThat(uri1.getKey()).isEqualTo("prod/client_actions"); + assertThat(uri1.getRegion() == null).isTrue(); // show info: log.info("Tested out URI1 : %s", uri1); AmazonS3URI uri2 = new AmazonS3URI("s3://some.big.bucket/long/complex/path"); - assertNotNull(uri2.getKey()); - assertNotNull(uri2.getBucket()); + assertThat(uri2.getKey()).isNotNull(); + assertThat(uri2.getBucket()).isNotNull(); - assertEquals(uri2.toString(), "s3://some.big.bucket/long/complex/path"); - assertEquals(uri2.getBucket(), "some.big.bucket"); - assertEquals(uri2.getKey(), "long/complex/path"); - assertTrue(uri2.getRegion() == null); + assertThat(uri2.toString()).isEqualTo("s3://some.big.bucket/long/complex/path"); + assertThat(uri2.getBucket()).isEqualTo("some.big.bucket"); + assertThat(uri2.getKey()).isEqualTo("long/complex/path"); + assertThat(uri2.getRegion() == null).isTrue(); // info: log.info("Tested out URI2 : %s", uri2); AmazonS3URI uri3 = new AmazonS3URI("s3://trino.kinesis.config/unit-test/trino-kinesis"); - assertNotNull(uri3.getKey()); - assertNotNull(uri3.getBucket()); + assertThat(uri3.getKey()).isNotNull(); + assertThat(uri3.getBucket()).isNotNull(); - assertEquals(uri3.toString(), "s3://trino.kinesis.config/unit-test/trino-kinesis"); - assertEquals(uri3.getBucket(), "trino.kinesis.config"); - assertEquals(uri3.getKey(), "unit-test/trino-kinesis"); + assertThat(uri3.toString()).isEqualTo("s3://trino.kinesis.config/unit-test/trino-kinesis"); + assertThat(uri3.getBucket()).isEqualTo("trino.kinesis.config"); + assertThat(uri3.getKey()).isEqualTo("unit-test/trino-kinesis"); } - @Parameters({ - "kinesis.test-table-description-location", - "kinesis.awsAccessKey", - "kinesis.awsSecretKey" - }) @Test - public void testTableReading(String tableDescriptionS3, String accessKey, String secretKey) + public void testTableReading() { // To run this test: setup an S3 bucket with a folder for unit testing, and put // MinimalTable.json in that folder. @@ -112,13 +120,13 @@ public void testTableReading(String tableDescriptionS3, String accessKey, String KinesisMetadata metadata = (KinesisMetadata) kinesisConnector.getMetadata(SESSION, new ConnectorTransactionHandle() {}); SchemaTableName tblName = new SchemaTableName("default", "test123"); KinesisTableHandle tableHandle = metadata.getTableHandle(SESSION, tblName); - assertNotNull(metadata); + assertThat(metadata).isNotNull(); SchemaTableName tableSchemaName = tableHandle.toSchemaTableName(); - assertEquals(tableSchemaName.getSchemaName(), "default"); - assertEquals(tableSchemaName.getTableName(), "test123"); - assertEquals(tableHandle.getStreamName(), "test123"); - assertEquals(tableHandle.getMessageDataFormat(), "json"); + assertThat(tableSchemaName.getSchemaName()).isEqualTo("default"); + assertThat(tableSchemaName.getTableName()).isEqualTo("test123"); + assertThat(tableHandle.getStreamName()).isEqualTo("test123"); + assertThat(tableHandle.getMessageDataFormat()).isEqualTo("json"); Map columnHandles = metadata.getColumnHandles(SESSION, tableHandle); - assertEquals(columnHandles.size(), 12); + assertThat(columnHandles.size()).isEqualTo(12); } } diff --git a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/util/TestUtils.java b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/util/TestUtils.java index 9786f76cdfd4..11398190803d 100644 --- a/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/util/TestUtils.java +++ b/plugin/trino-kinesis/src/test/java/io/trino/plugin/kinesis/util/TestUtils.java @@ -26,7 +26,7 @@ import java.util.Map; import static java.util.Objects.requireNonNull; -import static org.testng.Assert.assertNotNull; +import static org.assertj.core.api.Assertions.assertThat; public final class TestUtils { @@ -39,7 +39,7 @@ public static KinesisConnector createConnector(KinesisPlugin plugin, Map