diff --git a/build.gradle b/build.gradle index 70f34bda..f2f2cec5 100644 --- a/build.gradle +++ b/build.gradle @@ -167,8 +167,10 @@ dependencies { implementation "com.google.guava:guava:33.0.0-jre" implementation "org.slf4j:slf4j-api:$slf4jVersion" - testImplementation "junit:junit:4.13.2" + testImplementation "org.junit.jupiter:junit-jupiter:$jupiterVersion" + testImplementation "org.junit.vintage:junit-vintage-engine:$jupiterVersion" testImplementation "org.mockito:mockito-core:$mockitoVersion" + testImplementation "org.mockito:mockito-junit-jupiter:$mockitoVersion" testImplementation "org.apache.kafka:connect-api:$kafkaVersion" testImplementation "commons-io:commons-io:2.14.0" testImplementation "org.apache.derby:derby:$derbyVersion" @@ -204,6 +206,10 @@ dependencies { integrationTestImplementation sourceSets.test.output } +test { + useJUnitPlatform() +} + task integrationTest(type: Test) { description = 'Runs the integration tests.' group = 'verification' diff --git a/src/test/java/io/aiven/connect/jdbc/JdbcSourceConnectorTest.java b/src/test/java/io/aiven/connect/jdbc/JdbcSourceConnectorTest.java index 13cd2d44..7d3d7c72 100644 --- a/src/test/java/io/aiven/connect/jdbc/JdbcSourceConnectorTest.java +++ b/src/test/java/io/aiven/connect/jdbc/JdbcSourceConnectorTest.java @@ -36,9 +36,9 @@ import io.aiven.connect.jdbc.util.ExpressionBuilder; import io.aiven.connect.jdbc.util.TableId; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.MockedConstruction; import static org.assertj.core.api.Assertions.assertThat; @@ -55,7 +55,7 @@ public class JdbcSourceConnectorTest { private EmbeddedDerby db; private Map connProps; - @Before + @BeforeEach public void setup() { connector = new JdbcSourceConnector(); db = new EmbeddedDerby(); @@ -65,7 +65,7 @@ public void setup() { connProps.put(JdbcSourceConnectorConfig.TOPIC_PREFIX_CONFIG, "test-"); } - @After + @AfterEach public void tearDown() throws Exception { db.close(); db.dropDatabase(); diff --git a/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTest.java b/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTest.java index 0c6f53a9..fa47b206 100644 --- a/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTest.java +++ b/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTest.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.math.BigDecimal; +import java.math.RoundingMode; import java.net.URL; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; @@ -64,8 +65,8 @@ import org.junit.runners.Parameterized; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @RunWith(Parameterized.class) @@ -356,7 +357,7 @@ public void bindFieldPrimitiveValues() throws SQLException { verifyBindField( ++index, Decimal.schema(0), - new BigDecimal("1.5").setScale(0, BigDecimal.ROUND_HALF_EVEN) + new BigDecimal("1.5").setScale(0, RoundingMode.HALF_EVEN) ).setBigDecimal(index, new BigDecimal(2)); final Calendar utcCalendar = DateTimeUtils.getTimeZoneCalendar(TimeZone.getTimeZone(ZoneOffset.UTC)); verifyBindField( @@ -400,22 +401,31 @@ public void bindFieldNull() throws SQLException { } } - @Test(expected = ConnectException.class) - public void bindFieldStructUnsupported() throws SQLException { + @Test + public void bindFieldStructUnsupported() { final Schema structSchema = SchemaBuilder.struct().field("test", Schema.BOOLEAN_SCHEMA).build(); - dialect.bindField(mock(PreparedStatement.class), 1, structSchema, new Struct(structSchema)); + assertThatThrownBy(() -> dialect.bindField(mock(PreparedStatement.class), 1, structSchema, + new Struct(structSchema))) + .isInstanceOf(ConnectException.class) + .hasMessage("Unsupported source data type: STRUCT"); } - @Test(expected = ConnectException.class) - public void bindFieldArrayUnsupported() throws SQLException { + @Test + public void bindFieldArrayUnsupported() { final Schema arraySchema = SchemaBuilder.array(Schema.INT8_SCHEMA); - dialect.bindField(mock(PreparedStatement.class), 1, arraySchema, Collections.emptyList()); + assertThatThrownBy( + () -> dialect.bindField(mock(PreparedStatement.class), 1, arraySchema, Collections.emptyList())) + .isInstanceOf(ConnectException.class) + .hasMessage("Unsupported source data type: ARRAY"); } - @Test(expected = ConnectException.class) - public void bindFieldMapUnsupported() throws SQLException { + @Test + public void bindFieldMapUnsupported() { final Schema mapSchema = SchemaBuilder.map(Schema.INT8_SCHEMA, Schema.INT8_SCHEMA); - dialect.bindField(mock(PreparedStatement.class), 1, mapSchema, Collections.emptyMap()); + assertThatThrownBy( + () -> dialect.bindField(mock(PreparedStatement.class), 1, mapSchema, Collections.emptyMap())) + .isInstanceOf(ConnectException.class) + .hasMessage("Unsupported source data type: MAP"); } protected void assertSanitizedUrl(final String url, final String expectedSanitizedUrl) { @@ -430,7 +440,7 @@ protected PreparedStatement verifyBindField(final int index, final Schema schema throws SQLException { final PreparedStatement statement = mock(PreparedStatement.class); dialect.bindField(statement, index, schema, value); - return verify(statement, times(1)); + return verify(statement); } /** diff --git a/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTypeTest.java b/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTypeTest.java deleted file mode 100644 index 3fd83cd6..00000000 --- a/src/test/java/io/aiven/connect/jdbc/dialect/BaseDialectTypeTest.java +++ /dev/null @@ -1,186 +0,0 @@ -/* - * Copyright 2019 Aiven Oy and jdbc-connector-for-apache-kafka project contributors - * Copyright 2017 Confluent Inc. - * - * 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.aiven.connect.jdbc.dialect; - -import java.math.BigDecimal; -import java.sql.ResultSet; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.kafka.connect.data.Field; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; - -import io.aiven.connect.jdbc.config.JdbcConfig; -import io.aiven.connect.jdbc.source.ColumnMapping; -import io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig; -import io.aiven.connect.jdbc.util.ColumnDefinition; -import io.aiven.connect.jdbc.util.ColumnId; -import io.aiven.connect.jdbc.util.TableId; - -import org.junit.Before; -import org.junit.Test; -import org.junit.runners.Parameterized; -import org.mockito.Mock; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.data.Offset.offset; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -public abstract class BaseDialectTypeTest { - - public static final boolean NULLABLE = true; - public static final boolean NOT_NULLABLE = false; - - public static final TableId TABLE_ID = new TableId(null, null, "MyTable"); - public static final ColumnId COLUMN_ID = new ColumnId(TABLE_ID, "columnA", "aliasA"); - - public static final BigDecimal BIG_DECIMAL = new BigDecimal(9.9); - public static final long LONG = Long.MAX_VALUE; - public static final int INT = Integer.MAX_VALUE; - public static final short SHORT = Short.MAX_VALUE; - public static final byte BYTE = Byte.MAX_VALUE; - public static final double DOUBLE = Double.MAX_VALUE; - - @Parameterized.Parameter(0) - public Schema.Type expectedType; - - @Parameterized.Parameter(1) - public Object expectedValue; - - @Parameterized.Parameter(2) - public JdbcSourceConnectorConfig.NumericMapping numMapping; - - @Parameterized.Parameter(3) - public boolean optional; - - @Parameterized.Parameter(4) - public int columnType; - - @Parameterized.Parameter(5) - public int precision; - - @Parameterized.Parameter(6) - public int scale; - - @Mock - ResultSet resultSet = mock(ResultSet.class); - - @Mock - ColumnDefinition columnDefn = mock(ColumnDefinition.class); - - protected boolean signed = true; - protected T dialect; - protected SchemaBuilder schemaBuilder; - protected DatabaseDialect.ColumnConverter converter; - - @Before - public void setup() throws Exception { - dialect = createDialect(); - } - - @SuppressWarnings("deprecation") - @Test - public void testValueConversionOnNumeric() throws Exception { - when(columnDefn.precision()).thenReturn(precision); - when(columnDefn.scale()).thenReturn(scale); - when(columnDefn.type()).thenReturn(columnType); - when(columnDefn.isOptional()).thenReturn(optional); - when(columnDefn.id()).thenReturn(COLUMN_ID); - when(columnDefn.isSignedNumber()).thenReturn(signed); - when(columnDefn.typeName()).thenReturn("parameterizedType"); - - dialect = createDialect(); - schemaBuilder = SchemaBuilder.struct(); - - // Check the schema field is created with the right type - dialect.addFieldToSchema(columnDefn, schemaBuilder); - final Schema schema = schemaBuilder.build(); - final List fields = schema.fields(); - assertThat(fields).hasSize(1); - final Field field = fields.get(0); - assertThat(field.schema().type()).isEqualTo(expectedType); - - // Set up the ResultSet - when(resultSet.getBigDecimal(1, scale)).thenReturn(BIG_DECIMAL); - when(resultSet.getBigDecimal(1, -scale)).thenReturn(BIG_DECIMAL); - when(resultSet.getBigDecimal(1)).thenReturn(BIG_DECIMAL); - when(resultSet.getLong(1)).thenReturn(LONG); - when(resultSet.getInt(1)).thenReturn(INT); - when(resultSet.getShort(1)).thenReturn(SHORT); - when(resultSet.getByte(1)).thenReturn(BYTE); - when(resultSet.getDouble(1)).thenReturn(DOUBLE); - - // Check the converter operates correctly - final ColumnMapping mapping = new ColumnMapping(columnDefn, 1, field); - converter = dialect.columnConverterFor( - mapping, - mapping.columnDefn(), - mapping.columnNumber(), - true - ); - final Object value = converter.convert(resultSet); - if (value instanceof Number && expectedValue instanceof Number) { - assertThat(((Number) value).floatValue()) - .isCloseTo(((Number) expectedValue).floatValue(), offset(0.01f)); - } else { - assertThat(value).isEqualTo(expectedValue); - } - } - - /** - * Create an instance of the dialect to be tested. - * - * @return the dialect; may not be null - */ - protected abstract T createDialect(); - - /** - * Create a {@link JdbcSourceConnectorConfig} with the specified URL and optional config props. - * - * @param url the database URL; may not be null - * @param propertyPairs optional set of config name-value pairs; must be an even number - * @return the config; never null - */ - protected JdbcSourceConnectorConfig sourceConfigWithUrl( - final String url, - final String... propertyPairs - ) { - final Map connProps = new HashMap<>(); - connProps.put(JdbcSourceConnectorConfig.MODE_CONFIG, JdbcSourceConnectorConfig.MODE_BULK); - connProps.put(JdbcSourceConnectorConfig.TOPIC_PREFIX_CONFIG, "test-"); - connProps.putAll(propertiesFromPairs(propertyPairs)); - connProps.put(JdbcConfig.CONNECTION_URL_CONFIG, url); - connProps.put(JdbcSourceConnectorConfig.NUMERIC_MAPPING_CONFIG, numMapping.toString()); - return new JdbcSourceConnectorConfig(connProps); - } - - protected Map propertiesFromPairs(final String... pairs) { - final Map props = new HashMap<>(); - assertThat(pairs.length % 2).as("Expecting even number of properties but found " + pairs.length) - .isZero(); - for (int i = 0; i != pairs.length; ++i) { - final String key = pairs[i]; - final String value = pairs[++i]; - props.put(key, value); - } - return props; - } -} diff --git a/src/test/java/io/aiven/connect/jdbc/dialect/DatabaseDialectsTest.java b/src/test/java/io/aiven/connect/jdbc/dialect/DatabaseDialectsTest.java index 9a60bc16..a1c8d376 100644 --- a/src/test/java/io/aiven/connect/jdbc/dialect/DatabaseDialectsTest.java +++ b/src/test/java/io/aiven/connect/jdbc/dialect/DatabaseDialectsTest.java @@ -26,10 +26,11 @@ import io.aiven.connect.jdbc.config.JdbcConfig; import io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig; -import org.junit.Test; +import org.junit.jupiter.api.Test; -import static junit.framework.TestCase.assertSame; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.Assertions.assertSame; public class DatabaseDialectsTest { @@ -104,14 +105,16 @@ public void shouldFindMockDialect() { assertDialect(MockDatabaseDialect.class, "jdbc:mock:argle"); } - @Test(expected = ConnectException.class) + @Test public void shouldNotFindDialectForInvalidUrl() { - DatabaseDialects.extractJdbcUrlInfo("jdbc:protocolinvalid;field=value;"); + assertThatThrownBy(() -> DatabaseDialects.extractJdbcUrlInfo("jdbc:protocolinvalid;field=value;")) + .isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void shouldNotFindDialectForInvalidUrlMissingJdbcPrefix() { - DatabaseDialects.extractJdbcUrlInfo("mysql://Server:port"); + assertThatThrownBy(() -> DatabaseDialects.extractJdbcUrlInfo("mysql://Server:port")) + .isInstanceOf(ConnectException.class); } private void assertDialect( diff --git a/src/test/java/io/aiven/connect/jdbc/dialect/GenericDatabaseDialectTypeTest.java b/src/test/java/io/aiven/connect/jdbc/dialect/GenericDatabaseDialectTypeTest.java index 74feb3c3..b7da966d 100644 --- a/src/test/java/io/aiven/connect/jdbc/dialect/GenericDatabaseDialectTypeTest.java +++ b/src/test/java/io/aiven/connect/jdbc/dialect/GenericDatabaseDialectTypeTest.java @@ -17,22 +17,59 @@ package io.aiven.connect.jdbc.dialect; +import java.math.BigDecimal; +import java.sql.ResultSet; import java.sql.Types; import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Stream; +import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import io.aiven.connect.jdbc.config.JdbcConfig; +import io.aiven.connect.jdbc.source.ColumnMapping; import io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig; +import io.aiven.connect.jdbc.util.ColumnDefinition; +import io.aiven.connect.jdbc.util.ColumnId; +import io.aiven.connect.jdbc.util.TableId; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mock; -@RunWith(Parameterized.class) -public class GenericDatabaseDialectTypeTest extends BaseDialectTypeTest { +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.data.Offset.offset; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; - @Parameterized.Parameters - public static Iterable mapping() { - return Arrays.asList( + +public class GenericDatabaseDialectTypeTest { + + public static final boolean NULLABLE = true; + public static final boolean NOT_NULLABLE = false; + public static final TableId TABLE_ID = new TableId(null, null, "MyTable"); + public static final ColumnId COLUMN_ID = new ColumnId(TABLE_ID, "columnA", "aliasA"); + public static final BigDecimal BIG_DECIMAL = new BigDecimal("9.9"); + public static final long LONG = Long.MAX_VALUE; + public static final int INT = Integer.MAX_VALUE; + public static final short SHORT = Short.MAX_VALUE; + public static final byte BYTE = Byte.MAX_VALUE; + public static final double DOUBLE = Double.MAX_VALUE; + protected boolean signed = true; + protected GenericDatabaseDialect dialect; + protected SchemaBuilder schemaBuilder; + protected DatabaseDialect.ColumnConverter converter; + @Mock + ResultSet resultSet = mock(ResultSet.class); + @Mock + ColumnDefinition columnDefn = mock(ColumnDefinition.class); + + public static Stream data() { + return Arrays.stream( new Object[][]{ // MAX_VALUE means this value doesn't matter // Parameter range 1-4 @@ -115,7 +152,7 @@ public static Iterable mapping() { {Schema.Type.BYTES, BIG_DECIMAL, JdbcSourceConnectorConfig.NumericMapping.BEST_FIT, NULLABLE, Types.NUMERIC, 19, -1}, - // floating point - fitting - non optional + // floating point - fitting - non-optional {Schema.Type.FLOAT64, DOUBLE, JdbcSourceConnectorConfig.NumericMapping.BEST_FIT, NOT_NULLABLE, Types.NUMERIC, 18, 127}, {Schema.Type.FLOAT64, DOUBLE, @@ -134,8 +171,92 @@ public static Iterable mapping() { ); } - @Override - protected GenericDatabaseDialect createDialect() { - return new GenericDatabaseDialect(sourceConfigWithUrl("jdbc:some:db")); + /** + * Create a {@link JdbcSourceConnectorConfig} with the specified URL and optional config props. + * + * @param url the database URL; may not be null + * @param propertyPairs optional set of config name-value pairs; must be an even number + * @return the config; never null + */ + protected static JdbcSourceConnectorConfig sourceConfigWithUrl( + final String url, + final JdbcSourceConnectorConfig.NumericMapping numMapping, + final String... propertyPairs + ) { + final Map connProps = new HashMap<>(); + connProps.put(JdbcSourceConnectorConfig.MODE_CONFIG, JdbcSourceConnectorConfig.MODE_BULK); + connProps.put(JdbcSourceConnectorConfig.TOPIC_PREFIX_CONFIG, "test-"); + connProps.putAll(propertiesFromPairs(propertyPairs)); + connProps.put(JdbcConfig.CONNECTION_URL_CONFIG, url); + connProps.put(JdbcSourceConnectorConfig.NUMERIC_MAPPING_CONFIG, numMapping.toString()); + return new JdbcSourceConnectorConfig(connProps); + } + + protected static Map propertiesFromPairs(final String... pairs) { + final Map props = new HashMap<>(); + assertThat(pairs.length % 2).as("Expecting even number of properties but found " + pairs.length) + .isZero(); + for (int i = 0; i != pairs.length; ++i) { + final String key = pairs[i]; + final String value = pairs[++i]; + props.put(key, value); + } + return props; + } + + @SuppressWarnings("deprecation") + @ParameterizedTest + @MethodSource("data") + public void testValueConversionOnNumeric(final Schema.Type expectedType, + final Object expectedValue, + final JdbcSourceConnectorConfig.NumericMapping numMapping, + final boolean optional, + final int columnType, + final int precision, + final int scale) throws Exception { + when(columnDefn.precision()).thenReturn(precision); + when(columnDefn.scale()).thenReturn(scale); + when(columnDefn.type()).thenReturn(columnType); + when(columnDefn.isOptional()).thenReturn(optional); + when(columnDefn.id()).thenReturn(COLUMN_ID); + when(columnDefn.isSignedNumber()).thenReturn(signed); + when(columnDefn.typeName()).thenReturn("parameterizedType"); + + dialect = new GenericDatabaseDialect(sourceConfigWithUrl("jdbc:some:db", numMapping)); + schemaBuilder = SchemaBuilder.struct(); + + // Check the schema field is created with the right type + dialect.addFieldToSchema(columnDefn, schemaBuilder); + final Schema schema = schemaBuilder.build(); + final List fields = schema.fields(); + assertThat(fields).hasSize(1); + final Field field = fields.get(0); + assertThat(field.schema().type()).isEqualTo(expectedType); + + // Set up the ResultSet + when(resultSet.getBigDecimal(1, scale)).thenReturn(BIG_DECIMAL); + when(resultSet.getBigDecimal(1, -scale)).thenReturn(BIG_DECIMAL); + when(resultSet.getBigDecimal(1)).thenReturn(BIG_DECIMAL); + when(resultSet.getLong(1)).thenReturn(LONG); + when(resultSet.getInt(1)).thenReturn(INT); + when(resultSet.getShort(1)).thenReturn(SHORT); + when(resultSet.getByte(1)).thenReturn(BYTE); + when(resultSet.getDouble(1)).thenReturn(DOUBLE); + + // Check the converter operates correctly + final ColumnMapping mapping = new ColumnMapping(columnDefn, 1, field); + converter = dialect.columnConverterFor( + mapping, + mapping.columnDefn(), + mapping.columnNumber(), + true + ); + final Object value = converter.convert(resultSet); + if (value instanceof Number && expectedValue instanceof Number) { + assertThat(((Number) value).floatValue()) + .isCloseTo(((Number) expectedValue).floatValue(), offset(0.01f)); + } else { + assertThat(value).isEqualTo(expectedValue); + } } } diff --git a/src/test/java/io/aiven/connect/jdbc/sink/BufferedRecordsTest.java b/src/test/java/io/aiven/connect/jdbc/sink/BufferedRecordsTest.java index 01da6637..ef63bd26 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/BufferedRecordsTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/BufferedRecordsTest.java @@ -36,9 +36,9 @@ import io.aiven.connect.jdbc.sink.metadata.FieldsMetadata; import io.aiven.connect.jdbc.util.TableId; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import static java.sql.Statement.SUCCESS_NO_INFO; @@ -56,12 +56,12 @@ public class BufferedRecordsTest { private final SqliteHelper sqliteHelper = new SqliteHelper(getClass().getSimpleName()); private final String dbUrl = sqliteHelper.sqliteUri(); - @Before + @BeforeEach public void setUp() throws IOException, SQLException { sqliteHelper.setUp(); } - @After + @AfterEach public void tearDown() throws IOException, SQLException { sqliteHelper.tearDown(); } @@ -146,7 +146,7 @@ public void testFlushSuccessNoInfo() throws SQLException { buffer.add(recordA); final Schema schemaB = SchemaBuilder.struct().field("name", Schema.STRING_SCHEMA).build(); - final Struct valueB = new Struct(schemaA).put("name", "cubb"); + final Struct valueB = new Struct(schemaB).put("name", "cubb"); final SinkRecord recordB = wrapInSinkRecord(valueB); buffer.add(recordB); buffer.flush(); diff --git a/src/test/java/io/aiven/connect/jdbc/sink/DbStructureTest.java b/src/test/java/io/aiven/connect/jdbc/sink/DbStructureTest.java index 0aa6390e..df5d20b7 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/DbStructureTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/DbStructureTest.java @@ -28,7 +28,7 @@ import io.aiven.connect.jdbc.sink.metadata.SinkRecordField; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; diff --git a/src/test/java/io/aiven/connect/jdbc/sink/JdbcDbWriterTest.java b/src/test/java/io/aiven/connect/jdbc/sink/JdbcDbWriterTest.java index a48417b8..83fd3579 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/JdbcDbWriterTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/JdbcDbWriterTest.java @@ -41,11 +41,12 @@ import io.aiven.connect.jdbc.util.TableDefinition; import io.aiven.connect.jdbc.util.TableId; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.data.Offset.offset; public class JdbcDbWriterTest { @@ -55,12 +56,12 @@ public class JdbcDbWriterTest { private JdbcDbWriter writer = null; private DatabaseDialect dialect; - @Before + @BeforeEach public void setUp() throws IOException, SQLException { sqliteHelper.setUp(); } - @After + @AfterEach public void tearDown() throws IOException, SQLException { if (writer != null) { writer.closeQuietly(); @@ -119,7 +120,7 @@ public void shouldSelectTableFromMapping() { assertThat(tableId.tableName()).isEqualTo("same_table"); } - @Test(expected = ConnectException.class) + @Test public void shouldThrowConnectExceptionForUnknownTopicToTableMapping() { final Map props = new HashMap<>(); props.put(JdbcSinkConfig.CONNECTION_URL_CONFIG, "jdbc://localhost"); @@ -130,7 +131,8 @@ public void shouldThrowConnectExceptionForUnknownTopicToTableMapping() { dialect = new SqliteDatabaseDialect(jdbcSinkConfig); final DbStructure dbStructure = new DbStructure(dialect); final JdbcDbWriter writer = new JdbcDbWriter(jdbcSinkConfig, dialect, dbStructure); - writer.generateTableNameFor("another_topic"); + assertThatThrownBy(() -> writer.generateTableNameFor("another_topic")) + .isInstanceOf(ConnectException.class); } @Test @@ -188,10 +190,11 @@ public void autoCreateWithAutoEvolve() throws SQLException { } } - @Test(expected = SQLException.class) - public void multiInsertWithKafkaPkFailsDueToUniqueConstraint() throws SQLException { - writeSameRecordTwiceExpectingSingleUpdate( - JdbcSinkConfig.InsertMode.INSERT, JdbcSinkConfig.PrimaryKeyMode.KAFKA, ""); + @Test + public void multiInsertWithKafkaPkFailsDueToUniqueConstraint() { + assertThatThrownBy(() -> writeSameRecordTwiceExpectingSingleUpdate( + JdbcSinkConfig.InsertMode.INSERT, JdbcSinkConfig.PrimaryKeyMode.KAFKA, "")) + .isInstanceOf(SQLException.class); } @Test @@ -200,10 +203,11 @@ public void idempotentUpsertWithKafkaPk() throws SQLException { JdbcSinkConfig.InsertMode.UPSERT, JdbcSinkConfig.PrimaryKeyMode.KAFKA, ""); } - @Test(expected = SQLException.class) - public void multiInsertWithRecordKeyPkFailsDueToUniqueConstraint() throws SQLException { - writeSameRecordTwiceExpectingSingleUpdate( - JdbcSinkConfig.InsertMode.INSERT, JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, ""); + @Test + public void multiInsertWithRecordKeyPkFailsDueToUniqueConstraint() { + assertThatThrownBy(() -> writeSameRecordTwiceExpectingSingleUpdate( + JdbcSinkConfig.InsertMode.INSERT, JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, "")) + .isInstanceOf(SQLException.class); } @Test @@ -212,10 +216,11 @@ public void idempotentUpsertWithRecordKeyPk() throws SQLException { JdbcSinkConfig.InsertMode.UPSERT, JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, ""); } - @Test(expected = SQLException.class) - public void multiInsertWithRecordValuePkFailsDueToUniqueConstraint() throws SQLException { - writeSameRecordTwiceExpectingSingleUpdate( - JdbcSinkConfig.InsertMode.INSERT, JdbcSinkConfig.PrimaryKeyMode.RECORD_VALUE, "author,title"); + @Test + public void multiInsertWithRecordValuePkFailsDueToUniqueConstraint() { + assertThatThrownBy(() -> writeSameRecordTwiceExpectingSingleUpdate( + JdbcSinkConfig.InsertMode.INSERT, JdbcSinkConfig.PrimaryKeyMode.RECORD_VALUE, "author,title")) + .isInstanceOf(SQLException.class); } @Test diff --git a/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkConfigTest.java b/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkConfigTest.java index af8ba7f2..b4e7527f 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkConfigTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkConfigTest.java @@ -22,9 +22,10 @@ import org.apache.kafka.common.config.ConfigException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.entry; public class JdbcSinkConfigTest { @@ -55,20 +56,22 @@ public void shouldParseTopicToTableMappings() { assertThat(config.topicsToTablesMapping).containsExactly(entry("t3", "tbl3")); } - @Test(expected = ConfigException.class) + @Test public void shouldThrowExceptionForWrongMappingFormat() { final Map props = new HashMap<>(); props.put(JdbcSinkConfig.TOPICS_TO_TABLES_MAPPING, "asd:asd,asd"); - new JdbcSinkConfig(props); + assertThatThrownBy(() -> new JdbcSinkConfig(props)) + .isInstanceOf(ConfigException.class); } - @Test(expected = ConfigException.class) + @Test public void shouldThrowExceptionForEmptyMappingFormat() { final Map props = new HashMap<>(); props.put(JdbcSinkConfig.TOPICS_TO_TABLES_MAPPING, ",,,,,,asd"); - new JdbcSinkConfig(props); + assertThatThrownBy(() -> new JdbcSinkConfig(props)) + .isInstanceOf(ConfigException.class); } } diff --git a/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkTaskTest.java b/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkTaskTest.java index 7e974ff8..6721e515 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkTaskTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/JdbcSinkTaskTest.java @@ -39,9 +39,9 @@ import io.aiven.connect.jdbc.config.JdbcConfig; import io.aiven.connect.jdbc.util.DateTimeUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -67,12 +67,12 @@ public class JdbcSinkTaskTest { .field("modified", Timestamp.SCHEMA) .build(); - @Before + @BeforeEach public void setUp() throws IOException, SQLException { sqliteHelper.setUp(); } - @After + @AfterEach public void tearDown() throws IOException, SQLException { sqliteHelper.tearDown(); } diff --git a/src/test/java/io/aiven/connect/jdbc/sink/PreparedStatementBinderTest.java b/src/test/java/io/aiven/connect/jdbc/sink/PreparedStatementBinderTest.java index 3f0756ff..dc2e74a0 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/PreparedStatementBinderTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/PreparedStatementBinderTest.java @@ -18,6 +18,7 @@ package io.aiven.connect.jdbc.sink; import java.math.BigDecimal; +import java.math.RoundingMode; import java.sql.PreparedStatement; import java.sql.SQLException; import java.text.ParseException; @@ -45,18 +46,17 @@ import io.aiven.connect.jdbc.sink.metadata.SchemaPair; import io.aiven.connect.jdbc.util.DateTimeUtils; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; public class PreparedStatementBinderTest { private DatabaseDialect dialect; - @Before + @BeforeEach public void beforeEach() { final Map props = new HashMap<>(); props.put(JdbcConfig.CONNECTION_URL_CONFIG, "jdbc:bogus:something"); @@ -97,7 +97,7 @@ public void bindRecordInsert() throws SQLException, ParseException { .put("double", -2436546.56457) .put("bytes", new byte[]{-32, 124}) .put("age", 30) - .put("decimal", new BigDecimal("1.5").setScale(0, BigDecimal.ROUND_HALF_EVEN)) + .put("decimal", new BigDecimal("1.5").setScale(0, RoundingMode.HALF_EVEN)) .put("date", new java.util.Date(0)) .put("time", new java.util.Date(1000)) .put("timestamp", new java.util.Date(100)); @@ -126,36 +126,33 @@ public void bindRecordInsert() throws SQLException, ParseException { int index = 1; // key field first - verify(statement, times(1)).setLong(index++, valueStruct.getInt64("long")); + verify(statement).setLong(index++, valueStruct.getInt64("long")); // rest in order of schema def - verify(statement, times(1)).setString(index++, valueStruct.getString("firstName")); - verify(statement, times(1)).setString(index++, valueStruct.getString("lastName")); - verify(statement, times(1)).setInt(index++, valueStruct.getInt32("age")); - verify(statement, times(1)).setBoolean(index++, valueStruct.getBoolean("bool")); - verify(statement, times(1)).setShort(index++, valueStruct.getInt16("short")); - verify(statement, times(1)).setByte(index++, valueStruct.getInt8("byte")); - verify(statement, times(1)).setFloat(index++, valueStruct.getFloat32("float")); - verify(statement, times(1)).setDouble(index++, valueStruct.getFloat64("double")); - verify(statement, times(1)).setBytes(index++, valueStruct.getBytes("bytes")); - verify(statement, times(1)).setBigDecimal(index++, (BigDecimal) valueStruct.get("decimal")); + verify(statement).setString(index++, valueStruct.getString("firstName")); + verify(statement).setString(index++, valueStruct.getString("lastName")); + verify(statement).setInt(index++, valueStruct.getInt32("age")); + verify(statement).setBoolean(index++, valueStruct.getBoolean("bool")); + verify(statement).setShort(index++, valueStruct.getInt16("short")); + verify(statement).setByte(index++, valueStruct.getInt8("byte")); + verify(statement).setFloat(index++, valueStruct.getFloat32("float")); + verify(statement).setDouble(index++, valueStruct.getFloat64("double")); + verify(statement).setBytes(index++, valueStruct.getBytes("bytes")); + verify(statement).setBigDecimal(index++, (BigDecimal) valueStruct.get("decimal")); final Calendar utcCalendar = DateTimeUtils.getTimeZoneCalendar(TimeZone.getTimeZone(ZoneOffset.UTC)); verify( - statement, - times(1) + statement ).setDate(index++, new java.sql.Date(((java.util.Date) valueStruct.get("date")).getTime()), utcCalendar); verify( - statement, - times(1) + statement ).setTime(index++, new java.sql.Time(((java.util.Date) valueStruct.get("time")).getTime()), utcCalendar); verify( - statement, - times(1) + statement ).setTimestamp( index++, new java.sql.Timestamp(((java.util.Date) valueStruct.get("timestamp")).getTime()), utcCalendar); // last field is optional and is null-valued in struct - verify(statement, times(1)).setObject(index++, null); + verify(statement).setObject(index++, null); } @Test @@ -192,9 +189,9 @@ public void bindRecordUpsertMode() throws SQLException, ParseException { int index = 1; // key field first - verify(statement, times(1)).setLong(index++, valueStruct.getInt64("long")); + verify(statement).setLong(index++, valueStruct.getInt64("long")); // rest in order of schema def - verify(statement, times(1)).setString(index++, valueStruct.getString("firstName")); + verify(statement).setString(index++, valueStruct.getString("firstName")); } @Test @@ -232,9 +229,9 @@ public void bindRecordUpdateMode() throws SQLException, ParseException { int index = 1; // non key first - verify(statement, times(1)).setString(index++, valueStruct.getString("firstName")); + verify(statement).setString(index++, valueStruct.getString("firstName")); // last the keys - verify(statement, times(1)).setLong(index++, valueStruct.getInt64("long")); + verify(statement).setLong(index++, valueStruct.getInt64("long")); } } diff --git a/src/test/java/io/aiven/connect/jdbc/sink/SqliteHelperTest.java b/src/test/java/io/aiven/connect/jdbc/sink/SqliteHelperTest.java index 1166d994..99dd5c64 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/SqliteHelperTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/SqliteHelperTest.java @@ -30,9 +30,9 @@ import io.aiven.connect.jdbc.util.TableDefinition; import io.aiven.connect.jdbc.util.TableId; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -40,12 +40,12 @@ public class SqliteHelperTest { private final SqliteHelper sqliteHelper = new SqliteHelper(getClass().getSimpleName()); - @Before + @BeforeEach public void setUp() throws IOException, SQLException { sqliteHelper.setUp(); } - @After + @AfterEach public void tearDown() throws IOException, SQLException { sqliteHelper.tearDown(); } diff --git a/src/test/java/io/aiven/connect/jdbc/sink/metadata/FieldsMetadataTest.java b/src/test/java/io/aiven/connect/jdbc/sink/metadata/FieldsMetadataTest.java index 7b48d4d2..7123c093 100644 --- a/src/test/java/io/aiven/connect/jdbc/sink/metadata/FieldsMetadataTest.java +++ b/src/test/java/io/aiven/connect/jdbc/sink/metadata/FieldsMetadataTest.java @@ -28,9 +28,10 @@ import io.aiven.connect.jdbc.sink.JdbcSinkConfig; import com.google.common.collect.Lists; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class FieldsMetadataTest { @@ -39,24 +40,26 @@ public class FieldsMetadataTest { .field("name", Schema.STRING_SCHEMA).build(); private static final Schema SIMPLE_MAP_SCHEMA = SchemaBuilder.map(SchemaBuilder.INT64_SCHEMA, Schema.STRING_SCHEMA); - @Test(expected = ConnectException.class) + @Test public void valueSchemaMustBePresentForPkModeRecordValue() { - extract( - JdbcSinkConfig.PrimaryKeyMode.RECORD_VALUE, - Collections.emptyList(), - SIMPLE_PRIMITIVE_SCHEMA, - null - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.RECORD_VALUE, + Collections.emptyList(), + SIMPLE_PRIMITIVE_SCHEMA, + null + )).isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void valueSchemaMustBeStructIfPresent() { - extract( - JdbcSinkConfig.PrimaryKeyMode.KAFKA, - Collections.emptyList(), - SIMPLE_PRIMITIVE_SCHEMA, - SIMPLE_PRIMITIVE_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.KAFKA, + Collections.emptyList(), + SIMPLE_PRIMITIVE_SCHEMA, + SIMPLE_PRIMITIVE_SCHEMA + )).isInstanceOf(ConnectException.class); } @Test @@ -78,14 +81,15 @@ public void missingValueSchemaCanBeOk() { ).allFields.keySet())).isEqualTo(List.of("__connect_topic", "__connect_partition", "__connect_offset")); } - @Test(expected = ConnectException.class) + @Test public void metadataMayNotBeEmpty() { - extract( - JdbcSinkConfig.PrimaryKeyMode.NONE, - Collections.emptyList(), - null, - null - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.NONE, + Collections.emptyList(), + null, + null + )).isInstanceOf(ConnectException.class); } @Test @@ -129,14 +133,15 @@ public void kafkaPkModeCustomNames() { assertThat(metadata.nonKeyFieldNames).isEqualTo(Collections.singleton("name")); } - @Test(expected = ConnectException.class) + @Test public void kafkaPkModeBadFieldSpec() { - extract( - JdbcSinkConfig.PrimaryKeyMode.KAFKA, - List.of("lone"), - null, - SIMPLE_STRUCT_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.KAFKA, + List.of("lone"), + null, + SIMPLE_STRUCT_SCHEMA + )).isInstanceOf(ConnectException.class); } /** @@ -166,54 +171,59 @@ public void recordKeyPkModePrimitiveKey() { assertThat(metadata.allFields.get("name").isOptional()).isFalse(); } - @Test(expected = ConnectException.class) + @Test public void recordKeyPkModeWithPrimitiveKeyButMultiplePkFieldsSpecified() { - extract( - JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, - List.of("pk1", "pk2"), - SIMPLE_PRIMITIVE_SCHEMA, - SIMPLE_STRUCT_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, + List.of("pk1", "pk2"), + SIMPLE_PRIMITIVE_SCHEMA, + SIMPLE_STRUCT_SCHEMA + )).isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void recordKeyPkModeButKeySchemaMissing() { - extract( - JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, - Collections.emptyList(), - null, - SIMPLE_STRUCT_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, + Collections.emptyList(), + null, + SIMPLE_STRUCT_SCHEMA + )).isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void recordKeyPkModeButKeySchemaAsNonStructCompositeType() { - extract( - JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, - Collections.emptyList(), - SIMPLE_MAP_SCHEMA, - SIMPLE_STRUCT_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, + Collections.emptyList(), + SIMPLE_MAP_SCHEMA, + SIMPLE_STRUCT_SCHEMA + )).isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void recordKeyPkModeWithStructKeyButMissingField() { - extract( - JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, - Collections.singletonList("nonexistent"), - SIMPLE_STRUCT_SCHEMA, - SIMPLE_STRUCT_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.RECORD_KEY, + Collections.singletonList("nonexistent"), + SIMPLE_STRUCT_SCHEMA, + SIMPLE_STRUCT_SCHEMA + )).isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void recordValuePkModeWithMissingPkField() { - extract( - JdbcSinkConfig.PrimaryKeyMode.RECORD_VALUE, - List.of("nonexistent"), - SIMPLE_PRIMITIVE_SCHEMA, - SIMPLE_STRUCT_SCHEMA - ); + assertThatThrownBy(() -> + extract( + JdbcSinkConfig.PrimaryKeyMode.RECORD_VALUE, + List.of("nonexistent"), + SIMPLE_PRIMITIVE_SCHEMA, + SIMPLE_STRUCT_SCHEMA + )).isInstanceOf(ConnectException.class); } @Test diff --git a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceConnectorConfigTest.java b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceConnectorConfigTest.java index d7404f07..726f3c58 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceConnectorConfigTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceConnectorConfigTest.java @@ -30,19 +30,19 @@ import io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig.CachedRecommenderValues; import io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig.CachingRecommender; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.junit.jupiter.MockitoExtension; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.util.Lists.list; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class JdbcSourceConnectorConfigTest { private EmbeddedDerby db; @@ -53,7 +53,7 @@ public class JdbcSourceConnectorConfigTest { private Recommender mockRecommender; private final MockTime time = new MockTime(); - @Before + @BeforeEach public void setup() throws Exception { configDef = null; results = null; @@ -72,7 +72,7 @@ public void setup() throws Exception { db.createTable("another_private_table", "id", "INT"); } - @After + @AfterEach public void cleanup() throws Exception { db.close(); db.dropDatabase(); diff --git a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskConversionTest.java b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskConversionTest.java index 670c1d48..fdabdb3c 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskConversionTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskConversionTest.java @@ -37,35 +37,26 @@ import org.apache.kafka.connect.data.Timestamp; import org.apache.kafka.connect.source.SourceRecord; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; // Tests conversion of data types and schemas. These use the types supported by Derby, which // might not cover everything in the SQL standards and definitely doesn't cover any non-standard // types, but should cover most of the JDBC types which is all we see anyway -@RunWith(Parameterized.class) public class JdbcSourceTaskConversionTest extends JdbcSourceTaskTestBase { - @Parameterized.Parameters - public static Object[] mapping() { - return new Object[]{false, true}; - } - - @Parameterized.Parameter - public boolean extendedMapping; + public boolean extendedMapping = true; - @Before + @BeforeEach public void setup() throws Exception { super.setup(); task.start(singleTableConfig(extendedMapping)); } - @After + @AfterEach public void tearDown() throws Exception { task.stop(); super.tearDown(); diff --git a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskLifecycleTest.java b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskLifecycleTest.java index 192b0f62..1929e237 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskLifecycleTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskLifecycleTest.java @@ -17,7 +17,6 @@ package io.aiven.connect.jdbc.source; -import java.sql.Connection; import java.util.List; import java.util.Map; @@ -26,37 +25,30 @@ import io.aiven.connect.jdbc.config.JdbcConfig; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mockConstruction; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@RunWith(MockitoJUnitRunner.class) public class JdbcSourceTaskLifecycleTest extends JdbcSourceTaskTestBase { - @Mock - private SourceConnectionProvider mockSourceConnectionProvider; - - @Mock - private Connection conn; - - @Test(expected = ConnectException.class) + @Test public void testMissingParentConfig() { final Map props = singleTableConfig(); props.remove(JdbcConfig.CONNECTION_URL_CONFIG); - task.start(props); + assertThatThrownBy(() -> task.start(props)) + .isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void testMissingTables() { final Map props = singleTableConfig(); props.remove(JdbcSourceTaskConfig.TABLES_CONFIG); - task.start(props); + assertThatThrownBy(() -> task.start(props)) + .isInstanceOf(ConnectException.class); } @Test diff --git a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskTestBase.java b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskTestBase.java index a1335a14..34b72212 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskTestBase.java +++ b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskTestBase.java @@ -28,11 +28,10 @@ import io.aiven.connect.jdbc.config.JdbcConfig; import io.aiven.connect.jdbc.util.TableId; -import org.junit.After; -import org.junit.Before; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; import org.mockito.Mock; -import static io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig.NumericMapping; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.when; @@ -45,9 +44,6 @@ public class JdbcSourceTaskTestBase { protected static final Map SINGLE_TABLE_PARTITION_WITH_VERSION = OffsetProtocols.sourcePartitionForProtocolV1(SINGLE_TABLE_ID); - protected static final EmbeddedDerby.TableName SINGLE_TABLE - = new EmbeddedDerby.TableName(SINGLE_TABLE_NAME); - protected static final String SECOND_TABLE_NAME = "test2"; protected static final Map SECOND_TABLE_PARTITION = new HashMap<>(); @@ -55,9 +51,6 @@ public class JdbcSourceTaskTestBase { SECOND_TABLE_PARTITION.put(JdbcSourceConnectorConstants.TABLE_NAME_KEY, SECOND_TABLE_NAME); } - protected static final EmbeddedDerby.TableName SECOND_TABLE - = new EmbeddedDerby.TableName(SECOND_TABLE_NAME); - protected static final String JOIN_TABLE_NAME = "users"; protected static final Map JOIN_QUERY_PARTITION = new HashMap<>(); @@ -66,9 +59,6 @@ public class JdbcSourceTaskTestBase { JdbcSourceConnectorConstants.QUERY_NAME_VALUE); } - protected static final EmbeddedDerby.TableName JOIN_TABLE - = new EmbeddedDerby.TableName(JOIN_TABLE_NAME); - protected static final String TOPIC_PREFIX = "test-"; protected Time time; @@ -79,14 +69,14 @@ public class JdbcSourceTaskTestBase { @Mock private OffsetStorageReader reader; - @Before + @BeforeEach public void setup() throws Exception { time = new MockTime(); task = new JdbcSourceTask(time); db = new EmbeddedDerby(); } - @After + @AfterEach public void tearDown() throws Exception { db.close(); db.dropDatabase(); @@ -103,7 +93,8 @@ protected Map singleTableConfig(final boolean completeMapping) { props.put(JdbcSourceConnectorConfig.MODE_CONFIG, JdbcSourceConnectorConfig.MODE_BULK); props.put(JdbcSourceTaskConfig.TOPIC_PREFIX_CONFIG, TOPIC_PREFIX); if (completeMapping) { - props.put(JdbcSourceTaskConfig.NUMERIC_MAPPING_CONFIG, NumericMapping.BEST_FIT.toString()); + props.put(JdbcSourceTaskConfig.NUMERIC_MAPPING_CONFIG, + JdbcSourceConnectorConfig.NumericMapping.BEST_FIT.toString()); } else { props.put(JdbcSourceTaskConfig.NUMERIC_PRECISION_MAPPING_CONFIG, "true"); } diff --git a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskUpdateTest.java b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskUpdateTest.java index a69259fe..5baaa3b0 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskUpdateTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/JdbcSourceTaskUpdateTest.java @@ -35,10 +35,10 @@ import io.aiven.connect.jdbc.config.JdbcConfig; import io.aiven.connect.jdbc.util.DateTimeUtils; -import org.junit.After; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.junit.MockitoJUnitRunner; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -47,7 +47,7 @@ // Tests of polling that return data updates, i.e. verifies the different behaviors for getting // incremental data updates from the database -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class JdbcSourceTaskUpdateTest extends JdbcSourceTaskTestBase { private static final Map QUERY_SOURCE_PARTITION = Collections.singletonMap(JdbcSourceConnectorConstants.QUERY_NAME_KEY, @@ -55,7 +55,7 @@ public class JdbcSourceTaskUpdateTest extends JdbcSourceTaskTestBase { private static final TimeZone UTC_TIME_ZONE = TimeZone.getTimeZone(ZoneOffset.UTC); - @After + @AfterEach public void tearDown() throws Exception { task.stop(); super.tearDown(); @@ -92,7 +92,7 @@ public void testBulkPeriodicLoad() throws Exception { assertRecordsTopic(records, TOPIC_PREFIX + SINGLE_TABLE_NAME); } - @Test(expected = ConnectException.class) + @Test public void testIncrementingInvalidColumn() throws Exception { expectInitializeNoOffsets(Arrays.asList( SINGLE_TABLE_PARTITION_WITH_VERSION, @@ -102,12 +102,11 @@ public void testIncrementingInvalidColumn() throws Exception { // Incrementing column must be NOT NULL db.createTable(SINGLE_TABLE_NAME, "id", "INT"); - startTask(null, "id", null); - - verify(taskContext).offsetStorageReader(); + assertThatThrownBy(() -> startTask(null, "id", null)) + .isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void testTimestampInvalidColumn() throws Exception { expectInitializeNoOffsets(Arrays.asList( SINGLE_TABLE_PARTITION_WITH_VERSION, @@ -117,9 +116,8 @@ public void testTimestampInvalidColumn() throws Exception { // Timestamp column must be NOT NULL db.createTable(SINGLE_TABLE_NAME, "modified", "TIMESTAMP"); - startTask("modified", null, null); - - verify(taskContext).offsetStorageReader(); + assertThatThrownBy(() -> startTask("modified", null, null)) + .isInstanceOf(ConnectException.class); } @Test diff --git a/src/test/java/io/aiven/connect/jdbc/source/NumericMappingConfigTest.java b/src/test/java/io/aiven/connect/jdbc/source/NumericMappingConfigTest.java index ebdb3101..15b4ff48 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/NumericMappingConfigTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/NumericMappingConfigTest.java @@ -20,22 +20,20 @@ import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import java.util.stream.Stream; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.MethodSource; import static io.aiven.connect.jdbc.source.JdbcSourceConnectorConfig.NumericMapping; import static org.assertj.core.api.Assertions.assertThat; -@RunWith(Parameterized.class) public class NumericMappingConfigTest { private Map props; - @Parameterized.Parameters - public static Iterable mapping() { - return Arrays.asList( + public static Stream mapping() { + return Arrays.stream( new Object[][]{ {NumericMapping.NONE, false, null}, {NumericMapping.NONE, false, "none"}, @@ -55,22 +53,15 @@ public static Iterable mapping() { ); } - @Parameterized.Parameter(0) - public NumericMapping expected; - - @Parameterized.Parameter(1) - public boolean precisionMapping; - - @Parameterized.Parameter(2) - public String extendedMapping; - - @Before + @BeforeEach public void setup() throws Exception { props = new HashMap<>(); } - @Test - public void testNumericMapping() { + @ParameterizedTest + @MethodSource("mapping") + public void testNumericMapping(final NumericMapping expected, final boolean precisionMapping, + final String extendedMapping) { props.put(JdbcSourceConnectorConfig.CONNECTION_URL_CONFIG, "jdbc:foo:bar"); props.put(JdbcSourceConnectorConfig.MODE_CONFIG, JdbcSourceConnectorConfig.MODE_BULK); props.put(JdbcSourceConnectorConfig.TOPIC_PREFIX_CONFIG, "test-"); diff --git a/src/test/java/io/aiven/connect/jdbc/source/TableMonitorThreadTest.java b/src/test/java/io/aiven/connect/jdbc/source/TableMonitorThreadTest.java index 84559a26..1ee79785 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/TableMonitorThreadTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/TableMonitorThreadTest.java @@ -35,14 +35,12 @@ import io.aiven.connect.jdbc.util.ExpressionBuilder; import io.aiven.connect.jdbc.util.TableId; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mock; import org.mockito.Mockito; -import org.mockito.MockitoAnnotations; -import org.mockito.invocation.InvocationOnMock; +import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.stubbing.Answer; import static org.assertj.core.api.Assertions.assertThat; @@ -54,7 +52,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@RunWith(Parameterized.class) +@ExtendWith(MockitoExtension.class) public class TableMonitorThreadTest { private static final long POLL_INTERVAL = 100; @@ -84,8 +82,6 @@ public class TableMonitorThreadTest { new HashSet<>(Arrays.asList("VIEW")) ); - private final boolean qualifiedTableNames; - private TableMonitorThread tableMonitorThread; @Mock @@ -97,24 +93,11 @@ public class TableMonitorThreadTest { @Mock private ConnectorContext context; - @Parameterized.Parameters - public static Iterable data() { - return Arrays.asList(false, true); - } - - public TableMonitorThreadTest(final boolean qualifiedTableNames) { - this.qualifiedTableNames = qualifiedTableNames; - } - - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - } - - @Test - public void testSingleLookup() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testSingleLookup(final boolean qualifiedTableNames) throws Exception { when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(null, null); + tableMonitorThread = newTableMonitorThread(null, null, qualifiedTableNames); final String expectedTableName; if (qualifiedTableNames) { expectTableNames(LIST_FOO, shutdownThread()); @@ -130,11 +113,12 @@ public void testSingleLookup() throws Exception { verify(dialect).expressionBuilder(); } - @Test - public void testWhitelist() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testWhitelist(final boolean qualifiedTableNames) throws Exception { final Set whitelist = new HashSet<>(Arrays.asList("foo", "bar")); when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(whitelist, null); + tableMonitorThread = newTableMonitorThread(whitelist, null, qualifiedTableNames); expectTableNames(LIST_FOO_BAR, shutdownThread()); tableMonitorThread.start(); @@ -144,11 +128,12 @@ public void testWhitelist() throws Exception { verify(dialect, atLeastOnce()).expressionBuilder(); } - @Test - public void testBlacklist() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testBlacklist(final boolean qualifiedTableNames) throws Exception { final Set blacklist = new HashSet<>(Arrays.asList("bar", "baz")); when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(null, blacklist); + tableMonitorThread = newTableMonitorThread(null, blacklist, qualifiedTableNames); expectTableNames(LIST_FOO_BAR_BAZ, shutdownThread()); tableMonitorThread.start(); @@ -158,12 +143,12 @@ public void testBlacklist() throws Exception { verify(dialect, atLeastOnce()).expressionBuilder(); } - @Test - public void testReconfigOnUpdate() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testReconfigOnUpdate(final boolean qualifiedTableNames) throws Exception { when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(null, null); - expectTableNames(LIST_FOO); - expectTableNames(LIST_FOO, checkTableNames("foo")); + tableMonitorThread = newTableMonitorThread(null, null, qualifiedTableNames); + when(connectionProvider.getConnection()).thenReturn(connection); // Change the result to trigger a task reconfiguration expectTableNames(LIST_FOO_BAR); @@ -179,15 +164,13 @@ public void testReconfigOnUpdate() throws Exception { verify(dialect, times(2)).expressionBuilder(); } - @Test - public void testInvalidConnection() throws Exception { - tableMonitorThread = newTableMonitorThread(null, null); - when(connectionProvider.getConnection()).thenAnswer(new Answer() { - @Override - public Connection answer(final InvocationOnMock invocation) throws Throwable { - tableMonitorThread.shutdown(); - throw new ConnectException("Simulated error with the db."); - } + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testInvalidConnection(final boolean qualifiedTableNames) throws Exception { + tableMonitorThread = newTableMonitorThread(null, null, qualifiedTableNames); + when(connectionProvider.getConnection()).thenAnswer((Answer) invocation -> { + tableMonitorThread.shutdown(); + throw new ConnectException("Simulated error with the db."); }); tableMonitorThread.start(); @@ -196,10 +179,11 @@ public Connection answer(final InvocationOnMock invocation) throws Throwable { verify(connectionProvider).getConnection(); } - @Test - public void testDuplicates() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testDuplicates(final boolean qualifiedTableNames) throws Exception { when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(null, null); + tableMonitorThread = newTableMonitorThread(null, null, qualifiedTableNames); expectTableNames(LIST_DUP_WITH_ALL, shutdownThread()); tableMonitorThread.start(); tableMonitorThread.join(); @@ -213,11 +197,12 @@ public void testDuplicates() throws Exception { verify(dialect).expressionBuilder(); } - @Test - public void testDuplicateWithUnqualifiedWhitelist() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testDuplicateWithUnqualifiedWhitelist(final boolean qualifiedTableNames) throws Exception { final Set whitelist = new HashSet<>(Arrays.asList("dup")); when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(whitelist, null); + tableMonitorThread = newTableMonitorThread(whitelist, null, qualifiedTableNames); expectTableNames(LIST_DUP_ONLY, shutdownThread()); tableMonitorThread.start(); @@ -232,11 +217,12 @@ public void testDuplicateWithUnqualifiedWhitelist() throws Exception { verify(dialect, atLeastOnce()).expressionBuilder(); } - @Test - public void testDuplicateWithUnqualifiedBlacklist() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testDuplicateWithUnqualifiedBlacklist(final boolean qualifiedTableNames) throws Exception { final Set blacklist = new HashSet<>(Arrays.asList("foo")); when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(null, blacklist); + tableMonitorThread = newTableMonitorThread(null, blacklist, qualifiedTableNames); expectTableNames(LIST_DUP_WITH_ALL, shutdownThread()); tableMonitorThread.start(); tableMonitorThread.join(); @@ -250,11 +236,12 @@ public void testDuplicateWithUnqualifiedBlacklist() throws Exception { verify(dialect, atLeastOnce()).expressionBuilder(); } - @Test - public void testDuplicateWithQualifiedWhitelist() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testDuplicateWithQualifiedWhitelist(final boolean qualifiedTableNames) throws Exception { final Set whitelist = new HashSet<>(Arrays.asList("dup1.dup", "foo")); when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(whitelist, null); + tableMonitorThread = newTableMonitorThread(whitelist, null, qualifiedTableNames); expectTableNames(LIST_DUP_WITH_ALL, shutdownThread()); tableMonitorThread.start(); @@ -269,11 +256,12 @@ public void testDuplicateWithQualifiedWhitelist() throws Exception { verify(dialect, atLeastOnce()).expressionBuilder(); } - @Test - public void testDuplicateWithQualifiedBlacklist() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testDuplicateWithQualifiedBlacklist(final boolean qualifiedTableNames) throws Exception { final Set blacklist = new HashSet<>(Arrays.asList("dup1.dup", "foo")); when(dialect.expressionBuilder()).thenReturn(ExpressionBuilder.create()); - tableMonitorThread = newTableMonitorThread(null, blacklist); + tableMonitorThread = newTableMonitorThread(null, blacklist, qualifiedTableNames); expectTableNames(LIST_DUP_WITH_ALL, shutdownThread()); tableMonitorThread.start(); @@ -287,15 +275,16 @@ public void testDuplicateWithQualifiedBlacklist() throws Exception { verify(dialect, atLeastOnce()).expressionBuilder(); } - private TableMonitorThread newTableMonitorThread(final Set whitelist, final Set blacklist) { + private TableMonitorThread newTableMonitorThread(final Set whitelist, final Set blacklist, + final boolean qualifiedTableNames) { return new TableMonitorThread( - dialect, - connectionProvider, - context, - POLL_INTERVAL, - whitelist, - blacklist, - qualifiedTableNames + dialect, + connectionProvider, + context, + POLL_INTERVAL, + whitelist, + blacklist, + qualifiedTableNames ); } diff --git a/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingCriteriaTest.java b/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingCriteriaTest.java index ad66e6df..64fbdcea 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingCriteriaTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingCriteriaTest.java @@ -33,10 +33,11 @@ import io.aiven.connect.jdbc.util.ColumnId; import io.aiven.connect.jdbc.util.TableId; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; public class TimestampIncrementingCriteriaTest { @@ -54,7 +55,7 @@ public class TimestampIncrementingCriteriaTest { private Struct record; private TimeZone utcTimeZone = TimeZone.getTimeZone(ZoneOffset.UTC); - @Before + @BeforeEach public void beforeEach() { criteria = new TimestampIncrementingCriteria(null, null, utcTimeZone); criteriaInc = new TimestampIncrementingCriteria(INCREMENTING_COLUMN, null, utcTimeZone); @@ -101,22 +102,24 @@ record = new Struct(schema).put("id", new BigDecimal(42)); assertExtractedOffset(42L, schema, record); } - @Test(expected = ConnectException.class) + @Test public void extractTooLargeDecimalOffset() { final Schema decimalSchema = Decimal.schema(0); schema = SchemaBuilder.struct().field("id", decimalSchema).build(); record = new Struct(schema).put( "id", new BigDecimal(Long.MAX_VALUE).add(new BigDecimal(1))); - assertExtractedOffset(42L, schema, record); + assertThatThrownBy(() -> assertExtractedOffset(42L, schema, record)) + .isInstanceOf(ConnectException.class); } - @Test(expected = ConnectException.class) + @Test public void extractFractionalDecimalOffset() { final Schema decimalSchema = Decimal.schema(2); schema = SchemaBuilder.struct().field("id", decimalSchema).build(); record = new Struct(schema).put("id", new BigDecimal("42.42")); - assertExtractedOffset(42L, schema, record); + assertThatThrownBy(() -> assertExtractedOffset(42L, schema, record)) + .isInstanceOf(ConnectException.class); } @Test diff --git a/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingOffsetTest.java b/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingOffsetTest.java index d2199c00..69843e22 100644 --- a/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingOffsetTest.java +++ b/src/test/java/io/aiven/connect/jdbc/source/TimestampIncrementingOffsetTest.java @@ -19,8 +19,8 @@ import java.sql.Timestamp; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; @@ -34,7 +34,7 @@ public class TimestampIncrementingOffsetTest { private Timestamp nanos; private TimestampIncrementingOffset nanosOffset; - @Before + @BeforeEach public void setUp() { final long millis = System.currentTimeMillis(); nanos = new Timestamp(millis); @@ -70,7 +70,6 @@ public void testGetIncrementingOffset() { @Test public void testGetTimestampOffset() { assertThat(unset.getTimestampOffset()).isNull(); - final Timestamp zero = new Timestamp(0); assertThat(tsOnly.getTimestampOffset()).isEqualTo(ts); assertThat(tsOnly.getTimestampOffset()).isEqualTo(ts); assertThat(incOnly.getTimestampOffset()).isNull(); diff --git a/src/test/java/io/aiven/connect/jdbc/util/CachedConnectionProviderTest.java b/src/test/java/io/aiven/connect/jdbc/util/CachedConnectionProviderTest.java index e3824a68..72749c47 100644 --- a/src/test/java/io/aiven/connect/jdbc/util/CachedConnectionProviderTest.java +++ b/src/test/java/io/aiven/connect/jdbc/util/CachedConnectionProviderTest.java @@ -22,10 +22,10 @@ import org.apache.kafka.connect.errors.ConnectException; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; +import org.mockito.junit.jupiter.MockitoExtension; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -34,7 +34,7 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class CachedConnectionProviderTest { @Mock diff --git a/src/test/java/io/aiven/connect/jdbc/util/IdentifierRulesTest.java b/src/test/java/io/aiven/connect/jdbc/util/IdentifierRulesTest.java index 78f03c8f..790daa33 100644 --- a/src/test/java/io/aiven/connect/jdbc/util/IdentifierRulesTest.java +++ b/src/test/java/io/aiven/connect/jdbc/util/IdentifierRulesTest.java @@ -17,10 +17,8 @@ package io.aiven.connect.jdbc.util; -import java.util.List; - -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -28,9 +26,8 @@ public class IdentifierRulesTest { private IdentifierRules rules; - private List parts; - @Before + @BeforeEach public void beforeEach() { rules = IdentifierRules.DEFAULT; } diff --git a/src/test/java/io/aiven/connect/jdbc/util/TimeZoneValidatorTest.java b/src/test/java/io/aiven/connect/jdbc/util/TimeZoneValidatorTest.java index b2d4fee3..fac61d98 100644 --- a/src/test/java/io/aiven/connect/jdbc/util/TimeZoneValidatorTest.java +++ b/src/test/java/io/aiven/connect/jdbc/util/TimeZoneValidatorTest.java @@ -21,7 +21,7 @@ import org.apache.kafka.common.config.ConfigException; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThatThrownBy;