From 4dbf96513b9d2491ea0c8a6ca359da6f42b519c3 Mon Sep 17 00:00:00 2001 From: Stephane Geneix Date: Fri, 26 Apr 2024 13:12:04 -0700 Subject: [PATCH] remove log4j from java cdk --- .../azure/AzureBlobStorageStreamCopier.kt | 85 ++++++------- .../AbstractJdbcCompatibleSourceOperations.kt | 16 +-- .../airbyte/cdk/db/jdbc/DateTimeConverter.kt | 33 ++--- .../cdk/db/jdbc/DefaultJdbcDatabase.kt | 24 ++-- .../cdk/db/jdbc/JdbcSourceOperations.kt | 22 ++-- .../cdk/db/jdbc/StreamingJdbcDatabase.kt | 11 +- .../streaming/AdaptiveStreamingQueryConfig.kt | 14 +-- .../jdbc/streaming/TwoStageSizeEstimator.kt | 18 +-- .../cdk/db/util/SSLCertificateUtils.kt | 8 +- .../base/AirbyteExceptionHandler.kt | 11 +- .../CommitOnStateAirbyteMessageConsumer.kt | 9 +- .../cdk/integrations/base/Destination.kt | 9 +- .../integrations/base/DestinationConfig.kt | 13 +- .../FailureTrackingAirbyteMessageConsumer.kt | 17 +-- .../integrations/base/IntegrationCliParser.kt | 7 +- .../integrations/base/IntegrationRunner.kt | 49 ++++---- .../adaptive/AdaptiveDestinationRunner.kt | 13 +- .../base/adaptive/AdaptiveSourceRunner.kt | 13 +- .../cdk/integrations/base/ssh/SshTunnel.kt | 23 ++-- .../base/ssh/SshWrappedDestination.kt | 24 ++-- .../integrations/base/ssh/SshWrappedSource.kt | 23 ++-- .../BufferedStreamConsumer.kt | 27 ++-- .../destination/jdbc/SqlOperations.kt | 6 +- .../normalization/SentryExceptionHelper.kt | 9 +- .../record_buffer/BaseSerializedBuffer.kt | 18 ++- .../destination/record_buffer/FileBuffer.kt | 8 +- .../record_buffer/InMemoryBuffer.kt | 5 +- .../InMemoryRecordBufferingStrategy.kt | 30 ++--- .../SerializedBufferingStrategy.kt | 54 +++----- .../util/ConnectorExceptionUtil.kt | 7 +- .../concurrent/ConcurrentStreamConsumer.kt | 34 ++---- .../base/IntegrationRunnerTest.kt | 6 +- .../LoggingInvocationInterceptor.kt | 6 +- .../airbyte/cdk/testutils/ContainerFactory.kt | 10 +- .../io/airbyte/cdk/testutils/TestDatabase.kt | 5 +- .../cdk/db/bigquery/BigQueryDatabase.kt | 15 +-- .../db/bigquery/BigQuerySourceOperations.kt | 15 ++- .../airbyte/cdk/db/mongodb/MongoDatabase.kt | 19 ++- .../io/airbyte/cdk/db/mongodb/MongoUtils.kt | 22 ++-- .../jdbc/AbstractJdbcDestination.kt | 10 +- .../jdbc/JdbcBufferedConsumerFactory.kt | 26 ++-- .../jdbc/copy/CopyConsumerFactory.kt | 19 ++- .../destination/jdbc/copy/CopyDestination.kt | 17 ++- .../jdbc/copy/SwitchingDestination.kt | 14 +-- .../typing_deduping/JdbcDestinationHandler.kt | 85 +++++-------- .../staging/SerialStagingConsumerFactory.kt | 8 +- .../jdbc/copy/SwitchingDestinationTest.kt | 8 +- .../destination/DestinationAcceptanceTest.kt | 7 +- .../DataTypeTestArgumentProvider.kt | 7 +- .../comparator/AdvancedTestDataComparator.kt | 6 +- .../comparator/BasicTestDataComparator.kt | 9 +- .../debezium/AirbyteDebeziumHandler.kt | 23 ++-- .../AirbyteFileOffsetBackingStore.kt | 8 +- .../internals/AirbyteSchemaHistoryStorage.kt | 32 ++--- .../internals/DebeziumConverterUtils.kt | 15 ++- .../internals/DebeziumMessageProducer.kt | 15 ++- .../internals/DebeziumRecordIterator.kt | 32 +++-- .../internals/DebeziumRecordPublisher.kt | 25 ++-- .../internals/DebeziumShutdownProcedure.kt | 14 +-- .../debezium/internals/RecordWaitTimeUtil.kt | 28 +++-- .../source/jdbc/AbstractJdbcSource.kt | 115 ++++++------------ .../source/jdbc/JdbcSSLConnectionUtils.kt | 41 +++---- .../integrations/source/jdbc/JdbcSource.kt | 10 +- .../source/relationaldb/AbstractDbSource.kt | 21 ++-- .../source/relationaldb/CdcStateManager.kt | 11 +- .../relationaldb/DbSourceDiscoverUtil.kt | 17 ++- .../relationaldb/RelationalDbQueryUtils.kt | 13 +- .../relationaldb/StateDecoratingIterator.kt | 41 +++---- .../relationaldb/state/CursorManager.kt | 50 +++----- .../state/CursorStateMessageProducer.kt | 13 +- .../relationaldb/state/LegacyStateManager.kt | 8 +- .../relationaldb/state/SourceStateIterator.kt | 6 +- .../relationaldb/state/StateGeneratorUtils.kt | 7 +- .../source/relationaldb/state/StateManager.kt | 15 +-- .../relationaldb/state/StateManagerFactory.kt | 35 +++--- .../relationaldb/state/StreamStateManager.kt | 19 ++- .../jdbc/DefaultJdbcSourceAcceptanceTest.kt | 29 ++--- .../source/jdbc/DefaultJdbcStressTest.kt | 10 +- .../source/jdbc/JdbcSourceStressTest.kt | 10 +- .../integrations/debezium/CdcSourceTest.kt | 6 +- .../source/jdbc/test/JdbcStressTest.kt | 6 +- .../source/AbstractSourceConnectorTest.kt | 4 - .../source/AbstractSourceDatabaseTypeTest.kt | 10 +- .../source/PythonSourceAcceptanceTest.kt | 7 +- .../source/SourceAcceptanceTest.kt | 7 +- .../standardtest/source/TestEnvConfigs.kt | 6 +- .../standardtest/source/fs/TestSourceMain.kt | 3 - .../AbstractSourceFillDbWithTestData.kt | 10 +- .../AbstractSourcePerformanceTest.kt | 11 +- .../airbyte-cdk/dependencies/build.gradle | 2 +- .../features/EnvVariableFeatureFlags.kt | 8 +- .../io/airbyte/commons/io/LineGobbler.kt | 19 ++- .../io/airbyte/commons/json/JsonPaths.kt | 7 +- .../kotlin/io/airbyte/commons/json/Jsons.kt | 8 +- .../commons/lang/CloseableShutdownHook.kt | 7 +- .../io/airbyte/commons/lang/Exceptions.kt | 9 +- .../commons/stream/StreamStatusUtils.kt | 13 +- .../airbyte/commons/util/CompositeIterator.kt | 10 +- .../kotlin/io/airbyte/configoss/EnvConfigs.kt | 16 ++- .../validation/json/JsonSchemaValidator.kt | 15 ++- .../airbyte/workers/TestHarnessUtilsTest.kt | 12 +- .../DefaultAirbyteStreamFactoryTest.kt | 10 +- .../io/airbyte/workers/TestHarnessUtils.kt | 17 ++- .../general/DbtTransformationRunner.kt | 7 +- .../DefaultCheckConnectionTestHarness.kt | 10 +- .../DefaultDiscoverCatalogTestHarness.kt | 7 +- .../general/DefaultGetSpecTestHarness.kt | 9 +- .../workers/helper/ConnectorConfigUpdater.kt | 8 +- .../internal/DefaultAirbyteDestination.kt | 18 ++- .../workers/internal/DefaultAirbyteSource.kt | 6 +- .../internal/DefaultAirbyteStreamFactory.kt | 16 +-- .../DefaultNormalizationRunner.kt | 7 +- .../NormalizationAirbyteStreamFactory.kt | 12 +- .../workers/process/DockerProcessFactory.kt | 7 +- .../destination/gcs/BaseGcsDestination.kt | 17 ++- .../destination/gcs/GcsStorageOperations.kt | 11 +- .../destination/gcs/avro/GcsAvroWriter.kt | 20 ++- .../destination/gcs/csv/GcsCsvWriter.kt | 18 ++- .../destination/gcs/jsonl/GcsJsonlWriter.kt | 15 +-- .../gcs/parquet/GcsParquetWriter.kt | 24 ++-- .../destination/gcs/util/GcsUtils.kt | 9 +- .../destination/gcs/writer/BaseGcsWriter.kt | 38 +++--- .../jdbc/copy/gcs/GcsStreamCopier.kt | 67 ++++------ .../gcs/GcsDestinationAcceptanceTest.kt | 7 +- .../destination/s3/BaseS3Destination.kt | 11 +- .../destination/s3/S3BaseChecks.kt | 22 ++-- .../destination/s3/S3ConsumerFactory.kt | 50 ++++---- .../destination/s3/S3DestinationConfig.kt | 8 +- .../s3/UploadFormatConfigFactory.kt | 8 +- .../destination/s3/avro/S3AvroWriter.kt | 18 ++- .../destination/s3/csv/S3CsvWriter.kt | 18 ++- .../destination/s3/jsonl/S3JsonlWriter.kt | 15 +-- .../destination/s3/parquet/S3ParquetWriter.kt | 8 +- .../destination/s3/util/JavaProcessRunner.kt | 12 +- .../destination/s3/writer/BaseS3Writer.kt | 36 +++--- .../s3/writer/ProductionWriterFactory.kt | 13 +- .../staging/StagingConsumerFactory.kt | 8 +- .../s3/S3DestinationAcceptanceTest.kt | 7 +- .../typing_deduping/AirbyteProtocolType.kt | 10 +- .../typing_deduping/AirbyteType.kt | 9 +- .../BaseDestinationV1V2Migrator.kt | 48 +++----- .../typing_deduping/CatalogParser.kt | 26 ++-- .../typing_deduping/DefaultTyperDeduper.kt | 99 ++++++--------- .../typing_deduping/TyperDeduperUtil.kt | 7 +- .../base/destination/typing_deduping/Union.kt | 13 +- .../BaseSqlGeneratorIntegrationTest.kt | 13 +- .../typing_deduping/BaseTypingDedupingTest.kt | 5 +- 147 files changed, 1103 insertions(+), 1534 deletions(-) diff --git a/airbyte-cdk/java/airbyte-cdk/azure-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/azure/AzureBlobStorageStreamCopier.kt b/airbyte-cdk/java/airbyte-cdk/azure-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/azure/AzureBlobStorageStreamCopier.kt index fc11a57fcfaa..4f7c158d6181 100644 --- a/airbyte-cdk/java/airbyte-cdk/azure-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/azure/AzureBlobStorageStreamCopier.kt +++ b/airbyte-cdk/java/airbyte-cdk/azure-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/azure/AzureBlobStorageStreamCopier.kt @@ -17,6 +17,7 @@ import io.airbyte.cdk.integrations.destination.jdbc.copy.StreamCopier import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.DestinationSyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.nio.charset.StandardCharsets import java.sql.SQLException @@ -26,8 +27,8 @@ import java.util.* import java.util.function.Consumer import org.apache.commons.csv.CSVFormat import org.apache.commons.csv.CSVPrinter -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class AzureBlobStorageStreamCopier( protected val stagingFolder: String, @@ -104,38 +105,32 @@ abstract class AzureBlobStorageStreamCopier( @Throws(Exception::class) override fun closeStagingUploader(hasFailed: Boolean) { - LOGGER.info("Uploading remaining data for {} stream.", streamName) + LOGGER.info { "Uploading remaining data for $streamName stream." } for (csvPrinter in csvPrinters.values) { csvPrinter.close() } - LOGGER.info("All data for {} stream uploaded.", streamName) + LOGGER.info { "All data for $streamName stream uploaded." } } @Throws(Exception::class) override fun createDestinationSchema() { - LOGGER.info("Creating schema in destination if it doesn't exist: {}", schemaName) + LOGGER.info { "Creating schema in destination if it doesn't exist: $schemaName" } sqlOperations.createSchemaIfNotExists(db, schemaName) } @Throws(Exception::class) override fun createTemporaryTable() { - LOGGER.info( - "Preparing tmp table in destination for stream: {}, schema: {}, tmp table name: {}.", - streamName, - schemaName, - tmpTableName - ) + LOGGER.info { + "Preparing tmp table in destination for stream: $streamName, schema: $schemaName, tmp table name: $tmpTableName." + } sqlOperations.createTableIfNotExists(db, schemaName, tmpTableName) } @Throws(Exception::class) override fun copyStagingFileToTemporaryTable() { - LOGGER.info( - "Starting copy to tmp table: {} in destination for stream: {}, schema: {}.", - tmpTableName, - streamName, - schemaName - ) + LOGGER.info { + "Starting copy to tmp table: $tmpTableName in destination for stream: $streamName, schema: $schemaName." + } for (azureStagingFile in azureStagingFiles) { copyAzureBlobCsvFileIntoTable( db, @@ -145,11 +140,9 @@ abstract class AzureBlobStorageStreamCopier( azureBlobConfig ) } - LOGGER.info( - "Copy to tmp table {} in destination for stream {} complete.", - tmpTableName, - streamName - ) + LOGGER.info { + "Copy to tmp table $tmpTableName in destination for stream $streamName complete." + } } private fun getFullAzurePath(azureStagingFile: String?): String { @@ -166,29 +159,24 @@ abstract class AzureBlobStorageStreamCopier( @Throws(Exception::class) override fun createDestinationTable(): String? { @Suppress("DEPRECATION") val destTableName = nameTransformer.getRawTableName(streamName) - LOGGER.info("Preparing table {} in destination.", destTableName) + LOGGER.info { "Preparing table $destTableName in destination." } sqlOperations.createTableIfNotExists(db, schemaName, destTableName) - LOGGER.info("Table {} in destination prepared.", tmpTableName) + LOGGER.info { "Table $tmpTableName in destination prepared." } return destTableName } @Throws(Exception::class) override fun generateMergeStatement(destTableName: String?): String { - LOGGER.info( - "Preparing to merge tmp table {} to dest table: {}, schema: {}, in destination.", - tmpTableName, - destTableName, - schemaName - ) + LOGGER.info { + "Preparing to merge tmp table $tmpTableName to dest table: $destTableName, schema: $schemaName, in destination." + } val queries = StringBuilder() if (destSyncMode == DestinationSyncMode.OVERWRITE) { queries.append(sqlOperations.truncateTableQuery(db, schemaName, destTableName)) - LOGGER.info( - "Destination OVERWRITE mode detected. Dest table: {}, schema: {}, truncated.", - destTableName, - schemaName - ) + LOGGER.info { + "Destination OVERWRITE mode detected. Dest table: $destTableName, schema: $schemaName, truncated." + } } queries.append(sqlOperations.insertTableQuery(db, schemaName, tmpTableName, destTableName)) return queries.toString() @@ -196,20 +184,20 @@ abstract class AzureBlobStorageStreamCopier( @Throws(Exception::class) override fun removeFileAndDropTmpTable() { - LOGGER.info("Begin cleaning azure blob staging files.") + LOGGER.info { "Begin cleaning azure blob staging files." } for (appendBlobClient in blobClients.values) { appendBlobClient.delete() } - LOGGER.info("Azure Blob staging files cleaned.") + LOGGER.info { "Azure Blob staging files cleaned." } - LOGGER.info("Begin cleaning {} tmp table in destination.", tmpTableName) + LOGGER.info { "Begin cleaning $tmpTableName tmp table in destination." } sqlOperations.dropTableIfExists(db, schemaName, tmpTableName) - LOGGER.info("{} tmp table in destination cleaned.", tmpTableName) + LOGGER.info { "$tmpTableName tmp table in destination cleaned." } } @Throws(Exception::class) override fun closeNonCurrentStagingFileWriters() { - LOGGER.info("Begin closing non current file writers") + LOGGER.info { "Begin closing non current file writers" } val removedKeys: MutableSet = HashSet() for (key in activeStagingWriterFileNames) { if (key != currentFile) { @@ -231,8 +219,7 @@ abstract class AzureBlobStorageStreamCopier( ) companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(AzureBlobStorageStreamCopier::class.java) + fun attemptAzureBlobWriteAndDelete(config: AzureBlobStorageConfig) { var appendBlobClient: AppendBlobClient? = null try { @@ -249,7 +236,7 @@ abstract class AzureBlobStorageStreamCopier( listCreatedBlob(containerClient) } finally { if (appendBlobClient != null && appendBlobClient.exists()) { - LOGGER.info("Deleting blob: " + appendBlobClient.blobName) + LOGGER.info { "Deleting blob: ${appendBlobClient.blobName}" } appendBlobClient.delete() } } @@ -260,16 +247,14 @@ abstract class AzureBlobStorageStreamCopier( .listBlobs() .forEach( Consumer { blobItem: BlobItem -> - LOGGER.info( - "Blob name: " + blobItem.name + "Snapshot: " + blobItem.snapshot - ) + LOGGER.info { "Blob name: ${blobItem.name} Snapshot: ${blobItem.snapshot}" } } ) } private fun writeTestDataIntoBlob(appendBlobClient: AppendBlobClient?) { val test = "test_data" - LOGGER.info("Writing test data to Azure Blob storage: $test") + LOGGER.info { "Writing test data to Azure Blob storage: $test" } val dataStream: InputStream = ByteArrayInputStream(test.toByteArray(StandardCharsets.UTF_8)) @@ -278,7 +263,7 @@ abstract class AzureBlobStorageStreamCopier( .appendBlock(dataStream, test.length.toLong()) .blobCommittedBlockCount - LOGGER.info("blobCommittedBlockCount: $blobCommittedBlockCount") + LOGGER.info { "blobCommittedBlockCount: $blobCommittedBlockCount" } } private fun getBlobContainerClient( @@ -291,9 +276,9 @@ abstract class AzureBlobStorageStreamCopier( if (!appendBlobClient.exists()) { appendBlobClient.create() - LOGGER.info("blobContainerClient created") + LOGGER.info { "blobContainerClient created" } } else { - LOGGER.info("blobContainerClient already exists") + LOGGER.info { "blobContainerClient already exists" } } return containerClient } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/AbstractJdbcCompatibleSourceOperations.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/AbstractJdbcCompatibleSourceOperations.kt index 49696e37ef2f..5ee46b5cf3d4 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/AbstractJdbcCompatibleSourceOperations.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/AbstractJdbcCompatibleSourceOperations.kt @@ -13,6 +13,7 @@ import io.airbyte.cdk.integrations.base.AirbyteTraceMessageUtility import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteRecordMessageMeta import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange +import io.github.oshai.kotlinlogging.KotlinLogging import java.math.BigDecimal import java.sql.* import java.sql.Date @@ -21,16 +22,12 @@ import java.time.* import java.time.chrono.IsoEra import java.time.format.DateTimeParseException import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Source operation skeleton for JDBC compatible databases. */ abstract class AbstractJdbcCompatibleSourceOperations : JdbcCompatibleSourceOperations { - private val LOGGER: Logger = - LoggerFactory.getLogger(AbstractJdbcCompatibleSourceOperations::class.java) - @Throws(SQLException::class) override fun convertDatabaseRowToAirbyteRecordData(queryContext: ResultSet): AirbyteRecordData { // the first call communicates with the database. after that the result is cached. @@ -47,12 +44,9 @@ abstract class AbstractJdbcCompatibleSourceOperations : copyToJsonField(queryContext, i, jsonNode) } catch (e: java.lang.Exception) { jsonNode.putNull(columnName) - LOGGER.info( - "Failed to serialize column: {}, of type {}, with error {}", - columnName, - columnTypeName, - e.message - ) + LOGGER.info { + "Failed to serialize column: $columnName, of type $columnTypeName, with error ${e.message}" + } AirbyteTraceMessageUtility.emitAnalyticsTrace(dataTypesSerializationErrorMessage()) metaChanges.add( AirbyteRecordMessageMetaChange() diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DateTimeConverter.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DateTimeConverter.kt index 16d03d8c04f0..61f32b968a9d 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DateTimeConverter.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DateTimeConverter.kt @@ -5,17 +5,18 @@ package io.airbyte.cdk.db.jdbc import com.fasterxml.jackson.databind.node.ObjectNode import io.airbyte.cdk.db.DataTypeUtils +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.* import java.time.* import java.time.format.DateTimeFormatter import java.util.concurrent.* import kotlin.math.abs import kotlin.math.min -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object DateTimeConverter { - private val LOGGER: Logger = LoggerFactory.getLogger(DateTimeConverter::class.java) + val TIME_WITH_TIMEZONE_FORMATTER: DateTimeFormatter = DateTimeFormatter.ofPattern( "HH:mm:ss[.][SSSSSSSSS][SSSSSSS][SSSSSS][SSSSS][SSSS][SSS][SS][S][''][XXX][XX][X]" @@ -34,7 +35,7 @@ object DateTimeConverter { else time.toString() } else { if (!loggedUnknownTimeWithTimeZoneClass) { - LOGGER.info("Unknown class for Time with timezone data type" + time.javaClass) + LOGGER.info { "Unknown class for Time with timezone data type ${time.javaClass}" } loggedUnknownTimeWithTimeZoneClass = true } val timetz = OffsetTime.parse(time.toString(), TIME_WITH_TIMEZONE_FORMATTER) @@ -78,9 +79,9 @@ object DateTimeConverter { return AbstractJdbcCompatibleSourceOperations.Companion.resolveEra(localDate, value) } else { if (!loggedUnknownTimestampWithTimeZoneClass) { - LOGGER.info( - "Unknown class for Timestamp with time zone data type" + timestamp.javaClass - ) + LOGGER.info { + "Unknown class for Timestamp with time zone data type ${timestamp.javaClass}" + } loggedUnknownTimestampWithTimeZoneClass = true } val instant = Instant.parse(timestamp.toString()) @@ -123,7 +124,7 @@ object DateTimeConverter { ) } else { if (!loggedUnknownTimestampClass) { - LOGGER.info("Unknown class for Timestamp data type" + timestamp.javaClass) + LOGGER.info { "Unknown class for Timestamp data type ${timestamp.javaClass}" } loggedUnknownTimestampClass = true } val localDateTime = LocalDateTime.parse(timestamp.toString()) @@ -158,7 +159,7 @@ object DateTimeConverter { return LocalDate.ofEpochDay(date.toLong()).format(DataTypeUtils.DATE_FORMATTER) } else { if (!loggedUnknownDateClass) { - LOGGER.info("Unknown class for Date data type" + date.javaClass) + LOGGER.info { "Unknown class for Date data type${date.javaClass}" } loggedUnknownDateClass = true } val localDate = LocalDate.parse(date.toString()) @@ -182,22 +183,22 @@ object DateTimeConverter { } else { val updatedValue = min(abs(value.toDouble()), LocalTime.MAX.toNanoOfDay().toDouble()).toLong() - LOGGER.debug( - "Time values must use number of nanoseconds greater than 0 and less than 86400000000000 but its {}, converting to {} ", - value, - updatedValue - ) + LOGGER.debug { + "Time values must use number of nanoseconds greater than 0 and less than 86400000000000 but its $value, converting to $updatedValue " + } return formatTime(LocalTime.ofNanoOfDay(updatedValue)) } } else { if (!loggedUnknownTimeClass) { - LOGGER.info("Unknown class for Time data type" + time.javaClass) + LOGGER.info { "Unknown class for Time data type ${time.javaClass}" } loggedUnknownTimeClass = true } val valueAsString = time.toString() if (valueAsString.startsWith("24")) { - LOGGER.debug("Time value {} is above range, converting to 23:59:59", valueAsString) + LOGGER.debug { + "Time value ${valueAsString} is above range, converting to 23:59:59" + } return LocalTime.MAX.toString() } return formatTime(LocalTime.parse(valueAsString)) diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DefaultJdbcDatabase.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DefaultJdbcDatabase.kt index d5b01105c5d0..407f399d646c 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DefaultJdbcDatabase.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/DefaultJdbcDatabase.kt @@ -8,14 +8,14 @@ import io.airbyte.cdk.db.JdbcCompatibleSourceOperations import io.airbyte.commons.exceptions.ConnectionErrorException import io.airbyte.commons.functional.CheckedConsumer import io.airbyte.commons.functional.CheckedFunction +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.* import java.util.* import java.util.function.Function import java.util.stream.Stream import javax.sql.DataSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Database object for interacting with a JDBC connection. Can be used for any JDBC compliant db. */ @@ -50,15 +50,13 @@ constructor( ): Stream { val connection = dataSource.connection return JdbcDatabase.Companion.toUnsafeStream(query.apply(connection), recordTransform) - .onClose( - Runnable { - try { - connection.close() - } catch (e: SQLException) { - throw RuntimeException(e) - } + .onClose { + try { + connection.close() + } catch (e: SQLException) { + throw RuntimeException(e) } - ) + } } @get:Throws(SQLException::class) @@ -125,7 +123,7 @@ constructor( .onClose( Runnable { try { - LOGGER.info("closing connection") + LOGGER.info { "closing connection" } connection.close() } catch (e: SQLException) { throw RuntimeException(e) @@ -133,8 +131,4 @@ constructor( } ) } - - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DefaultJdbcDatabase::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/JdbcSourceOperations.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/JdbcSourceOperations.kt index 5d68630b7105..4f56226cc452 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/JdbcSourceOperations.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/JdbcSourceOperations.kt @@ -7,13 +7,13 @@ import com.fasterxml.jackson.databind.JsonNode import com.fasterxml.jackson.databind.node.ObjectNode import io.airbyte.cdk.db.SourceOperations import io.airbyte.protocol.models.JsonSchemaType +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.* import java.time.OffsetDateTime import java.time.OffsetTime import java.time.format.DateTimeParseException -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Implementation of source operations with standard JDBC types. */ open class JdbcSourceOperations : AbstractJdbcCompatibleSourceOperations(), SourceOperations { @@ -130,15 +130,11 @@ open class JdbcSourceOperations : try { return JDBCType.valueOf(field[JdbcConstants.INTERNAL_COLUMN_TYPE].asInt()) } catch (ex: IllegalArgumentException) { - LOGGER.warn( - String.format( - "Could not convert column: %s from table: %s.%s with type: %s. Casting to VARCHAR.", - field[JdbcConstants.INTERNAL_COLUMN_NAME], - field[JdbcConstants.INTERNAL_SCHEMA_NAME], - field[JdbcConstants.INTERNAL_TABLE_NAME], - field[JdbcConstants.INTERNAL_COLUMN_TYPE] - ) - ) + LOGGER.warn { + "Could not convert column: ${field[JdbcConstants.INTERNAL_COLUMN_NAME]} from table: " + + "${field[JdbcConstants.INTERNAL_SCHEMA_NAME]}.${field[JdbcConstants.INTERNAL_TABLE_NAME]} " + + "with type: ${field[JdbcConstants.INTERNAL_COLUMN_TYPE]}. Casting to VARCHAR." + } return JDBCType.VARCHAR } } @@ -177,7 +173,5 @@ open class JdbcSourceOperations : } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(JdbcSourceOperations::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/StreamingJdbcDatabase.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/StreamingJdbcDatabase.kt index 9d826fbdebb6..8caf72590850 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/StreamingJdbcDatabase.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/StreamingJdbcDatabase.kt @@ -7,6 +7,7 @@ import com.google.errorprone.annotations.MustBeClosed import io.airbyte.cdk.db.JdbcCompatibleSourceOperations import io.airbyte.cdk.db.jdbc.streaming.JdbcStreamingQueryConfig import io.airbyte.commons.functional.CheckedFunction +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.* import java.util.Spliterators.AbstractSpliterator import java.util.function.Consumer @@ -14,8 +15,8 @@ import java.util.function.Supplier import java.util.stream.Stream import java.util.stream.StreamSupport import javax.sql.DataSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** * This database allows a developer to specify a [JdbcStreamingQueryConfig]. This allows the @@ -95,7 +96,7 @@ class StreamingJdbcDatabase( action.accept(dataRow) return true } catch (e: SQLException) { - LOGGER.error("SQLState: {}, Message: {}", e.sqlState, e.message) + LOGGER.error { "SQLState: ${e.sqlState}, Message: ${e.message}" } streamException = e isStreamFailed = true throw RuntimeException(e) @@ -105,8 +106,4 @@ class StreamingJdbcDatabase( false ) } - - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(StreamingJdbcDatabase::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/AdaptiveStreamingQueryConfig.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/AdaptiveStreamingQueryConfig.kt index 82d07922727e..f4163376ca6b 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/AdaptiveStreamingQueryConfig.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/AdaptiveStreamingQueryConfig.kt @@ -3,9 +3,10 @@ */ package io.airbyte.cdk.db.jdbc.streaming +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} open class AdaptiveStreamingQueryConfig : JdbcStreamingQueryConfig { private val fetchSizeEstimator: FetchSizeEstimator = TwoStageSizeEstimator.Companion.instance @@ -20,7 +21,7 @@ open class AdaptiveStreamingQueryConfig : JdbcStreamingQueryConfig { connection.autoCommit = false statement.fetchSize = FetchSizeConstants.INITIAL_SAMPLE_SIZE currentFetchSize = FetchSizeConstants.INITIAL_SAMPLE_SIZE - LOGGER.info("Set initial fetch size: {} rows", statement.fetchSize) + LOGGER.info { "Set initial fetch size: ${statement.fetchSize} rows" } } @Throws(SQLException::class) @@ -29,14 +30,9 @@ open class AdaptiveStreamingQueryConfig : JdbcStreamingQueryConfig { val newFetchSize = fetchSizeEstimator.fetchSize if (newFetchSize.isPresent && currentFetchSize != newFetchSize.get()) { - LOGGER.info("Set new fetch size: {} rows", newFetchSize.get()) + LOGGER.info { "Set new fetch size: ${newFetchSize.get()} rows" } resultSet.fetchSize = newFetchSize.get() currentFetchSize = newFetchSize.get() } } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(AdaptiveStreamingQueryConfig::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/TwoStageSizeEstimator.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/TwoStageSizeEstimator.kt index 16d485861809..37c8220e750d 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/TwoStageSizeEstimator.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/streaming/TwoStageSizeEstimator.kt @@ -4,11 +4,11 @@ package io.airbyte.cdk.db.jdbc.streaming import com.google.common.annotations.VisibleForTesting +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import kotlin.math.max -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This estimator first uses the [InitialSizeEstimator] to calculate an initial fetch size by * sampling the first N rows consecutively, and then switches to [SamplingSizeEstimator] to @@ -57,7 +57,6 @@ class TwoStageSizeEstimator private constructor() : FetchSizeEstimator { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(TwoStageSizeEstimator::class.java) val instance: TwoStageSizeEstimator get() = TwoStageSizeEstimator() @@ -65,10 +64,9 @@ class TwoStageSizeEstimator private constructor() : FetchSizeEstimator { @VisibleForTesting fun getTargetBufferByteSize(maxMemory: Long?): Long { if (maxMemory == null || maxMemory == Long.MAX_VALUE) { - LOGGER.info( - "No max memory limit found, use min JDBC buffer size: {}", - FetchSizeConstants.MIN_BUFFER_BYTE_SIZE - ) + LOGGER.info { + "No max memory limit found, use min JDBC buffer size: ${FetchSizeConstants.MIN_BUFFER_BYTE_SIZE}" + } return FetchSizeConstants.MIN_BUFFER_BYTE_SIZE } val targetBufferByteSize = @@ -79,11 +77,7 @@ class TwoStageSizeEstimator private constructor() : FetchSizeEstimator { targetBufferByteSize.toDouble() ) .toLong() - LOGGER.info( - "Max memory limit: {}, JDBC buffer size: {}", - maxMemory, - finalBufferByteSize - ) + LOGGER.info { "Max memory limit: $maxMemory, JDBC buffer size: $finalBufferByteSize" } return finalBufferByteSize } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/util/SSLCertificateUtils.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/util/SSLCertificateUtils.kt index d97741dd76a0..0c8ef8aaa3c9 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/util/SSLCertificateUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/util/SSLCertificateUtils.kt @@ -3,6 +3,7 @@ */ package io.airbyte.cdk.db.util +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.net.URI import java.nio.charset.StandardCharsets @@ -19,15 +20,14 @@ import java.util.* import java.util.concurrent.* import javax.net.ssl.SSLContext import org.apache.http.ssl.SSLContexts -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * General SSL utilities used for certificate and keystore operations related to secured db * connections. */ object SSLCertificateUtils { - private val LOGGER: Logger = LoggerFactory.getLogger(SSLCertificateUtils::class.java) + private const val PKCS_12 = "PKCS12" private const val X509 = "X.509" private val RANDOM: Random = SecureRandom() @@ -61,7 +61,7 @@ object SSLCertificateUtils { @Throws(IOException::class, InterruptedException::class) private fun runProcess(cmd: String, run: Runtime) { - LOGGER.debug("running [{}]", cmd) + LOGGER.debug { "running [$cmd]" } @Suppress("deprecation") val p = run.exec(cmd) if (!p.waitFor(30, TimeUnit.SECONDS)) { p.destroy() diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/AirbyteExceptionHandler.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/AirbyteExceptionHandler.kt index 62af52d91aad..5c6e4af804d8 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/AirbyteExceptionHandler.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/AirbyteExceptionHandler.kt @@ -5,12 +5,13 @@ package io.airbyte.cdk.integrations.base import com.fasterxml.jackson.databind.JsonNode import com.google.common.annotations.VisibleForTesting +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.regex.Pattern import javax.validation.constraints.NotNull import org.apache.commons.lang3.exception.ExceptionUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class AirbyteExceptionHandler : Thread.UncaughtExceptionHandler { override fun uncaughtException(thread: Thread, throwable: Throwable) { @@ -27,7 +28,7 @@ class AirbyteExceptionHandler : Thread.UncaughtExceptionHandler { // from the spec: // https://docs.google.com/document/d/1ctrj3Yh_GjtQ93aND-WH3ocqGxsmxyC3jfiarrF6NY0/edit# try { - LOGGER.error(logMessage, throwable) + LOGGER.error(throwable) { logMessage } // Attempt to deinterpolate the error message before emitting a trace message val mangledMessage: String? // If any exception in the chain is of a deinterpolatable type, find it and @@ -74,7 +75,7 @@ class AirbyteExceptionHandler : Thread.UncaughtExceptionHandler { AirbyteTraceMessageUtility.emitCustomErrorTrace(throwable.message, mangledMessage) } } catch (t: Throwable) { - LOGGER.error("exception in the exception handler", t) + LOGGER.error(t) { "exception in the exception handler" } } finally { terminate() } @@ -87,7 +88,7 @@ class AirbyteExceptionHandler : Thread.UncaughtExceptionHandler { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(AirbyteExceptionHandler::class.java) + const val logMessage: String = "Something went wrong in the connector. See the logs for more details." diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/CommitOnStateAirbyteMessageConsumer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/CommitOnStateAirbyteMessageConsumer.kt index bd789e689f78..2ea383a75b7f 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/CommitOnStateAirbyteMessageConsumer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/CommitOnStateAirbyteMessageConsumer.kt @@ -4,10 +4,10 @@ package io.airbyte.cdk.integrations.base import io.airbyte.protocol.models.v0.AirbyteMessage +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Minimal abstract class intended to handle the case where the destination can commit records every * time a state message appears. This class does that commit and then immediately emits the state @@ -26,9 +26,4 @@ abstract class CommitOnStateAirbyteMessageConsumer( } @Throws(Exception::class) abstract fun commit() - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(CommitOnStateAirbyteMessageConsumer::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/Destination.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/Destination.kt index 8c4e243bda31..cde7a75724db 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/Destination.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/Destination.kt @@ -10,9 +10,10 @@ import com.google.common.annotations.VisibleForTesting import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} interface Destination : Integration { /** @@ -104,8 +105,6 @@ interface Destination : Integration { } companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(ShimToSerializedAirbyteMessageConsumer::class.java) /** * Consumes an [AirbyteMessage] for processing. * @@ -126,7 +125,7 @@ interface Destination : Integration { consumer!!.accept(messageOptional.get()) } else { check(!isStateMessage(inputString)) { "Invalid state message: $inputString" } - LOGGER.error("Received invalid message: $inputString") + LOGGER.error { "Received invalid message: $inputString" } } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/DestinationConfig.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/DestinationConfig.kt index cdfbfc4f0373..666c807ed6e4 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/DestinationConfig.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/DestinationConfig.kt @@ -5,9 +5,9 @@ package io.airbyte.cdk.integrations.base import com.fasterxml.jackson.databind.JsonNode import com.google.common.annotations.VisibleForTesting -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import io.github.oshai.kotlinlogging.KotlinLogging +private val LOGGER = KotlinLogging.logger {} /** Singleton of destination config for easy lookup of values. */ class DestinationConfig private constructor() { // whether the destination fully supports Destinations V2 @@ -19,7 +19,7 @@ class DestinationConfig private constructor() { fun getNodeValue(key: String?): JsonNode? { val node = config!!.root!![key] if (node == null) { - LOGGER.debug("Cannot find node with key {} ", key) + LOGGER.debug { "Cannot find node with key $key" } } return node } @@ -28,7 +28,7 @@ class DestinationConfig private constructor() { fun getTextValue(key: String?): String { val node = getNodeValue(key) if (node == null || !node.isTextual) { - LOGGER.debug("Cannot retrieve text value for node with key {}", key) + LOGGER.debug { "Cannot retrieve text value for node with key $key" } return "" } return node.asText() @@ -38,14 +38,13 @@ class DestinationConfig private constructor() { fun getBooleanValue(key: String?): Boolean { val node = getNodeValue(key) if (node == null || !node.isBoolean) { - LOGGER.debug("Cannot retrieve boolean value for node with key {}", key) + LOGGER.debug { "Cannot retrieve boolean value for node with key $key" } return false } return node.asBoolean() } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DestinationConfig::class.java) private var config: DestinationConfig? = null @@ -62,7 +61,7 @@ class DestinationConfig private constructor() { config!!.root = root config!!.isV2Destination = isV2Destination } else { - LOGGER.warn("Singleton was already initialized.") + LOGGER.warn { "Singleton was already initialized." } } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/FailureTrackingAirbyteMessageConsumer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/FailureTrackingAirbyteMessageConsumer.kt index 7375360996ec..e275071ff9bc 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/FailureTrackingAirbyteMessageConsumer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/FailureTrackingAirbyteMessageConsumer.kt @@ -4,9 +4,9 @@ package io.airbyte.cdk.integrations.base import io.airbyte.protocol.models.v0.AirbyteMessage -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import io.github.oshai.kotlinlogging.KotlinLogging +private val LOGGER = KotlinLogging.logger {} /** * Minimal abstract class intended to provide a consistent structure to classes seeking to implement * the [AirbyteMessageConsumer] interface. The original interface methods are wrapped in generic @@ -36,7 +36,7 @@ abstract class FailureTrackingAirbyteMessageConsumer : AirbyteMessageConsumer { try { startTracked() } catch (e: Exception) { - LOGGER.error("Exception while starting consumer", e) + LOGGER.error(e) { "Exception while starting consumer" } hasFailed = true throw e } @@ -58,7 +58,7 @@ abstract class FailureTrackingAirbyteMessageConsumer : AirbyteMessageConsumer { try { acceptTracked(message) } catch (e: Exception) { - LOGGER.error("Exception while accepting message", e) + LOGGER.error(e) { "Exception while accepting message" } hasFailed = true throw e } @@ -69,15 +69,10 @@ abstract class FailureTrackingAirbyteMessageConsumer : AirbyteMessageConsumer { @Throws(Exception::class) override fun close() { if (hasFailed) { - LOGGER.warn("Airbyte message consumer: failed.") + LOGGER.warn { "Airbyte message consumer: failed." } } else { - LOGGER.info("Airbyte message consumer: succeeded.") + LOGGER.info { "Airbyte message consumer: succeeded." } } close(hasFailed) } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(FailureTrackingAirbyteMessageConsumer::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationCliParser.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationCliParser.kt index cec1564d076e..8e125326767f 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationCliParser.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationCliParser.kt @@ -5,14 +5,14 @@ package io.airbyte.cdk.integrations.base import com.google.common.base.Preconditions import io.airbyte.commons.cli.Clis +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.util.* import org.apache.commons.cli.Option import org.apache.commons.cli.OptionGroup import org.apache.commons.cli.Options -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} // todo (cgardens) - use argparse4j.github.io instead of org.apache.commons.cli to leverage better // sub-parser support. /** Parses command line args to a type safe config object for each command type. */ @@ -23,7 +23,6 @@ class IntegrationCliParser { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(IntegrationCliParser::class.java) private val COMMAND_GROUP: OptionGroup @@ -142,7 +141,7 @@ class IntegrationCliParser { for (option in parsed.options) { argsMap[option.longOpt] = option.value } - LOGGER.info("integration args: {}", argsMap) + LOGGER.info { "integration args: $argsMap" } return when (command) { Command.SPEC -> { diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunner.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunner.kt index 41fc8651f727..7a4a7ade3325 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunner.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunner.kt @@ -24,6 +24,7 @@ import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.validation.json.JsonSchemaValidator +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.lang.reflect.Method import java.nio.charset.StandardCharsets @@ -34,9 +35,8 @@ import java.util.concurrent.* import java.util.function.Consumer import org.apache.commons.lang3.ThreadUtils import org.apache.commons.lang3.concurrent.BasicThreadFactory -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Accepts EITHER a destination or a source. Routes commands from the commandline to the appropriate * methods on the integration. Keeps itself DRY for methods that are common between source and @@ -120,10 +120,10 @@ internal constructor( } @Throws(Exception::class) - private fun runInternal(parsed: IntegrationConfig?) { - LOGGER.info("Running integration: {}", integration.javaClass.name) - LOGGER.info("Command: {}", parsed!!.command) - LOGGER.info("Integration config: {}", parsed) + private fun runInternal(parsed: IntegrationConfig) { + LOGGER.info { "Running integration: ${integration.javaClass.name}" } + LOGGER.info { "Command: ${parsed.command}" } + LOGGER.info { "Integration config: $parsed" } try { when (parsed.command) { @@ -178,7 +178,7 @@ internal constructor( parsed.getStatePath().map { path: Path -> parseConfig(path) } try { if (featureFlags.concurrentSourceStreamRead()) { - LOGGER.info("Concurrent source stream read enabled.") + LOGGER.info { "Concurrent source stream read enabled." } readConcurrent(config, catalog, stateOptional) } else { readSerial(config, catalog, stateOptional) @@ -264,7 +264,7 @@ internal constructor( throw e } - LOGGER.info("Completed integration: {}", integration.javaClass.name) + LOGGER.info { "Completed integration: ${integration.javaClass.name}" } } private fun produceMessages( @@ -272,11 +272,11 @@ internal constructor( recordCollector: Consumer ) { messageIterator.airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("Producing messages for stream {}...", s) + LOGGER.debug { "Producing messages for stream $s..." } } messageIterator.forEachRemaining(recordCollector) messageIterator.airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("Finished producing messages for stream {}...", s) + LOGGER.debug { "Finished producing messages for stream $s..." } } } @@ -316,7 +316,7 @@ internal constructor( } } } catch (e: Exception) { - LOGGER.error("Unable to perform concurrent read.", e) + LOGGER.error(e) { "Unable to perform concurrent read." } throw e } finally { stopOrphanedThreads() @@ -353,7 +353,7 @@ internal constructor( produceMessages(stream, streamStatusTrackingRecordConsumer) } catch (e: Exception) { stream.airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.error("Failed to consume from stream {}.", s, e) + LOGGER.error(e) { "Failed to consume from stream $s." } } throw RuntimeException(e) } @@ -368,7 +368,6 @@ internal constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(IntegrationRunner::class.java) private val threadCreationInfo: InheritableThreadLocal = object : InheritableThreadLocal() { override fun childValue(parentValue: ThreadCreationInfo): ThreadCreationInfo { @@ -420,7 +419,7 @@ internal constructor( consumer: SerializedAirbyteMessageConsumer, inputStream: InputStream = System.`in` ) { - LOGGER.info("Starting buffered read of input stream") + LOGGER.info { "Starting buffered read of input stream" } consumer.start() inputStream.bufferedReader(StandardCharsets.UTF_8).use { var emptyLines = 0 @@ -434,15 +433,15 @@ internal constructor( // TODO: Monitor the logs for occurrences of this log line and if this isn't // an issue, remove it. if (emptyLines % 1_000 == 0 && emptyLines < 10_000) { - LOGGER.warn("Encountered $emptyLines empty lines during execution") + LOGGER.warn { "Encountered $emptyLines empty lines during execution" } } } } if (emptyLines > 0) { - LOGGER.warn("Encountered $emptyLines empty lines in the input stream.") + LOGGER.warn { "Encountered $emptyLines empty lines in the input stream." } } } - LOGGER.info("Finished buffered read of input stream") + LOGGER.info { "Finished buffered read of input stream" } } @JvmStatic @@ -483,15 +482,15 @@ internal constructor( val runningThreads = ThreadUtils.getAllThreads().filter(::filterOrphanedThread) if (runningThreads.isNotEmpty()) { - LOGGER.warn( + LOGGER.warn { """ The main thread is exiting while children non-daemon threads from a connector are still active. Ideally, this situation should not happen... Please check with maintainers if the connector or library code should safely clean up its threads before quitting instead. - The main thread is: {} - """.trimIndent(), - dumpThread(currentThread) - ) + The main thread is: ${dumpThread(currentThread)} + """.trimIndent() + } + val scheduledExecutorService = Executors.newSingleThreadScheduledExecutor( BasicThreadFactory @@ -506,7 +505,7 @@ internal constructor( "Active non-daemon thread: " + dumpThread(runningThread) + "\ncreationStack=${getThreadCreationInfo(runningThread)}" - LOGGER.warn(str) + LOGGER.warn { str } // even though the main thread is already shutting down, we still leave some // chances to the children // threads to close properly on their own. @@ -524,9 +523,9 @@ internal constructor( !runningThread.isDaemon && runningThread.name != currentThread.name } ) { - LOGGER.error( + LOGGER.error { "Failed to interrupt children non-daemon threads, forcefully exiting NOW...\n" - ) + } exitHook.run() } }, diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveDestinationRunner.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveDestinationRunner.kt index 40c9797bf2af..568d2d2fd014 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveDestinationRunner.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveDestinationRunner.kt @@ -6,16 +6,15 @@ package io.airbyte.cdk.integrations.base.adaptive import io.airbyte.cdk.integrations.base.Destination import io.airbyte.cdk.integrations.base.IntegrationRunner import io.airbyte.commons.features.EnvVariableFeatureFlags +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class launches different variants of a destination connector based on where Airbyte is * deployed. */ object AdaptiveDestinationRunner { - private val LOGGER: Logger = LoggerFactory.getLogger(AdaptiveDestinationRunner::class.java) private const val DEPLOYMENT_MODE_KEY = EnvVariableFeatureFlags.DEPLOYMENT_MODE private const val CLOUD_MODE = "CLOUD" @@ -52,12 +51,12 @@ object AdaptiveDestinationRunner { ) { private val destination: Destination get() { - LOGGER.info("Running destination under deployment mode: {}", deploymentMode) + LOGGER.info { "Running destination under deployment mode: $deploymentMode" } if (deploymentMode != null && deploymentMode == CLOUD_MODE) { return cloudDestinationSupplier.get() } if (deploymentMode == null) { - LOGGER.warn("Deployment mode is null, default to OSS mode") + LOGGER.warn { "Deployment mode is null, default to OSS mode" } } return ossDestinationSupplier.get() } @@ -65,9 +64,9 @@ object AdaptiveDestinationRunner { @Throws(Exception::class) fun run(args: Array) { val destination = destination - LOGGER.info("Starting destination: {}", destination.javaClass.name) + LOGGER.info { "Starting destination: ${destination.javaClass.name}" } IntegrationRunner(destination).run(args) - LOGGER.info("Completed destination: {}", destination.javaClass.name) + LOGGER.info { "Completed destination: ${destination.javaClass.name}" } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveSourceRunner.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveSourceRunner.kt index 73514b4a0a08..5ad4cb4d6760 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveSourceRunner.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/adaptive/AdaptiveSourceRunner.kt @@ -6,15 +6,14 @@ package io.airbyte.cdk.integrations.base.adaptive import io.airbyte.cdk.integrations.base.IntegrationRunner import io.airbyte.cdk.integrations.base.Source import io.airbyte.commons.features.EnvVariableFeatureFlags +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class launches different variants of a source connector based on where Airbyte is deployed. */ object AdaptiveSourceRunner { - private val LOGGER: Logger = LoggerFactory.getLogger(AdaptiveSourceRunner::class.java) const val DEPLOYMENT_MODE_KEY: String = EnvVariableFeatureFlags.DEPLOYMENT_MODE const val CLOUD_MODE: String = "CLOUD" @@ -46,12 +45,12 @@ object AdaptiveSourceRunner { ) { private val source: Source get() { - LOGGER.info("Running source under deployment mode: {}", deploymentMode) + LOGGER.info { "Running source under deployment mode: $deploymentMode" } if (deploymentMode != null && deploymentMode == CLOUD_MODE) { return cloudSourceSupplier.get() } if (deploymentMode == null) { - LOGGER.warn("Deployment mode is null, default to OSS mode") + LOGGER.warn { "Deployment mode is null, default to OSS mode" } } return ossSourceSupplier.get() } @@ -59,9 +58,9 @@ object AdaptiveSourceRunner { @Throws(Exception::class) fun run(args: Array) { val source = source - LOGGER.info("Starting source: {}", source.javaClass.name) + LOGGER.info { "Starting source: ${source.javaClass.name}" } IntegrationRunner(source).run(args) - LOGGER.info("Completed source: {}", source.javaClass.name) + LOGGER.info { "Completed source: ${source.javaClass.name}" } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshTunnel.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshTunnel.kt index 6d6d6f891620..c6383ae6b551 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshTunnel.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshTunnel.kt @@ -11,6 +11,7 @@ import io.airbyte.commons.functional.CheckedConsumer import io.airbyte.commons.functional.CheckedFunction import io.airbyte.commons.json.Jsons import io.airbyte.commons.string.Strings +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.net.InetSocketAddress import java.net.MalformedURLException @@ -30,9 +31,8 @@ import org.apache.sshd.common.util.security.SecurityUtils import org.apache.sshd.core.CoreModuleProperties import org.apache.sshd.server.forward.AcceptAllForwardingFilter import org.bouncycastle.jce.provider.BouncyCastleProvider -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} // todo (cgardens) - this needs unit tests. it is currently tested transitively via source postgres // integration tests. /** @@ -296,7 +296,7 @@ constructor( globalHeartbeatInterval: Duration, idleTimeout: Duration ): SshClient { - LOGGER.info("Creating SSH client with Heartbeat and Keepalive enabled") + LOGGER.info { "Creating SSH client with Heartbeat and Keepalive enabled" } val client = createClient() // Session level heartbeat using SSH_MSG_IGNORE every second. client.setSessionHeartbeat( @@ -351,14 +351,9 @@ constructor( // try to connect tunnelLocalPort = address.port - LOGGER.info( - String.format( - "Established tunneling session to %s:%d. Port forwarding started on %s ", - remoteServiceHost, - remoteServicePort, - address.toInetSocketAddress() - ) - ) + LOGGER.info { + "Established tunneling session to $remoteServiceHost:$remoteServicePort. Port forwarding started on ${address.toInetSocketAddress()} " + } return session } catch (e: IOException) { if ( @@ -403,7 +398,7 @@ constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(SshTunnel::class.java) + const val SSH_TIMEOUT_DISPLAY_MESSAGE: String = "Timed out while opening a SSH Tunnel. Please double check the given SSH configurations and try again." @@ -425,7 +420,7 @@ constructor( TunnelMethod.valueOf(method.asText().trim { it <= ' ' }) } .orElse(TunnelMethod.NO_TUNNEL) - LOGGER.info("Starting connection with method: {}", tunnelMethod) + LOGGER.info { "Starting connection with method: $tunnelMethod" } return SshTunnel( config, @@ -489,7 +484,7 @@ constructor( TunnelMethod.valueOf(method.asText().trim { it <= ' ' }) } .orElse(TunnelMethod.NO_TUNNEL) - LOGGER.info("Starting connection with method: {}", tunnelMethod) + LOGGER.info { "Starting connection with method: $tunnelMethod" } return SshTunnel( config, diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedDestination.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedDestination.kt index 24f2340384be..a346098f6852 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedDestination.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedDestination.kt @@ -16,11 +16,11 @@ import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConnectorSpecification +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Decorates a Destination with an SSH Tunnel using the standard configuration that Airbyte uses for * configuring SSH. @@ -87,10 +87,9 @@ class SshWrappedDestination : Destination { delegateConsumer = delegate.getConsumer(tunnel.configInTunnel, catalog, outputRecordCollector) } catch (e: Exception) { - LOGGER.error( - "Exception occurred while getting the delegate consumer, closing SSH tunnel", - e - ) + LOGGER.error(e) { + "Exception occurred while getting the delegate consumer, closing SSH tunnel" + } tunnel.close() throw e } @@ -110,7 +109,7 @@ class SshWrappedDestination : Destination { val connectionOptionsConfig: Optional = Jsons.getOptional(clone, SshTunnel.Companion.CONNECTION_OPTIONS_KEY) if (connectionOptionsConfig.isEmpty) { - LOGGER.info("No SSH connection options found, using defaults") + LOGGER.info { "No SSH connection options found, using defaults" } if (clone is ObjectNode) { // Defensive check, it will always be object node val connectionOptions = clone.putObject(SshTunnel.Companion.CONNECTION_OPTIONS_KEY) connectionOptions.put( @@ -133,10 +132,9 @@ class SshWrappedDestination : Destination { outputRecordCollector ) } catch (e: Exception) { - LOGGER.error( - "Exception occurred while getting the delegate consumer, closing SSH tunnel", - e - ) + LOGGER.error(e) { + "Exception occurred while getting the delegate consumer, closing SSH tunnel" + } tunnel.close() throw e } @@ -155,7 +153,5 @@ class SshWrappedDestination : Destination { override val isV2Destination: Boolean get() = delegate.isV2Destination - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(SshWrappedDestination::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedSource.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedSource.kt index a9800fa955fb..30dcd2b36095 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedSource.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/ssh/SshWrappedSource.kt @@ -9,9 +9,10 @@ import io.airbyte.cdk.integrations.base.Source import io.airbyte.commons.util.AutoCloseableIterator import io.airbyte.commons.util.AutoCloseableIterators import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class SshWrappedSource : Source { private val delegate: Source @@ -68,10 +69,9 @@ class SshWrappedSource : Source { try { delegateRead = delegate.read(tunnel.configInTunnel, catalog, state) } catch (e: Exception) { - LOGGER.error( - "Exception occurred while getting the delegate read iterator, closing SSH tunnel", - e - ) + LOGGER.error(e) { + "Exception occurred while getting the delegate read iterator, closing SSH tunnel" + } tunnel.close() throw e } @@ -88,16 +88,13 @@ class SshWrappedSource : Source { try { return delegate.readStreams(tunnel.configInTunnel, catalog, state) } catch (e: Exception) { - LOGGER.error( - "Exception occurred while getting the delegate read stream iterators, closing SSH tunnel", - e - ) + LOGGER.error(e) { + "Exception occurred while getting the delegate read stream iterators, closing SSH tunnel" + } tunnel.close() throw e } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(SshWrappedSource::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.kt index 4a7021eec42b..ccf15b902710 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumer.kt @@ -18,12 +18,12 @@ import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Duration import java.time.Instant import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class consumes AirbyteMessages from the worker. * @@ -135,7 +135,7 @@ internal constructor( hasStarted = true nextFlushDeadline = Instant.now().plus(bufferFlushFrequency) streamToIgnoredRecordCount.clear() - LOGGER.info("{} started.", BufferedStreamConsumer::class.java) + LOGGER.info { "${BufferedStreamConsumer::class.java} started." } onStart.call() } @@ -188,7 +188,7 @@ internal constructor( } else if (msg.type == AirbyteMessage.Type.STATE) { stateManager.addState(msg) } else { - LOGGER.warn("Unexpected message: " + msg.type) + LOGGER.warn { "Unexpected message: ${msg.type}" } } periodicBufferFlush() } @@ -222,12 +222,12 @@ internal constructor( // When the last time the buffered has been flushed exceed the frequency, flush the current // buffer before receiving incoming AirbyteMessage if (Instant.now().isAfter(nextFlushDeadline)) { - LOGGER.info("Periodic buffer flush started") + LOGGER.info { "Periodic buffer flush started" } try { bufferingStrategy.flushAllBuffers() markStatesAsFlushedToDestination() } catch (e: Exception) { - LOGGER.error("Periodic buffer flush failed", e) + LOGGER.error(e) { "Periodic buffer flush failed" } throw e } } @@ -250,16 +250,14 @@ internal constructor( streamToIgnoredRecordCount.forEach { (pair: AirbyteStreamNameNamespacePair?, count: Long?) -> - LOGGER.warn( - "A total of {} record(s) of data from stream {} were invalid and were ignored.", - count, - pair - ) + LOGGER.warn { + "A total of $count record(s) of data from stream $pair were invalid and were ignored." + } } if (hasFailed) { - LOGGER.error("executing on failed close procedure.") + LOGGER.error { "executing on failed close procedure." } } else { - LOGGER.info("executing on success close procedure.") + LOGGER.info { "executing on success close procedure." } // When flushing the buffer, this will call the respective #flushBufferFunction which // bundles // the flush and commit operation, so if successful then mark state as committed @@ -293,13 +291,12 @@ internal constructor( stateManager.listCommitted()!!.forEach(outputRecordCollector) } catch (e: Exception) { - LOGGER.error("Close failed.", e) + LOGGER.error(e) { "Close failed." } throw e } } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BufferedStreamConsumer::class.java) private fun throwUnrecognizedStream( catalog: ConfiguredAirbyteCatalog?, diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/SqlOperations.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/SqlOperations.kt index 7c67b240acaa..77a028ff31f7 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/SqlOperations.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/SqlOperations.kt @@ -6,8 +6,6 @@ package io.airbyte.cdk.integrations.destination.jdbc import com.fasterxml.jackson.databind.JsonNode import io.airbyte.cdk.db.jdbc.JdbcDatabase import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage -import org.slf4j.Logger -import org.slf4j.LoggerFactory /** * SQL queries required for successfully syncing to a destination connector. These operations @@ -139,7 +137,5 @@ interface SqlOperations { */ val isSchemaRequired: Boolean - companion object { - val LOGGER: Logger = LoggerFactory.getLogger(SqlOperations::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/normalization/SentryExceptionHelper.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/normalization/SentryExceptionHelper.kt index c43b0589060f..af78632ee365 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/normalization/SentryExceptionHelper.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/normalization/SentryExceptionHelper.kt @@ -3,16 +3,15 @@ */ package io.airbyte.cdk.integrations.destination.normalization +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This is copied out of platform * (https://github.com/airbytehq/airbyte-platform/blob/main/airbyte-persistence/job-persistence/src/main/java/io/airbyte/persistence/job/errorreporter/SentryExceptionHelper.java#L257) */ object SentryExceptionHelper { - private val LOGGER: Logger = LoggerFactory.getLogger(SentryExceptionHelper::class.java) fun getUsefulErrorMessageAndTypeFromDbtError(stacktrace: String): Map { // the dbt 'stacktrace' is really just all the log messages at 'error' level, stuck @@ -149,9 +148,9 @@ object SentryExceptionHelper { } catch (e: ArrayIndexOutOfBoundsException) { // this means our logic is slightly off, our assumption of where error lines are is // incorrect - LOGGER.warn( + LOGGER.warn { "Failed trying to parse useful error message out of dbt error, defaulting to full stacktrace" - ) + } } } if (errorMessageAndType.isEmpty()) { diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/BaseSerializedBuffer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/BaseSerializedBuffer.kt index 7aa6932b5697..a837a6114e7f 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/BaseSerializedBuffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/BaseSerializedBuffer.kt @@ -5,12 +5,12 @@ package io.airbyte.cdk.integrations.destination.record_buffer import com.google.common.io.CountingOutputStream import io.airbyte.protocol.models.v0.AirbyteRecordMessage +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream import org.apache.commons.io.FileUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Base implementation of a [SerializableBuffer]. It is composed of a [BufferStorage] where the * actual data is being stored in a serialized format. @@ -132,7 +132,7 @@ protected constructor(private val bufferStorage: BufferStorage) : SerializableBu get() { if (useCompression && !bufferStorage.filename.endsWith(GZ_SUFFIX)) { if (bufferStorage.file.renameTo(File(bufferStorage.filename + GZ_SUFFIX))) { - LOGGER.info("Renaming compressed file to include .gz file extension") + LOGGER.info { "Renaming compressed file to include .gz file extension" } } } return bufferStorage.file @@ -147,17 +147,15 @@ protected constructor(private val bufferStorage: BufferStorage) : SerializableBu override fun flush() { if (inputStream == null && !isClosed) { flushWriter() - LOGGER.debug("Wrapping up compression and write GZIP trailer data.") + LOGGER.debug { "Wrapping up compression and write GZIP trailer data." } compressedBuffer?.flush() compressedBuffer?.close() closeWriter() bufferStorage.close() inputStream = convertToInputStream() - LOGGER.info( - "Finished writing data to {} ({})", - filename, - FileUtils.byteCountToDisplaySize(byteCounter.count) - ) + LOGGER.info { + "Finished writing data to $filename (${FileUtils.byteCountToDisplaySize(byteCounter.count)})" + } } } @@ -182,7 +180,7 @@ protected constructor(private val bufferStorage: BufferStorage) : SerializableBu override val maxConcurrentStreamsInBuffer: Int = bufferStorage.maxConcurrentStreamsInBuffer companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BaseSerializedBuffer::class.java) + private const val GZ_SUFFIX = ".gz" } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/FileBuffer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/FileBuffer.kt index 6b3d21fc0b72..468c495c97c7 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/FileBuffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/FileBuffer.kt @@ -3,11 +3,12 @@ */ package io.airbyte.cdk.integrations.destination.record_buffer +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.nio.file.Files import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class FileBuffer : BufferStorage { private val fileExtension: String @@ -71,12 +72,11 @@ class FileBuffer : BufferStorage { @Throws(IOException::class) override fun deleteFile() { - LOGGER.info("Deleting tempFile data {}", filename) + LOGGER.info { "Deleting tempFile data $filename" } Files.deleteIfExists(file.toPath()) } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(FileBuffer::class.java) /* * We limit number of stream being buffered simultaneously anyway (limit how many files are diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryBuffer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryBuffer.kt index 05bf5c932cca..1d05c25e3ec4 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryBuffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryBuffer.kt @@ -3,11 +3,12 @@ */ package io.airbyte.cdk.integrations.destination.record_buffer -import io.airbyte.cdk.integrations.destination.jdbc.SqlOperations.Companion.LOGGER +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.nio.file.Files import java.util.* +private val LOGGER = KotlinLogging.logger {} /** * Instead of storing buffered data on disk like the [FileBuffer], this [BufferStorage] accumulates * message data in-memory instead. Thus, a bigger heap size would be required. @@ -63,7 +64,7 @@ class InMemoryBuffer(private val fileExtension: String) : BufferStorage { override fun deleteFile() { var pathToDelete = tempFile?.toPath() if (pathToDelete != null) { - LOGGER.info("Deleting tempFile data {}", filename) + LOGGER.info { "Deleting tempFile data $filename" } Files.deleteIfExists(pathToDelete) } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.kt index 92ed3b5fb71e..4b5e59b7feb8 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/InMemoryRecordBufferingStrategy.kt @@ -9,11 +9,11 @@ import io.airbyte.cdk.integrations.destination.buffered_stream_consumer.RecordWr import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import org.apache.commons.io.FileUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This is the default implementation of a [BufferStorage] to be backward compatible. Data is being * buffered in a [<] as they are being consumed. @@ -66,29 +66,24 @@ class InMemoryRecordBufferingStrategy( stream: AirbyteStreamNameNamespacePair, buffer: SerializableBuffer ) { - LOGGER.info( - "Flushing single stream {}: {} records", - stream.name, - streamBuffer[stream]!!.size - ) + LOGGER.info { + "Flushing single stream ${stream.name}: ${streamBuffer[stream]!!.size} records" + } recordWriter.accept(stream, streamBuffer[stream]!!) - LOGGER.info("Flushing completed for {}", stream.name) + LOGGER.info { "Flushing completed for ${stream.name}" } } @Throws(Exception::class) override fun flushAllBuffers() { for ((key, value) in streamBuffer) { - LOGGER.info( - "Flushing {}: {} records ({})", - key.name, - value.size, - FileUtils.byteCountToDisplaySize(bufferSizeInBytes) - ) + LOGGER.info { + "Flushing ${key.name}: ${value.size} records (${FileUtils.byteCountToDisplaySize(bufferSizeInBytes)})" + } recordWriter.accept(key, value) if (checkAndRemoveRecordWriter != null) { fileName = checkAndRemoveRecordWriter.apply(key, fileName) } - LOGGER.info("Flushing completed for {}", key.name) + LOGGER.info { "Flushing completed for ${key.name}" } } close() clear() @@ -100,9 +95,4 @@ class InMemoryRecordBufferingStrategy( } @Throws(Exception::class) override fun close() {} - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(InMemoryRecordBufferingStrategy::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/SerializedBufferingStrategy.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/SerializedBufferingStrategy.kt index 5249d5313f0c..2d50f34e21cc 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/SerializedBufferingStrategy.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/record_buffer/SerializedBufferingStrategy.kt @@ -7,11 +7,11 @@ import io.airbyte.cdk.integrations.util.ConnectorExceptionUtil import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import org.apache.commons.io.FileUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Buffering Strategy used to convert [io.airbyte.protocol.models.AirbyteRecordMessage] into a * stream of bytes to more readily save and transmit information @@ -90,16 +90,13 @@ class SerializedBufferingStrategy */ private fun getOrCreateBuffer(stream: AirbyteStreamNameNamespacePair): SerializableBuffer { return allBuffers.computeIfAbsent(stream) { _: AirbyteStreamNameNamespacePair -> - LOGGER.info( - "Starting a new buffer for stream {} (current state: {} in {} buffers)", - stream.name, - FileUtils.byteCountToDisplaySize(totalBufferSizeInBytes), - allBuffers.size - ) + LOGGER.info { + "Starting a new buffer for stream ${stream.name} (current state: ${FileUtils.byteCountToDisplaySize(totalBufferSizeInBytes)} in ${allBuffers.size} buffers)" + } try { return@computeIfAbsent onCreateBuffer.apply(stream, catalog)!! } catch (e: Exception) { - LOGGER.error("Failed to create a new buffer for stream {}", stream.name, e) + LOGGER.error(e) { "Failed to create a new buffer for stream ${stream.name}" } throw RuntimeException(e) } } @@ -110,32 +107,26 @@ class SerializedBufferingStrategy stream: AirbyteStreamNameNamespacePair, buffer: SerializableBuffer ) { - LOGGER.info( - "Flushing buffer of stream {} ({})", - stream.name, - FileUtils.byteCountToDisplaySize(buffer.byteCount) - ) + LOGGER.info { + "Flushing buffer of stream ${stream.name} (${FileUtils.byteCountToDisplaySize(buffer.byteCount)})" + } onStreamFlush.accept(stream, buffer) totalBufferSizeInBytes -= buffer.byteCount allBuffers.remove(stream) - LOGGER.info("Flushing completed for {}", stream.name) + LOGGER.info { "Flushing completed for ${stream.name}" } } @Throws(Exception::class) override fun flushAllBuffers() { - LOGGER.info( - "Flushing all {} current buffers ({} in total)", - allBuffers.size, - FileUtils.byteCountToDisplaySize(totalBufferSizeInBytes) - ) + LOGGER.info { + "Flushing all ${allBuffers.size} current buffers (${FileUtils.byteCountToDisplaySize(totalBufferSizeInBytes)} in total)" + } for ((stream, buffer) in allBuffers) { - LOGGER.info( - "Flushing buffer of stream {} ({})", - stream.name, - FileUtils.byteCountToDisplaySize(buffer.byteCount) - ) + LOGGER.info { + "Flushing buffer of stream ${stream.name} (${FileUtils.byteCountToDisplaySize(buffer.byteCount)})" + } onStreamFlush.accept(stream, buffer) - LOGGER.info("Flushing completed for {}", stream.name) + LOGGER.info { "Flushing completed for ${stream.name}" } } close() clear() @@ -144,7 +135,7 @@ class SerializedBufferingStrategy @Throws(Exception::class) override fun clear() { - LOGGER.debug("Reset all buffers") + LOGGER.debug { "Reset all buffers" } allBuffers = HashMap() } @@ -153,11 +144,11 @@ class SerializedBufferingStrategy val exceptionsThrown: MutableList = ArrayList() for ((stream, buffer) in allBuffers) { try { - LOGGER.info("Closing buffer for stream {}", stream.name) + LOGGER.info { "Closing buffer for stream ${stream.name}" } buffer.close() } catch (e: Exception) { exceptionsThrown.add(e) - LOGGER.error("Exception while closing stream buffer", e) + LOGGER.error(e) { "Exception while closing stream buffer" } } } @@ -166,9 +157,4 @@ class SerializedBufferingStrategy exceptionsThrown ) } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(SerializedBufferingStrategy::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/ConnectorExceptionUtil.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/ConnectorExceptionUtil.kt index 38f185f459dd..dba7ae8c4eef 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/ConnectorExceptionUtil.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/ConnectorExceptionUtil.kt @@ -9,16 +9,15 @@ import io.airbyte.commons.exceptions.ConfigErrorException import io.airbyte.commons.exceptions.ConnectionErrorException import io.airbyte.commons.exceptions.TransientErrorException import io.airbyte.commons.functional.Either +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.EOFException import java.sql.SQLException import java.sql.SQLSyntaxErrorException import org.apache.commons.lang3.exception.ExceptionUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Utility class defining methods for handling configuration exceptions in connectors. */ object ConnectorExceptionUtil { - private val LOGGER: Logger = LoggerFactory.getLogger(ConnectorExceptionUtil::class.java) const val COMMON_EXCEPTION_MESSAGE_TEMPLATE: String = "Could not connect with provided configuration. Error: %s" @@ -113,7 +112,7 @@ object ConnectorExceptionUtil { throwables.joinToString("\n") { throwable: Throwable -> ExceptionUtils.getStackTrace(throwable) } - LOGGER.error("$initialMessage$stacktraces\nRethrowing first exception.") + LOGGER.error { "$initialMessage$stacktraces\nRethrowing first exception." } throw throwables.iterator().next() } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumer.kt index 6c392c3cd132..082b94ffc0cf 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumer.kt @@ -9,14 +9,14 @@ import io.airbyte.commons.stream.StreamStatusUtils import io.airbyte.commons.util.AutoCloseableIterator import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.AirbyteMessage +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.concurrent.* import java.util.concurrent.ThreadPoolExecutor.AbortPolicy import java.util.function.Consumer import kotlin.math.min -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * [Consumer] implementation that consumes [AirbyteMessage] records from each provided stream * concurrently. @@ -88,9 +88,9 @@ class ConcurrentStreamConsumer( * Wait for the submitted streams to complete before returning. This uses the join() method to allow * all streams to complete even if one or more encounters an exception. */ - LOGGER.debug("Waiting for all streams to complete....") + LOGGER.debug { "Waiting for all streams to complete...." } CompletableFuture.allOf(*futures.toTypedArray>()).join() - LOGGER.debug("Completed consuming from all streams.") + LOGGER.debug { "Completed consuming from all streams." } } val exception: Optional @@ -101,7 +101,7 @@ class ConcurrentStreamConsumer( * captured during execution. */ get() = - if (!exceptions.isEmpty()) { + if (exceptions.isNotEmpty()) { Optional.of(exceptions[0]) } else { Optional.empty() @@ -136,18 +136,16 @@ class ConcurrentStreamConsumer( Optional.ofNullable(System.getenv("DEFAULT_CONCURRENT_STREAM_CONSUMER_THREADS")) .map { s: String -> s.toInt() } .orElseGet { Runtime.getRuntime().availableProcessors() } - LOGGER.debug( - "Default parallelism: {}, Requested parallelism: {}", - defaultPoolSize, - requestedParallelism - ) + LOGGER.debug { + "Default parallelism: $defaultPoolSize, Requested parallelism: $requestedParallelism" + } val parallelism = min( defaultPoolSize.toDouble(), (if (requestedParallelism > 0) requestedParallelism else 1).toDouble() ) .toInt() - LOGGER.debug("Computed concurrent stream consumer parallelism: {}", parallelism) + LOGGER.debug { "Computed concurrent stream consumer parallelism: $parallelism" } return parallelism } @@ -179,18 +177,18 @@ class ConcurrentStreamConsumer( try { stream.use { stream.airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("Consuming from stream {}...", s) + LOGGER.debug { "Consuming from stream $s..." } } StreamStatusUtils.emitStartStreamStatus(stream, streamStatusEmitter) streamConsumer.accept(stream) StreamStatusUtils.emitCompleteStreamStatus(stream, streamStatusEmitter) stream.airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("Consumption from stream {} complete.", s) + LOGGER.debug { "Consumption from stream $s complete." } } } } catch (e: Exception) { stream.airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.error("Unable to consume from stream {}.", s, e) + LOGGER.error(e) { "Unable to consume from stream $s." } } StreamStatusUtils.emitIncompleteStreamStatus(stream, streamStatusEmitter) exceptions.add(e) @@ -213,12 +211,7 @@ class ConcurrentStreamConsumer( if (stream.airbyteStream.isPresent) { val airbyteStream = stream.airbyteStream.get() thread.name = - String.format( - "%s-%s-%s", - CONCURRENT_STREAM_THREAD_NAME, - airbyteStream.namespace, - airbyteStream.name - ) + "$CONCURRENT_STREAM_THREAD_NAME-${airbyteStream.namespace}-${airbyteStream.name}" } else { thread.name = CONCURRENT_STREAM_THREAD_NAME } @@ -245,7 +238,6 @@ class ConcurrentStreamConsumer( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(ConcurrentStreamConsumer::class.java) /** Name of threads spawned by the [ConcurrentStreamConsumer]. */ const val CONCURRENT_STREAM_THREAD_NAME: String = "concurrent-stream-thread" diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunnerTest.kt b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunnerTest.kt index 30d2b646b9ae..5bf2809155c7 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunnerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunnerTest.kt @@ -14,6 +14,7 @@ import io.airbyte.commons.util.AutoCloseableIterators import io.airbyte.commons.util.MoreIterators import io.airbyte.protocol.models.v0.* import io.airbyte.validation.json.JsonSchemaValidator +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.net.URI import java.nio.charset.StandardCharsets @@ -32,8 +33,8 @@ import org.junit.jupiter.api.Test import org.mockito.Mockito import org.mockito.kotlin.any import org.mockito.kotlin.mock -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} internal class IntegrationRunnerTest { private lateinit var cliParser: IntegrationCliParser @@ -610,7 +611,6 @@ ${Jsons.serialize(message2)}""".toByteArray( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(IntegrationRunnerTest::class.java) private const val CONFIG_FILE_NAME = "config.json" private const val CONFIGURED_CATALOG_FILE_NAME = "configured_catalog.json" diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/extensions/LoggingInvocationInterceptor.kt b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/extensions/LoggingInvocationInterceptor.kt index effb0a3348c7..f15596af805d 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/extensions/LoggingInvocationInterceptor.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/extensions/LoggingInvocationInterceptor.kt @@ -3,6 +3,7 @@ */ package io.airbyte.cdk.extensions +import io.github.oshai.kotlinlogging.KotlinLogging import java.lang.reflect.* import java.time.Duration import java.time.Instant @@ -20,9 +21,8 @@ import org.junit.jupiter.api.extension.DynamicTestInvocationContext import org.junit.jupiter.api.extension.ExtensionContext import org.junit.jupiter.api.extension.InvocationInterceptor import org.junit.jupiter.api.extension.ReflectiveInvocationContext -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * By default, junit only output logs to the console, and nothing makes it into log4j logs. This * class fixes that by using the interceptor facility to print progress and timing information. This @@ -345,8 +345,6 @@ class LoggingInvocationInterceptor : InvocationInterceptor { } companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(LoggingInvocationInterceptor::class.java) private val JUNIT_METHOD_EXECUTION_TIMEOUT_PROPERTY_NAME: String = "JunitMethodExecutionTimeout" } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/ContainerFactory.kt b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/ContainerFactory.kt index 886785d5ff3b..127d3ca8dcc1 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/ContainerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/ContainerFactory.kt @@ -6,6 +6,9 @@ package io.airbyte.cdk.testutils import com.google.common.collect.Lists import io.airbyte.commons.logging.LoggingHelper import io.airbyte.commons.logging.MdcScope +import io.github.oshai.kotlinlogging.DelegatingKLogger +import io.github.oshai.kotlinlogging.KLogger +import io.github.oshai.kotlinlogging.KotlinLogging import java.lang.reflect.InvocationTargetException import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentMap @@ -14,12 +17,12 @@ import java.util.function.Consumer import java.util.function.Supplier import kotlin.concurrent.Volatile import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.testcontainers.containers.GenericContainer import org.testcontainers.containers.output.OutputFrame import org.testcontainers.containers.output.Slf4jLogConsumer import org.testcontainers.utility.DockerImageName +private val LOGGER: KLogger = KotlinLogging.logger {} /** * ContainerFactory is the companion to [TestDatabase] and provides it with suitable testcontainer * instances. @@ -191,8 +194,9 @@ abstract class ContainerFactory> { Lists.transform(namedContainerModifiers) { c: NamedContainerModifier -> c.name() } ) val container = createNewContainer(imageName) + @Suppress("unchecked_cast") val logConsumer: Slf4jLogConsumer = - object : Slf4jLogConsumer(LOGGER) { + object : Slf4jLogConsumer((LOGGER as DelegatingKLogger).underlyingLogger) { override fun accept(frame: OutputFrame) { if (frame.utf8StringWithoutLineEnding.trim { it <= ' ' }.isNotEmpty()) { super.accept(frame) @@ -219,8 +223,6 @@ abstract class ContainerFactory> { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(ContainerFactory::class.java) - private val SHARED_CONTAINERS: ConcurrentMap, ContainerOrException> = ConcurrentHashMap() private val containerId: AtomicInteger = AtomicInteger(0) diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/TestDatabase.kt b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/TestDatabase.kt index d386c720169b..2567b744947b 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/TestDatabase.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/testutils/TestDatabase.kt @@ -15,6 +15,7 @@ import io.airbyte.cdk.integrations.JdbcConnector import io.airbyte.cdk.integrations.util.HostPortResolver import io.airbyte.commons.json.Jsons import io.airbyte.commons.string.Strings +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.UncheckedIOException import java.sql.SQLException @@ -28,10 +29,9 @@ import javax.sql.DataSource import kotlin.concurrent.Volatile import org.jooq.DSLContext import org.jooq.SQLDialect -import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.testcontainers.containers.JdbcDatabaseContainer +private val LOGGER = KotlinLogging.logger {} /** * TestDatabase provides a convenient pattern for interacting with databases when testing SQL * database sources. The basic idea is to share the same database testcontainer instance for all @@ -307,7 +307,6 @@ protected constructor(val container: C) : AutoCloseable { } companion object { - private val LOGGER: Logger? = LoggerFactory.getLogger(TestDatabase::class.java) private val nextDatabaseId: AtomicInteger = AtomicInteger(0) diff --git a/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQueryDatabase.kt b/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQueryDatabase.kt index aeb4cb550a44..a85b1698adcb 100644 --- a/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQueryDatabase.kt +++ b/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQueryDatabase.kt @@ -11,6 +11,7 @@ import com.google.common.base.Charsets import com.google.common.collect.ImmutableMap import com.google.common.collect.Streams import io.airbyte.cdk.db.SqlDatabase +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.ByteArrayInputStream import java.io.IOException import java.sql.SQLException @@ -19,10 +20,10 @@ import java.util.function.Consumer import java.util.stream.Stream import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.tuple.ImmutablePair -import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.threeten.bp.Duration +private val LOGGER = KotlinLogging.logger {} + class BigQueryDatabase @JvmOverloads constructor( @@ -84,10 +85,10 @@ constructor( val result = executeQuery(bigQuery, getQueryConfig(sql, emptyList())) if (result.getLeft() == null) { throw SQLException( - "BigQuery request is failed with error: " + result.getRight() + ". SQL: " + sql + "BigQuery request is failed with error: ${result.getRight()}. SQL: ${sql}" ) } - LOGGER.info("BigQuery successfully finished execution SQL: $sql") + LOGGER.info { "BigQuery successfully finished execution SQL: $sql" } } @Throws(Exception::class) @@ -192,9 +193,9 @@ constructor( val success = bigQuery.delete(dataSetId, option) if (success) { - LOGGER.info("BQ Dataset $dataSetId deleted...") + LOGGER.info { "BQ Dataset $dataSetId deleted..." } } else { - LOGGER.info("BQ Dataset cleanup for $dataSetId failed!") + LOGGER.info { "BQ Dataset cleanup for $dataSetId failed!" } } } @@ -218,7 +219,7 @@ constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryDatabase::class.java) + private const val AGENT_TEMPLATE = "%s (GPN: Airbyte; staging)" } } diff --git a/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQuerySourceOperations.kt b/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQuerySourceOperations.kt index d2e04a06f509..49f9cfa0cf35 100644 --- a/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQuerySourceOperations.kt +++ b/airbyte-cdk/java/airbyte-cdk/datastore-bigquery/src/main/kotlin/io/airbyte/cdk/db/bigquery/BigQuerySourceOperations.kt @@ -23,13 +23,14 @@ import io.airbyte.cdk.db.util.JsonUtil.putLongValueIntoJson import io.airbyte.cdk.db.util.JsonUtil.putStringValueIntoJson import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.JsonSchemaType +import io.github.oshai.kotlinlogging.KotlinLogging import java.text.DateFormat import java.text.ParseException import java.text.SimpleDateFormat import java.util.* import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class BigQuerySourceOperations : SourceOperations { private val BIG_QUERY_DATE_FORMAT: DateFormat = SimpleDateFormat("yyyy-MM-dd") @@ -136,7 +137,7 @@ class BigQuerySourceOperations : SourceOperations value } } catch (e: Exception) { - LOGGER.error( - String.format("Failed to get BsonValue for field type %s", type), - e.message - ) + LOGGER.error(e) { "Failed to get BsonValue for field type $type" } return value } } @@ -148,7 +145,7 @@ object MongoUtils { readDocument(reader, objectNode, columnNames) } } catch (e: Exception) { - LOGGER.error("Exception while parsing BsonDocument: {}", e.message) + LOGGER.error { "Exception while parsing BsonDocument: ${e.message}" } throw RuntimeException(e) } } @@ -207,10 +204,9 @@ object MongoUtils { if (data.isTextual) data.asText() else data.toString() ) } else { - LOGGER.debug( - "WARNING Field list out of sync, Document doesn't contain field: {}", - fieldName - ) + LOGGER.debug { + "WARNING Field list out of sync, Document doesn't contain field: $fieldName" + } } } } @@ -453,7 +449,7 @@ object MongoUtils { // Override the default codec registry return document.toBsonDocument(BsonDocument::class.java, customCodecRegistry) } catch (e: Exception) { - LOGGER.error("Exception while converting Document to BsonDocument: {}", e.message) + LOGGER.error { "Exception while converting Document to BsonDocument: ${e.message}" } throw RuntimeException(e) } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/AbstractJdbcDestination.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/AbstractJdbcDestination.kt index f17d7f16b513..9e3b1dc174d9 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/AbstractJdbcDestination.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/AbstractJdbcDestination.kt @@ -38,6 +38,7 @@ import io.airbyte.integrations.base.destination.typing_deduping.migrators.Minimu import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.Connection import java.sql.ResultSet import java.sql.SQLException @@ -45,8 +46,8 @@ import java.util.* import java.util.function.Consumer import javax.sql.DataSource import org.apache.commons.lang3.NotImplementedException -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class AbstractJdbcDestination( driverClass: String, @@ -101,7 +102,7 @@ abstract class AbstractJdbcDestination = ArrayList() for (writeConfig in writeConfigs) { val schemaName = writeConfig.outputSchemaName val dstTableName = writeConfig.outputTableName - LOGGER.info( - "Preparing raw table in destination started for stream {}. schema: {}, table name: {}", - writeConfig.streamName, - schemaName, - dstTableName - ) + LOGGER.info { + "Preparing raw table in destination started for stream ${writeConfig.streamName}. schema: $schemaName, table name: $dstTableName" + } sqlOperations.createSchemaIfNotExists(database, schemaName) sqlOperations.createTableIfNotExists(database, schemaName, dstTableName) when (writeConfig.syncMode) { @@ -262,7 +256,7 @@ object JdbcBufferedConsumerFactory { } } sqlOperations.executeTransaction(database, queryList) - LOGGER.info("Preparing raw tables in destination completed.") + LOGGER.info { "Preparing raw tables in destination completed." } typerDeduper.prepareFinalTables() } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyConsumerFactory.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyConsumerFactory.kt index 2a50769c402a..f58aa2543638 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyConsumerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyConsumerFactory.kt @@ -14,15 +14,15 @@ import io.airbyte.cdk.integrations.destination.jdbc.SqlOperations import io.airbyte.cdk.integrations.destination.jdbc.constants.GlobalDataSizeConstants import io.airbyte.cdk.integrations.destination.record_buffer.InMemoryRecordBufferingStrategy import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Consumer import javax.sql.DataSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} // TODO: Delete this class, this is only used in StarburstGalaxyDestination object CopyConsumerFactory { - private val LOGGER: Logger = LoggerFactory.getLogger(CopyConsumerFactory::class.java) fun create( outputRecordCollector: Consumer, @@ -158,11 +158,9 @@ object CopyConsumerFactory { return OnCloseFunction { hasFailed: Boolean, _: Map -> pairToIgnoredRecordCount.forEach { (pair: AirbyteStreamNameNamespacePair?, count: Long?) -> - LOGGER.warn( - "A total of {} record(s) of data from stream {} were invalid and were ignored.", - count, - pair - ) + LOGGER.warn { + "A total of $count record(s) of data from stream $pair were invalid and were ignored." + } } closeAsOneTransaction(pairToCopier, hasFailed, database, sqlOperations, dataSource) } @@ -193,9 +191,8 @@ object CopyConsumerFactory { queries.add(mergeQuery) } } catch (e: Exception) { - val message = - String.format("Failed to finalize copy to temp table due to: %s", e) - LOGGER.error(message) + val message = "Failed to finalize copy to temp table due to: $e" + LOGGER.error { message } failed = true if (firstException == null) { firstException = e diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyDestination.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyDestination.kt index 952fe9a2cf77..3752ef30d091 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyDestination.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/CopyDestination.kt @@ -16,9 +16,10 @@ import io.airbyte.cdk.integrations.destination.jdbc.AbstractJdbcDestination import io.airbyte.cdk.integrations.destination.jdbc.SqlOperations import io.airbyte.commons.exceptions.ConnectionErrorException import io.airbyte.protocol.models.v0.AirbyteConnectionStatus +import io.github.oshai.kotlinlogging.KotlinLogging import javax.sql.DataSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} // TODO: Delete this class, this is only used in StarburstGalaxyDestination abstract class CopyDestination : BaseConnector, Destination { @@ -52,7 +53,7 @@ abstract class CopyDestination : BaseConnector, Destination { try { checkPersistence(config) } catch (e: Exception) { - LOGGER.error("Exception attempting to access the staging persistence: ", e) + LOGGER.error(e) { "Exception attempting to access the staging persistence: " } return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) .withMessage( @@ -73,14 +74,14 @@ abstract class CopyDestination : BaseConnector, Destination { return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) } catch (ex: ConnectionErrorException) { - LOGGER.info("Exception while checking connection: ", ex) + LOGGER.info { "Exception while checking connection: $ex" } val message = getErrorMessage(ex.stateCode, ex.errorCode, ex.exceptionMessage, ex) emitConfigErrorTrace(ex, message) return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) .withMessage(message) } catch (e: Exception) { - LOGGER.error("Exception attempting to connect to the warehouse: ", e) + LOGGER.error(e) { "Exception attempting to connect to the warehouse: " } return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) .withMessage( @@ -93,7 +94,7 @@ abstract class CopyDestination : BaseConnector, Destination { try { close(dataSource) } catch (e: Exception) { - LOGGER.warn("Unable to close data source.", e) + LOGGER.warn(e) { "Unable to close data source." } } } } @@ -113,7 +114,5 @@ abstract class CopyDestination : BaseConnector, Destination { ) } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CopyDestination::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestination.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestination.kt index f880a4243e41..a0df83a35e9e 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestination.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestination.kt @@ -12,12 +12,12 @@ import io.airbyte.cdk.integrations.base.SerializedAirbyteMessageConsumer import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Consumer import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Multiple configs may allow you to sync data to the destination in multiple ways. * @@ -50,7 +50,7 @@ open class SwitchingDestination>( @Throws(Exception::class) override fun check(config: JsonNode): AirbyteConnectionStatus? { val destinationType = configToType.apply(config) - LOGGER.info("Using destination type: " + destinationType.name) + LOGGER.info { "Using destination type: ${destinationType.name}" } return typeToDestination[destinationType]!!.check(config) } @@ -61,7 +61,7 @@ open class SwitchingDestination>( outputRecordCollector: Consumer ): AirbyteMessageConsumer? { val destinationType = configToType.apply(config) - LOGGER.info("Using destination type: " + destinationType.name) + LOGGER.info { "Using destination type: ${destinationType.name}" } return typeToDestination[destinationType]!!.getConsumer( config, catalog, @@ -76,7 +76,7 @@ open class SwitchingDestination>( outputRecordCollector: Consumer ): SerializedAirbyteMessageConsumer? { val destinationType = configToType.apply(config) - LOGGER.info("Using destination type: " + destinationType.name) + LOGGER.info { "Using destination type: ${destinationType.name}" } return typeToDestination[destinationType]!!.getSerializedMessageConsumer( config, catalog, @@ -84,7 +84,5 @@ open class SwitchingDestination>( ) } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(SwitchingDestination::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcDestinationHandler.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcDestinationHandler.kt index a0d197e1413f..bab110833fcd 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcDestinationHandler.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcDestinationHandler.kt @@ -12,11 +12,11 @@ import io.airbyte.cdk.integrations.destination.jdbc.TableDefinition import io.airbyte.cdk.integrations.util.ConnectorExceptionUtil.getResultsOrLogAndThrowFirst import io.airbyte.commons.concurrency.CompletableFutures import io.airbyte.commons.exceptions.SQLRuntimeException -import io.airbyte.commons.functional.CheckedFunction import io.airbyte.commons.json.Jsons import io.airbyte.integrations.base.destination.typing_deduping.* import io.airbyte.integrations.base.destination.typing_deduping.Struct import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.* import java.time.Instant import java.time.OffsetDateTime @@ -24,7 +24,6 @@ import java.time.temporal.ChronoUnit import java.util.* import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletionStage -import java.util.function.Predicate import org.jooq.Condition import org.jooq.DSLContext import org.jooq.DataType @@ -35,8 +34,8 @@ import org.jooq.impl.DSL.field import org.jooq.impl.DSL.quotedName import org.jooq.impl.DSL.table import org.jooq.impl.SQLDataType -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class JdbcDestinationHandler( // JDBC's "catalog name" refers to e.g. the Postgres/Mysql database. @@ -85,18 +84,15 @@ abstract class JdbcDestinationHandler( private fun getInitialRawTableState(id: StreamId): InitialRawTableStatus { val tableExists = jdbcDatabase.executeMetadataQuery { dbmetadata: DatabaseMetaData -> - LOGGER.info( - "Retrieving table from Db metadata: {} {} {}", - catalogName, - id.rawNamespace, - id.rawName - ) + LOGGER.info { + "Retrieving table from Db metadata: $catalogName ${id.rawNamespace} ${id.rawName}" + } try { getTableFromMetadata(dbmetadata, id).use { table -> return@executeMetadataQuery table.next() } } catch (e: SQLException) { - LOGGER.error("Failed to retrieve table info from metadata", e) + LOGGER.error(e) { "Failed to retrieve table info from metadata" } throw SQLRuntimeException(e) } } @@ -108,7 +104,7 @@ abstract class JdbcDestinationHandler( } jdbcDatabase .unsafeQuery( - CheckedFunction { conn: Connection -> + { conn: Connection -> conn.prepareStatement( dslContext .select(field("MIN(_airbyte_extracted_at)").`as`("min_timestamp")) @@ -117,15 +113,13 @@ abstract class JdbcDestinationHandler( .sql ) }, - CheckedFunction { record: ResultSet -> record.getTimestamp("min_timestamp") } + { record: ResultSet -> record.getTimestamp("min_timestamp") } ) .use { timestampStream -> // Filter for nonNull values in case the query returned NULL (i.e. no unloaded // records). val minUnloadedTimestamp: Optional = - timestampStream - .filter(Predicate { obj: Timestamp -> Objects.nonNull(obj) }) - .findFirst() + timestampStream.filter { obj: Timestamp -> Objects.nonNull(obj) }.findFirst() if (minUnloadedTimestamp.isPresent) { // Decrement by 1 second since timestamp precision varies between databases. val ts = @@ -137,7 +131,7 @@ abstract class JdbcDestinationHandler( } jdbcDatabase .unsafeQuery( - CheckedFunction { conn: Connection -> + { conn: Connection -> conn.prepareStatement( dslContext .select(field("MAX(_airbyte_extracted_at)").`as`("min_timestamp")) @@ -145,15 +139,13 @@ abstract class JdbcDestinationHandler( .sql ) }, - CheckedFunction { record: ResultSet -> record.getTimestamp("min_timestamp") } + { record: ResultSet -> record.getTimestamp("min_timestamp") } ) .use { timestampStream -> // Filter for nonNull values in case the query returned NULL (i.e. no raw records at // all). val minUnloadedTimestamp: Optional = - timestampStream - .filter(Predicate { obj: Timestamp -> Objects.nonNull(obj) }) - .findFirst() + timestampStream.filter { obj: Timestamp -> Objects.nonNull(obj) }.findFirst() return InitialRawTableStatus( true, false, @@ -168,27 +160,21 @@ abstract class JdbcDestinationHandler( val queryId = UUID.randomUUID() for (transaction in transactions) { val transactionId = UUID.randomUUID() - LOGGER.info( - "Executing sql {}-{}: {}", - queryId, - transactionId, - java.lang.String.join("\n", transaction) - ) + LOGGER.info { + "Executing sql $queryId-$transactionId: ${transactions.joinToString("\n")}" + } val startTime = System.currentTimeMillis() try { jdbcDatabase.executeWithinTransaction(transaction) } catch (e: SQLException) { - LOGGER.error("Sql {}-{} failed", queryId, transactionId, e) + LOGGER.error(e) { "Sql $queryId-$transactionId failed" } throw e } - LOGGER.info( - "Sql {}-{} completed in {} ms", - queryId, - transactionId, - System.currentTimeMillis() - startTime - ) + LOGGER.info { + "Sql $queryId-$transactionId completed in ${System.currentTimeMillis() - startTime} ms" + } } } @@ -302,7 +288,7 @@ abstract class JdbcDestinationHandler( airbyteStreamNameNamespacePair to toDestinationState(stateNode) } } catch (e: Exception) { - LOGGER.warn("Failed to retrieve destination states", e) + LOGGER.warn(e) { "Failed to retrieve destination states" } return emptyMap() } } @@ -472,7 +458,7 @@ abstract class JdbcDestinationHandler( executeWithinTransaction(listOf(deleteStates, insertStates)) } catch (e: Exception) { - LOGGER.warn("Failed to commit destination states", e) + LOGGER.warn(e) { "Failed to commit destination states" } } } @@ -492,10 +478,10 @@ abstract class JdbcDestinationHandler( protected abstract fun toDestinationState(json: JsonNode): DestinationState companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(JdbcDestinationHandler::class.java) - protected const val DESTINATION_STATE_TABLE_NAME = "_airbyte_destination_state" - protected const val DESTINATION_STATE_TABLE_COLUMN_NAME = "name" - protected const val DESTINATION_STATE_TABLE_COLUMN_NAMESPACE = "namespace" + + private const val DESTINATION_STATE_TABLE_NAME = "_airbyte_destination_state" + private const val DESTINATION_STATE_TABLE_COLUMN_NAME = "name" + private const val DESTINATION_STATE_TABLE_COLUMN_NAMESPACE = "namespace" private const val DESTINATION_STATE_TABLE_COLUMN_STATE = "destination_state" private const val DESTINATION_STATE_TABLE_COLUMN_UPDATED_AT = "updated_at" @@ -513,12 +499,9 @@ abstract class JdbcDestinationHandler( // needed. Snowflake // requires uppercase) val columnDefinitions = LinkedHashMap() - LOGGER.info( - "Retrieving existing columns for {}.{}.{}", - catalogName, - schemaName, - tableName - ) + LOGGER.info { + "Retrieving existing columns for $catalogName.$schemaName.$tableName" + } try { dbMetadata.getColumns(catalogName, schemaName, tableName, null).use { columns -> @@ -537,13 +520,9 @@ abstract class JdbcDestinationHandler( } } } catch (e: SQLException) { - LOGGER.error( - "Failed to retrieve column info for {}.{}.{}", - catalogName, - schemaName, - tableName, - e - ) + LOGGER.error(e) { + "Failed to retrieve column info for $catalogName.$schemaName.$tableName" + } throw SQLRuntimeException(e) } columnDefinitions diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialStagingConsumerFactory.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialStagingConsumerFactory.kt index 546ad3dec5da..1bc80893274f 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialStagingConsumerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialStagingConsumerFactory.kt @@ -18,13 +18,13 @@ import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStream import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Instant import java.util.UUID import java.util.function.Consumer import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Uses both Factory and Consumer design pattern to create a single point of creation for consuming * [AirbyteMessage] for processing @@ -84,8 +84,6 @@ open class SerialStagingConsumerFactory { } companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(SerialStagingConsumerFactory::class.java) // using a random string here as a placeholder for the moment. // This would avoid mixing data in the staging area between different syncs (especially if @@ -166,7 +164,7 @@ open class SerialStagingConsumerFactory { syncMode, SYNC_DATETIME ) - LOGGER.info("Write config: {}", writeConfig) + LOGGER.info { "Write config: $writeConfig" } writeConfig } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestinationTest.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestinationTest.kt index 48dcc916175b..766a6e9a5823 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestinationTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/SwitchingDestinationTest.kt @@ -42,7 +42,7 @@ internal class SwitchingDestinationTest { val switchingDestination = SwitchingDestination( SwitchingEnum::class.java, - { c: JsonNode -> SwitchingEnum.INSERT }, + { SwitchingEnum.INSERT }, destinationMap ) @@ -65,11 +65,7 @@ internal class SwitchingDestinationTest { @Throws(Exception::class) fun testCopy() { val switchingDestination = - SwitchingDestination( - SwitchingEnum::class.java, - { c: JsonNode -> SwitchingEnum.COPY }, - destinationMap - ) + SwitchingDestination(SwitchingEnum::class.java, { SwitchingEnum.COPY }, destinationMap) switchingDestination.getConsumer( Mockito.mock(JsonNode::class.java), diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/DestinationAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/DestinationAcceptanceTest.kt index caa84d1dfa9a..fd5f9e967692 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/DestinationAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/DestinationAcceptanceTest.kt @@ -52,6 +52,7 @@ import io.airbyte.workers.normalization.NormalizationRunner import io.airbyte.workers.process.AirbyteIntegrationLauncher import io.airbyte.workers.process.DockerProcessFactory import io.airbyte.workers.process.ProcessFactory +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.UncheckedIOException import java.net.URISyntaxException @@ -71,8 +72,8 @@ import org.junit.jupiter.params.provider.Arguments import org.junit.jupiter.params.provider.ArgumentsProvider import org.junit.jupiter.params.provider.ArgumentsSource import org.mockito.Mockito -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class DestinationAcceptanceTest { protected var testSchemas: HashSet = HashSet() @@ -2179,8 +2180,6 @@ abstract class DestinationAcceptanceTest { private const val DUMMY_CATALOG_NAME = "DummyCatalog" - private val LOGGER: Logger = LoggerFactory.getLogger(DestinationAcceptanceTest::class.java) - /** * Reverses a list by creating a new list with the same elements of the input list and then * reversing it. The input list will not be altered. diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/argproviders/DataTypeTestArgumentProvider.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/argproviders/DataTypeTestArgumentProvider.kt index ce2e18ab7807..ad2a21a8e0c4 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/argproviders/DataTypeTestArgumentProvider.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/argproviders/DataTypeTestArgumentProvider.kt @@ -5,12 +5,13 @@ package io.airbyte.cdk.integrations.standardtest.destination.argproviders import io.airbyte.cdk.integrations.standardtest.destination.ProtocolVersion import io.airbyte.cdk.integrations.standardtest.destination.argproviders.util.ArgumentProviderUtil +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.stream.Stream import org.junit.jupiter.api.extension.ExtensionContext import org.junit.jupiter.params.provider.Arguments import org.junit.jupiter.params.provider.ArgumentsProvider -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DataTypeTestArgumentProvider : ArgumentsProvider { private lateinit var protocolVersion: ProtocolVersion @@ -94,8 +95,6 @@ class DataTypeTestArgumentProvider : ArgumentsProvider { ) : DataArgumentsProvider.CatalogMessageTestConfigPair(catalogFile, messageFile) companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(DataTypeTestArgumentProvider::class.java) const val INTEGER_TYPE_CATALOG: String = "data_type_integer_type_test_catalog.json" const val NUMBER_TYPE_CATALOG: String = "data_type_number_type_test_catalog.json" diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/AdvancedTestDataComparator.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/AdvancedTestDataComparator.kt index f7f042d250df..f95c0ea43c5e 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/AdvancedTestDataComparator.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/AdvancedTestDataComparator.kt @@ -4,13 +4,14 @@ package io.airbyte.cdk.integrations.standardtest.destination.comparator import com.fasterxml.jackson.databind.JsonNode +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.ZoneOffset import java.time.ZonedDateTime import java.time.format.DateTimeFormatter import java.time.format.DateTimeParseException import org.junit.jupiter.api.Assertions -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} open class AdvancedTestDataComparator : TestDataComparator { override fun assertSameData(expected: List, actual: List) { @@ -241,7 +242,6 @@ open class AdvancedTestDataComparator : TestDataComparator { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(AdvancedTestDataComparator::class.java) const val AIRBYTE_DATE_FORMAT: String = "yyyy-MM-dd" const val AIRBYTE_DATETIME_FORMAT: String = "yyyy-MM-dd'T'HH:mm:ss" diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/BasicTestDataComparator.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/BasicTestDataComparator.kt index e18d2ea54508..650b81265cf8 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/BasicTestDataComparator.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/comparator/BasicTestDataComparator.kt @@ -4,10 +4,11 @@ package io.airbyte.cdk.integrations.standardtest.destination.comparator import com.fasterxml.jackson.databind.JsonNode +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Function import org.junit.jupiter.api.Assertions -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class BasicTestDataComparator(private val nameResolver: Function>) : TestDataComparator { @@ -41,7 +42,5 @@ class BasicTestDataComparator(private val nameResolver: Function( Duration.between(lastReport, Instant.now()) .compareTo(Companion.REPORT_DURATION) > 0 ) { - LOGGER.info( - "CDC events queue size: {}. remaining {}", - this.size, - this.remainingCapacity() - ) + LOGGER.info { + "CDC events queue size: ${this.size}. remaining ${this.remainingCapacity()}" + } synchronized(this) { lastReport = Instant.now() } } } @@ -73,11 +71,10 @@ class AirbyteDebeziumHandler( cdcSavedInfoFetcher: CdcSavedInfoFetcher, cdcStateHandler: CdcStateHandler ): AutoCloseableIterator { - LOGGER.info("Using CDC: {}", true) - LOGGER.info( - "Using DBZ version: {}", - DebeziumEngine::class.java.getPackage().implementationVersion - ) + LOGGER.info { "Using CDC: true" } + LOGGER.info { + "Using DBZ version: ${DebeziumEngine::class.java.getPackage().implementationVersion}" + } val offsetManager: AirbyteFileOffsetBackingStore = AirbyteFileOffsetBackingStore.Companion.initializeState( cdcSavedInfoFetcher.savedOffset, @@ -142,7 +139,7 @@ class AirbyteDebeziumHandler( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(AirbyteDebeziumHandler::class.java) + private val REPORT_DURATION: Duration = Duration.of(10, ChronoUnit.SECONDS) /** diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteFileOffsetBackingStore.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteFileOffsetBackingStore.kt index b65ba98a136c..19f09bc21721 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteFileOffsetBackingStore.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteFileOffsetBackingStore.kt @@ -6,6 +6,7 @@ package io.airbyte.cdk.integrations.debezium.internals import com.fasterxml.jackson.databind.JsonNode import com.google.common.base.Preconditions import io.airbyte.commons.json.Jsons +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.EOFException import java.io.IOException import java.io.ObjectOutputStream @@ -19,9 +20,8 @@ import java.util.function.BiFunction import org.apache.commons.io.FileUtils import org.apache.kafka.connect.errors.ConnectException import org.apache.kafka.connect.util.SafeObjectInputStream -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class handles reading and writing a debezium offset file. In many cases it is duplicating * logic in debezium because that logic is not exposed in the public API. We mostly treat the @@ -71,7 +71,7 @@ class AirbyteFileOffsetBackingStore( return mapAsString } - LOGGER.info("Mutating sate to make it Debezium 2.1 compatible") + LOGGER.info { "Mutating sate to make it Debezium 2.1 compatible" } val newKey = if (dbName.isPresent) SQL_SERVER_STATE_MUTATION.apply(key.substring(i, i1 + 1), dbName.get()) @@ -157,8 +157,6 @@ class AirbyteFileOffsetBackingStore( } companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(AirbyteFileOffsetBackingStore::class.java) private val SQL_SERVER_STATE_MUTATION = BiFunction { key: String, databaseName: String -> (key.substring(0, key.length - 2) + ",\"database\":\"" + diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteSchemaHistoryStorage.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteSchemaHistoryStorage.kt index 0bfd427108b9..8e2e07a803d4 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteSchemaHistoryStorage.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/AirbyteSchemaHistoryStorage.kt @@ -8,6 +8,7 @@ import com.google.common.annotations.VisibleForTesting import io.airbyte.commons.json.Jsons import io.debezium.document.DocumentReader import io.debezium.document.DocumentWriter +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.nio.charset.Charset import java.nio.charset.StandardCharsets @@ -19,9 +20,8 @@ import java.util.* import java.util.zip.GZIPInputStream import java.util.zip.GZIPOutputStream import org.apache.commons.io.FileUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * The purpose of this class is : to , 1. Read the contents of the file [.path] which contains the * schema history at the end of the sync so that it can be saved in state for future syncs. Check @@ -40,33 +40,27 @@ class AirbyteSchemaHistoryStorage( fun read(): SchemaHistory { val fileSizeMB = path.toFile().length().toDouble() / (ONE_MB) if ((fileSizeMB > SIZE_LIMIT_TO_COMPRESS_MB) && compressSchemaHistoryForState) { - LOGGER.info( - "File Size {} MB is greater than the size limit of {} MB, compressing the content of the file.", - fileSizeMB, - SIZE_LIMIT_TO_COMPRESS_MB - ) + LOGGER.info { + "File Size $fileSizeMB MB is greater than the size limit of $SIZE_LIMIT_TO_COMPRESS_MB MB, compressing the content of the file." + } val schemaHistory = readCompressed() val compressedSizeMB = calculateSizeOfStringInMB(schemaHistory) if (fileSizeMB > compressedSizeMB) { - LOGGER.info("Content Size post compression is {} MB ", compressedSizeMB) + LOGGER.info { "Content Size post compression is $compressedSizeMB MB " } } else { throw RuntimeException( - "Compressing increased the size of the content. Size before compression " + - fileSizeMB + - ", after compression " + - compressedSizeMB + "Compressing increased the size of the content. Size before compression ${fileSizeMB}MB " + + ", after compression ${compressedSizeMB}MB" ) } return SchemaHistory(schemaHistory, true) } if (compressSchemaHistoryForState) { - LOGGER.info( - "File Size {} MB is less than the size limit of {} MB, reading the content of the file without compression.", - fileSizeMB, - SIZE_LIMIT_TO_COMPRESS_MB - ) + LOGGER.info { + "File Size $fileSizeMB MB is less than the size limit of $SIZE_LIMIT_TO_COMPRESS_MB MB, reading the content of the file without compression." + } } else { - LOGGER.info("File Size {} MB.", fileSizeMB) + LOGGER.info { "File Size $fileSizeMB MB." } } val schemaHistory = readUncompressed() return SchemaHistory(schemaHistory, false) @@ -221,8 +215,6 @@ class AirbyteSchemaHistoryStorage( } companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(AirbyteSchemaHistoryStorage::class.java) private const val SIZE_LIMIT_TO_COMPRESS_MB: Long = 1 const val ONE_MB: Int = 1024 * 1024 private val UTF8: Charset = StandardCharsets.UTF_8 diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumConverterUtils.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumConverterUtils.kt index a2044acb064f..fde47d831274 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumConverterUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumConverterUtils.kt @@ -6,14 +6,15 @@ package io.airbyte.cdk.integrations.debezium.internals import io.airbyte.cdk.db.DataTypeUtils.toISO8601String import io.airbyte.cdk.db.DataTypeUtils.toISO8601StringWithMicroseconds import io.debezium.spi.converter.RelationalColumn +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.Date import java.sql.Timestamp import java.time.Duration import java.time.LocalDate import java.time.LocalDateTime import java.time.format.DateTimeParseException -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DebeziumConverterUtils private constructor() { init { @@ -21,7 +22,6 @@ class DebeziumConverterUtils private constructor() { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DebeziumConverterUtils::class.java) /** TODO : Replace usage of this method with [io.airbyte.cdk.db.jdbc.DateTimeConverter] */ @JvmStatic @@ -51,14 +51,13 @@ class DebeziumConverterUtils private constructor() { try { return LocalDateTime.parse(input).toString() } catch (e: DateTimeParseException) { - LOGGER.warn("Cannot convert value '{}' to LocalDateTime type", input) + LOGGER.warn { "Cannot convert value '$input' to LocalDateTime type" } return input.toString() } } - LOGGER.warn( - "Uncovered date class type '{}'. Use default converter", - input.javaClass.name - ) + LOGGER.warn { + "Uncovered date class type '${input.javaClass.name}'. Use default converter" + } return input.toString() } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumMessageProducer.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumMessageProducer.kt index df14cada6b29..5a8b0c077296 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumMessageProducer.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumMessageProducer.kt @@ -9,10 +9,11 @@ import io.airbyte.cdk.integrations.source.relationaldb.state.SourceStateMessageP import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import org.apache.kafka.connect.errors.ConnectException -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DebeziumMessageProducer( private val cdcStateHandler: CdcStateHandler, @@ -62,7 +63,7 @@ class DebeziumMessageProducer( override fun generateStateMessageAtCheckpoint( stream: ConfiguredAirbyteStream? ): AirbyteStateMessage { - LOGGER.info("Sending CDC checkpoint state message.") + LOGGER.info { "Sending CDC checkpoint state message." } val stateMessage = createStateMessage(checkpointOffsetToSend) previousCheckpointOffset.clear() previousCheckpointOffset.putAll(checkpointOffsetToSend) @@ -87,9 +88,9 @@ class DebeziumMessageProducer( checkpointOffsetToSend.putAll(temporalOffset) } } catch (e: ConnectException) { - LOGGER.warn( + LOGGER.warn { "Offset file is being written by Debezium. Skipping CDC checkpoint in this loop." - ) + } } } @@ -140,7 +141,5 @@ class DebeziumMessageProducer( return message } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DebeziumMessageProducer::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordIterator.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordIterator.kt index b15e2ce58334..9822c298d122 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordIterator.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordIterator.kt @@ -9,6 +9,7 @@ import io.airbyte.cdk.integrations.debezium.CdcTargetPosition import io.airbyte.commons.lang.MoreBooleans import io.airbyte.commons.util.AutoCloseableIterator import io.debezium.engine.ChangeEvent +import io.github.oshai.kotlinlogging.KotlinLogging import java.lang.reflect.Field import java.time.Duration import java.time.LocalDateTime @@ -16,9 +17,8 @@ import java.util.* import java.util.concurrent.* import java.util.function.Supplier import org.apache.kafka.connect.source.SourceRecord -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * The record iterator is the consumer (in the producer / consumer relationship with debezium) * responsible for 1. making sure every record produced by the record publisher is processed 2. @@ -85,7 +85,7 @@ class DebeziumRecordIterator( ) ) } - LOGGER.info("no record found. polling again.") + LOGGER.info { "no record found. polling again." } maxInstanceOfNoRecordsFound++ continue } @@ -131,7 +131,7 @@ class DebeziumRecordIterator( } if (!signalledDebeziumEngineShutdown) { - LOGGER.warn("Debezium engine has not been signalled to shutdown, this is unexpected") + LOGGER.warn { "Debezium engine has not been signalled to shutdown, this is unexpected" } } // Read the records that Debezium might have fetched right at the time we called shutdown @@ -190,10 +190,9 @@ class DebeziumRecordIterator( } val timeElapsedSinceLastHeartbeatTs = Duration.between(this.tsLastHeartbeat, LocalDateTime.now()) - LOGGER.info( - "Time since last hb_pos change {}s", - timeElapsedSinceLastHeartbeatTs.toSeconds() - ) + LOGGER.info { + "Time since last hb_pos change ${timeElapsedSinceLastHeartbeatTs.toSeconds()}s" + } // wait time for no change in heartbeat position is half of initial waitTime return timeElapsedSinceLastHeartbeatTs.compareTo(firstRecordWaitTime.dividedBy(2)) > 0 } @@ -202,7 +201,7 @@ class DebeziumRecordIterator( if (signalledDebeziumEngineShutdown) { return } - LOGGER.info(closeLogMessage) + LOGGER.info { closeLogMessage } debeziumShutdownProcedure.initiateShutdownProcedure() signalledDebeziumEngineShutdown = true } @@ -230,25 +229,22 @@ class DebeziumRecordIterator( heartbeatEventSourceField[eventClass] = f if (heartbeatEventSourceField.size > 1) { - LOGGER.warn( - "Field Cache size growing beyond expected size of 1, size is " + - heartbeatEventSourceField.size - ) + LOGGER.warn { + "Field Cache size growing beyond expected size of 1, size is ${heartbeatEventSourceField.size}" + } } } val sr = f!![heartbeatEvent] as SourceRecord return targetPosition.extractPositionFromHeartbeatOffset(sr.sourceOffset()) } catch (e: NoSuchFieldException) { - LOGGER.info("failed to get heartbeat source offset") + LOGGER.info { "failed to get heartbeat source offset" } throw RuntimeException(e) } catch (e: IllegalAccessException) { - LOGGER.info("failed to get heartbeat source offset") + LOGGER.info { "failed to get heartbeat source offset" } throw RuntimeException(e) } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DebeziumRecordIterator::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordPublisher.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordPublisher.kt index 50506b6f8122..61da64bd6dbc 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordPublisher.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumRecordPublisher.kt @@ -7,13 +7,13 @@ import io.debezium.engine.ChangeEvent import io.debezium.engine.DebeziumEngine import io.debezium.engine.format.Json import io.debezium.engine.spi.OffsetCommitPolicy +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.concurrent.* import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicReference -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * The purpose of this class is to initialize and spawn the debezium engine with the right * properties to fetch records @@ -57,11 +57,10 @@ class DebeziumRecordPublisher(private val debeziumPropertiesManager: DebeziumPro } } .using { success: Boolean, message: String?, error: Throwable? -> - LOGGER.info( - "Debezium engine shutdown. Engine terminated successfully : {}", - success, - ) - LOGGER.info(message) + LOGGER.info { + "Debezium engine shutdown. Engine terminated successfully : $success" + } + LOGGER.info { message } if (!success) { if (error != null) { thrownError.set(error) @@ -77,19 +76,19 @@ class DebeziumRecordPublisher(private val debeziumPropertiesManager: DebeziumPro .using( object : DebeziumEngine.ConnectorCallback { override fun connectorStarted() { - LOGGER.info("DebeziumEngine notify: connector started") + LOGGER.info { "DebeziumEngine notify: connector started" } } override fun connectorStopped() { - LOGGER.info("DebeziumEngine notify: connector stopped") + LOGGER.info { "DebeziumEngine notify: connector stopped" } } override fun taskStarted() { - LOGGER.info("DebeziumEngine notify: task started") + LOGGER.info { "DebeziumEngine notify: task started" } } override fun taskStopped() { - LOGGER.info("DebeziumEngine notify: task stopped") + LOGGER.info { "DebeziumEngine notify: task stopped" } } }, ) @@ -127,7 +126,5 @@ class DebeziumRecordPublisher(private val debeziumPropertiesManager: DebeziumPro } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DebeziumRecordPublisher::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumShutdownProcedure.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumShutdownProcedure.kt index d6bbe6652fe1..af2c00fe4f27 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumShutdownProcedure.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/DebeziumShutdownProcedure.kt @@ -5,11 +5,11 @@ package io.airbyte.cdk.integrations.debezium.internals import io.airbyte.commons.concurrency.VoidCallable import io.airbyte.commons.lang.MoreBooleans +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.concurrent.* import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class has the logic for shutting down Debezium Engine in graceful manner. We made it Generic * to allow us to write tests easily. @@ -62,9 +62,9 @@ class DebeziumShutdownProcedure( val recordsRemainingAfterShutdown: LinkedBlockingQueue get() { if (!hasTransferThreadShutdown) { - LOGGER.warn( + LOGGER.warn { "Queue transfer thread has not shut down, some records might be missing." - ) + } } return targetQueue } @@ -80,7 +80,7 @@ class DebeziumShutdownProcedure( */ fun initiateShutdownProcedure() { if (hasEngineShutDown()) { - LOGGER.info("Debezium Engine has already shut down.") + LOGGER.info { "Debezium Engine has already shut down." } return } var exceptionDuringEngineClose: Exception? = null @@ -119,7 +119,5 @@ class DebeziumShutdownProcedure( } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DebeziumShutdownProcedure::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RecordWaitTimeUtil.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RecordWaitTimeUtil.kt index 00c1fa1acfae..1e77096f71ce 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RecordWaitTimeUtil.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RecordWaitTimeUtil.kt @@ -4,13 +4,13 @@ package io.airbyte.cdk.integrations.debezium.internals import com.fasterxml.jackson.databind.JsonNode +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Duration import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object RecordWaitTimeUtil { - private val LOGGER: Logger = LoggerFactory.getLogger(RecordWaitTimeUtil::class.java) val MIN_FIRST_RECORD_WAIT_TIME: Duration = Duration.ofMinutes(2) val MAX_FIRST_RECORD_WAIT_TIME: Duration = Duration.ofMinutes(60) @@ -51,21 +51,21 @@ object RecordWaitTimeUtil { if (firstRecordWaitSeconds.isPresent) { firstRecordWaitTime = Duration.ofSeconds(firstRecordWaitSeconds.get().toLong()) if (!isTest && firstRecordWaitTime.compareTo(MIN_FIRST_RECORD_WAIT_TIME) < 0) { - LOGGER.warn( - "First record waiting time is overridden to {} minutes, which is the min time allowed for safety.", - MIN_FIRST_RECORD_WAIT_TIME.toMinutes() - ) + LOGGER.warn { + "First record waiting time is overridden to ${MIN_FIRST_RECORD_WAIT_TIME.toMinutes()} minutes, " + + "which is the min time allowed for safety." + } firstRecordWaitTime = MIN_FIRST_RECORD_WAIT_TIME } else if (!isTest && firstRecordWaitTime.compareTo(MAX_FIRST_RECORD_WAIT_TIME) > 0) { - LOGGER.warn( - "First record waiting time is overridden to {} minutes, which is the max time allowed for safety.", - MAX_FIRST_RECORD_WAIT_TIME.toMinutes() - ) + LOGGER.warn { + "First record waiting time is overridden to ${MAX_FIRST_RECORD_WAIT_TIME.toMinutes()} minutes, " + + "which is the max time allowed for safety." + } firstRecordWaitTime = MAX_FIRST_RECORD_WAIT_TIME } } - LOGGER.info("First record waiting time: {} seconds", firstRecordWaitTime.seconds) + LOGGER.info { "First record waiting time: ${firstRecordWaitTime.seconds} seconds" } return firstRecordWaitTime } @@ -78,7 +78,9 @@ object RecordWaitTimeUtil { // In tests, reuse the initial_waiting_seconds property to speed things up. subsequentRecordWaitTime = Duration.ofSeconds(firstRecordWaitSeconds.get().toLong()) } - LOGGER.info("Subsequent record waiting time: {} seconds", subsequentRecordWaitTime.seconds) + LOGGER.info { + "Subsequent record waiting time: ${subsequentRecordWaitTime.seconds} seconds" + } return subsequentRecordWaitTime } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/AbstractJdbcSource.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/AbstractJdbcSource.kt index 1f19ffa47028..5fa961038cdb 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/AbstractJdbcSource.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/AbstractJdbcSource.kt @@ -60,6 +60,7 @@ import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import io.airbyte.protocol.models.v0.SyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.Connection import java.sql.PreparedStatement import java.sql.ResultSet @@ -73,9 +74,8 @@ import java.util.function.Supplier import java.util.stream.Collectors import javax.sql.DataSource import org.apache.commons.lang3.tuple.ImmutablePair -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class contains helper functions and boilerplate for implementing a source connector for a * relational DB source which can be accessed via JDBC driver. If you are implementing a connector @@ -161,7 +161,7 @@ abstract class AbstractJdbcSource( syncMode: SyncMode, cursorField: Optional ): AutoCloseableIterator { - AbstractDbSource.LOGGER.info("Queueing query for table: {}", tableName) + LOGGER.info { "Queueing query for table: $tableName" } val airbyteStream = AirbyteStreamUtils.convertFromNameAndNamespace(tableName, schemaName) return AutoCloseableIterators.lazyIterator( Supplier> { @@ -169,10 +169,7 @@ abstract class AbstractJdbcSource( val stream = database.unsafeQuery( { connection: Connection -> - AbstractDbSource.LOGGER.info( - "Preparing query for table: {}", - tableName - ) + LOGGER.info { "Preparing query for table: $tableName" } val fullTableName: String = RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting( schemaName, @@ -190,11 +187,7 @@ abstract class AbstractJdbcSource( ) val sql = java.lang.StringBuilder( - String.format( - "SELECT %s FROM %s", - wrappedColumnNames, - fullTableName - ) + "SELECT $wrappedColumnNames FROM $fullTableName" ) // if the connector emits intermediate states, the incremental query // must be sorted by the cursor @@ -204,15 +197,13 @@ abstract class AbstractJdbcSource( ) { val quotedCursorField: String = enquoteIdentifier(cursorField.get(), quoteString) - sql.append(String.format(" ORDER BY %s ASC", quotedCursorField)) + sql.append(" ORDER BY $quotedCursorField ASC") } val preparedStatement = connection.prepareStatement(sql.toString()) - AbstractDbSource.LOGGER.info( - "Executing query for table {}: {}", - tableName, - preparedStatement - ) + LOGGER.info { + "Executing query for table $tableName: $preparedStatement" + } preparedStatement }, sourceOperations::convertDatabaseRowToAirbyteRecordData @@ -241,9 +232,9 @@ abstract class AbstractJdbcSource( ): List> { return ImmutableList.of( CheckedConsumer { database: JdbcDatabase -> - LOGGER.info( + LOGGER.info { "Attempting to get metadata from the database to see if we can connect." - ) + } database.bufferedResultSetQuery( CheckedFunction { connection: Connection -> connection.metaData.catalogs }, CheckedFunction { queryResult: ResultSet -> @@ -266,7 +257,7 @@ abstract class AbstractJdbcSource( schema: String? ): List>> { val internalSchemas: Set = HashSet(excludedInternalNameSpaces) - LOGGER.info("Internal schemas to exclude: {}", internalSchemas) + LOGGER.info { "Internal schemas to exclude: $internalSchemas" } val tablesWithSelectGrantPrivilege = getPrivilegesTableForCurrentUser(database, schema) return database @@ -300,15 +291,11 @@ abstract class AbstractJdbcSource( .map { f: JsonNode -> val datatype = sourceOperations.getDatabaseFieldType(f) val jsonType = getAirbyteType(datatype) - LOGGER.debug( - "Table {} column {} (type {}[{}], nullable {}) -> {}", - fields[0].get(INTERNAL_TABLE_NAME).asText(), - f.get(INTERNAL_COLUMN_NAME).asText(), - f.get(INTERNAL_COLUMN_TYPE_NAME).asText(), - f.get(INTERNAL_COLUMN_SIZE).asInt(), - f.get(INTERNAL_IS_NULLABLE).asBoolean(), - jsonType - ) + LOGGER.debug { + "Table ${fields[0].get(INTERNAL_TABLE_NAME).asText()} column ${f.get(INTERNAL_COLUMN_NAME).asText()}" + + "(type ${f.get(INTERNAL_COLUMN_TYPE_NAME).asText()}[${f.get(INTERNAL_COLUMN_SIZE).asInt()}], " + + "nullable ${f.get(INTERNAL_IS_NULLABLE).asBoolean()}) -> $jsonType" + } object : CommonField( f.get(INTERNAL_COLUMN_NAME).asText(), @@ -407,10 +394,7 @@ abstract class AbstractJdbcSource( database: JdbcDatabase, tableInfos: List>> ): Map> { - LOGGER.info( - "Discover primary keys for tables: " + - tableInfos.map { obj: TableInfo> -> obj.name }.toSet() - ) + LOGGER.info { "Discover primary keys for tables: ${tableInfos.map { it.name }}" } try { // Get all primary keys without specifying a table name val tablePrimaryKeys = @@ -439,20 +423,14 @@ abstract class AbstractJdbcSource( return tablePrimaryKeys } } catch (e: SQLException) { - LOGGER.debug( - String.format( - "Could not retrieve primary keys without a table name (%s), retrying", - e - ) - ) + LOGGER.debug { "Could not retrieve primary keys without a table name ($e), retrying" } } // Get primary keys one table at a time return tableInfos .stream() .collect( - Collectors.toMap>, String, MutableList>( - Function>, String> { - tableInfo: TableInfo> -> + Collectors.toMap( + { tableInfo: TableInfo> -> getFullyQualifiedTableName(tableInfo.nameSpace, tableInfo.name) }, Function>, MutableList> toMap@{ @@ -484,13 +462,7 @@ abstract class AbstractJdbcSource( mutableListOf() ) } catch (e: SQLException) { - LOGGER.error( - String.format( - "Could not retrieve primary keys for %s: %s", - streamName, - e - ) - ) + LOGGER.error { "Could not retrieve primary keys for $streamName: $e" } return@toMap mutableListOf() } } @@ -510,7 +482,7 @@ abstract class AbstractJdbcSource( cursorInfo: CursorInfo, cursorFieldType: Datatype ): AutoCloseableIterator { - AbstractDbSource.LOGGER.info("Queueing query for table: {}", tableName) + LOGGER.info { "Queueing query for table: $tableName" } val airbyteStream = AirbyteStreamUtils.convertFromNameAndNamespace(tableName, schemaName) return AutoCloseableIterators.lazyIterator( { @@ -518,10 +490,8 @@ abstract class AbstractJdbcSource( val stream = database.unsafeQuery( { connection: Connection -> - AbstractDbSource.LOGGER.info( - "Preparing query for table: {}", - tableName - ) + LOGGER.info { "Preparing query for table: $tableName" } + val fullTableName: String = RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting( schemaName, @@ -542,12 +512,9 @@ abstract class AbstractJdbcSource( cursorFieldType, cursorInfo.cursor ) - AbstractDbSource.LOGGER.info( - "Table {} cursor count: expected {}, actual {}", - tableName, - cursorInfo.cursorRecordCount, - actualRecordCount - ) + LOGGER.info { + "Table $tableName cursor count: expected ${cursorInfo.cursorRecordCount}, actual $actualRecordCount" + } operator = if (actualRecordCount == cursorInfo.cursorRecordCount) { ">" @@ -580,11 +547,9 @@ abstract class AbstractJdbcSource( sql.append(String.format(" ORDER BY %s ASC", quotedCursorField)) } val preparedStatement = connection.prepareStatement(sql.toString()) - AbstractDbSource.LOGGER.info( - "Executing query for table {}: {}", - tableName, - preparedStatement - ) + LOGGER.info { + "Executing query for table $tableName: $preparedStatement" + } sourceOperations.setCursorField( preparedStatement, 1, @@ -635,12 +600,7 @@ abstract class AbstractJdbcSource( val cursorRecordStatement: PreparedStatement if (cursor == null) { val cursorRecordQuery = - String.format( - "SELECT COUNT(*) AS %s FROM %s WHERE %s IS NULL", - columnName, - fullTableName, - quotedCursorField - ) + "SELECT COUNT(*) AS $columnName FROM $fullTableName WHERE $quotedCursorField IS NULL" cursorRecordStatement = connection.prepareStatement(cursorRecordQuery) } else { val cursorRecordQuery = @@ -708,11 +668,9 @@ abstract class AbstractJdbcSource( */ @Throws(SQLException::class) override fun logPreSyncDebugData(database: JdbcDatabase, catalog: ConfiguredAirbyteCatalog?) { - LOGGER.info( - "Data source product recognized as {}:{}", - database.metaData.databaseProductName, - database.metaData.databaseProductVersion - ) + LOGGER.info { + "Data source product recognized as ${database.metaData.databaseProductName}:${database.metaData.databaseProductVersion}" + } } override fun close() { @@ -721,7 +679,7 @@ abstract class AbstractJdbcSource( try { close(d) } catch (e: Exception) { - LOGGER.warn("Unable to close data source.", e) + LOGGER.warn(e) { "Unable to close data source." } } } ) @@ -756,7 +714,6 @@ abstract class AbstractJdbcSource( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(AbstractJdbcSource::class.java) /** * Aggregate list of @param entries of StreamName and PrimaryKey and diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSSLConnectionUtils.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSSLConnectionUtils.kt index 8578ecd42f1a..e399be12b3f0 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSSLConnectionUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSSLConnectionUtils.kt @@ -7,6 +7,7 @@ import com.fasterxml.jackson.databind.JsonNode import io.airbyte.cdk.db.jdbc.JdbcUtils import io.airbyte.cdk.db.util.SSLCertificateUtils.keyStoreFromCertificate import io.airbyte.cdk.db.util.SSLCertificateUtils.keyStoreFromClientCertificate +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.net.MalformedURLException import java.net.URI @@ -20,8 +21,8 @@ import java.util.* import org.apache.commons.lang3.RandomStringUtils import org.apache.commons.lang3.tuple.ImmutablePair import org.apache.commons.lang3.tuple.Pair -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class JdbcSSLConnectionUtils { var caCertKeyStorePair: Pair? = null @@ -57,8 +58,7 @@ class JdbcSSLConnectionUtils { const val TRUST_KEY_STORE_TYPE: String = "trustCertificateKeyStoreType" const val KEY_STORE_TYPE_PKCS12: String = "PKCS12" const val PARAM_MODE: String = "mode" - private val LOGGER: Logger = - LoggerFactory.getLogger(JdbcSSLConnectionUtils::class.java.javaClass) + const val PARAM_CA_CERTIFICATE: String = "ca_certificate" const val PARAM_CLIENT_CERTIFICATE: String = "client_certificate" const val PARAM_CLIENT_KEY: String = "client_key" @@ -73,10 +73,10 @@ class JdbcSSLConnectionUtils { */ @JvmStatic fun parseSSLConfig(config: JsonNode): Map { - LOGGER.debug("source config: {}", config) + LOGGER.debug { "source config: $config" } - var caCertKeyStorePair: Pair? = null - var clientCertKeyStorePair: Pair? = null + var caCertKeyStorePair: Pair? + var clientCertKeyStorePair: Pair? val additionalParameters: MutableMap = HashMap() // assume ssl if not explicitly mentioned. if (!config.has(JdbcUtils.SSL_KEY) || config[JdbcUtils.SSL_KEY].asBoolean()) { @@ -86,15 +86,10 @@ class JdbcSSLConnectionUtils { SslMode.bySpec(specMode) .orElseThrow { IllegalArgumentException("unexpected ssl mode") } .name - if (Objects.isNull(caCertKeyStorePair)) { - caCertKeyStorePair = prepareCACertificateKeyStore(config) - } + caCertKeyStorePair = prepareCACertificateKeyStore(config) - if (Objects.nonNull(caCertKeyStorePair)) { - LOGGER.debug( - "uri for ca cert keystore: {}", - caCertKeyStorePair!!.left.toString() - ) + if (null != caCertKeyStorePair) { + LOGGER.debug { "uri for ca cert keystore: ${caCertKeyStorePair.left}" } try { additionalParameters.putAll( java.util.Map.of( @@ -111,16 +106,12 @@ class JdbcSSLConnectionUtils { } } - if (Objects.isNull(clientCertKeyStorePair)) { - clientCertKeyStorePair = prepareClientCertificateKeyStore(config) - } + clientCertKeyStorePair = prepareClientCertificateKeyStore(config) - if (Objects.nonNull(clientCertKeyStorePair)) { - LOGGER.debug( - "uri for client cert keystore: {} / {}", - clientCertKeyStorePair!!.left.toString(), - clientCertKeyStorePair.right - ) + if (null != clientCertKeyStorePair) { + LOGGER.debug { + "uri for client cert keystore: ${clientCertKeyStorePair.left} / ${clientCertKeyStorePair.right}" + } try { additionalParameters.putAll( java.util.Map.of( @@ -140,7 +131,7 @@ class JdbcSSLConnectionUtils { additionalParameters[SSL_MODE] = SslMode.DISABLED.name } } - LOGGER.debug("additional params: {}", additionalParameters) + LOGGER.debug { "additional params: $additionalParameters" } return additionalParameters } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSource.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSource.kt index 7e1f9b312534..136c2376af13 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSource.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSource.kt @@ -9,10 +9,11 @@ import io.airbyte.cdk.db.jdbc.JdbcUtils import io.airbyte.cdk.db.jdbc.streaming.AdaptiveStreamingQueryConfig import io.airbyte.cdk.integrations.base.IntegrationRunner import io.airbyte.cdk.integrations.base.Source +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.JDBCType import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class JdbcSource : AbstractJdbcSource( @@ -30,15 +31,14 @@ class JdbcSource : get() = setOf("information_schema", "pg_catalog", "pg_internal", "catalog_history") companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(JdbcSource::class.java) @Throws(Exception::class) @JvmStatic fun main(args: Array) { val source: Source = JdbcSource() - LOGGER.info("starting source: {}", JdbcSource::class.java) + LOGGER.info { "starting source: ${JdbcSource::class.java}" } IntegrationRunner(source).run(args) - LOGGER.info("completed source: {}", JdbcSource::class.java) + LOGGER.info { "completed source: ${JdbcSource::class.java}" } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/AbstractDbSource.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/AbstractDbSource.kt index 19e8ba0e212e..06f5a99ab440 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/AbstractDbSource.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/AbstractDbSource.kt @@ -33,15 +33,15 @@ import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.CommonField import io.airbyte.protocol.models.JsonSchemaType import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.SQLException import java.time.Duration import java.time.Instant import java.util.* import java.util.concurrent.atomic.AtomicLong import java.util.stream.Stream -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class contains helper functions and boilerplate for implementing a source connector for a DB * source of both non-relational and relational type @@ -71,7 +71,7 @@ protected constructor(driverClassName: String) : .withMessage(message) } catch (e: Exception) { addExceptionToTrace(e) - LOGGER.info("Exception while checking connection: ", e) + LOGGER.info { "Exception while checking connection: $e" } return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) .withMessage( @@ -175,9 +175,9 @@ protected constructor(driverClassName: String) : AirbyteTraceMessageUtility::emitStreamStatusTrace ) ) { - LOGGER.info("Closing database connection pool.") + LOGGER.info { "Closing database connection pool." } Exceptions.toRuntime { this.close() } - LOGGER.info("Closed database connection pool.") + LOGGER.info { "Closed database connection pool." } } } @@ -368,10 +368,9 @@ protected constructor(driverClassName: String) : val fullyQualifiedTableName = DbSourceDiscoverUtil.getFullyQualifiedTableName(stream.namespace, stream.name) if (!tableNameToTable.containsKey(fullyQualifiedTableName)) { - LOGGER.info( - "Skipping stream {} because it is not in the source", - fullyQualifiedTableName - ) + LOGGER.info { + "Skipping stream $fullyQualifiedTableName because it is not in the source" + } continue } @@ -511,7 +510,7 @@ protected constructor(driverClassName: String) : ) { r: AirbyteMessage -> val count = recordCount.incrementAndGet() if (count % 10000 == 0L) { - LOGGER.info("Reading stream {}. Records read: {}", streamName, count) + LOGGER.info { "Reading stream $streamName. Records read: $count" } } r } @@ -797,8 +796,6 @@ protected constructor(driverClassName: String) : const val READ_TRACE_OPERATION_NAME: String = "read-operation" @JvmStatic - protected val LOGGER: Logger = LoggerFactory.getLogger(AbstractDbSource::class.java) - private fun getMessageIterator( recordIterator: AutoCloseableIterator, streamName: String, diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/CdcStateManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/CdcStateManager.kt index 8a628ae73230..7cc3a2ce0aeb 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/CdcStateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/CdcStateManager.kt @@ -7,9 +7,10 @@ import io.airbyte.cdk.integrations.source.relationaldb.models.CdcState import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class CdcStateManager( private val initialState: CdcState?, @@ -26,7 +27,7 @@ class CdcStateManager( if (initialStreamsSynced != null) Collections.unmodifiableSet(initialStreamsSynced) else null this.rawStateMessage = stateMessage - LOGGER.info("Initialized CDC state") + LOGGER.info { "Initialized CDC state" } } var cdcState: CdcState? @@ -44,7 +45,5 @@ class CdcStateManager( '}' } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CdcStateManager::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/DbSourceDiscoverUtil.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/DbSourceDiscoverUtil.kt index 59023764e39d..32dba8308ce8 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/DbSourceDiscoverUtil.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/DbSourceDiscoverUtil.kt @@ -11,14 +11,14 @@ import io.airbyte.protocol.models.v0.AirbyteCatalog import io.airbyte.protocol.models.v0.CatalogHelpers import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.SyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Contains utilities and helper classes for discovering schemas in database sources. */ object DbSourceDiscoverUtil { - private val LOGGER: Logger = LoggerFactory.getLogger(DbSourceDiscoverUtil::class.java) + private val AIRBYTE_METADATA: List = mutableListOf("_ab_cdc_lsn", "_ab_cdc_updated_at", "_ab_cdc_deleted_at") @@ -68,13 +68,10 @@ object DbSourceDiscoverUtil { } if (!mismatchedFields.isEmpty()) { - LOGGER.warn( - "Source schema changed for table {}! Potential mismatches: {}. Actual schema: {}. Catalog schema: {}", - fullyQualifiedTableName, - java.lang.String.join(", ", mismatchedFields.toString()), - currentJsonSchema, - catalogSchema - ) + LOGGER.warn { + "Source schema changed for table $fullyQualifiedTableName! Potential mismatches: " + + "${mismatchedFields.joinToString(", ")}. Actual schema: $currentJsonSchema. Catalog schema: $catalogSchema" + } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbQueryUtils.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbQueryUtils.kt index 67b79bc3c7d8..a4f614aa9a36 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbQueryUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbQueryUtils.kt @@ -9,13 +9,12 @@ import io.airbyte.commons.stream.AirbyteStreamUtils import io.airbyte.commons.util.AutoCloseableIterator import io.airbyte.commons.util.AutoCloseableIterators import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Utility class for methods to query a relational db. */ object RelationalDbQueryUtils { - private val LOGGER: Logger = LoggerFactory.getLogger(RelationalDbQueryUtils::class.java) @JvmStatic fun getIdentifierWithQuoting(identifier: String, quoteString: String): String { @@ -76,7 +75,7 @@ object RelationalDbQueryUtils { return AutoCloseableIterators.lazyIterator( { try { - LOGGER.info("Queueing query: {}", sqlQuery) + LOGGER.info { "Queueing query: $sqlQuery" } val stream = database!!.unsafeQuery(sqlQuery) return@lazyIterator AutoCloseableIterators.fromStream( stream, @@ -93,10 +92,10 @@ object RelationalDbQueryUtils { @JvmStatic fun logStreamSyncStatus(streams: List, syncType: String?) { if (streams.isEmpty()) { - LOGGER.info("No Streams will be synced via {}.", syncType) + LOGGER.info { "No Streams will be synced via $syncType." } } else { - LOGGER.info("Streams to be synced via {} : {}", syncType, streams.size) - LOGGER.info("Streams: {}", prettyPrintConfiguredAirbyteStreamList(streams)) + LOGGER.info { "Streams to be synced via $syncType : ${streams.size}" } + LOGGER.info { "Streams: ${prettyPrintConfiguredAirbyteStreamList(streams)}" } } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/StateDecoratingIterator.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/StateDecoratingIterator.kt index 5560d1e25f5b..9c4c1119b81a 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/StateDecoratingIterator.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/StateDecoratingIterator.kt @@ -10,9 +10,11 @@ import io.airbyte.protocol.models.JsonSchemaPrimitiveUtil import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStateStats import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import kotlin.jvm.optionals.getOrNull + +private val LOGGER = KotlinLogging.logger {} @Deprecated("") class StateDecoratingIterator( @@ -141,10 +143,10 @@ class StateDecoratingIterator( } else if (cursorComparison == 0) { currentMaxCursorRecordCount++ } else if (cursorComparison > 0 && stateEmissionFrequency > 0) { - LOGGER.warn( + LOGGER.warn { "Intermediate state emission feature requires records to be processed in order according to the cursor value. Otherwise, " + "data loss can occur." - ) + } } } @@ -156,7 +158,7 @@ class StateDecoratingIterator( } catch (e: Exception) { emitIntermediateState = true hasCaughtException = true - LOGGER.error("Message iterator failed to read next record.", e) + LOGGER.error(e) { "Message iterator failed to read next record." } optionalIntermediateMessage = intermediateMessage return optionalIntermediateMessage.orElse(endOfData()) } @@ -205,38 +207,27 @@ class StateDecoratingIterator( fun createStateMessage(isFinalState: Boolean, recordCount: Int): AirbyteMessage { val stateMessage = stateManager.updateAndEmit(pair, currentMaxCursor, currentMaxCursorRecordCount) - val cursorInfo = stateManager.getCursorInfo(pair) + val cursorInfo = stateManager.getCursorInfo(pair).getOrNull() // logging once every 100 messages to reduce log verbosity if (recordCount % 100 == 0) { - LOGGER.info( - "State report for stream {} - original: {} = {} (count {}) -> latest: {} = {} (count {})", - pair, - cursorInfo.map { obj: CursorInfo -> obj.originalCursorField }.orElse(null), - cursorInfo.map { obj: CursorInfo -> obj.originalCursor }.orElse(null), - cursorInfo.map { obj: CursorInfo -> obj.originalCursorRecordCount }.orElse(null), - cursorInfo.map { obj: CursorInfo -> obj.cursorField }.orElse(null), - cursorInfo.map { obj: CursorInfo -> obj.cursor }.orElse(null), - cursorInfo.map { obj: CursorInfo -> obj.cursorRecordCount }.orElse(null) - ) + LOGGER.info { + "State report for stream $pair - original: ${cursorInfo?.originalCursorField} = ${cursorInfo?.originalCursor}" + + " (count ${cursorInfo?.originalCursorRecordCount}) -> latest: ${cursorInfo?.cursorField} = " + + "${cursorInfo?.cursor} (count ${cursorInfo?.cursorRecordCount})" + } } stateMessage?.withSourceStats(AirbyteStateStats().withRecordCount(recordCount.toDouble())) if (isFinalState) { hasEmittedFinalState = true if (stateManager.getCursor(pair).isEmpty) { - LOGGER.warn( - "Cursor for stream {} was null. This stream will replicate all records on the next run", - pair - ) + LOGGER.warn { + "Cursor for stream $pair was null. This stream will replicate all records on the next run" + } } } return AirbyteMessage().withType(AirbyteMessage.Type.STATE).withState(stateMessage) } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(@Suppress("deprecation") StateDecoratingIterator::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorManager.kt index cf3a41646b6b..1844a6ebe4ff 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorManager.kt @@ -6,13 +6,13 @@ package io.airbyte.cdk.integrations.source.relationaldb.state import com.google.common.annotations.VisibleForTesting import io.airbyte.cdk.integrations.source.relationaldb.CursorInfo import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.concurrent.* import java.util.function.Function import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Manages the map of streams to current cursor values for state management. * @@ -202,44 +202,36 @@ class CursorManager( // indicates a CDC stream // and we shouldn't log anything. if (cursor != null) { - LOGGER.info( - "Found matching cursor in state. Stream: {}. Cursor Field: {} Value: {} Count: {}", - pair, - cursorField, - cursor, - cursorRecordCount - ) + LOGGER.info { + "Found matching cursor in state. Stream: $pair. Cursor Field: $cursorField Value: $cursor Count: $cursorRecordCount" + } } // if cursor field in catalog and state are different. } else { cursor = null cursorRecordCount = 0L - LOGGER.info( - "Found cursor field. Does not match previous cursor field. Stream: {}. Original Cursor Field: {} (count {}). New Cursor Field: {}. Resetting cursor value.", - pair, - originalCursorField, - originalCursorRecordCount, - cursorField - ) + LOGGER.info { + "Found cursor field. Does not match previous cursor field. Stream: $pair." + + " Original Cursor Field: $originalCursorField (count $originalCursorRecordCount). " + + "New Cursor Field: $cursorField. Resetting cursor value." + } } // if cursor field is not set in state but is set in catalog. } else { - LOGGER.info( - "No cursor field set in catalog but not present in state. Stream: {}, New Cursor Field: {}. Resetting cursor value", - pair, - cursorField - ) + LOGGER.info { + "No cursor field set in catalog but not present in state. " + + "Stream: $pair, New Cursor Field: $cursorField. Resetting cursor value" + } cursor = null cursorRecordCount = 0L } // if cursor field is not set in catalog. } else { - LOGGER.info( - "Cursor field set in state but not present in catalog. Stream: {}. Original Cursor Field: {}. Original value: {}. Resetting cursor.", - pair, - originalCursorField, - originalCursor - ) + LOGGER.info { + "Cursor field set in state but not present in catalog. " + + "Stream: $pair. Original Cursor Field: $originalCursorField. " + + "Original value: $originalCursor. Resetting cursor." + } cursorField = null cursor = null cursorRecordCount = 0L @@ -291,7 +283,5 @@ class CursorManager( return getCursorInfo(pair).map { obj: CursorInfo -> obj.cursor } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CursorManager::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorStateMessageProducer.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorStateMessageProducer.kt index c7a1424db3a2..c065d28a5a23 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorStateMessageProducer.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/CursorStateMessageProducer.kt @@ -11,9 +11,10 @@ import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class CursorStateMessageProducer( private val stateManager: StateManager?, @@ -40,10 +41,10 @@ class CursorStateMessageProducer( val message = intermediateStateMessage intermediateStateMessage = null if (cursorOutOfOrderDetected) { - LOGGER.warn( + LOGGER.warn { "Intermediate state emission feature requires records to be processed in order according to the cursor value. Otherwise, " + "data loss can occur." - ) + } } return message } @@ -112,7 +113,7 @@ class CursorStateMessageProducer( // logging once every 100 messages to reduce log verbosity if (currentCursorRecordCount % LOG_FREQUENCY == 0) { - LOGGER.info("State report for stream {}: {}", pair, cursorInfo) + LOGGER.info { "State report for stream $pair: $cursorInfo" } } return stateMessage @@ -131,7 +132,7 @@ class CursorStateMessageProducer( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CursorStateMessageProducer::class.java) + private const val LOG_FREQUENCY = 100 } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManager.kt index 45317487c8ad..9815960b3157 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManager.kt @@ -10,12 +10,12 @@ import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Function import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Legacy implementation (pre-per-stream state support) of the [StateManager] interface. * @@ -70,7 +70,7 @@ class LegacyStateManager(dbState: DbState, catalog: ConfiguredAirbyteCatalog) : .withCdc(isCdc) .withCdcState(cdcStateManager.cdcState) - LOGGER.debug("Generated legacy state for {} streams", dbState.streams.size) + LOGGER.debug { "Generated legacy state for ${dbState.streams.size} streams" } return AirbyteStateMessage() .withType(AirbyteStateMessage.AirbyteStateType.LEGACY) .withData(Jsons.jsonNode(dbState)) @@ -97,8 +97,6 @@ class LegacyStateManager(dbState: DbState, catalog: ConfiguredAirbyteCatalog) : } companion object { - private val LOGGER: Logger = - @Suppress("deprecation") LoggerFactory.getLogger(LegacyStateManager::class.java) /** [Function] that extracts the cursor from the stream state. */ private val CURSOR_FUNCTION = DbStreamState::getCursor diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/SourceStateIterator.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/SourceStateIterator.kt index 7c150c44a69d..bd97b0112724 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/SourceStateIterator.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/SourceStateIterator.kt @@ -11,8 +11,6 @@ import io.airbyte.protocol.models.v0.SyncMode import java.time.Duration import java.time.Instant import java.time.OffsetDateTime -import org.slf4j.Logger -import org.slf4j.LoggerFactory open class SourceStateIterator( private val messageIterator: Iterator, @@ -107,7 +105,5 @@ open class SourceStateIterator( return false } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(SourceStateIterator::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateGeneratorUtils.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateGeneratorUtils.kt index 817a9245ae5f..aee75cf6fc79 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateGeneratorUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateGeneratorUtils.kt @@ -14,14 +14,13 @@ import io.airbyte.configoss.StateType import io.airbyte.configoss.StateWrapper import io.airbyte.configoss.helpers.StateMessageHelper import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Collection of utilities that facilitate the generation of state objects. */ object StateGeneratorUtils { - private val LOGGER: Logger = LoggerFactory.getLogger(StateGeneratorUtils::class.java) /** [Function] that extracts the cursor from the stream state. */ val CURSOR_FUNCTION: Function = @@ -162,7 +161,7 @@ object StateGeneratorUtils { try { return Optional.ofNullable(Jsons.`object`(state.streamState, DbStreamState::class.java)) } catch (e: IllegalArgumentException) { - LOGGER.error("Unable to extract state.", e) + LOGGER.error(e) { "Unable to extract state." } return Optional.empty() } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManager.kt index 06d79f3c2e90..1d1f62d053f2 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManager.kt @@ -8,10 +8,10 @@ import io.airbyte.cdk.integrations.source.relationaldb.CdcStateManager import io.airbyte.cdk.integrations.source.relationaldb.CursorInfo import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Defines a manager that manages connector state. Connector state is used to keep track of the data * synced by the connector. @@ -161,16 +161,9 @@ interface StateManager { if (cursorRecordCount > 0L) { cursorInfo.get().setCursorRecordCount(cursorRecordCount) } - LOGGER.debug( - "Updating cursor value for {} to {} (count {})...", - pair, - cursor, - cursorRecordCount - ) + LOGGER.debug { "Updating cursor value for $pair to $cursor (count $cursorRecordCount)..." } return emit(Optional.ofNullable(pair)) } - companion object { - val LOGGER: Logger = LoggerFactory.getLogger(StateManager::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManagerFactory.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManagerFactory.kt index 57c9879a8b43..40020b6b31c1 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManagerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StateManagerFactory.kt @@ -8,12 +8,11 @@ import io.airbyte.commons.exceptions.ConfigErrorException import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import io.github.oshai.kotlinlogging.KotlinLogging +private val LOGGER = KotlinLogging.logger {} /** Factory class that creates [StateManager] instances based on the provided state. */ object StateManagerFactory { - private val LOGGER: Logger = LoggerFactory.getLogger(StateManagerFactory::class.java) /** * Creates a [StateManager] based on the provided state object and catalog. This method will @@ -37,10 +36,9 @@ object StateManagerFactory { val airbyteStateMessage = initialState[0] when (supportedStateType) { AirbyteStateMessage.AirbyteStateType.LEGACY -> { - LOGGER.info( - "Legacy state manager selected to manage state object with type {}.", - airbyteStateMessage.type - ) + LOGGER.info { + "Legacy state manager selected to manage state object with type ${airbyteStateMessage.type}." + } @Suppress("deprecation") val retVal: StateManager = LegacyStateManager( @@ -50,24 +48,21 @@ object StateManagerFactory { return retVal } AirbyteStateMessage.AirbyteStateType.GLOBAL -> { - LOGGER.info( - "Global state manager selected to manage state object with type {}.", - airbyteStateMessage.type - ) + LOGGER.info { + "Global state manager selected to manage state object with type ${airbyteStateMessage.type}." + } return GlobalStateManager(generateGlobalState(airbyteStateMessage), catalog) } AirbyteStateMessage.AirbyteStateType.STREAM -> { - LOGGER.info( - "Stream state manager selected to manage state object with type {}.", - airbyteStateMessage.type - ) + LOGGER.info { + "Stream state manager selected to manage state object with type ${airbyteStateMessage.type}." + } return StreamStateManager(generateStreamState(initialState), catalog) } else -> { - LOGGER.info( - "Stream state manager selected to manage state object with type {}.", - airbyteStateMessage.type - ) + LOGGER.info { + "Stream state manager selected to manage state object with type ${airbyteStateMessage.type}." + } return StreamStateManager(generateStreamState(initialState), catalog) } } @@ -102,7 +97,7 @@ object StateManagerFactory { AirbyteStateMessage.AirbyteStateType.LEGACY -> { globalStateMessage = StateGeneratorUtils.convertLegacyStateToGlobalState(airbyteStateMessage) - LOGGER.info("Legacy state converted to global state.", airbyteStateMessage.type) + LOGGER.info { "Legacy state converted to global state." } } AirbyteStateMessage.AirbyteStateType.GLOBAL -> {} else -> {} diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManager.kt index 52c408d25d89..b7f4563e67ec 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManager.kt @@ -9,11 +9,11 @@ import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.AirbyteStreamState import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Per-stream implementation of the [StateManager] interface. * @@ -56,7 +56,7 @@ open class StreamStateManager val cursorInfo = Optional.ofNullable(pairToCursorInfoMap[pair.get()]) if (cursorInfo.isPresent) { - LOGGER.debug("Generating state message for {}...", pair) + LOGGER.debug { "Generating state message for $pair..." } return AirbyteStateMessage() .withType( AirbyteStateMessage.AirbyteStateType.STREAM @@ -69,23 +69,20 @@ open class StreamStateManager StateGeneratorUtils.generateStreamState(pair.get(), cursorInfo.get()) ) } else { - LOGGER.warn( - "Cursor information could not be located in state for stream {}. Returning a new, empty state message...", - pair - ) + LOGGER.warn { + "Cursor information could not be located in state for stream $pair. Returning a new, empty state message..." + } return AirbyteStateMessage() .withType(AirbyteStateMessage.AirbyteStateType.STREAM) .withStream(AirbyteStreamState()) } } else { - LOGGER.warn("Stream not provided. Returning a new, empty state message...") + LOGGER.warn { "Stream not provided. Returning a new, empty state message..." } return AirbyteStateMessage() .withType(AirbyteStateMessage.AirbyteStateType.STREAM) .withStream(AirbyteStreamState()) } } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(StreamStateManager::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcSourceAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcSourceAcceptanceTest.kt index 0ab7f9349cd9..4f4dc9a40106 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcSourceAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcSourceAcceptanceTest.kt @@ -19,6 +19,7 @@ import io.airbyte.cdk.integrations.util.HostPortResolver.resolvePort import io.airbyte.cdk.testutils.TestDatabase import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteStateMessage +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.JDBCType import java.util.function.Supplier import java.util.stream.Stream @@ -27,10 +28,10 @@ import org.junit.jupiter.api.AfterAll import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.BeforeAll import org.junit.jupiter.api.Test -import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.testcontainers.containers.PostgreSQLContainer +private val LOGGER = KotlinLogging.logger {} + /** * Runs the acceptance tests in the source-jdbc test module. We want this module to run these tests * itself as a sanity check. The trade off here is that this class is duplicated from the one used @@ -40,7 +41,7 @@ internal class DefaultJdbcSourceAcceptanceTest : JdbcSourceAcceptanceTest< DefaultJdbcSourceAcceptanceTest.PostgresTestSource, BareBonesTestDatabase>() { override fun config(): JsonNode { - return testdb?.testConfigBuilder()?.build()!! + return testdb.testConfigBuilder().build() } override fun source(): PostgresTestSource { @@ -48,7 +49,7 @@ internal class DefaultJdbcSourceAcceptanceTest : } override fun createTestDatabase(): BareBonesTestDatabase { - return BareBonesTestDatabase(PSQL_CONTAINER).initialized()!! + return BareBonesTestDatabase(PSQL_CONTAINER).initialized() } public override fun supportsSchemas(): Boolean { @@ -111,7 +112,6 @@ internal class DefaultJdbcSourceAcceptanceTest : } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(PostgresTestSource::class.java) val DRIVER_CLASS: String = DatabaseDriver.POSTGRESQL.driverClassName @@ -119,9 +119,9 @@ internal class DefaultJdbcSourceAcceptanceTest : @JvmStatic fun main(args: Array) { val source: Source = PostgresTestSource() - LOGGER.info("starting source: {}", PostgresTestSource::class.java) + LOGGER.info { "starting source: ${PostgresTestSource::class.java}" } IntegrationRunner(source).run(args) - LOGGER.info("completed source: {}", PostgresTestSource::class.java) + LOGGER.info { "completed source: ${PostgresTestSource::class.java}" } } } } @@ -181,15 +181,12 @@ internal class DefaultJdbcSourceAcceptanceTest : fun testCustomParametersOverwriteDefaultParametersExpectException() { val connectionPropertiesUrl = "ssl=false" val config = - testdb?.let { - getConfigWithConnectionProperties( - PSQL_CONTAINER, - it.databaseName, - connectionPropertiesUrl - ) - } - val customParameters = - parseJdbcParameters(config!!, JdbcUtils.CONNECTION_PROPERTIES_KEY, "&") + getConfigWithConnectionProperties( + PSQL_CONTAINER, + testdb.databaseName, + connectionPropertiesUrl + ) + val customParameters = parseJdbcParameters(config, JdbcUtils.CONNECTION_PROPERTIES_KEY, "&") val defaultParameters = mapOf("ssl" to "true", "sslmode" to "require") Assertions.assertThrows(IllegalArgumentException::class.java) { JdbcDataSourceUtils.assertCustomParametersDontOverwriteDefaultParameters( diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcStressTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcStressTest.kt index 85902d6915ad..ff2388e8a6a6 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcStressTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/DefaultJdbcStressTest.kt @@ -15,6 +15,7 @@ import io.airbyte.cdk.testutils.PostgreSQLContainerHelper.runSqlScript import io.airbyte.commons.io.IOs import io.airbyte.commons.json.Jsons import io.airbyte.commons.string.Strings +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.JDBCType import java.util.* import java.util.function.Supplier @@ -22,11 +23,11 @@ import org.junit.jupiter.api.AfterAll import org.junit.jupiter.api.BeforeAll import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Disabled -import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.testcontainers.containers.PostgreSQLContainer import org.testcontainers.utility.MountableFile +private val LOGGER = KotlinLogging.logger {} + /** * Runs the stress tests in the source-jdbc test module. We want this module to run these tests * itself as a sanity check. The trade off here is that this class is duplicated from the one used @@ -119,7 +120,6 @@ internal class DefaultJdbcStressTest : JdbcStressTest() { setOf("information_schema", "pg_catalog", "pg_internal", "catalog_history") companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(PostgresTestSource::class.java) val DRIVER_CLASS: String = DatabaseDriver.POSTGRESQL.driverClassName @@ -127,9 +127,9 @@ internal class DefaultJdbcStressTest : JdbcStressTest() { @JvmStatic fun main(args: Array) { val source: Source = PostgresTestSource() - LOGGER.info("starting source: {}", PostgresTestSource::class.java) + LOGGER.info { "starting source: ${PostgresTestSource::class.java}" } IntegrationRunner(source).run(args) - LOGGER.info("completed source: {}", PostgresTestSource::class.java) + LOGGER.info { "completed source: ${PostgresTestSource::class.java}" } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSourceStressTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSourceStressTest.kt index e5f333152dc0..e5b961f3848d 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSourceStressTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/jdbc/JdbcSourceStressTest.kt @@ -15,6 +15,7 @@ import io.airbyte.cdk.testutils.PostgreSQLContainerHelper.runSqlScript import io.airbyte.commons.io.IOs import io.airbyte.commons.json.Jsons import io.airbyte.commons.string.Strings +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.JDBCType import java.util.* import java.util.function.Supplier @@ -22,11 +23,11 @@ import org.junit.jupiter.api.AfterAll import org.junit.jupiter.api.BeforeAll import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Disabled -import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.testcontainers.containers.PostgreSQLContainer import org.testcontainers.utility.MountableFile +private val LOGGER = KotlinLogging.logger {} + /** * Runs the stress tests in the source-jdbc test module. We want this module to run these tests * itself as a sanity check. The trade off here is that this class is duplicated from the one used @@ -104,7 +105,6 @@ internal class JdbcSourceStressTest : JdbcStressTest() { setOf("information_schema", "pg_catalog", "pg_internal", "catalog_history") companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(PostgresTestSource::class.java) val DRIVER_CLASS: String = DatabaseDriver.POSTGRESQL.driverClassName @@ -112,9 +112,9 @@ internal class JdbcSourceStressTest : JdbcStressTest() { @JvmStatic fun main(args: Array) { val source: Source = PostgresTestSource() - LOGGER.info("starting source: {}", PostgresTestSource::class.java) + LOGGER.info { "starting source: ${PostgresTestSource::class.java}" } IntegrationRunner(source).run(args) - LOGGER.info("completed source: {}", PostgresTestSource::class.java) + LOGGER.info { "completed source: ${PostgresTestSource::class.java}" } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/debezium/CdcSourceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/debezium/CdcSourceTest.kt index 286bffedbd94..af5dd37b7a1f 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/debezium/CdcSourceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/debezium/CdcSourceTest.kt @@ -14,14 +14,15 @@ import io.airbyte.commons.util.AutoCloseableIterators import io.airbyte.protocol.models.Field import io.airbyte.protocol.models.JsonSchemaType import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Consumer import org.junit.jupiter.api.AfterEach import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.BeforeEach import org.junit.jupiter.api.Test -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class CdcSourceTest> { @JvmField protected var testdb: T = createTestDatabase() @@ -1340,7 +1341,6 @@ abstract class CdcSourceTest> { ) {} companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CdcSourceTest::class.java) const val MODELS_STREAM_NAME: String = "models" const val MODELS_STREAM_NAME_2: String = "models_2" diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcStressTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcStressTest.kt index 9fbd2d1252fc..b2a8f2d40e29 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcStressTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcStressTest.kt @@ -19,13 +19,14 @@ import io.airbyte.commons.string.Strings import io.airbyte.protocol.models.Field import io.airbyte.protocol.models.JsonSchemaType import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.math.BigDecimal import java.sql.Connection import java.util.* import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** * Runs a "large" amount of data through a JdbcSource to ensure that it streams / chunks records. @@ -224,7 +225,6 @@ abstract class JdbcStressTest { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(JdbcStressTest::class.java) // this will get rounded down to the nearest 1000th. private const val TOTAL_RECORDS = 10000000L diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceConnectorTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceConnectorTest.kt index 9b42e6852800..4761398a496c 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceConnectorTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceConnectorTest.kt @@ -38,8 +38,6 @@ import org.junit.jupiter.api.BeforeEach import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers import org.mockito.Mockito -import org.slf4j.Logger -import org.slf4j.LoggerFactory /** * This abstract class contains helpful functionality and boilerplate for testing a source @@ -348,8 +346,6 @@ abstract class AbstractSourceConnectorTest { } companion object { - protected val LOGGER: Logger = - LoggerFactory.getLogger(AbstractSourceConnectorTest::class.java) private const val JOB_ID = 0L.toString() private const val JOB_ATTEMPT = 0 diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceDatabaseTypeTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceDatabaseTypeTest.kt index 055fd87d5141..674aee9a7432 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceDatabaseTypeTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/AbstractSourceDatabaseTypeTest.kt @@ -10,6 +10,7 @@ import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.Field import io.airbyte.protocol.models.JsonSchemaType import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.sql.SQLException import java.util.function.Consumer @@ -17,8 +18,8 @@ import org.apache.commons.lang3.StringUtils import org.jooq.DSLContext import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** * This abstract class contains common helpers and boilerplate for comprehensively testing that all @@ -384,9 +385,4 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } .map { obj: AirbyteMessage -> obj.state } } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(AbstractSourceDatabaseTypeTest::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/PythonSourceAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/PythonSourceAcceptanceTest.kt index 90a090fcb6e6..1019769f807e 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/PythonSourceAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/PythonSourceAcceptanceTest.kt @@ -13,6 +13,7 @@ import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConnectorSpecification import io.airbyte.workers.TestHarnessUtils +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.nio.file.Files import java.nio.file.Path @@ -20,8 +21,8 @@ import java.util.* import java.util.concurrent.TimeUnit import java.util.function.Consumer import org.junit.jupiter.api.Assertions -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** * Extends TestSource such that it can be called using resources pulled from the file system. Will @@ -148,7 +149,7 @@ class PythonSourceAcceptanceTest : SourceAcceptanceTest() { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(PythonSourceAcceptanceTest::class.java) + private const val OUTPUT_FILENAME = "output.json" var IMAGE_NAME: String = "dummy_image_name" diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/SourceAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/SourceAcceptanceTest.kt index df12f2f87707..93f3def3b401 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/SourceAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/SourceAcceptanceTest.kt @@ -10,11 +10,12 @@ import com.google.common.collect.Sets import io.airbyte.commons.json.Jsons import io.airbyte.configoss.StandardCheckConnectionOutput import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class SourceAcceptanceTest : AbstractSourceConnectorTest() { /** @@ -427,8 +428,6 @@ abstract class SourceAcceptanceTest : AbstractSourceConnectorTest() { const val CDC_DEFAULT_CURSOR: String = "_ab_cdc_cursor" const val CDC_EVENT_SERIAL_NO: String = "_ab_cdc_event_serial_no" - private val LOGGER: Logger = LoggerFactory.getLogger(SourceAcceptanceTest::class.java) - @JvmStatic protected fun filterRecords( messages: Collection diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestEnvConfigs.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestEnvConfigs.kt index 18e56fd1b5c5..4acf7e7be88e 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestEnvConfigs.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestEnvConfigs.kt @@ -6,11 +6,12 @@ package io.airbyte.cdk.integrations.standardtest.source import io.airbyte.commons.lang.Exceptions import io.airbyte.commons.map.MoreMaps import io.airbyte.commons.version.AirbyteVersion +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Function import java.util.function.Supplier -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** * This class passes environment variable to the DockerProcessFactory that runs the source in the @@ -116,7 +117,6 @@ class TestEnvConfigs private constructor(envMap: Map) { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(TestEnvConfigs::class.java) // env variable names const val AIRBYTE_ROLE: String = "AIRBYTE_ROLE" diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/fs/TestSourceMain.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/fs/TestSourceMain.kt index b1552e38d7c2..cee110620da0 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/fs/TestSourceMain.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/fs/TestSourceMain.kt @@ -8,15 +8,12 @@ import java.nio.file.Path import net.sourceforge.argparse4j.ArgumentParsers import net.sourceforge.argparse4j.inf.ArgumentParserException import net.sourceforge.argparse4j.inf.Namespace -import org.slf4j.Logger -import org.slf4j.LoggerFactory /** * Parse command line arguments and inject them into the test class before running the test. Then * runs the tests. */ object TestSourceMain { - private val LOGGER: Logger = LoggerFactory.getLogger(TestSourceMain::class.java) @JvmStatic fun main(args: Array) { diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourceFillDbWithTestData.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourceFillDbWithTestData.kt index 34e9911fecf1..af35037c15bc 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourceFillDbWithTestData.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourceFillDbWithTestData.kt @@ -4,6 +4,7 @@ package io.airbyte.cdk.integrations.standardtest.source.performancetest import io.airbyte.cdk.db.Database +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.stream.Stream import org.jooq.DSLContext @@ -12,8 +13,8 @@ import org.junit.jupiter.api.TestInstance import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.Arguments import org.junit.jupiter.params.provider.MethodSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** This abstract class contains common methods for Fill Db scripts. */ @TestInstance(TestInstance.Lifecycle.PER_CLASS) @@ -62,7 +63,7 @@ abstract class AbstractSourceFillDbWithTestData : AbstractSourceBasePerformanceT ctx.fetch(String.format(insertQueryTemplate, currentTableName)) } - c.info("Finished processing for stream $currentSteamNumber") + LOGGER.info { "Finished processing for stream $currentSteamNumber" } } null } @@ -146,9 +147,6 @@ abstract class AbstractSourceFillDbWithTestData : AbstractSourceBasePerformanceT "CREATE TABLE %s.%s(id INTEGER PRIMARY KEY, %s)" private const val INSERT_INTO_DB_TABLE_QUERY_TEMPLATE = "INSERT INTO %s.%s (%s) VALUES %s" private const val TEST_DB_FIELD_TYPE = "varchar(10)" - - protected val c: Logger = - LoggerFactory.getLogger(AbstractSourceFillDbWithTestData::class.java) private const val TEST_VALUE_TEMPLATE_POSTGRES = "\'Value id_placeholder\'" } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourcePerformanceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourcePerformanceTest.kt index b72205662374..168ad60c8233 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourcePerformanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/performancetest/AbstractSourcePerformanceTest.kt @@ -9,14 +9,15 @@ import io.airbyte.cdk.integrations.standardtest.source.TestDestinationEnv import io.airbyte.protocol.models.Field import io.airbyte.protocol.models.JsonSchemaType import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.stream.Stream import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.TestInstance import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.Arguments import org.junit.jupiter.params.provider.MethodSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** This abstract class contains common methods for Performance tests. */ @TestInstance(TestInstance.Lifecycle.PER_CLASS) @@ -84,7 +85,7 @@ abstract class AbstractSourcePerformanceTest : AbstractSourceBasePerformanceTest if (failedStreamsMap.isNotEmpty()) { Assertions.fail("Non all messages were delivered. $failedStreamsMap") } - c.info("Finished all checks, no issues found for {} of streams", checkStatusMap.size) + LOGGER.info("Finished all checks, no issues found for {} of streams", checkStatusMap.size) } protected fun prepareMapWithExpectedRecords( @@ -151,7 +152,5 @@ abstract class AbstractSourcePerformanceTest : AbstractSourceBasePerformanceTest return ConfiguredAirbyteCatalog().withStreams(streams) } - companion object { - protected val c: Logger = LoggerFactory.getLogger(AbstractSourcePerformanceTest::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/build.gradle b/airbyte-cdk/java/airbyte-cdk/dependencies/build.gradle index b941fb213f30..8f527f3fe41e 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/build.gradle +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/build.gradle @@ -28,7 +28,7 @@ dependencies { api 'org.apache.logging.log4j:log4j-slf4j2-impl:2.21.1' api 'org.slf4j:log4j-over-slf4j:2.0.11' api 'org.slf4j:slf4j-api:2.0.11' - api 'io.github.oshai:kotlin-logging-jvm:5.1.0' + api 'io.github.oshai:kotlin-logging-jvm:6.0.9' api 'org.jetbrains.kotlinx:kotlinx-coroutines-core:1.8.0' diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/features/EnvVariableFeatureFlags.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/features/EnvVariableFeatureFlags.kt index df2e4ec1894b..5106ad19f598 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/features/EnvVariableFeatureFlags.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/features/EnvVariableFeatureFlags.kt @@ -3,9 +3,10 @@ */ package io.airbyte.commons.features +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val log = KotlinLogging.logger {} class EnvVariableFeatureFlags : FeatureFlags { override fun autoDetectSchema(): Boolean { @@ -51,13 +52,12 @@ class EnvVariableFeatureFlags : FeatureFlags { if (value != null && !value.isEmpty()) { return parser.apply(value) } else { - log.debug("Using default value for environment variable {}: '{}'", key, defaultValue) + log.debug { "Using default value for environment variable $key: '$defaultValue'" } return defaultValue } } companion object { - private val log: Logger = LoggerFactory.getLogger(EnvVariableFeatureFlags::class.java) const val AUTO_DETECT_SCHEMA: String = "AUTO_DETECT_SCHEMA" diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/io/LineGobbler.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/io/LineGobbler.kt index 8f238f2e17ac..c90bef7f9ce4 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/io/LineGobbler.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/io/LineGobbler.kt @@ -5,6 +5,7 @@ package io.airbyte.commons.io import io.airbyte.commons.concurrency.VoidCallable import io.airbyte.commons.logging.MdcScope +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.BufferedReader import java.io.ByteArrayInputStream import java.io.IOException @@ -13,10 +14,10 @@ import java.nio.charset.StandardCharsets import java.util.concurrent.ExecutorService import java.util.concurrent.Executors import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory import org.slf4j.MDC +private val LOGGER = KotlinLogging.logger {} + class LineGobbler @JvmOverloads internal constructor( @@ -46,26 +47,24 @@ internal constructor( line = `is`.readLine() } } catch (i: IOException) { - LOGGER.warn( - "{} gobbler IOException: {}. Typically happens when cancelling a job.", - caller, - i.message - ) + LOGGER.warn { + "$caller gobbler IOException: ${i.message}. Typically happens when cancelling a job." + } } catch (e: Exception) { - LOGGER.error("{} gobbler error when reading stream", caller, e) + LOGGER.error(e) { "$caller gobbler error when reading stream" } } finally { executor.shutdown() } } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(LineGobbler::class.java) + private const val GENERIC = "generic" @JvmOverloads fun gobble( message: String, - consumer: Consumer = Consumer { msg: String -> LOGGER.info(msg) } + consumer: Consumer = Consumer { msg: String -> LOGGER.info { msg } } ) { val stringAsSteam: InputStream = ByteArrayInputStream(message.toByteArray(StandardCharsets.UTF_8)) diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonPaths.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonPaths.kt index 10495a84ed00..6dac8d1ccd5e 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonPaths.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonPaths.kt @@ -15,11 +15,11 @@ import com.jayway.jsonpath.spi.json.JsonProvider import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider import com.jayway.jsonpath.spi.mapper.MappingProvider import io.airbyte.commons.util.MoreIterators +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.BiFunction -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * JSONPath is specification for querying JSON objects. More information about the specification can * be found here: https://goessner.net/articles/JsonPath/. For those familiar with jq, JSONPath will @@ -38,7 +38,6 @@ import org.slf4j.LoggerFactory * possible for a query to return more than one value. */ object JsonPaths { - private val LOGGER: Logger = LoggerFactory.getLogger(JsonPaths::class.java) const val JSON_PATH_START_CHARACTER: String = "$" const val JSON_PATH_LIST_SPLAT: String = "[*]" @@ -292,7 +291,7 @@ object JsonPaths { try { return replaceAtJsonNodeLoud(json, jsonPath, replacement) } catch (e: PathNotFoundException) { - LOGGER.debug("Path not found", e) + LOGGER.debug(e) { "Path not found" } return Jsons.clone(json) // defensive copy in failure case. } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/Jsons.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/Jsons.kt index 81998a9cf621..fe1b22fa9307 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/Jsons.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/Jsons.kt @@ -18,15 +18,15 @@ import com.google.common.base.Charsets import com.google.common.base.Preconditions import io.airbyte.commons.jackson.MoreMappers import io.airbyte.commons.stream.MoreStreams +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.File import java.io.IOException import java.util.* import java.util.function.BiConsumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object Jsons { - private val LOGGER: Logger = LoggerFactory.getLogger(Jsons::class.java) // Object Mapper is thread-safe private val OBJECT_MAPPER: ObjectMapper = MoreMappers.initMapper() @@ -434,7 +434,7 @@ object Jsons { sb.append(traceElement.toString()) } } - LOGGER.warn("Failed to deserialize json due to {}", sb) + LOGGER.warn { "Failed to deserialize json due to $sb" } return Optional.empty() } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/CloseableShutdownHook.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/CloseableShutdownHook.kt index c08cf4deffe4..08da99dd14ea 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/CloseableShutdownHook.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/CloseableShutdownHook.kt @@ -4,10 +4,10 @@ package io.airbyte.commons.lang import com.google.common.annotations.VisibleForTesting +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.stream.Stream -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val log = KotlinLogging.logger {} /** * Registers a shutdown hook that calls the close method of the provided objects. If an object does * not support either the [AutoCloseable] or [Closeable] interface, it will be ignored. @@ -17,7 +17,6 @@ import org.slf4j.LoggerFactory * application framework is introduced to the project that can provide object lifecycle management. */ object CloseableShutdownHook { - private val log: Logger = LoggerFactory.getLogger(CloseableShutdownHook::class.java) /** * Registers a runtime shutdown hook with the application for each provided closeable object. @@ -49,7 +48,7 @@ object CloseableShutdownHook { try { autoCloseable.close() } catch (e: Exception) { - log.error("Unable to close object {}.", autoCloseable.javaClass.name, e) + log.error(e) { "Unable to close object ${autoCloseable.javaClass.name}." } } } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/Exceptions.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/Exceptions.kt index 41d21d2ae270..b9814af7807f 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/Exceptions.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/lang/Exceptions.kt @@ -3,14 +3,13 @@ */ package io.airbyte.commons.lang -import java.lang.invoke.MethodHandles +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.concurrent.Callable import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val log = KotlinLogging.logger {} object Exceptions { - private val log: Logger = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()) /** * Catch a checked exception and rethrow as a [RuntimeException] @@ -59,7 +58,7 @@ object Exceptions { try { procedure.call() } catch (e: Exception) { - log.error("Swallowed error.", e) + log.error(e) { "Swallowed error." } } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/stream/StreamStatusUtils.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/stream/StreamStatusUtils.kt index aa7513a54b69..5857264edd55 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/stream/StreamStatusUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/stream/StreamStatusUtils.kt @@ -8,14 +8,13 @@ import io.airbyte.commons.util.AutoCloseableIterator import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** Collection of utility methods that support the generation of stream status updates. */ object StreamStatusUtils { - private val LOGGER: Logger = LoggerFactory.getLogger(StreamStatusUtils::class.java) /** * Creates a new [Consumer] that wraps the provided [Consumer] with stream status reporting @@ -89,7 +88,7 @@ object StreamStatusUtils { statusEmitter: Optional> ) { airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("RUNNING -> {}", s) + LOGGER.debug { "RUNNING -> $s" } emitStreamStatus( s, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.RUNNING, @@ -135,7 +134,7 @@ object StreamStatusUtils { statusEmitter: Optional> ) { airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("STARTING -> {}", s) + LOGGER.debug { "STARTING -> $s" } emitStreamStatus( s, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.STARTED, @@ -181,7 +180,7 @@ object StreamStatusUtils { statusEmitter: Optional> ) { airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("COMPLETE -> {}", s) + LOGGER.debug { "COMPLETE -> $s" } emitStreamStatus( s, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE, @@ -227,7 +226,7 @@ object StreamStatusUtils { statusEmitter: Optional> ) { airbyteStream.ifPresent { s: AirbyteStreamNameNamespacePair -> - LOGGER.debug("INCOMPLETE -> {}", s) + LOGGER.debug { "INCOMPLETE -> $s" } emitStreamStatus( s, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.INCOMPLETE, diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/util/CompositeIterator.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/util/CompositeIterator.kt index 10d900e482fc..a871c860cf7c 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/util/CompositeIterator.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/util/CompositeIterator.kt @@ -8,11 +8,11 @@ import com.google.common.collect.AbstractIterator import io.airbyte.commons.stream.AirbyteStreamStatusHolder import io.airbyte.commons.stream.StreamStatusUtils import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Consumer -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Composes multiple [AutoCloseableIterator]s. For each internal iterator, after the first time its * [Iterator.hasNext] function returns false, the composite iterator will call @@ -125,7 +125,7 @@ internal constructor( try { iterator.close() } catch (e: Exception) { - LOGGER.error("exception while closing", e) + LOGGER.error(e) { "exception while closing" } exceptions.add(e) } } @@ -142,7 +142,5 @@ internal constructor( Preconditions.checkState(!hasClosed) } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CompositeIterator::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/EnvConfigs.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/EnvConfigs.kt index 6bdad3700067..58f11c184d6e 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/EnvConfigs.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/EnvConfigs.kt @@ -3,10 +3,11 @@ */ package io.airbyte.configoss +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class EnvConfigs @JvmOverloads constructor(envMap: Map = System.getenv()) : Configs { @@ -35,14 +36,12 @@ class EnvConfigs @JvmOverloads constructor(envMap: Map = Syste isSecret: Boolean ): T { val value = getEnv.apply(key) - if (value != null && !value.isEmpty()) { + if (!value.isNullOrEmpty()) { return parser.apply(value) } else { - LOGGER.info( - "Using default value for environment variable {}: '{}'", - key, - if (isSecret) "*****" else defaultValue - ) + LOGGER.info { + "Using default value for environment variable $key: '${if (isSecret) "*****" else defaultValue.toString()}'" + } return defaultValue } } @@ -52,7 +51,6 @@ class EnvConfigs @JvmOverloads constructor(envMap: Map = Syste } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(EnvConfigs::class.java) // env variable names const val SPEC_CACHE_BUCKET: String = "SPEC_CACHE_BUCKET" diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/validation/json/JsonSchemaValidator.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/validation/json/JsonSchemaValidator.kt index 9ab5247b4e0b..24fc828cc4b0 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/validation/json/JsonSchemaValidator.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/validation/json/JsonSchemaValidator.kt @@ -7,6 +7,7 @@ import com.fasterxml.jackson.databind.JsonNode import com.google.common.annotations.VisibleForTesting import com.google.common.base.Preconditions import com.networknt.schema.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.File import java.io.IOException import java.net.URI @@ -14,8 +15,8 @@ import java.net.URISyntaxException import me.andrz.jackson.JsonContext import me.andrz.jackson.JsonReferenceException import me.andrz.jackson.JsonReferenceProcessor -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class JsonSchemaValidator @VisibleForTesting constructor(private val baseUri: URI?) { private val jsonSchemaFactory: JsonSchemaFactory = @@ -76,11 +77,10 @@ class JsonSchemaValidator @VisibleForTesting constructor(private val baseUri: UR fun test(schemaJson: JsonNode, objectJson: JsonNode): Boolean { val validationMessages = validateInternal(schemaJson, objectJson) - if (!validationMessages.isEmpty()) { - LOGGER.info( - "JSON schema validation failed. \nerrors: {}", - validationMessages.joinToString(", ") - ) + if (validationMessages.isNotEmpty()) { + LOGGER.info { + "JSON schema validation failed. \nerrors: ${validationMessages.joinToString(", ")}" + } } return validationMessages.isEmpty() @@ -176,7 +176,6 @@ class JsonSchemaValidator @VisibleForTesting constructor(private val baseUri: UR } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(JsonSchemaValidator::class.java) // This URI just needs to point at any path in the same directory as // /app/WellKnownTypes.json diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/TestHarnessUtilsTest.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/TestHarnessUtilsTest.kt index d5d97e264200..baf5e898eac0 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/TestHarnessUtilsTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/TestHarnessUtilsTest.kt @@ -6,6 +6,7 @@ package io.airbyte.workers import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair import io.airbyte.workers.internal.HeartbeatMonitor import io.airbyte.workers.test_utils.TestConfigHelpers +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Duration import java.time.temporal.ChronoUnit import java.util.concurrent.TimeUnit @@ -18,8 +19,8 @@ import org.junit.jupiter.api.Test import org.mockito.Mockito import org.mockito.invocation.InvocationOnMock import org.mockito.kotlin.mock -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} internal class TestHarnessUtilsTest { @Nested @@ -127,7 +128,6 @@ internal class TestHarnessUtilsTest { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(GentleCloseWithHeartbeat::class.java) /** * As long as the the heartbeatMonitor detects a heartbeat, the process will be allowed to @@ -160,7 +160,7 @@ internal class TestHarnessUtilsTest { try { process.waitFor(checkHeartbeatDuration.toMillis(), TimeUnit.MILLISECONDS) } catch (e: InterruptedException) { - LOGGER.error("Exception while waiting for process to finish", e) + LOGGER.error(e) { "Exception while waiting for process to finish" } } } @@ -168,9 +168,9 @@ internal class TestHarnessUtilsTest { try { process.waitFor(gracefulShutdownDuration.toMillis(), TimeUnit.MILLISECONDS) } catch (e: InterruptedException) { - LOGGER.error( + LOGGER.error { "Exception during grace period for process to finish. This can happen when cancelling jobs." - ) + } } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactoryTest.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactoryTest.kt index 039560b8d627..64233b7dfb8e 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactoryTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactoryTest.kt @@ -8,6 +8,8 @@ import io.airbyte.commons.logging.MdcScope import io.airbyte.protocol.models.AirbyteLogMessage import io.airbyte.protocol.models.AirbyteMessage import io.airbyte.workers.test_utils.AirbyteMessageUtils +import io.github.oshai.kotlinlogging.KLogger +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.BufferedReader import java.io.ByteArrayInputStream import java.io.InputStream @@ -20,17 +22,19 @@ import org.junit.jupiter.api.Disabled import org.junit.jupiter.api.Test import org.mockito.ArgumentMatchers import org.mockito.Mockito -import org.slf4j.Logger +import org.mockito.kotlin.mock + +private val logger = KotlinLogging.logger {} internal class DefaultAirbyteStreamFactoryTest { private lateinit var protocolPredicate: AirbyteProtocolPredicate - private lateinit var logger: Logger + private lateinit var logger: KLogger @BeforeEach fun setup() { protocolPredicate = Mockito.mock(AirbyteProtocolPredicate::class.java) Mockito.`when`(protocolPredicate.test(ArgumentMatchers.any())).thenReturn(true) - logger = Mockito.mock(Logger::class.java) + logger = mock() } @Test diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/TestHarnessUtils.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/TestHarnessUtils.kt index 1375945d605d..5009d9ff13f9 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/TestHarnessUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/TestHarnessUtils.kt @@ -11,6 +11,7 @@ import io.airbyte.protocol.models.* import io.airbyte.workers.exception.TestHarnessException import io.airbyte.workers.helper.FailureHelper import io.airbyte.workers.internal.AirbyteStreamFactory +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.* import java.nio.charset.StandardCharsets import java.time.Duration @@ -18,12 +19,10 @@ import java.time.temporal.ChronoUnit import java.util.* import java.util.concurrent.TimeUnit import java.util.stream.Collectors -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} // TODO:(Issue-4824): Figure out how to log Docker process information. object TestHarnessUtils { - private val LOGGER: Logger = LoggerFactory.getLogger(TestHarnessUtils::class.java) fun gentleClose(process: Process?, timeout: Long, timeUnit: TimeUnit?) { if (process == null) { @@ -32,7 +31,7 @@ object TestHarnessUtils { if (process.info() != null) { process.info().commandLine().ifPresent { commandLine: String -> - LOGGER.debug("Gently closing process {}", commandLine) + LOGGER.debug { "Gently closing process $commandLine" } } } @@ -41,7 +40,7 @@ object TestHarnessUtils { process.waitFor(timeout, timeUnit) } } catch (e: InterruptedException) { - LOGGER.error("Exception while while waiting for process to finish", e) + LOGGER.error(e) { "Exception while while waiting for process to finish" } } if (process.isAlive) { @@ -57,13 +56,13 @@ object TestHarnessUtils { process.destroy() process.waitFor(lastChanceDuration.toMillis(), TimeUnit.MILLISECONDS) if (process.isAlive) { - LOGGER.warn( + LOGGER.warn { "Process is still alive after calling destroy. Attempting to destroy forcibly..." - ) + } process.destroyForcibly() } } catch (e: InterruptedException) { - LOGGER.error("Exception when closing process.", e) + LOGGER.error(e) { "Exception when closing process." } } } @@ -71,7 +70,7 @@ object TestHarnessUtils { try { process.waitFor() } catch (e: InterruptedException) { - LOGGER.error("Exception while while waiting for process to finish", e) + LOGGER.error(e) { "Exception while while waiting for process to finish" } } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DbtTransformationRunner.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DbtTransformationRunner.kt index 8de44daf0e48..44405ca40339 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DbtTransformationRunner.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DbtTransformationRunner.kt @@ -17,12 +17,13 @@ import io.airbyte.workers.exception.TestHarnessException import io.airbyte.workers.normalization.NormalizationRunner import io.airbyte.workers.process.Metadata import io.airbyte.workers.process.ProcessFactory +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.util.* import java.util.concurrent.TimeUnit import org.apache.tools.ant.types.Commandline -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DbtTransformationRunner( private val processFactory: ProcessFactory, @@ -157,7 +158,7 @@ class DbtTransformationRunner( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DbtTransformationRunner::class.java) + private const val DBT_ENTRYPOINT_SH = "entrypoint.sh" private val CONTAINER_LOG_MDC_BUILDER: MdcScope.Builder = MdcScope.Builder() diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultCheckConnectionTestHarness.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultCheckConnectionTestHarness.kt index d40865d35efe..52a97e9548ae 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultCheckConnectionTestHarness.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultCheckConnectionTestHarness.kt @@ -15,9 +15,10 @@ import io.airbyte.workers.helper.ConnectorConfigUpdater import io.airbyte.workers.internal.AirbyteStreamFactory import io.airbyte.workers.internal.DefaultAirbyteStreamFactory import io.airbyte.workers.process.IntegrationLauncher +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DefaultCheckConnectionTestHarness @JvmOverloads @@ -124,9 +125,4 @@ constructor( override fun cancel() { TestHarnessUtils.cancelProcess(process) } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(DefaultCheckConnectionTestHarness::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultDiscoverCatalogTestHarness.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultDiscoverCatalogTestHarness.kt index fb67ac881dbb..2365e94fbe75 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultDiscoverCatalogTestHarness.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultDiscoverCatalogTestHarness.kt @@ -19,11 +19,12 @@ import io.airbyte.workers.helper.ConnectorConfigUpdater import io.airbyte.workers.internal.AirbyteStreamFactory import io.airbyte.workers.internal.DefaultAirbyteStreamFactory import io.airbyte.workers.process.IntegrationLauncher +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.util.* import kotlin.concurrent.Volatile -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DefaultDiscoverCatalogTestHarness @JvmOverloads @@ -140,8 +141,6 @@ constructor( } companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(DefaultDiscoverCatalogTestHarness::class.java) private const val WRITE_DISCOVER_CATALOG_LOGS_TAG = "call to write discover schema result" } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultGetSpecTestHarness.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultGetSpecTestHarness.kt index b584a394adcf..0636257789de 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultGetSpecTestHarness.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/general/DefaultGetSpecTestHarness.kt @@ -13,9 +13,10 @@ import io.airbyte.workers.exception.TestHarnessException import io.airbyte.workers.internal.AirbyteStreamFactory import io.airbyte.workers.internal.DefaultAirbyteStreamFactory import io.airbyte.workers.process.IntegrationLauncher +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DefaultGetSpecTestHarness @JvmOverloads @@ -78,7 +79,5 @@ constructor( TestHarnessUtils.cancelProcess(process) } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DefaultGetSpecTestHarness::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/ConnectorConfigUpdater.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/ConnectorConfigUpdater.kt index be819be4c67a..afa3504ba3a1 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/ConnectorConfigUpdater.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/ConnectorConfigUpdater.kt @@ -13,11 +13,11 @@ import io.airbyte.api.client.model.generated.SourceIdRequestBody import io.airbyte.api.client.model.generated.SourceUpdate import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.Config +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.charset.StandardCharsets import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Helper class for workers to persist updates to Source/Destination configs emitted from * AirbyteControlMessages. @@ -101,7 +101,5 @@ class ConnectorConfigUpdater( ) } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(ConnectorConfigUpdater::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteDestination.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteDestination.kt index 4fdcbde7e11b..cb1e10ec944c 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteDestination.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteDestination.kt @@ -5,7 +5,6 @@ package io.airbyte.workers.internal import com.google.common.base.Charsets import com.google.common.base.Preconditions -import io.airbyte.cdk.extensions.TestContext import io.airbyte.commons.io.IOs import io.airbyte.commons.io.LineGobbler import io.airbyte.commons.json.Jsons @@ -19,6 +18,7 @@ import io.airbyte.workers.TestHarnessUtils import io.airbyte.workers.WorkerConstants import io.airbyte.workers.exception.TestHarnessException import io.airbyte.workers.process.IntegrationLauncher +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.BufferedWriter import java.io.IOException import java.io.OutputStreamWriter @@ -32,8 +32,8 @@ import kotlin.collections.Map import kotlin.collections.Set import kotlin.collections.contains import kotlin.collections.setOf -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DefaultAirbyteDestination @JvmOverloads @@ -179,15 +179,11 @@ constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DefaultAirbyteDestination::class.java) - fun createContainerLogMdcBuilder(): MdcScope.Builder { - val currentTest = TestContext.CURRENT_TEST_NAME.get() - val logPrefix = - if (currentTest == null) "destination" else "destination(${currentTest})" - return MdcScope.Builder() - .setLogPrefix(logPrefix) + + fun createContainerLogMdcBuilder(): MdcScope.Builder = + MdcScope.Builder() + .setLogPrefix("destination") .setPrefixColor(LoggingHelper.Color.YELLOW_BACKGROUND) - } val IGNORED_EXIT_CODES: Set = setOf( 0, // Normal exit diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteSource.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteSource.kt index 794b63e60607..32b6ff1c4597 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteSource.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteSource.kt @@ -18,6 +18,7 @@ import io.airbyte.protocol.models.AirbyteMessage import io.airbyte.workers.TestHarnessUtils import io.airbyte.workers.WorkerConstants import io.airbyte.workers.process.IntegrationLauncher +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.time.Duration import java.time.temporal.ChronoUnit @@ -28,8 +29,8 @@ import kotlin.collections.Iterator import kotlin.collections.Set import kotlin.collections.contains import kotlin.collections.setOf -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DefaultAirbyteSource @VisibleForTesting @@ -200,7 +201,6 @@ internal constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DefaultAirbyteSource::class.java) private val HEARTBEAT_FRESH_DURATION: Duration = Duration.of(5, ChronoUnit.MINUTES) private val GRACEFUL_SHUTDOWN_DURATION: Duration = Duration.of(1, ChronoUnit.MINUTES) diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactory.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactory.kt index 4b0ec4d4511e..c6874b29bf22 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/DefaultAirbyteStreamFactory.kt @@ -9,6 +9,8 @@ import io.airbyte.commons.json.Jsons import io.airbyte.commons.logging.MdcScope import io.airbyte.protocol.models.AirbyteLogMessage import io.airbyte.protocol.models.AirbyteMessage +import io.github.oshai.kotlinlogging.KLogger +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.BufferedReader import java.lang.reflect.InvocationTargetException import java.nio.charset.StandardCharsets @@ -17,9 +19,8 @@ import java.text.StringCharacterIterator import java.time.Instant import java.util.* import java.util.stream.Stream -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Creates a stream from an input stream. The produced stream attempts to parse each line of the * InputStream into a AirbyteMessage. If the line cannot be parsed into a AirbyteMessage it is @@ -34,7 +35,7 @@ class DefaultAirbyteStreamFactory : AirbyteStreamFactory { private val containerLogMdcBuilder: MdcScope.Builder private val protocolValidator: AirbyteProtocolPredicate - protected val logger: Logger + protected val logger: KLogger private val maxMemory: Long private val exceptionClass: Optional> @@ -56,7 +57,7 @@ class DefaultAirbyteStreamFactory : AirbyteStreamFactory { */ internal constructor( protocolPredicate: AirbyteProtocolPredicate, - logger: Logger, + logger: KLogger, containerLogMdcBuilder: MdcScope.Builder, messageSizeExceptionClass: Optional> ) { @@ -70,7 +71,7 @@ class DefaultAirbyteStreamFactory : AirbyteStreamFactory { @VisibleForTesting internal constructor( protocolPredicate: AirbyteProtocolPredicate, - logger: Logger, + logger: KLogger, containerLogMdcBuilder: MdcScope.Builder, messageSizeExceptionClass: Optional>, maxMemory: Long @@ -185,9 +186,4 @@ class DefaultAirbyteStreamFactory : AirbyteStreamFactory { } return String.format("%.1f %cB", bytes / 1000.0, ci.current()) } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(DefaultAirbyteStreamFactory::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/DefaultNormalizationRunner.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/DefaultNormalizationRunner.kt index c5641a2c97fe..2d6fc33529ca 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/DefaultNormalizationRunner.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/DefaultNormalizationRunner.kt @@ -22,14 +22,15 @@ import io.airbyte.workers.WorkerConstants import io.airbyte.workers.exception.TestHarnessException import io.airbyte.workers.process.Metadata import io.airbyte.workers.process.ProcessFactory +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.util.* import java.util.concurrent.TimeUnit import java.util.function.Function import java.util.stream.Collectors import java.util.stream.Stream -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DefaultNormalizationRunner( private val processFactory: ProcessFactory, @@ -264,7 +265,7 @@ class DefaultNormalizationRunner( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DefaultNormalizationRunner::class.java) + private val CONTAINER_LOG_MDC_BUILDER: MdcScope.Builder = MdcScope.Builder() .setLogPrefix("normalization") diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/NormalizationAirbyteStreamFactory.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/NormalizationAirbyteStreamFactory.kt index a2cc295a6ae1..8b28711d289e 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/NormalizationAirbyteStreamFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/normalization/NormalizationAirbyteStreamFactory.kt @@ -10,11 +10,12 @@ import io.airbyte.commons.logging.MdcScope import io.airbyte.protocol.models.AirbyteLogMessage import io.airbyte.protocol.models.AirbyteMessage import io.airbyte.workers.internal.AirbyteStreamFactory +import io.github.oshai.kotlinlogging.KLogger +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.BufferedReader import java.util.stream.Stream -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Creates a stream from an input stream. The produced stream attempts to parse each line of the * InputStream into a AirbyteMessage. If the line cannot be parsed into a AirbyteMessage it is @@ -26,7 +27,7 @@ import org.slf4j.LoggerFactory */ class NormalizationAirbyteStreamFactory internal constructor( - private val logger: Logger, + private val logger: KLogger, private val containerLogMdcBuilder: MdcScope.Builder ) : AirbyteStreamFactory { val dbtErrors: MutableList = ArrayList() @@ -110,9 +111,4 @@ internal constructor( else -> logger.info(logMessage.message) } } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(NormalizationAirbyteStreamFactory::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/process/DockerProcessFactory.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/process/DockerProcessFactory.kt index 63995d4fb46a..e58319655d9d 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/process/DockerProcessFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/process/DockerProcessFactory.kt @@ -15,6 +15,7 @@ import io.airbyte.configoss.AllowedHosts import io.airbyte.configoss.ResourceRequirements import io.airbyte.workers.TestHarnessUtils import io.airbyte.workers.exception.TestHarnessException +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.nio.file.Files import java.nio.file.Path @@ -22,8 +23,8 @@ import java.util.* import java.util.concurrent.TimeUnit import java.util.function.Function import org.apache.commons.lang3.StringUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class DockerProcessFactory( private val workspaceRoot: Path, @@ -185,7 +186,7 @@ class DockerProcessFactory( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(DockerProcessFactory::class.java) + private const val DOCKER_NAME_LEN_LIMIT = 128 private val DATA_MOUNT_DESTINATION: Path = Path.of("/data") diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/BaseGcsDestination.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/BaseGcsDestination.kt index 30caeeffa849..cdc9f795edf8 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/BaseGcsDestination.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/BaseGcsDestination.kt @@ -20,10 +20,11 @@ import io.airbyte.cdk.integrations.destination.s3.SerializedBufferFactory.Compan import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Consumer import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class BaseGcsDestination : BaseConnector(), Destination { private val nameTransformer: NamingConventionTransformer = GcsNameTransformer() @@ -46,18 +47,16 @@ abstract class BaseGcsDestination : BaseConnector(), Destination { return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) } catch (e: AmazonS3Exception) { - LOGGER.error("Exception attempting to access the Gcs bucket", e) + LOGGER.error(e) { "Exception attempting to access the Gcs bucket" } val message = getErrorMessage(e.errorCode, 0, e.message, e) emitConfigErrorTrace(e, message) return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) .withMessage(message) } catch (e: Exception) { - LOGGER.error( - "Exception attempting to access the Gcs bucket: {}. Please make sure you account has all of these roles: " + - EXPECTED_ROLES, - e - ) + LOGGER.error(e) { + "Exception attempting to access the Gcs bucket: {}. Please make sure you account has all of these roles: $EXPECTED_ROLES" + } emitConfigErrorTrace(e, e.message) return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) @@ -90,7 +89,7 @@ abstract class BaseGcsDestination : BaseConnector(), Destination { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BaseGcsDestination::class.java) + const val EXPECTED_ROLES: String = ("storage.multipartUploads.abort, storage.multipartUploads.create, " + "storage.objects.create, storage.objects.delete, storage.objects.get, storage.objects.list") diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsStorageOperations.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsStorageOperations.kt index 618f8bde637d..c1496cd5178b 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsStorageOperations.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsStorageOperations.kt @@ -8,8 +8,9 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest import io.airbyte.cdk.integrations.destination.NamingConventionTransformer import io.airbyte.cdk.integrations.destination.s3.S3DestinationConfig import io.airbyte.cdk.integrations.destination.s3.S3StorageOperations -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import io.github.oshai.kotlinlogging.KotlinLogging + +private val LOGGER = KotlinLogging.logger {} class GcsStorageOperations( nameTransformer: NamingConventionTransformer, @@ -31,7 +32,7 @@ class GcsStorageOperations( keysToDelete: List ) { for (keyToDelete in keysToDelete) { - LOGGER.info("Deleting object {}", keyToDelete.key) + LOGGER.info { "Deleting object ${keyToDelete.key}" } s3Client.deleteObject(bucket, keyToDelete.key) } } @@ -40,7 +41,5 @@ class GcsStorageOperations( return HashMap() } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(GcsStorageOperations::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/avro/GcsAvroWriter.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/avro/GcsAvroWriter.kt index d5fd5734e130..a65b28c6147f 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/avro/GcsAvroWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/avro/GcsAvroWriter.kt @@ -19,16 +19,17 @@ import io.airbyte.cdk.integrations.destination.s3.util.StreamTransferManagerFact import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.sql.Timestamp import java.util.* import org.apache.avro.file.DataFileWriter import org.apache.avro.generic.GenericData import org.apache.avro.generic.GenericDatumWriter -import org.slf4j.Logger -import org.slf4j.LoggerFactory import tech.allegro.schema.json2avro.converter.JsonAvroConverter +private val LOGGER = KotlinLogging.logger {} + class GcsAvroWriter @JvmOverloads constructor( @@ -61,18 +62,15 @@ constructor( false, true ) - LOGGER.info("Avro schema for stream {}: {}", stream.name, schema!!.toString(false)) + LOGGER.info { "Avro schema for stream ${stream.name}: ${schema!!.toString(false)}" } val outputFilename: String = getOutputFilename(uploadTimestamp, FileUploadFormat.AVRO) outputPath = java.lang.String.join("/", outputPrefix, outputFilename) fileLocation = String.format("gs://%s/%s", config.bucketName, outputPath) - LOGGER.info( - "Full GCS path for stream '{}': {}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { + "Full GCS path for stream '${stream.name}': ${config.bucketName}/$outputPath" + } this.avroRecordFactory = AvroRecordFactory(schema, converter) this.uploadManager = @@ -120,7 +118,5 @@ constructor( override val fileFormat: FileUploadFormat get() = FileUploadFormat.AVRO - companion object { - protected val LOGGER: Logger = LoggerFactory.getLogger(GcsAvroWriter::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/csv/GcsCsvWriter.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/csv/GcsCsvWriter.kt index 71a6ff30817e..e59e37173c49 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/csv/GcsCsvWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/csv/GcsCsvWriter.kt @@ -18,6 +18,7 @@ import io.airbyte.cdk.integrations.destination.s3.util.StreamTransferManagerFact import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.PrintWriter import java.nio.charset.StandardCharsets @@ -26,8 +27,8 @@ import java.util.* import org.apache.commons.csv.CSVFormat import org.apache.commons.csv.CSVPrinter import org.apache.commons.csv.QuoteMode -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class GcsCsvWriter( config: GcsDestinationConfig, @@ -51,12 +52,9 @@ class GcsCsvWriter( outputPath = java.lang.String.join("/", outputPrefix, outputFilename) fileLocation = String.format("gs://%s/%s", config.bucketName, outputPath) - LOGGER.info( - "Full GCS path for stream '{}': {}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { + "Full GCS path for stream '${stream.name}': ${config.bucketName}/$outputPath" + } this.uploadManager = create(config.bucketName, outputPath, s3Client) @@ -101,7 +99,5 @@ class GcsCsvWriter( override val fileFormat: FileUploadFormat get() = FileUploadFormat.CSV - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(GcsCsvWriter::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/jsonl/GcsJsonlWriter.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/jsonl/GcsJsonlWriter.kt index 0682d5ff1d0e..8b0141248d78 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/jsonl/GcsJsonlWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/jsonl/GcsJsonlWriter.kt @@ -18,13 +18,14 @@ import io.airbyte.commons.jackson.MoreMappers import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.PrintWriter import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class GcsJsonlWriter( config: GcsDestinationConfig, @@ -44,12 +45,9 @@ class GcsJsonlWriter( outputPath = java.lang.String.join("/", outputPrefix, outputFilename) fileLocation = String.format("gs://%s/%s", config.bucketName, outputPath) - LOGGER.info( - "Full GCS path for stream '{}': {}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { + "Full GCS path for stream '${stream.name}': ${config.bucketName}/$outputPath" + } this.uploadManager = create(config.bucketName, outputPath, s3Client).get() @@ -88,7 +86,6 @@ class GcsJsonlWriter( get() = FileUploadFormat.JSONL companion object { - protected val LOGGER: Logger = LoggerFactory.getLogger(GcsJsonlWriter::class.java) private val MAPPER: ObjectMapper = MoreMappers.initMapper() } diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/parquet/GcsParquetWriter.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/parquet/GcsParquetWriter.kt index 187a1c6d7260..acab641dd868 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/parquet/GcsParquetWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/parquet/GcsParquetWriter.kt @@ -16,6 +16,7 @@ import io.airbyte.cdk.integrations.destination.s3.parquet.UploadParquetFormatCon import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.net.URI import java.sql.Timestamp @@ -27,10 +28,10 @@ import org.apache.hadoop.fs.Path import org.apache.parquet.avro.AvroParquetWriter import org.apache.parquet.hadoop.ParquetWriter import org.apache.parquet.hadoop.util.HadoopOutputFile -import org.slf4j.Logger -import org.slf4j.LoggerFactory import tech.allegro.schema.json2avro.converter.JsonAvroConverter +private val LOGGER = KotlinLogging.logger {} + class GcsParquetWriter( config: GcsDestinationConfig, s3Client: AmazonS3, @@ -48,19 +49,14 @@ class GcsParquetWriter( val outputFilename: String = BaseGcsWriter.Companion.getOutputFilename(uploadTimestamp, FileUploadFormat.PARQUET) outputPath = java.lang.String.join("/", outputPrefix, outputFilename) - LOGGER.info( - "Storage path for stream '{}': {}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { "Storage path for stream '${stream.name}': ${config.bucketName}/$outputPath" } fileLocation = String.format("s3a://%s/%s/%s", config.bucketName, outputPrefix, outputFilename) val uri = URI(fileLocation) val path = Path(uri) - LOGGER.info("Full GCS path for stream '{}': {}", stream.name, path) + LOGGER.info { "Full GCS path for stream '${stream.name}': $path" } val formatConfig = config.formatConfig as UploadParquetFormatConfig val hadoopConfig = getHadoopConfig(config) @@ -93,13 +89,13 @@ class GcsParquetWriter( @Throws(IOException::class) override fun close(hasFailed: Boolean) { if (hasFailed) { - LOGGER.warn("Failure detected. Aborting upload of stream '{}'...", stream.name) + LOGGER.warn { "Failure detected. Aborting upload of stream '${stream.name}'..." } parquetWriter.close() - LOGGER.warn("Upload of stream '{}' aborted.", stream.name) + LOGGER.warn { "Upload of stream '${stream.name}' aborted." } } else { - LOGGER.info("Uploading remaining data for stream '{}'.", stream.name) + LOGGER.info { "Uploading remaining data for stream '${stream.name}'." } parquetWriter.close() - LOGGER.info("Upload completed for stream '{}'.", stream.name) + LOGGER.info { "Upload completed for stream '${stream.name}'." } } } @@ -107,7 +103,7 @@ class GcsParquetWriter( get() = FileUploadFormat.PARQUET companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(GcsParquetWriter::class.java) + private val MAPPER = ObjectMapper() fun getHadoopConfig(config: GcsDestinationConfig): Configuration { diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/util/GcsUtils.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/util/GcsUtils.kt index aad36d1cc9b0..7c27594d592d 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/util/GcsUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/util/GcsUtils.kt @@ -5,14 +5,15 @@ package io.airbyte.cdk.integrations.destination.gcs.util import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.integrations.destination.s3.avro.AvroConstants +import io.github.oshai.kotlinlogging.KotlinLogging import org.apache.avro.LogicalTypes import org.apache.avro.Schema import org.apache.avro.SchemaBuilder -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object GcsUtils { - private val LOGGER: Logger = LoggerFactory.getLogger(GcsUtils::class.java) + private val UUID_SCHEMA: Schema = LogicalTypes.uuid().addToSchema(Schema.create(Schema.Type.STRING)) private val TIMESTAMP_MILLIS_SCHEMA: Schema = @@ -26,7 +27,7 @@ object GcsUtils { appendAirbyteFields: Boolean, useDestinationsV2Columns: Boolean ): Schema? { - LOGGER.info("Default schema.") + LOGGER.info { "Default schema." } val stdName = AvroConstants.NAME_TRANSFORMER.getIdentifier(name) val stdNamespace = AvroConstants.NAME_TRANSFORMER.getNamespace(namespace) var builder = SchemaBuilder.record(stdName).namespace(stdNamespace) diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/writer/BaseGcsWriter.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/writer/BaseGcsWriter.kt index d6798291cc47..0924ac60f468 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/writer/BaseGcsWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/gcs/writer/BaseGcsWriter.kt @@ -14,14 +14,14 @@ import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteStream import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import io.airbyte.protocol.models.v0.DestinationSyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.sql.Timestamp import java.text.DateFormat import java.text.SimpleDateFormat import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * The base implementation takes care of the following: * @@ -48,13 +48,13 @@ protected constructor( try { val bucket = config.bucketName if (!gcsBucketExist(s3Client, bucket)) { - LOGGER.info("Bucket {} does not exist; creating...", bucket) + LOGGER.info { "Bucket $bucket does not exist; creating..." } s3Client.createBucket(bucket) - LOGGER.info("Bucket {} has been created.", bucket) + LOGGER.info { "Bucket $bucket has been created." } } if (syncMode == DestinationSyncMode.OVERWRITE) { - LOGGER.info("Overwrite mode") + LOGGER.info { "Overwrite mode" } val keysToDelete: MutableList = LinkedList() val objects = s3Client.listObjects(bucket, outputPrefix).objectSummaries for (`object` in objects) { @@ -62,24 +62,21 @@ protected constructor( } if (keysToDelete.size > 0) { - LOGGER.info( - "Purging non-empty output path for stream '{}' under OVERWRITE mode...", - stream.name - ) + LOGGER.info { + "Purging non-empty output path for stream '${stream.name}' under OVERWRITE mode..." + } // Google Cloud Storage doesn't accept request to delete multiple objects for (keyToDelete in keysToDelete) { s3Client.deleteObject(bucket, keyToDelete.key) } - LOGGER.info( - "Deleted {} file(s) for stream '{}'.", - keysToDelete.size, - stream.name - ) + LOGGER.info { + "Deleted ${keysToDelete.size} file(s) for stream '${stream.name}'." + } } - LOGGER.info("Overwrite is finished") + LOGGER.info { "Overwrite is finished" } } } catch (e: Exception) { - LOGGER.error("Failed to initialize: ", e) + LOGGER.error(e) { "Failed to initialize: " } closeWhenFail() throw e } @@ -102,13 +99,13 @@ protected constructor( @Throws(IOException::class) override fun close(hasFailed: Boolean) { if (hasFailed) { - LOGGER.warn("Failure detected. Aborting upload of stream '{}'...", stream.name) + LOGGER.warn { "Failure detected. Aborting upload of stream '${stream.name}'..." } closeWhenFail() - LOGGER.warn("Upload of stream '{}' aborted.", stream.name) + LOGGER.warn { "Upload of stream '${stream.name}' aborted." } } else { - LOGGER.info("Uploading remaining data for stream '{}'.", stream.name) + LOGGER.info { "Uploading remaining data for stream '${stream.name}'." } closeWhenSucceed() - LOGGER.info("Upload completed for stream '{}'.", stream.name) + LOGGER.info { "Upload completed for stream '${stream.name}'." } } } @@ -125,7 +122,6 @@ protected constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BaseGcsWriter::class.java) // Filename: __0. fun getOutputFilename(timestamp: Timestamp, format: FileUploadFormat): String { diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/gcs/GcsStreamCopier.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/gcs/GcsStreamCopier.kt index 5cf12f783693..9013a119d779 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/gcs/GcsStreamCopier.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/copy/gcs/GcsStreamCopier.kt @@ -19,6 +19,7 @@ import io.airbyte.cdk.integrations.destination.jdbc.copy.StreamCopier import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.DestinationSyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.ByteArrayInputStream import java.io.IOException import java.io.InputStream @@ -31,8 +32,8 @@ import java.time.Instant import java.util.* import org.apache.commons.csv.CSVFormat import org.apache.commons.csv.CSVPrinter -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class GcsStreamCopier( protected val stagingFolder: String, @@ -104,24 +105,21 @@ abstract class GcsStreamCopier( @Throws(Exception::class) override fun closeStagingUploader(hasFailed: Boolean) { - LOGGER.info("Uploading remaining data for {} stream.", streamName) + LOGGER.info { "Uploading remaining data for $streamName stream." } for (csvPrinter in csvPrinters.values) { csvPrinter.close() } for (channel in channels.values) { channel.close() } - LOGGER.info("All data for {} stream uploaded.", streamName) + LOGGER.info { "All data for $streamName stream uploaded." } } @Throws(Exception::class) override fun copyStagingFileToTemporaryTable() { - LOGGER.info( - "Starting copy to tmp table: {} in destination for stream: {}, schema: {}.", - tmpTableName, - streamName, - schemaName - ) + LOGGER.info { + "Starting copy to tmp table: $tmpTableName in destination for stream: $streamName, schema: $schemaName." + } for (gcsStagingFile in gcsStagingFiles) { copyGcsCsvFileIntoTable( db, @@ -131,72 +129,62 @@ abstract class GcsStreamCopier( gcsConfig ) } - LOGGER.info( - "Copy to tmp table {} in destination for stream {} complete.", - tmpTableName, - streamName - ) + LOGGER.info { + "Copy to tmp table $tmpTableName in destination for stream $streamName complete." + } } @Throws(Exception::class) override fun removeFileAndDropTmpTable() { for (gcsStagingFile in gcsStagingFiles) { - LOGGER.info("Begin cleaning gcs staging file {}.", gcsStagingFile) + LOGGER.info { "Begin cleaning gcs staging file $gcsStagingFile." } val blobId = BlobId.of(gcsConfig.bucketName, gcsStagingFile) if (storageClient[blobId].exists()) { storageClient.delete(blobId) } - LOGGER.info("GCS staging file {} cleaned.", gcsStagingFile) + LOGGER.info { "GCS staging file $gcsStagingFile cleaned." } } - LOGGER.info("Begin cleaning {} tmp table in destination.", tmpTableName) + LOGGER.info { "Begin cleaning $tmpTableName tmp table in destination." } sqlOperations.dropTableIfExists(db, schemaName, tmpTableName) - LOGGER.info("{} tmp table in destination cleaned.", tmpTableName) + LOGGER.info { "$tmpTableName tmp table in destination cleaned." } } @Throws(Exception::class) override fun createDestinationSchema() { - LOGGER.info("Creating schema in destination if it doesn't exist: {}", schemaName) + LOGGER.info { "Creating schema in destination if it doesn't exist: $schemaName" } sqlOperations.createSchemaIfNotExists(db, schemaName) } @Throws(Exception::class) override fun createTemporaryTable() { - LOGGER.info( - "Preparing tmp table in destination for stream: {}, schema: {}, tmp table name: {}.", - streamName, - schemaName, - tmpTableName - ) + LOGGER.info { + "Preparing tmp table in destination for stream: $streamName, schema: $schemaName, tmp table name: $tmpTableName." + } sqlOperations.createTableIfNotExists(db, schemaName, tmpTableName) } @Throws(Exception::class) override fun createDestinationTable(): String? { val destTableName = @Suppress("deprecation") nameTransformer.getRawTableName(streamName) - LOGGER.info("Preparing table {} in destination.", destTableName) + LOGGER.info { "Preparing table $destTableName in destination." } sqlOperations.createTableIfNotExists(db, schemaName, destTableName) - LOGGER.info("Table {} in destination prepared.", tmpTableName) + LOGGER.info { "Table $tmpTableName in destination prepared." } return destTableName } @Throws(Exception::class) override fun generateMergeStatement(destTableName: String?): String { - LOGGER.info( - "Preparing to merge tmp table {} to dest table: {}, schema: {}, in destination.", - tmpTableName, - destTableName, - schemaName - ) + LOGGER.info { + "Preparing to merge tmp table $tmpTableName to dest table: $destTableName, schema: $schemaName, in destination." + } val queries = StringBuilder() if (destSyncMode == DestinationSyncMode.OVERWRITE) { queries.append(sqlOperations.truncateTableQuery(db, schemaName, destTableName)) - LOGGER.info( - "Destination OVERWRITE mode detected. Dest table: {}, schema: {}, will be truncated.", - destTableName, - schemaName - ) + LOGGER.info { + "Destination OVERWRITE mode detected. Dest table: $destTableName, schema: $schemaName, will be truncated." + } } queries.append(sqlOperations.insertTableQuery(db, schemaName, tmpTableName, destTableName)) return queries.toString() @@ -216,7 +204,6 @@ abstract class GcsStreamCopier( ) companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(GcsStreamCopier::class.java) // It is optimal to write every 10,000,000 records (BATCH_SIZE * MAX_PER_FILE_PART_COUNT) to // a new diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsDestinationAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsDestinationAcceptanceTest.kt index 78dd8c8c51e5..d9ec3d570b65 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsDestinationAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsDestinationAcceptanceTest.kt @@ -22,6 +22,7 @@ import io.airbyte.commons.jackson.MoreMappers import io.airbyte.commons.json.Jsons import io.airbyte.configoss.StandardCheckConnectionOutput import io.airbyte.protocol.models.v0.AirbyteConnectionStatus +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.util.* import org.apache.commons.lang3.RandomStringUtils @@ -30,8 +31,8 @@ import org.joda.time.DateTimeZone import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test import org.mockito.Mockito.mock -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} /** * When adding a new GCS destination acceptance test, extend this class and do the following: @@ -265,8 +266,6 @@ abstract class GcsDestinationAcceptanceTest(protected val outputFormat: FileUplo } companion object { - protected val LOGGER: Logger = - LoggerFactory.getLogger(GcsDestinationAcceptanceTest::class.java) @JvmStatic protected val MAPPER: ObjectMapper = MoreMappers.initMapper() protected const val SECRET_FILE_PATH: String = "secrets/config.json" diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/BaseS3Destination.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/BaseS3Destination.kt index 9e5c6eca1f0d..731747034d21 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/BaseS3Destination.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/BaseS3Destination.kt @@ -15,10 +15,11 @@ import io.airbyte.cdk.integrations.destination.s3.util.S3NameTransformer import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Consumer import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class BaseS3Destination protected constructor( @@ -47,7 +48,7 @@ protected constructor( return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) } catch (e: Exception) { - LOGGER.error("Exception attempting to access the S3 bucket: ", e) + LOGGER.error(e) { "Exception attempting to access the S3 bucket: " } return AirbyteConnectionStatus() .withStatus(AirbyteConnectionStatus.Status.FAILED) .withMessage( @@ -80,7 +81,5 @@ protected constructor( abstract fun storageProvider(): StorageProvider - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BaseS3Destination::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3BaseChecks.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3BaseChecks.kt index 42c8e44b73f0..1b15a8156464 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3BaseChecks.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3BaseChecks.kt @@ -8,17 +8,17 @@ import com.amazonaws.services.s3.model.ListObjectsRequest import com.google.common.annotations.VisibleForTesting import com.google.common.base.Strings import io.airbyte.cdk.integrations.destination.s3.util.StreamTransferManagerFactory.create +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.PrintWriter import java.nio.charset.StandardCharsets import java.util.* import org.apache.commons.csv.CSVFormat import org.apache.commons.csv.CSVPrinter -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object S3BaseChecks { - private val LOGGER: Logger = LoggerFactory.getLogger(S3BaseChecks::class.java) /** * Note that this method completely ignores s3Config.getBucketPath(), in favor of the bucketPath @@ -35,9 +35,9 @@ object S3BaseChecks { @JvmStatic fun testSingleUpload(s3Client: AmazonS3, bucketName: String?, bucketPath: String) { - LOGGER.info( + LOGGER.info { "Started testing if all required credentials assigned to user for single file uploading" - ) + } val prefix = if (bucketPath.endsWith("/")) bucketPath else "$bucketPath/" val testFile = prefix + "test_" + System.currentTimeMillis() try { @@ -45,15 +45,15 @@ object S3BaseChecks { } finally { s3Client.deleteObject(bucketName, testFile) } - LOGGER.info("Finished checking for normal upload mode") + LOGGER.info { "Finished checking for normal upload mode" } } @JvmStatic @Throws(IOException::class) fun testMultipartUpload(s3Client: AmazonS3, bucketName: String?, bucketPath: String) { - LOGGER.info( + LOGGER.info { "Started testing if all required credentials assigned to user for multipart upload" - ) + } val prefix = if (bucketPath.endsWith("/")) bucketPath else "$bucketPath/" val testFile = prefix + "test_" + System.currentTimeMillis() val manager = create(bucketName, testFile, s3Client).get() @@ -83,7 +83,7 @@ object S3BaseChecks { } s3Client.deleteObject(bucketName, testFile) } - LOGGER.info("Finished verification for multipart upload mode") + LOGGER.info { "Finished verification for multipart upload mode" } } /** @@ -148,9 +148,9 @@ object S3BaseChecks { } fun testIAMUserHasListObjectPermission(s3: AmazonS3, bucketName: String?) { - LOGGER.info("Started testing if IAM user can call listObjects on the destination bucket") + LOGGER.info { "Started testing if IAM user can call listObjects on the destination bucket" } val request = ListObjectsRequest().withBucketName(bucketName).withMaxKeys(1) s3.listObjects(request) - LOGGER.info("Finished checking for listObjects permission") + LOGGER.info { "Finished checking for listObjects permission" } } } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3ConsumerFactory.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3ConsumerFactory.kt index fd5a45cc3fbd..ae0f121758d2 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3ConsumerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3ConsumerFactory.kt @@ -16,13 +16,13 @@ import io.airbyte.cdk.integrations.destination.record_buffer.SerializableBuffer import io.airbyte.cdk.integrations.destination.record_buffer.SerializedBufferingStrategy import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.function.Consumer import java.util.function.Function -import org.apache.commons.io.FileUtils import org.joda.time.DateTime import org.joda.time.DateTimeZone -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class S3ConsumerFactory { fun create( @@ -53,35 +53,31 @@ class S3ConsumerFactory { writeConfigs: List ): OnStartFunction { return OnStartFunction { - LOGGER.info("Preparing bucket in destination started for {} streams", writeConfigs.size) + LOGGER.info { + "Preparing bucket in destination started for ${writeConfigs.size} streams" + } for (writeConfig in writeConfigs) { if (writeConfig.syncMode == DestinationSyncMode.OVERWRITE) { val namespace = writeConfig.namespace val stream = writeConfig.streamName val outputBucketPath = writeConfig.outputBucketPath val pathFormat = writeConfig.pathFormat - LOGGER.info( - "Clearing storage area in destination started for namespace {} stream {} bucketObject {} pathFormat {}", - namespace, - stream, - outputBucketPath, - pathFormat - ) + LOGGER.info { + "Clearing storage area in destination started for namespace $namespace " + + "stream $stream bucketObject $outputBucketPath pathFormat $pathFormat" + } storageOperations.cleanUpBucketObject( namespace, stream, outputBucketPath, pathFormat ) - LOGGER.info( - "Clearing storage area in destination completed for namespace {} stream {} bucketObject {}", - namespace, - stream, - outputBucketPath - ) + LOGGER.info { + "Clearing storage area in destination completed for namespace $namespace stream $stream bucketObject $outputBucketPath" + } } } - LOGGER.info("Preparing storage area in destination completed.") + LOGGER.info { "Preparing storage area in destination completed." } } } @@ -95,11 +91,9 @@ class S3ConsumerFactory { return FlushBufferFunction { pair: AirbyteStreamNameNamespacePair, writer: SerializableBuffer -> - LOGGER.info( - "Flushing buffer for stream {} ({}) to storage", - pair.name, - FileUtils.byteCountToDisplaySize(writer.byteCount) - ) + LOGGER.info { + "Flushing buffer for stream ${pair.name} ({FileUtils.byteCountToDisplaySize(writer.byteCount)}) to storage" + } require(pairToWriteConfig.containsKey(pair)) { String.format( "Message contained record from a stream %s that was not in the catalog. \ncatalog: %s", @@ -121,7 +115,7 @@ class S3ConsumerFactory { ) } } catch (e: Exception) { - LOGGER.error("Failed to flush and upload buffer to storage:", e) + LOGGER.error(e) { "Failed to flush and upload buffer to storage:" } throw RuntimeException("Failed to upload buffer to storage", e) } } @@ -133,7 +127,7 @@ class S3ConsumerFactory { ): OnCloseFunction { return OnCloseFunction { hasFailed: Boolean, _: Map -> if (hasFailed) { - LOGGER.info("Cleaning up destination started for {} streams", writeConfigs.size) + LOGGER.info { "Cleaning up destination started for ${writeConfigs.size} streams" } for (writeConfig in writeConfigs) { storageOperations.cleanUpBucketObject( writeConfig.fullOutputPath, @@ -141,13 +135,13 @@ class S3ConsumerFactory { ) writeConfig.clearStoredFiles() } - LOGGER.info("Cleaning up destination completed.") + LOGGER.info { "Cleaning up destination completed." } } } } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(S3ConsumerFactory::class.java) + private val SYNC_DATETIME: DateTime = DateTime.now(DateTimeZone.UTC) private fun createWriteConfigs( @@ -189,7 +183,7 @@ class S3ConsumerFactory { fullOutputPath!!, syncMode ) - LOGGER.info("Write config: {}", writeConfig) + LOGGER.info { "Write config: $writeConfig" } writeConfig } } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationConfig.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationConfig.kt index 6e6a29551b03..48ce99db7610 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationConfig.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationConfig.kt @@ -12,11 +12,11 @@ import com.amazonaws.services.s3.AmazonS3ClientBuilder import com.fasterxml.jackson.databind.JsonNode import io.airbyte.cdk.integrations.destination.s3.constant.S3Constants import io.airbyte.cdk.integrations.destination.s3.credential.* +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import javax.annotation.Nonnull -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * An S3 configuration. Typical usage sets at most one of `bucketPath` (necessary for more delicate * data syncing to S3) @@ -109,7 +109,7 @@ open class S3DestinationConfig { } protected open fun createS3Client(): AmazonS3 { - LOGGER.info("Creating S3 client...") + LOGGER.info { "Creating S3 client..." } val credentialsProvider = s3CredentialConfig!!.s3CredentialsProvider val credentialType = s3CredentialConfig.credentialType @@ -277,7 +277,7 @@ open class S3DestinationConfig { } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(S3DestinationConfig::class.java) + private const val R2_INSTANCE_URL = "https://%s.r2.cloudflarestorage.com" @JvmStatic diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/UploadFormatConfigFactory.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/UploadFormatConfigFactory.kt index 039bad09d0be..084126cce5bd 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/UploadFormatConfigFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/UploadFormatConfigFactory.kt @@ -8,16 +8,16 @@ import io.airbyte.cdk.integrations.destination.s3.avro.UploadAvroFormatConfig import io.airbyte.cdk.integrations.destination.s3.csv.UploadCsvFormatConfig import io.airbyte.cdk.integrations.destination.s3.jsonl.UploadJsonlFormatConfig import io.airbyte.cdk.integrations.destination.s3.parquet.UploadParquetFormatConfig +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object UploadFormatConfigFactory { - internal val LOGGER: Logger = LoggerFactory.getLogger(UploadFormatConfigFactory::class.java) fun getUploadFormatConfig(config: JsonNode): UploadFormatConfig { val formatConfig = config["format"] - LOGGER.info("File upload format config: {}", formatConfig.toString()) + LOGGER.info { "File upload format config: $formatConfig" } val formatType = FileUploadFormat.valueOf( formatConfig["format_type"].asText().uppercase(Locale.getDefault()) diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/S3AvroWriter.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/S3AvroWriter.kt index 0b1516354a30..ec9e545055b0 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/S3AvroWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/S3AvroWriter.kt @@ -15,6 +15,7 @@ import io.airbyte.cdk.integrations.destination.s3.writer.BaseS3Writer import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.sql.Timestamp import java.util.* @@ -22,10 +23,10 @@ import org.apache.avro.Schema import org.apache.avro.file.DataFileWriter import org.apache.avro.generic.GenericData import org.apache.avro.generic.GenericDatumWriter -import org.slf4j.Logger -import org.slf4j.LoggerFactory import tech.allegro.schema.json2avro.converter.JsonAvroConverter +private val LOGGER = KotlinLogging.logger {} + class S3AvroWriter( config: S3DestinationConfig, s3Client: AmazonS3, @@ -54,12 +55,9 @@ class S3AvroWriter( outputPath = java.lang.String.join("/", outputPrefix, outputFilename) - LOGGER.info( - "Full S3 path for stream '{}': s3://{}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { + "Full S3 path for stream '${stream.name}': s3://${config.bucketName}/$outputPath" + } fileLocation = String.format("gs://%s/%s", config.bucketName, outputPath) this.avroRecordFactory = AvroRecordFactory(schema, converter) @@ -105,7 +103,5 @@ class S3AvroWriter( dataFileWriter.append(record) } - companion object { - protected val LOGGER: Logger = LoggerFactory.getLogger(S3AvroWriter::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/S3CsvWriter.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/S3CsvWriter.kt index 19bdd0d2c773..260aa7d2d70f 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/S3CsvWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/S3CsvWriter.kt @@ -16,6 +16,7 @@ import io.airbyte.cdk.integrations.destination.s3.writer.BaseS3Writer import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.PrintWriter import java.nio.charset.StandardCharsets @@ -24,8 +25,8 @@ import java.util.* import org.apache.commons.csv.CSVFormat import org.apache.commons.csv.CSVPrinter import org.apache.commons.csv.QuoteMode -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class S3CsvWriter private constructor( @@ -63,12 +64,9 @@ private constructor( ) this.outputPath = java.lang.String.join("/", outputPrefix, outputFilename) - LOGGER.info( - "Full S3 path for stream '{}': s3://{}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { + "Full S3 path for stream '${stream.name}': s3://${config.bucketName}/$outputPath" + } fileLocation = String.format("gs://%s/%s", config.bucketName, outputPath) this.uploadManager = @@ -179,7 +177,5 @@ private constructor( csvPrinter.printRecord(csvSheetGenerator.getDataRow(formattedData)) } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(S3CsvWriter::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/jsonl/S3JsonlWriter.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/jsonl/S3JsonlWriter.kt index e0ebc203dab3..bb982eb3b167 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/jsonl/S3JsonlWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/jsonl/S3JsonlWriter.kt @@ -19,13 +19,14 @@ import io.airbyte.commons.jackson.MoreMappers import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.io.PrintWriter import java.nio.charset.StandardCharsets import java.sql.Timestamp import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class S3JsonlWriter( config: S3DestinationConfig, @@ -51,12 +52,9 @@ class S3JsonlWriter( ) outputPath = java.lang.String.join("/", outputPrefix, outputFilename) - LOGGER.info( - "Full S3 path for stream '{}': s3://{}/{}", - stream.name, - config.bucketName, - outputPath - ) + LOGGER.info { + "Full S3 path for stream '${stream.name}': s3://${config.bucketName}/$outputPath" + } fileLocation = String.format("gs://%s/%s", config.bucketName, outputPath) this.uploadManager = create(config.bucketName, outputPath, s3Client).get() @@ -95,7 +93,6 @@ class S3JsonlWriter( } companion object { - protected val LOGGER: Logger = LoggerFactory.getLogger(S3JsonlWriter::class.java) private val MAPPER: ObjectMapper = MoreMappers.initMapper() } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/S3ParquetWriter.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/S3ParquetWriter.kt index 9326782b1aab..7159fd6359fb 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/S3ParquetWriter.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/S3ParquetWriter.kt @@ -14,6 +14,7 @@ import io.airbyte.cdk.integrations.destination.s3.writer.BaseS3Writer import io.airbyte.cdk.integrations.destination.s3.writer.DestinationFileWriter import io.airbyte.protocol.models.v0.AirbyteRecordMessage import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.net.URI import java.sql.Timestamp @@ -27,10 +28,10 @@ import org.apache.parquet.avro.AvroParquetWriter import org.apache.parquet.avro.AvroWriteSupport import org.apache.parquet.hadoop.ParquetWriter import org.apache.parquet.hadoop.util.HadoopOutputFile -import org.slf4j.Logger -import org.slf4j.LoggerFactory import tech.allegro.schema.json2avro.converter.JsonAvroConverter +private val LOGGER = KotlinLogging.logger {} + class S3ParquetWriter( config: S3DestinationConfig, s3Client: AmazonS3, @@ -59,7 +60,7 @@ class S3ParquetWriter( override val fileLocation: String = String.format("s3a://%s/%s", config.bucketName, outputPath) init { - LOGGER.info("Full S3 path for stream '{}': {}", stream.name, fileLocation) + LOGGER.info { "Full S3 path for stream '${stream.name}': $fileLocation" } val path = Path(URI(fileLocation)) val formatConfig = config.formatConfig as UploadParquetFormatConfig @@ -112,7 +113,6 @@ class S3ParquetWriter( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(S3ParquetWriter::class.java) @JvmStatic fun getHadoopConfig(config: S3DestinationConfig): Configuration { diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/util/JavaProcessRunner.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/util/JavaProcessRunner.kt index 7e90344ef395..6392606e049d 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/util/JavaProcessRunner.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/util/JavaProcessRunner.kt @@ -4,24 +4,24 @@ package io.airbyte.cdk.integrations.destination.s3.util import io.airbyte.commons.io.LineGobbler +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.File import java.io.IOException import java.util.* import java.util.concurrent.TimeUnit -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} object JavaProcessRunner { - private val LOGGER: Logger = LoggerFactory.getLogger(JavaProcessRunner::class.java) @Throws(IOException::class, InterruptedException::class) fun runProcess(path: String, run: Runtime, vararg commands: String?) { - LOGGER.info("Running process: " + Arrays.asList(*commands)) + LOGGER.info { "Running process: ${listOf(*commands)}" } val pr = if (path == System.getProperty("user.dir")) run.exec(commands) else run.exec(commands, null, File(path)) - LineGobbler.gobble(`is` = pr.errorStream, { LOGGER.warn(it) }) - LineGobbler.gobble(`is` = pr.inputStream, { LOGGER.info(it) }) + LineGobbler.gobble(`is` = pr.errorStream, { LOGGER.warn { it } }) + LineGobbler.gobble(`is` = pr.inputStream, { LOGGER.info { it } }) if (!pr.waitFor(10, TimeUnit.MINUTES)) { pr.destroy() throw RuntimeException("Timeout while executing: " + commands.contentToString()) diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/BaseS3Writer.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/BaseS3Writer.kt index f78086c40907..83225ca17d0f 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/BaseS3Writer.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/BaseS3Writer.kt @@ -13,14 +13,14 @@ import io.airbyte.cdk.integrations.destination.s3.util.S3OutputPathHelper import io.airbyte.protocol.models.v0.AirbyteStream import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import io.airbyte.protocol.models.v0.DestinationSyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.text.DateFormat import java.text.SimpleDateFormat import java.util.* import org.apache.commons.lang3.StringUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * The base implementation takes care of the following: * @@ -48,13 +48,13 @@ protected constructor( try { val bucket = config.bucketName if (!s3Client.doesBucketExistV2(bucket)) { - LOGGER.info("Bucket {} does not exist; creating...", bucket) + LOGGER.info { "Bucket $bucket does not exist; creating..." } s3Client.createBucket(bucket) - LOGGER.info("Bucket {} has been created.", bucket) + LOGGER.info { "Bucket $bucket has been created." } } if (syncMode == DestinationSyncMode.OVERWRITE) { - LOGGER.info("Overwrite mode") + LOGGER.info { "Overwrite mode" } val keysToDelete: MutableList = LinkedList() val objects = s3Client.listObjects(bucket, outputPrefix).objectSummaries for (`object` in objects) { @@ -62,21 +62,18 @@ protected constructor( } if (keysToDelete.size > 0) { - LOGGER.info( - "Purging non-empty output path for stream '{}' under OVERWRITE mode...", - stream.name - ) + LOGGER.info { + "Purging non-empty output path for stream '${stream.name}' under OVERWRITE mode..." + } val result = s3Client.deleteObjects(DeleteObjectsRequest(bucket).withKeys(keysToDelete)) - LOGGER.info( - "Deleted {} file(s) for stream '{}'.", - result.deletedObjects.size, - stream.name - ) + LOGGER.info { + "Deleted ${result.deletedObjects.size} file(s) for stream '${stream.name}'." + } } } } catch (e: Exception) { - LOGGER.error("Failed to initialize: ", e) + LOGGER.error(e) { "Failed to initialize: " } closeWhenFail() throw e } @@ -86,13 +83,13 @@ protected constructor( @Throws(IOException::class) override fun close(hasFailed: Boolean) { if (hasFailed) { - LOGGER.warn("Failure detected. Aborting upload of stream '{}'...", stream.name) + LOGGER.warn { "Failure detected. Aborting upload of stream '${stream.name}'..." } closeWhenFail() - LOGGER.warn("Upload of stream '{}' aborted.", stream.name) + LOGGER.warn { "Upload of stream '${stream.name}' aborted." } } else { - LOGGER.info("Uploading remaining data for stream '{}'.", stream.name) + LOGGER.info { "Uploading remaining data for stream '${stream.name}'." } closeWhenSucceed() - LOGGER.info("Upload completed for stream '{}'.", stream.name) + LOGGER.info { "Upload completed for stream '${stream.name}'." } } } @@ -109,7 +106,6 @@ protected constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BaseS3Writer::class.java) private val s3FilenameTemplateManager = S3FilenameTemplateManager() private const val DEFAULT_SUFFIX = "_0" diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/ProductionWriterFactory.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/ProductionWriterFactory.kt index 9085350c5087..a28d56a83e83 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/ProductionWriterFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/writer/ProductionWriterFactory.kt @@ -13,9 +13,10 @@ import io.airbyte.cdk.integrations.destination.s3.csv.S3CsvWriter import io.airbyte.cdk.integrations.destination.s3.jsonl.S3JsonlWriter import io.airbyte.cdk.integrations.destination.s3.parquet.S3ParquetWriter import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.sql.Timestamp -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class ProductionWriterFactory : S3WriterFactory { @Throws(Exception::class) @@ -29,13 +30,13 @@ class ProductionWriterFactory : S3WriterFactory { if (format == FileUploadFormat.AVRO || format == FileUploadFormat.PARQUET) { val stream = configuredStream.stream - LOGGER.info("Json schema for stream {}: {}", stream.name, stream.jsonSchema) + LOGGER.info { "Json schema for stream ${stream.name}: ${stream.jsonSchema}" } val schemaConverter = JsonToAvroSchemaConverter() val avroSchema = schemaConverter.getAvroSchema(stream.jsonSchema, stream.name, stream.namespace) - LOGGER.info("Avro schema for stream {}: {}", stream.name, avroSchema.toString(false)) + LOGGER.info { "Avro schema for stream ${stream.name}: ${avroSchema.toString(false)}" } return if (format == FileUploadFormat.AVRO) { S3AvroWriter( @@ -69,7 +70,5 @@ class ProductionWriterFactory : S3WriterFactory { throw RuntimeException("Unexpected S3 destination format: $format") } - companion object { - protected val LOGGER: Logger = LoggerFactory.getLogger(ProductionWriterFactory::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/StagingConsumerFactory.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/StagingConsumerFactory.kt index 007e30d433c9..93acb51c45e2 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/StagingConsumerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/StagingConsumerFactory.kt @@ -19,19 +19,20 @@ import io.airbyte.cdk.integrations.destination.jdbc.WriteConfig import io.airbyte.commons.exceptions.ConfigErrorException import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog import io.airbyte.integrations.base.destination.typing_deduping.TyperDeduper +import io.airbyte.protocol.models.v0.* import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteStream import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import io.airbyte.protocol.models.v0.StreamDescriptor +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Instant import java.util.Optional import java.util.concurrent.Executors import java.util.function.Consumer import java.util.function.Function -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * Uses both Factory and Consumer design pattern to create a single point of creation for consuming * [AirbyteMessage] for processing @@ -156,7 +157,6 @@ private constructor( } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(StagingConsumerFactory::class.java) private val SYNC_DATETIME: Instant = Instant.now() @@ -301,7 +301,7 @@ private constructor( syncMode, SYNC_DATETIME ) - LOGGER.info("Write config: {}", writeConfig) + LOGGER.info { "Write config: $writeConfig" } writeConfig } } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationAcceptanceTest.kt index 2fbf1213db21..c9bb00f694bf 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3DestinationAcceptanceTest.kt @@ -17,15 +17,15 @@ import io.airbyte.cdk.integrations.standardtest.destination.comparator.TestDataC import io.airbyte.commons.io.IOs import io.airbyte.commons.jackson.MoreMappers import io.airbyte.commons.json.Jsons +import io.github.oshai.kotlinlogging.KotlinLogging import java.nio.file.Path import java.util.* import org.apache.commons.lang3.RandomStringUtils import org.joda.time.DateTime import org.joda.time.DateTimeZone import org.mockito.Mockito.mock -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * When adding a new S3 destination acceptance test, extend this class and do the following: * * Implement [.getFormatConfig] that returns a [UploadFormatConfig] @@ -185,8 +185,7 @@ protected constructor(protected val outputFormat: FileUploadFormat) : Destinatio } companion object { - protected val LOGGER: Logger = - LoggerFactory.getLogger(S3DestinationAcceptanceTest::class.java) + @JvmStatic protected val MAPPER: ObjectMapper = MoreMappers.initMapper() } } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteProtocolType.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteProtocolType.kt index ce79d8cfcb99..3350658124a7 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteProtocolType.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteProtocolType.kt @@ -4,8 +4,10 @@ package io.airbyte.integrations.base.destination.typing_deduping import com.fasterxml.jackson.databind.JsonNode +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* +private val LOGGER = KotlinLogging.logger {} /** * Protocol types are ordered by precedence in the case of a Union that contains multiple types. * Priority is given to wider scope types over narrower ones. (Note that because of dedup logic in @@ -31,13 +33,7 @@ enum class AirbyteProtocolType : AirbyteType { try { return valueOf(type.uppercase(Locale.getDefault())) } catch (e: IllegalArgumentException) { - AirbyteType.Companion.LOGGER.error( - String.format( - "Could not find matching AirbyteProtocolType for \"%s\": %s", - type, - e - ) - ) + LOGGER.error { "Could not find matching AirbyteProtocolType for \"$type\": $e" } return UNKNOWN } } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.kt index e0d7412c4c21..707f358cda78 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/AirbyteType.kt @@ -6,8 +6,9 @@ package io.airbyte.integrations.base.destination.typing_deduping import com.fasterxml.jackson.databind.JsonNode import com.fasterxml.jackson.databind.node.ObjectNode import com.fasterxml.jackson.databind.node.TextNode -import org.slf4j.Logger -import org.slf4j.LoggerFactory +import io.github.oshai.kotlinlogging.KotlinLogging + +private val LOGGER = KotlinLogging.logger {} interface AirbyteType { val typeName: String @@ -49,7 +50,7 @@ interface AirbyteType { } return AirbyteProtocolType.Companion.fromJson(schema) } catch (e: Exception) { - LOGGER.error("Exception parsing JSON schema {}: {}; returning UNKNOWN.", schema, e) + LOGGER.error { "Exception parsing JSON schema $schema: $e; returning UNKNOWN." } return AirbyteProtocolType.UNKNOWN } } @@ -121,7 +122,5 @@ interface AirbyteType { (schemaClone as ObjectNode).put("type", type) return schemaClone } - - val LOGGER: Logger = LoggerFactory.getLogger(AirbyteType::class.java) } } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseDestinationV1V2Migrator.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseDestinationV1V2Migrator.kt index 52da4e7b1bf4..268ab985bf2e 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseDestinationV1V2Migrator.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseDestinationV1V2Migrator.kt @@ -5,9 +5,10 @@ package io.airbyte.integrations.base.destination.typing_deduping import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.protocol.models.v0.DestinationSyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} abstract class BaseDestinationV1V2Migrator : DestinationV1V2Migrator { @Throws(Exception::class) @@ -16,19 +17,18 @@ abstract class BaseDestinationV1V2Migrator : Destination destinationHandler: DestinationHandler<*>, streamConfig: StreamConfig ) { - LOGGER.info( - "Assessing whether migration is necessary for stream {}", - streamConfig.id.finalName - ) + LOGGER.info { + "Assessing whether migration is necessary for stream ${streamConfig.id.finalName}" + } + if (shouldMigrate(streamConfig)) { - LOGGER.info("Starting v2 Migration for stream {}", streamConfig.id.finalName) + LOGGER.info { "Starting v2 Migration for stream ${streamConfig.id.finalName}" } migrate(sqlGenerator, destinationHandler, streamConfig) - LOGGER.info( - "V2 Migration completed successfully for stream {}", - streamConfig.id.finalName - ) + LOGGER.info { + "V2 Migration completed successfully for stream ${streamConfig.id.finalName}" + } } else { - LOGGER.info("No Migration Required for stream: {}", streamConfig.id.finalName) + LOGGER.info { "No Migration Required for stream: ${streamConfig.id.finalName}" } } } @@ -41,22 +41,19 @@ abstract class BaseDestinationV1V2Migrator : Destination @Throws(Exception::class) fun shouldMigrate(streamConfig: StreamConfig): Boolean { val v1RawTable = convertToV1RawName(streamConfig) - LOGGER.info( - "Checking whether v1 raw table {} in dataset {} exists", - v1RawTable.tableName, - v1RawTable.namespace - ) + LOGGER.info { + "Checking whether v1 raw table ${v1RawTable.tableName} in dataset ${v1RawTable.namespace} exists" + } val syncModeNeedsMigration = isMigrationRequiredForSyncMode(streamConfig.destinationSyncMode) val noValidV2RawTableExists = !doesValidV2RawTableAlreadyExist(streamConfig) val aValidV1RawTableExists = doesValidV1RawTableExist(v1RawTable.namespace, v1RawTable.tableName) - LOGGER.info( - "Migration Info: Required for Sync mode: {}, No existing v2 raw tables: {}, A v1 raw table exists: {}", - syncModeNeedsMigration, - noValidV2RawTableExists, - aValidV1RawTableExists - ) + LOGGER.info { + "Migration Info: Required for Sync mode: $syncModeNeedsMigration, " + + "No existing v2 raw tables: $noValidV2RawTableExists, " + + "A v1 raw table exists: $aValidV1RawTableExists" + } return syncModeNeedsMigration && noValidV2RawTableExists && aValidV1RawTableExists } @@ -219,9 +216,4 @@ abstract class BaseDestinationV1V2Migrator : Destination * @return the valid v1 name and namespace for the same stream */ abstract fun convertToV1RawName(streamConfig: StreamConfig): NamespacedTableName - - companion object { - protected val LOGGER: Logger = - LoggerFactory.getLogger(BaseDestinationV1V2Migrator::class.java) - } } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.kt index f92cd7712246..07ae4123d338 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CatalogParser.kt @@ -8,11 +8,12 @@ import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addStr import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.Optional import java.util.function.Consumer import org.apache.commons.codec.digest.DigestUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory + +private val LOGGER = KotlinLogging.logger {} class CatalogParser @JvmOverloads @@ -40,11 +41,7 @@ constructor( val originalNamespace = stream.stream.namespace val originalName = stream.stream.name - LOGGER.info( - "Detected table name collision for {}.{}", - originalNamespace, - originalName - ) + LOGGER.info { "Detected table name collision for $originalNamespace.$originalName" } // ... this logic is ported from legacy normalization, and maybe should change? // We're taking a hash of the quoted namespace and the unquoted stream name @@ -106,7 +103,7 @@ constructor( } ) } - LOGGER.info("Running sync with stream configs: $streamConfigs") + LOGGER.info { "Running sync with stream configs: $streamConfigs" } return ParsedCatalog(streamConfigs) } @@ -184,12 +181,9 @@ constructor( // as-is. columnId = originalColumnId } else { - LOGGER.info( - "Detected column name collision for {}.{}.{}", - stream.stream.namespace, - stream.stream.name, - key, - ) + LOGGER.info { + "Detected column name collision for ${stream.stream.namespace}.${stream.stream.name}.$key" + } // One of the existing columns has the same name. We need to handle this collision. // Append _1, _2, _3, ... to the column name until we find one that doesn't collide. var i = 1 @@ -279,7 +273,5 @@ constructor( return newColumnId } - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(CatalogParser::class.java) - } + companion object {} } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/DefaultTyperDeduper.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/DefaultTyperDeduper.kt index 0bca4473a567..6a646d531b69 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/DefaultTyperDeduper.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/DefaultTyperDeduper.kt @@ -14,15 +14,15 @@ import io.airbyte.integrations.base.destination.typing_deduping.migrators.Migrat import io.airbyte.integrations.base.destination.typing_deduping.migrators.MinimumDestinationState import io.airbyte.protocol.models.v0.DestinationSyncMode import io.airbyte.protocol.models.v0.StreamDescriptor +import io.github.oshai.kotlinlogging.KotlinLogging import java.util.* import java.util.concurrent.* import java.util.function.Supplier import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.concurrent.BasicThreadFactory import org.apache.commons.lang3.tuple.Pair -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * An abstraction over SqlGenerator and DestinationHandler. Destinations will still need to call * `new CatalogParser(new FooSqlGenerator()).parseCatalog()`, but should otherwise avoid interacting @@ -118,7 +118,7 @@ class DefaultTyperDeduper( override fun prepareFinalTables() { check(!::overwriteStreamsWithTmpTable.isInitialized) { "Tables were already prepared." } overwriteStreamsWithTmpTable = ConcurrentHashMap.newKeySet() - LOGGER.info("Preparing tables") + LOGGER.info { "Preparing tables" } val prepareTablesFutureResult = CompletableFutures.allOf( @@ -151,7 +151,7 @@ class DefaultTyperDeduper( val stream = initialState.streamConfig try { if (initialState.isFinalTablePresent) { - LOGGER.info("Final Table exists for stream {}", stream.id.finalName) + LOGGER.info { "Final Table exists for stream ${stream.id.finalName}" } // The table already exists. Decide whether we're writing to it directly, or // using a tmp table. if (stream.destinationSyncMode == DestinationSyncMode.OVERWRITE) { @@ -168,15 +168,14 @@ class DefaultTyperDeduper( true ) ) - LOGGER.info( - "Using temp final table for stream {}, will overwrite existing table at end of sync", - stream.id.finalName - ) + LOGGER.info { + "Using temp final table for stream ${stream.id.finalName}, will overwrite existing table at end of sync" + } } else { - LOGGER.info( - "Final Table for stream {} is empty and matches the expected v2 format, writing to table directly", - stream.id.finalName - ) + LOGGER.info { + "Final Table for stream ${stream.id.finalName} is empty and matches the expected v2 format, " + + "writing to table directly" + } } } else if ( initialState.isSchemaMismatch || @@ -193,10 +192,9 @@ class DefaultTyperDeduper( ) } } else { - LOGGER.info( - "Final Table does not exist for stream {}, creating.", - stream.id.finalName - ) + LOGGER.info { + "Final Table does not exist for stream ${stream.id.finalName}, creating." + } // The table doesn't exist. Create it. Don't force. destinationHandler.execute( sqlGenerator.createTable(stream, NO_SUFFIX, false) @@ -211,11 +209,9 @@ class DefaultTyperDeduper( return@supplyAsync } catch (e: Exception) { - LOGGER.error( - "Exception occurred while preparing tables for stream " + - stream.id.originalName, - e - ) + LOGGER.error(e) { + "Exception occurred while preparing tables for stream ${stream.id.originalName}" + } throw RuntimeException(e) } }, @@ -229,11 +225,7 @@ class DefaultTyperDeduper( val task = typeAndDedupeTask(streamConfig) FutureUtils.reduceExceptions( setOf(task), - String.format( - "The Following Exceptions were thrown while typing and deduping %s.%s:\n", - originalNamespace, - originalName - ) + "The Following Exceptions were thrown while typing and deduping ${originalNamespace}.${originalName}:\n", ) } @@ -244,11 +236,9 @@ class DefaultTyperDeduper( // For example, if T+D setup fails, but the consumer tries to run T+D on all streams // during close, // we should skip it. - LOGGER.warn( - "Skipping typing and deduping for {}.{} because we could not set up the tables for this stream.", - originalNamespace, - originalName - ) + LOGGER.warn { + "Skipping typing and deduping for $originalNamespace.$originalName because we could not set up the tables for this stream." + } return false } return true @@ -276,10 +266,9 @@ class DefaultTyperDeduper( ) return@supplyAsync Optional.empty() } catch (e: Exception) { - LOGGER.error( - "Exception occurred while typing and deduping stream $originalName", - e - ) + LOGGER.error(e) { + "Exception occurred while typing and deduping stream $originalName" + } return@supplyAsync Optional.of(e) } }, @@ -289,7 +278,7 @@ class DefaultTyperDeduper( @Throws(Exception::class) override fun typeAndDedupe(streamSyncSummaries: Map) { - LOGGER.info("Typing and deduping all tables") + LOGGER.info { "Typing and deduping all tables" } val typeAndDedupeTasks: MutableSet>> = HashSet() parsedCatalog.streams .filter { streamConfig: StreamConfig -> @@ -318,11 +307,9 @@ class DefaultTyperDeduper( // Run T+D if either of those conditions are true. val shouldRunTypingDeduping = nonzeroRecords || unprocessedRecordsPreexist if (!shouldRunTypingDeduping) { - LOGGER.info( - "Skipping typing and deduping for stream {}.{} because it had no records during this sync and no unprocessed records from a previous sync.", - streamConfig.id.originalNamespace, - streamConfig.id.originalName - ) + LOGGER.info { + "Skipping typing and deduping for stream ${streamConfig.id.originalNamespace}.${streamConfig.id.originalName} because it had no records during this sync and no unprocessed records from a previous sync." + } } shouldRunTypingDeduping } @@ -344,7 +331,7 @@ class DefaultTyperDeduper( */ @Throws(Exception::class) override fun commitFinalTables() { - LOGGER.info("Committing final tables") + LOGGER.info { "Committing final tables" } val tableCommitTasks: MutableSet>> = HashSet() for (streamConfig in parsedCatalog.streams) { if ( @@ -352,11 +339,10 @@ class DefaultTyperDeduper( Pair.of(streamConfig.id.originalNamespace, streamConfig.id.originalName) ) ) { - LOGGER.warn( - "Skipping committing final table for for {}.{} because we could not set up the tables for this stream.", - streamConfig.id.originalNamespace, - streamConfig.id.originalName - ) + LOGGER.warn { + "Skipping committing final table for for ${streamConfig.id.originalNamespace}.${streamConfig.id.originalName} " + + "because we could not set up the tables for this stream." + } continue } if (DestinationSyncMode.OVERWRITE == streamConfig.destinationSyncMode) { @@ -380,19 +366,15 @@ class DefaultTyperDeduper( if (!StringUtils.isEmpty(finalSuffix)) { val overwriteFinalTable = sqlGenerator.overwriteFinalTable(streamId, finalSuffix) - LOGGER.info( - "Overwriting final table with tmp table for stream {}.{}", - streamId.originalNamespace, - streamId.originalName - ) + LOGGER.info { + "Overwriting final table with tmp table for stream ${streamId.originalNamespace}.${streamId.originalName}" + } try { destinationHandler.execute(overwriteFinalTable) } catch (e: Exception) { - LOGGER.error( - "Exception Occurred while committing final table for stream " + - streamId.originalName, - e - ) + LOGGER.error(e) { + "Exception Occurred while committing final table for stream ${streamId.originalName}" + } return@supplyAsync Optional.of(e) } } @@ -408,12 +390,11 @@ class DefaultTyperDeduper( } override fun cleanup() { - LOGGER.info("Cleaning Up type-and-dedupe thread pool") + LOGGER.info { "Cleaning Up type-and-dedupe thread pool" } executorService.shutdown() } companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(TyperDeduper::class.java) private const val NO_SUFFIX = "" private const val TMP_OVERWRITE_TABLE_SUFFIX = "_airbyte_tmp" diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/TyperDeduperUtil.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/TyperDeduperUtil.kt index 076547cdc3e3..c1e409fb6632 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/TyperDeduperUtil.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/TyperDeduperUtil.kt @@ -47,12 +47,7 @@ object TyperDeduperUtil { Map>> = currentStates.associate { it.streamConfig.id to - runMigrationsAsync( - executorService, - destinationHandler, - migration, - it, - ) + runMigrationsAsync(executorService, destinationHandler, migration, it) } val migrationResultFutures = CompletableFutures.allOf(futures.values.toList()).toCompletableFuture().join() diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Union.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Union.kt index c919bfe48812..06d5be1a45e9 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Union.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Union.kt @@ -3,6 +3,9 @@ */ package io.airbyte.integrations.base.destination.typing_deduping +import io.github.oshai.kotlinlogging.KotlinLogging + +private val LOGGER = KotlinLogging.logger {} /** * Represents a {type: [a, b, ...]} schema. This is theoretically equivalent to {oneOf: [{type: a}, * {type: b}, ...]} but legacy normalization only handles the {type: [...]} schemas. @@ -25,12 +28,10 @@ data class Union(val options: List) : AirbyteType { * @throws IllegalArgumentException if we cannot extract columns from this schema */ fun asColumns(): LinkedHashMap { - AirbyteType.LOGGER.warn("asColumns options=$options") + LOGGER.warn { "asColumns options=$options" } val numObjectOptions = options.filterIsInstance().count() if (numObjectOptions > 1) { - AirbyteType.LOGGER.error( - "Can't extract columns from a schema with multiple object options" - ) + LOGGER.error { "Can't extract columns from a schema with multiple object options" } return LinkedHashMap() } @@ -38,10 +39,10 @@ data class Union(val options: List) : AirbyteType { try { retVal = options.filterIsInstance().first().properties } catch (_: NoSuchElementException) { - AirbyteType.LOGGER.error("Can't extract columns from a schema with no object options") + LOGGER.error { "Can't extract columns from a schema with no object options" } retVal = LinkedHashMap() } - AirbyteType.LOGGER.warn("asColumns retVal=$retVal") + LOGGER.warn { "asColumns retVal=$retVal" } return retVal } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseSqlGeneratorIntegrationTest.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseSqlGeneratorIntegrationTest.kt index 3245cd02cfee..d5e12cbfca20 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseSqlGeneratorIntegrationTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseSqlGeneratorIntegrationTest.kt @@ -14,6 +14,7 @@ import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import io.airbyte.protocol.models.v0.DestinationSyncMode import io.airbyte.protocol.models.v0.SyncMode +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Instant import java.util.* import java.util.function.Consumer @@ -32,9 +33,8 @@ import org.junit.jupiter.api.parallel.ExecutionMode import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource import org.mockito.kotlin.mock -import org.slf4j.Logger -import org.slf4j.LoggerFactory +private val LOGGER = KotlinLogging.logger {} /** * This class exercises [SqlGenerator] implementations. All destinations should extend this class * for their respective implementation. Subclasses are encouraged to add additional tests with @@ -261,7 +261,7 @@ abstract class BaseSqlGeneratorIntegrationTest