From 741c4e074568c535658140f896f63eaf5b43ad13 Mon Sep 17 00:00:00 2001 From: Stephane Geneix Date: Wed, 24 Apr 2024 06:54:48 -0700 Subject: [PATCH] remove calls to java stream() --- .../base/AirbyteExceptionHandler.kt | 22 +- .../integrations/base/IntegrationRunner.kt | 11 +- .../destination/StandardNameTransformer.kt | 1 - .../destination/async/AsyncStreamConsumer.kt | 16 +- .../destination/async/DetectStreamToFlush.kt | 41 +-- .../destination/async/FlushWorkers.kt | 39 +-- .../async/buffers/BufferDequeue.kt | 7 +- .../async/buffers/MemoryAwareMessageBatch.kt | 8 +- .../async/state/GlobalAsyncStateManager.kt | 14 +- .../DestStreamStateLifecycleManager.kt | 42 ++- .../util/ConnectorExceptionUtil.kt | 10 +- .../concurrent/ConcurrentStreamConsumer.kt | 1 - .../cdk/db/jdbc/TestStreamingJdbcDatabase.kt | 2 +- .../base/IntegrationRunnerTest.kt | 13 +- .../async/AsyncStreamConsumerTest.kt | 33 ++- .../state/GlobalAsyncStateManagerTest.kt | 244 ++++-------------- .../BufferedStreamConsumerTest.kt | 20 +- .../ConcurrentStreamConsumerTest.kt | 2 +- .../base/ssh/SshBastionContainer.kt | 9 +- .../cdk/integrations/util/HostPortResolver.kt | 9 +- .../io/airbyte/cdk/db/mongodb/MongoUtils.kt | 2 - .../jdbc/JdbcBufferedConsumerFactory.kt | 7 +- .../typing_deduping/JdbcDestinationHandler.kt | 44 ++-- .../jdbc/typing_deduping/JdbcSqlGenerator.kt | 36 +-- .../destination/staging/SerialFlush.kt | 8 +- .../staging/SerialStagingConsumerFactory.kt | 8 +- .../destination/DestinationAcceptanceTest.kt | 55 +--- .../comparator/AdvancedTestDataComparator.kt | 13 +- .../JdbcSqlGeneratorIntegrationTest.kt | 2 - .../debezium/AirbyteDebeziumHandler.kt | 3 +- .../AirbyteFileOffsetBackingStore.kt | 30 +-- .../RelationalDbDebeziumPropertiesManager.kt | 15 +- .../source/jdbc/AbstractJdbcSource.kt | 43 ++- .../source/relationaldb/AbstractDbSource.kt | 35 +-- .../relationaldb/DbSourceDiscoverUtil.kt | 46 ++-- .../relationaldb/InvalidCursorInfoUtil.kt | 7 +- .../relationaldb/RelationalDbQueryUtils.kt | 8 +- .../relationaldb/RelationalDbReadUtil.kt | 2 - .../state/AbstractStateManager.kt | 2 +- .../relationaldb/state/CursorManager.kt | 29 +-- .../relationaldb/state/GlobalStateManager.kt | 6 +- .../relationaldb/state/StateGeneratorUtils.kt | 8 +- .../relationaldb/state/StreamStateManager.kt | 2 +- .../state/GlobalStateManagerTest.kt | 20 +- .../state/LegacyStateManagerTest.kt | 20 +- .../state/StreamStateManagerTest.kt | 12 +- .../integrations/debezium/CdcSourceTest.kt | 18 +- .../jdbc/test/JdbcSourceAcceptanceTest.kt | 105 +++----- .../source/AbstractSourceDatabaseTypeTest.kt | 22 +- .../source/PythonSourceAcceptanceTest.kt | 9 +- .../source/SourceAcceptanceTest.kt | 6 +- .../standardtest/source/TestDataHolder.kt | 4 +- .../standardtest/source/TestEnvConfigs.kt | 30 +-- .../AbstractSourcePerformanceTest.kt | 13 +- .../io/airbyte/commons/json/JsonPaths.kt | 1 - .../io/airbyte/commons/json/JsonSchemas.kt | 3 +- .../configoss/helpers/StateMessageHelper.kt | 4 +- .../validation/json/JsonSchemaValidator.kt | 2 - .../concurrency/CompletableFuturesTest.kt | 4 +- .../io/airbyte/commons/json/JsonPathsTest.kt | 5 +- .../io/airbyte/workers/TestHarnessUtils.kt | 37 +-- .../DefaultCheckConnectionTestHarness.kt | 9 +- .../DefaultDiscoverCatalogTestHarness.kt | 7 +- .../general/DefaultGetSpecTestHarness.kt | 7 +- .../workers/helper/CatalogClientConverters.kt | 2 - .../airbyte/workers/helper/FailureHelper.kt | 2 +- .../DefaultNormalizationRunner.kt | 12 +- ...GcsAvroParquetDestinationAcceptanceTest.kt | 62 ++--- .../gcs/GcsDestinationAcceptanceTest.kt | 11 +- .../destination/s3/S3ConsumerFactory.kt | 13 +- .../destination/s3/S3StorageOperations.kt | 12 +- .../s3/avro/AvroSerializedBuffer.kt | 15 +- .../s3/avro/JsonToAvroSchemaConverter.kt | 49 ++-- .../destination/s3/csv/CsvSerializedBuffer.kt | 23 +- .../csv/RootLevelFlatteningSheetGenerator.kt | 1 - .../s3/parquet/ParquetSerializedBuffer.kt | 12 +- .../staging/StagingConsumerFactory.kt | 21 +- .../s3/avro/JsonFieldNameUpdaterTest.kt | 12 +- .../s3/avro/JsonToAvroConverterTest.kt | 80 +++--- .../S3AvroParquetDestinationAcceptanceTest.kt | 64 ++--- .../s3/S3DestinationAcceptanceTest.kt | 4 +- .../typing_deduping/AirbyteType.kt | 1 - .../typing_deduping/CatalogParser.kt | 21 +- .../typing_deduping/CollectionUtils.kt | 9 +- .../typing_deduping/DefaultTyperDeduper.kt | 1 - .../typing_deduping/FutureUtils.kt | 1 - .../typing_deduping/ParsedCatalog.kt | 42 ++- .../base/destination/typing_deduping/Sql.kt | 19 +- .../typing_deduping/TyperDeduperUtil.kt | 20 +- .../BaseSqlGeneratorIntegrationTest.kt | 22 +- .../typing_deduping/BaseTypingDedupingTest.kt | 34 +-- .../typing_deduping/RecordDiffer.kt | 10 +- 92 files changed, 561 insertions(+), 1307 deletions(-) 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 de346a65b6ac..2c68a7ca0257 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 @@ -42,25 +42,21 @@ class AirbyteExceptionHandler : Thread.UncaughtExceptionHandler { // message. // This assumes that any wrapping exceptions are just noise (e.g. runtime exception). val deinterpolatableException = - ExceptionUtils.getThrowableList(throwable) - .stream() - .filter { t: Throwable -> - THROWABLES_TO_DEINTERPOLATE.stream().anyMatch { - deinterpolatableClass: Class -> - deinterpolatableClass.isAssignableFrom(t.javaClass) - } + ExceptionUtils.getThrowableList(throwable).firstOrNull { t: Throwable -> + THROWABLES_TO_DEINTERPOLATE.any { deinterpolatableClass: Class -> + deinterpolatableClass.isAssignableFrom(t.javaClass) } - .findFirst() + } val messageWasMangled: Boolean - if (deinterpolatableException.isPresent) { - val originalMessage = deinterpolatableException.get().message + if (deinterpolatableException != null) { + val originalMessage = deinterpolatableException.message mangledMessage = STRINGS_TO_DEINTERPOLATE - .stream() // Sort the strings longest to shortest, in case any target string is + // Sort the strings longest to shortest, in case any target string is // a substring of another // e.g. "airbyte_internal" should be swapped out before "airbyte" - .sorted(Comparator.comparing { obj: String -> obj.length }.reversed()) - .reduce(originalMessage) { message: String?, targetString: String? -> + .sortedWith(Comparator.comparing { obj: String -> obj.length }.reversed()) + .fold(originalMessage) { message: String?, targetString: String? -> deinterpolate(message, targetString) } messageWasMangled = mangledMessage != originalMessage 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 4682f51e43da..3287df5f0f93 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 @@ -30,7 +30,6 @@ import java.nio.file.Path import java.util.* import java.util.concurrent.* import java.util.function.Consumer -import java.util.function.Predicate import org.apache.commons.lang3.ThreadUtils import org.apache.commons.lang3.concurrent.BasicThreadFactory import org.slf4j.Logger @@ -280,7 +279,7 @@ internal constructor( * stream consumer. */ val partitionSize = streamConsumer.parallelism - val partitions = Lists.partition(streams.stream().toList(), partitionSize) + val partitions = Lists.partition(streams.toList(), partitionSize) // Submit each stream partition for concurrent execution partitions.forEach( @@ -352,11 +351,10 @@ internal constructor( * active so long as the database connection pool is open. */ @VisibleForTesting - val ORPHANED_THREAD_FILTER: Predicate = Predicate { runningThread: Thread -> + fun filterOrphanedThread(runningThread: Thread): Boolean = (runningThread.name != Thread.currentThread().name && !runningThread.isDaemon && TYPE_AND_DEDUPE_THREAD_NAME != runningThread.name) - } const val INTERRUPT_THREAD_DELAY_MINUTES: Int = 1 const val EXIT_THREAD_DELAY_MINUTES: Int = 2 @@ -424,8 +422,7 @@ internal constructor( ) { val currentThread = Thread.currentThread() - val runningThreads = - ThreadUtils.getAllThreads().stream().filter(ORPHANED_THREAD_FILTER).toList() + val runningThreads = ThreadUtils.getAllThreads().filter(::filterOrphanedThread).toList() if (runningThreads.isNotEmpty()) { LOGGER.warn( """ @@ -461,7 +458,7 @@ internal constructor( scheduledExecutorService.schedule( { if ( - ThreadUtils.getAllThreads().stream().anyMatch { runningThread: Thread -> + ThreadUtils.getAllThreads().any { runningThread: Thread -> !runningThread.isDaemon && runningThread.name != currentThread.name } ) { diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/StandardNameTransformer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/StandardNameTransformer.kt index e403a688f30b..106160b776fb 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/StandardNameTransformer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/StandardNameTransformer.kt @@ -74,7 +74,6 @@ open class StandardNameTransformer : NamingConventionTransformer { } else if (root.isArray) { return Jsons.jsonNode( MoreIterators.toList(root.elements()) - .stream() .map { r: JsonNode -> formatJsonPath(r) } .toList() ) diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumer.kt index 291a4606ebda..5de1126fa29e 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumer.kt @@ -28,7 +28,6 @@ import java.util.concurrent.ExecutorService import java.util.concurrent.Executors import java.util.concurrent.atomic.AtomicLong import java.util.function.Consumer -import java.util.stream.Collectors import kotlin.jvm.optionals.getOrNull private val logger = KotlinLogging.logger {} @@ -155,18 +154,11 @@ class AsyncStreamConsumer( bufferManager.close() val streamSyncSummaries = - streamNames - .stream() - .collect( - Collectors.toMap( - { streamDescriptor: StreamDescriptor -> streamDescriptor }, - { streamDescriptor: StreamDescriptor -> - StreamSyncSummary( - Optional.of(getRecordCounter(streamDescriptor).get()), - ) - }, - ), + streamNames.associateWith { streamDescriptor: StreamDescriptor -> + StreamSyncSummary( + Optional.of(getRecordCounter(streamDescriptor).get()), ) + } onClose.accept(hasFailed, streamSyncSummaries) // as this throws an exception, we need to be after all other close functions. diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/DetectStreamToFlush.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/DetectStreamToFlush.kt index 69d8fc255c19..92e43b748b6d 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/DetectStreamToFlush.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/DetectStreamToFlush.kt @@ -15,7 +15,6 @@ import java.util.Optional import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentMap import java.util.concurrent.atomic.AtomicBoolean -import java.util.stream.Collectors import kotlin.math.min private val logger = KotlinLogging.logger {} @@ -194,12 +193,10 @@ internal constructor( ) val workersWithBatchesSize = runningWorkerBatchesSizes - .stream() .filter { obj: Optional -> obj.isPresent } - .mapToLong { obj: Optional -> obj.get() } - .sum() + .sumOf { obj: Optional -> obj.get() } val workersWithoutBatchesCount = - runningWorkerBatchesSizes.stream().filter { obj: Optional -> obj.isEmpty }.count() + runningWorkerBatchesSizes.count { obj: Optional -> obj.isEmpty } val workersWithoutBatchesSizeEstimate = (min( flusher.optimalBatchSizeBytes.toDouble(), @@ -232,36 +229,20 @@ internal constructor( fun orderStreamsByPriority(streams: Set): List { // eagerly pull attributes so that values are consistent throughout comparison val sdToQueueSize = - streams - .stream() - .collect( - Collectors.toMap( - { s: StreamDescriptor -> s }, - { streamDescriptor: StreamDescriptor -> - bufferDequeue.getQueueSizeBytes( - streamDescriptor, - ) - }, - ), + streams.associateWith { streamDescriptor: StreamDescriptor -> + bufferDequeue.getQueueSizeBytes( + streamDescriptor, ) + } val sdToTimeOfLastRecord = - streams - .stream() - .collect( - Collectors.toMap( - { s: StreamDescriptor -> s }, - { streamDescriptor: StreamDescriptor -> - bufferDequeue.getTimeOfLastRecord( - streamDescriptor, - ) - }, - ), + streams.associateWith { streamDescriptor: StreamDescriptor -> + bufferDequeue.getTimeOfLastRecord( + streamDescriptor, ) - + } return streams - .stream() - .sorted( + .sortedWith( Comparator.comparing( { s: StreamDescriptor -> sdToQueueSize[s]!!.orElseThrow() }, Comparator.reverseOrder(), diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/FlushWorkers.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/FlushWorkers.kt index aa4d0baae060..52377b504b9a 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/FlushWorkers.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/FlushWorkers.kt @@ -20,7 +20,6 @@ import java.util.concurrent.ThreadPoolExecutor import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import java.util.function.Consumer -import java.util.stream.Collectors private val logger = KotlinLogging.logger {} @@ -153,14 +152,10 @@ constructor( ) val stateIdToCount = batch.data - .stream() .map(StreamAwareQueue.MessageWithMeta::stateId) - .collect( - Collectors.groupingBy( - { stateId: Long -> stateId }, - Collectors.counting(), - ), - ) + .groupingBy { it } + .eachCount() + .mapValues { it.value.toLong() } logger.info { "Flush Worker (${humanReadableFlushWorkerId( flushWorkerId, @@ -169,14 +164,7 @@ constructor( )} bytes." } - flusher.flush( - desc, - batch.data - .stream() - .map( - StreamAwareQueue.MessageWithMeta::message, - ), - ) + flusher.flush(desc, batch.data.map { it.message }.stream()) batch.flushStates(stateIdToCount, outputRecordCollector) } logger.info { @@ -207,22 +195,12 @@ constructor( // wait for all buffers to be flushed. while (true) { val streamDescriptorToRemainingRecords = - bufferDequeue.bufferedStreams - .stream() - .collect( - Collectors.toMap( - { desc: StreamDescriptor -> desc }, - { desc: StreamDescriptor -> - bufferDequeue.getQueueSizeInRecords(desc).orElseThrow() - }, - ), - ) - - val anyRecordsLeft = - streamDescriptorToRemainingRecords.values.stream().anyMatch { size: Long -> - size > 0 + bufferDequeue.bufferedStreams.associateWith { desc: StreamDescriptor -> + bufferDequeue.getQueueSizeInRecords(desc).orElseThrow() } + val anyRecordsLeft = streamDescriptorToRemainingRecords.values.any { it > 0 } + if (!anyRecordsLeft) { break } @@ -234,7 +212,6 @@ constructor( ) .append(System.lineSeparator()) streamDescriptorToRemainingRecords.entries - .stream() .filter { entry: Map.Entry -> entry.value > 0 } .forEach { entry: Map.Entry -> workerInfo.append( diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/BufferDequeue.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/BufferDequeue.kt index 012e22abe37f..c1d18172bce6 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/BufferDequeue.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/BufferDequeue.kt @@ -109,12 +109,7 @@ class BufferDequeue( get() = memoryManager.maxMemoryBytes val totalGlobalQueueSizeBytes: Long - get() = - buffers.values - .stream() - .map { obj: StreamAwareQueue -> obj.currentMemoryUsage } - .mapToLong { obj: Long -> obj } - .sum() + get() = buffers.values.sumOf { obj: StreamAwareQueue -> obj.currentMemoryUsage } fun getQueueSizeInRecords(streamDescriptor: StreamDescriptor): Optional { return getBuffer(streamDescriptor).map { buf: StreamAwareQueue -> buf.size().toLong() } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/MemoryAwareMessageBatch.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/MemoryAwareMessageBatch.kt index 15338f73201a..86a66a207089 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/MemoryAwareMessageBatch.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/buffers/MemoryAwareMessageBatch.kt @@ -36,13 +36,13 @@ class MemoryAwareMessageBatch( * can be flushed back to platform via stateManager. */ fun flushStates( - stateIdToCount: Map, + stateIdToCount: Map, outputRecordCollector: Consumer, ) { - stateIdToCount.forEach { (stateId: Long?, count: Long?) -> + stateIdToCount.forEach { (stateId: Long, count: Long) -> stateManager.decrement( - stateId!!, - count!!, + stateId, + count, ) } stateManager.flushStates(outputRecordCollector) diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManager.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManager.kt index e494e7ac9a0f..2db1c727fd07 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManager.kt @@ -282,8 +282,7 @@ class GlobalAsyncStateManager(private val memoryManager: GlobalMemoryManager) { synchronized(lock) { aliasIds.addAll( descToStateIdQ.values - .stream() - .flatMap { obj: LinkedBlockingDeque -> obj.stream() } + .flatMap { obj: LinkedBlockingDeque -> obj } .toList(), ) descToStateIdQ.clear() @@ -292,19 +291,12 @@ class GlobalAsyncStateManager(private val memoryManager: GlobalMemoryManager) { descToStateIdQ[SENTINEL_GLOBAL_DESC] = LinkedBlockingDeque() descToStateIdQ[SENTINEL_GLOBAL_DESC]!!.add(retroactiveGlobalStateId) - val combinedCounter: Long = - stateIdToCounter.values - .stream() - .mapToLong { obj: AtomicLong -> obj.get() } - .sum() + val combinedCounter: Long = stateIdToCounter.values.sumOf { it.get() } stateIdToCounter.clear() stateIdToCounter[retroactiveGlobalStateId] = AtomicLong(combinedCounter) val statsCounter: Long = - stateIdToCounterForPopulatingDestinationStats.values - .stream() - .mapToLong { obj: AtomicLong -> obj.get() } - .sum() + stateIdToCounterForPopulatingDestinationStats.values.sumOf { it.get() } stateIdToCounterForPopulatingDestinationStats.clear() stateIdToCounterForPopulatingDestinationStats.put( retroactiveGlobalStateId, diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManager.kt b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManager.kt index 278107786ef0..39a82dfbd016 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/dest_state_lifecycle_manager/DestStreamStateLifecycleManager.kt @@ -10,8 +10,6 @@ import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.StreamDescriptor import java.util.* -import java.util.function.Supplier -import java.util.stream.Collectors /** * This [DestStateLifecycleManager] handles any state where the state messages are scoped by stream. @@ -125,29 +123,25 @@ class DestStreamStateLifecycleManager(private val defaultNamespace: String?) : private fun listStatesInOrder( streamToState: Map ): Queue { - return streamToState.entries - .stream() // typically, we support by namespace and then stream name, so we retain - // that pattern here. - .sorted( - Comparator.comparing, String>( - { entry: Map.Entry -> - entry.key.namespace - }, - Comparator.nullsFirst(Comparator.naturalOrder()) - ) // namespace is allowed to be null - .thenComparing { entry: Map.Entry - -> - entry.key.name - } - ) - .map { obj: Map.Entry -> - obj.value - } - .collect( - Collectors.toCollection>( - Supplier> { LinkedList() } + return LinkedList( + streamToState.entries + + // typically, we support by namespace and then stream name, so we retain + // that pattern here. + .sortedWith( + Comparator.comparing, String>( + { entry: Map.Entry -> + entry.key.namespace + }, + Comparator.nullsFirst(Comparator.naturalOrder()) + ) // namespace is allowed to be null + .thenComparing { + entry: Map.Entry -> + entry.key.name + } ) - ) + .map { obj: Map.Entry -> obj.value } + ) } /** 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 a3a900c25b8a..755bf002edf9 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 @@ -10,7 +10,6 @@ import io.airbyte.commons.exceptions.ConnectionErrorException import io.airbyte.commons.functional.Either import java.sql.SQLException import java.sql.SQLSyntaxErrorException -import java.util.stream.Collectors import org.apache.commons.lang3.exception.ExceptionUtils import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -81,10 +80,9 @@ object ConnectorExceptionUtil { fun logAllAndThrowFirst(initialMessage: String, throwables: Collection) { if (!throwables.isEmpty()) { val stacktraces = - throwables - .stream() - .map { throwable: Throwable -> ExceptionUtils.getStackTrace(throwable) } - .collect(Collectors.joining("\n")) + throwables.joinToString("\n") { throwable: Throwable -> + ExceptionUtils.getStackTrace(throwable) + } LOGGER.error("$initialMessage$stacktraces\nRethrowing first exception.") throw throwables.iterator().next() } @@ -100,7 +98,7 @@ object ConnectorExceptionUtil { logAllAndThrowFirst(initialMessage, throwables) } // No need to filter on isRight since isLeft will throw before reaching this line. - return eithers.stream().map { obj: Either -> obj.right!! }.toList() + return eithers.map { obj: Either -> obj.right!! }.toList() } private fun isConfigErrorException(e: Throwable?): Boolean { 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 444b619af41f..22b826e747db 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 @@ -77,7 +77,6 @@ class ConcurrentStreamConsumer( */ val futures: Collection> = streams - .stream() .map { stream: AutoCloseableIterator -> ConcurrentStreamRunnable(stream, this) } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/db/jdbc/TestStreamingJdbcDatabase.kt b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/db/jdbc/TestStreamingJdbcDatabase.kt index d389d9a07615..3693a8aef75e 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/db/jdbc/TestStreamingJdbcDatabase.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/db/jdbc/TestStreamingJdbcDatabase.kt @@ -144,7 +144,7 @@ internal class TestStreamingJdbcDatabase { // This check assumes that FetchSizeConstants.TARGET_BUFFER_BYTE_SIZE = 200 MB. // Update this check if the buffer size constant is changed. Assertions.assertEquals(2, fetchSizes.size) - val sortedSizes = fetchSizes.stream().sorted().toList() + val sortedSizes = fetchSizes.sorted().toList() Assertions.assertTrue(sortedSizes[0] < FetchSizeConstants.INITIAL_SAMPLE_SIZE) Assertions.assertEquals(FetchSizeConstants.INITIAL_SAMPLE_SIZE, sortedSizes[1]) } 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 a460bd79ae56..30d2b646b9ae 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 @@ -65,7 +65,6 @@ internal class IntegrationRunnerTest { val testName = Thread.currentThread().name ThreadUtils.getAllThreads() - .stream() .filter { runningThread: Thread -> !runningThread.isDaemon } .forEach { runningThread: Thread -> runningThread.name = testName } } @@ -470,10 +469,8 @@ ${Jsons.serialize(message2)}""".toByteArray( throw RuntimeException(e) } val runningThreads = - ThreadUtils.getAllThreads() - .stream() - .filter(IntegrationRunner.ORPHANED_THREAD_FILTER) - .toList() + ThreadUtils.getAllThreads().filter(IntegrationRunner::filterOrphanedThread) + // all threads should be interrupted Assertions.assertEquals(listOf(), runningThreads) Assertions.assertEquals(1, caughtExceptions.size) @@ -498,10 +495,8 @@ ${Jsons.serialize(message2)}""".toByteArray( } val runningThreads = - ThreadUtils.getAllThreads() - .stream() - .filter(IntegrationRunner.ORPHANED_THREAD_FILTER) - .toList() + ThreadUtils.getAllThreads().filter(IntegrationRunner::filterOrphanedThread) + // a thread that refuses to be interrupted should remain Assertions.assertEquals(1, runningThreads.size) Assertions.assertEquals(1, caughtExceptions.size) diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumerTest.kt b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumerTest.kt index 3ebe3e1a1004..7b49b5d410ac 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/AsyncStreamConsumerTest.kt @@ -556,24 +556,21 @@ class AsyncStreamConsumerTest { .toList() val expRecords = - allRecords - .stream() - .map { m: AirbyteMessage -> - PartialAirbyteMessage() - .withType(AirbyteMessage.Type.RECORD) - .withRecord( - PartialAirbyteRecordMessage() - .withStream(m.record.stream) - .withNamespace(m.record.namespace) - .withData(m.record.data), - ) - .withSerialized( - Jsons.serialize( - m.record.data, - ), - ) - } - .toList() + allRecords.map { m: AirbyteMessage -> + PartialAirbyteMessage() + .withType(AirbyteMessage.Type.RECORD) + .withRecord( + PartialAirbyteRecordMessage() + .withStream(m.record.stream) + .withNamespace(m.record.namespace) + .withData(m.record.data), + ) + .withSerialized( + Jsons.serialize( + m.record.data, + ), + ) + } assertEquals(expRecords, actualRecords) } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManagerTest.kt b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManagerTest.kt index f790ab406022..8a15090c8485 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManagerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/async/state/GlobalAsyncStateManagerTest.kt @@ -16,7 +16,6 @@ import io.airbyte.protocol.models.v0.AirbyteStateMessage import io.airbyte.protocol.models.v0.AirbyteStateStats import io.airbyte.protocol.models.v0.AirbyteStreamState import io.airbyte.protocol.models.v0.StreamDescriptor -import java.util.stream.Collectors import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertNotEquals import org.junit.jupiter.api.Assertions.assertThrows @@ -212,14 +211,7 @@ class GlobalAsyncStateManagerTest { } // because no state message has been tracked, there is nothing to flush yet. val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals(0, stateWithStats.size) stateManager.trackState(STREAM1_STATE_MESSAGE1, STATE_MSG_SIZE, DEFAULT_NAMESPACE) @@ -231,14 +223,7 @@ class GlobalAsyncStateManagerTest { val expectedDestinationStats = AirbyteStateStats().withRecordCount(2.0) val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -249,9 +234,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats2.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats2.values.toList()) } private fun attachDestinationStateStats( @@ -279,14 +264,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(0.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } // assertEquals( listOf( @@ -298,9 +276,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) assertThrows( IllegalArgumentException::class.java, @@ -349,14 +327,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(30.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -367,9 +338,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) } @Test @@ -388,14 +359,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(10.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -406,9 +370,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) emittedStatesFromDestination.clear() @@ -421,14 +385,7 @@ class GlobalAsyncStateManagerTest { ) } val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -439,9 +396,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats2.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats2.values.toList()) } @Test @@ -460,14 +417,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(10.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -478,9 +428,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) emittedStatesFromDestination.clear() stateManager.trackState(GLOBAL_STATE_MESSAGE2, STATE_MSG_SIZE, DEFAULT_NAMESPACE) @@ -491,14 +441,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats2 = AirbyteStateStats().withRecordCount(0.0) val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -509,11 +452,11 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats2, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) assertEquals( listOf(expectedDestinationStats2), - stateWithStats2.values.stream().toList(), + stateWithStats2.values.toList(), ) emittedStatesFromDestination.clear() @@ -526,14 +469,7 @@ class GlobalAsyncStateManagerTest { ) } val stateWithStats3 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -544,9 +480,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats3.keys.stream().toList(), + stateWithStats3.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats3.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats3.values.toList()) } @Test @@ -568,14 +504,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(20.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -586,9 +515,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) emittedStatesFromDestination.clear() val afterConvertId0: Long = simulateIncomingRecords(STREAM1_DESC, 10, stateManager) @@ -602,14 +531,7 @@ class GlobalAsyncStateManagerTest { ) } val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -620,9 +542,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats2.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats2.values.toList()) } } @@ -643,14 +565,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(0.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -661,9 +576,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) assertThrows( IllegalArgumentException::class.java, @@ -692,14 +607,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(3.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -710,9 +618,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) emittedStatesFromDestination.clear() @@ -726,14 +634,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats2 = AirbyteStateStats().withRecordCount(10.0) val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -744,11 +645,11 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats2, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) assertEquals( listOf(expectedDestinationStats2), - stateWithStats2.values.stream().toList(), + stateWithStats2.values.toList(), ) } @@ -768,14 +669,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(3.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -786,9 +680,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) emittedStatesFromDestination.clear() stateManager.trackState(STREAM1_STATE_MESSAGE2, STATE_MSG_SIZE, DEFAULT_NAMESPACE) @@ -798,14 +692,7 @@ class GlobalAsyncStateManagerTest { ) } val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } val expectedDestinationStats2 = AirbyteStateStats().withRecordCount(0.0) assertEquals( listOf( @@ -817,11 +704,11 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats2, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) assertEquals( listOf(expectedDestinationStats2), - stateWithStats2.values.stream().toList(), + stateWithStats2.values.toList(), ) emittedStatesFromDestination.clear() @@ -834,14 +721,7 @@ class GlobalAsyncStateManagerTest { ) } val stateWithStats3 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } val expectedDestinationStats3 = AirbyteStateStats().withRecordCount(10.0) assertEquals( listOf( @@ -853,11 +733,11 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats3, ), ), - stateWithStats3.keys.stream().toList(), + stateWithStats3.keys.toList(), ) assertEquals( listOf(expectedDestinationStats3), - stateWithStats3.values.stream().toList(), + stateWithStats3.values.toList(), ) } @@ -879,14 +759,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats = AirbyteStateStats().withRecordCount(3.0) val stateWithStats = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -897,9 +770,9 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats, ), ), - stateWithStats.keys.stream().toList(), + stateWithStats.keys.toList(), ) - assertEquals(listOf(expectedDestinationStats), stateWithStats.values.stream().toList()) + assertEquals(listOf(expectedDestinationStats), stateWithStats.values.toList()) emittedStatesFromDestination.clear() stateManager.decrement(stream2StateId, 4) @@ -919,14 +792,7 @@ class GlobalAsyncStateManagerTest { } val expectedDestinationStats2 = AirbyteStateStats().withRecordCount(7.0) val stateWithStats2 = - emittedStatesFromDestination - .stream() - .collect( - Collectors.toMap( - { c: AirbyteMessage -> c }, - { c: AirbyteMessage -> c.state?.destinationStats }, - ), - ) + emittedStatesFromDestination.associateWith { it.state?.destinationStats } assertEquals( listOf( attachDestinationStateStats( @@ -937,11 +803,11 @@ class GlobalAsyncStateManagerTest { expectedDestinationStats2, ), ), - stateWithStats2.keys.stream().toList(), + stateWithStats2.keys.toList(), ) assertEquals( listOf(expectedDestinationStats2), - stateWithStats2.values.stream().toList(), + stateWithStats2.values.toList(), ) } } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.kt b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.kt index 4eb6dc79d1bf..2889871675bc 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/destination/buffered_stream_consumer/BufferedStreamConsumerTest.kt @@ -17,7 +17,6 @@ import java.time.Instant import java.util.* import java.util.concurrent.TimeUnit import java.util.function.Consumer -import java.util.stream.Stream import org.apache.commons.lang3.RandomStringUtils import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.BeforeEach @@ -147,8 +146,7 @@ class BufferedStreamConsumerTest { val expectedRecords = Lists.newArrayList(expectedRecordsBatch1, expectedRecordsBatch2) - .stream() - .flatMap { obj: List -> obj.stream() } + .flatMap { obj: List -> obj } .toList() verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecords) @@ -237,10 +235,8 @@ class BufferedStreamConsumerTest { val expectedRecordsStream1 = generateRecords(1000) val expectedRecordsStream2 = expectedRecordsStream1 - .stream() .map { `object`: AirbyteMessage -> Jsons.clone(`object`) } - .peek { m: AirbyteMessage -> m.record.withStream(STREAM_NAME2) } - .toList() + .onEach { m: AirbyteMessage -> m.record.withStream(STREAM_NAME2) } consumer.start() consumeRecords(consumer, expectedRecordsStream1) @@ -262,10 +258,8 @@ class BufferedStreamConsumerTest { val expectedRecordsStream1 = generateRecords(1000) val expectedRecordsStream2 = expectedRecordsStream1 - .stream() .map { `object`: AirbyteMessage -> Jsons.clone(`object`) } - .peek { m: AirbyteMessage -> m.record.withStream(STREAM_NAME2) } - .toList() + .onEach { m: AirbyteMessage -> m.record.withStream(STREAM_NAME2) } consumer.start() consumeRecords(consumer, expectedRecordsStream1) @@ -308,8 +302,7 @@ class BufferedStreamConsumerTest { verifyRecords( STREAM_NAME, SCHEMA_NAME, - Stream.concat(expectedRecordsStream1.stream(), expectedRecordsStream1Batch2.stream()) - .toList() + expectedRecordsStream1 + expectedRecordsStream1Batch2 ) Mockito.verify(outputRecordCollector).accept(STATE_MESSAGE1) } @@ -343,8 +336,7 @@ class BufferedStreamConsumerTest { verifyRecords( STREAM_NAME, SCHEMA_NAME, - Stream.concat(expectedRecordsStream1.stream(), expectedRecordsStream1Batch2.stream()) - .toList() + expectedRecordsStream1 + expectedRecordsStream1Batch2 ) verifyRecords(STREAM_NAME, SCHEMA_NAME, expectedRecordsStream1Batch3) // expects two STATE messages returned since one will be flushed after periodic flushing @@ -588,7 +580,7 @@ class BufferedStreamConsumerTest { Mockito.verify(recordWriter) .accept( AirbyteStreamNameNamespacePair(streamName, namespace), - expectedRecords.stream().map { obj: AirbyteMessage -> obj.record }.toList() + expectedRecords.map { obj: AirbyteMessage -> obj.record }.toList() ) } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumerTest.kt b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumerTest.kt index 98f8676ce555..d23ef0455900 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/test/kotlin/io/airbyte/cdk/integrations/util/concurrent/ConcurrentStreamConsumerTest.kt @@ -107,7 +107,7 @@ internal class ConcurrentStreamConsumerTest { val concurrentStreamConsumer = ConcurrentStreamConsumer(streamConsumer, streams.size) val partitionSize = concurrentStreamConsumer.parallelism - val partitions = Lists.partition(streams.stream().toList(), partitionSize) + val partitions = Lists.partition(streams.toList(), partitionSize) for (partition in partitions) { Assertions.assertDoesNotThrow { concurrentStreamConsumer.accept(partition) } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/base/ssh/SshBastionContainer.kt b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/base/ssh/SshBastionContainer.kt index 09f2a9f235f9..c0503081fe1f 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/base/ssh/SshBastionContainer.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/base/ssh/SshBastionContainer.kt @@ -144,13 +144,8 @@ class SshBastionContainer : AutoCloseable { */ fun getInnerContainerAddress(container: Container<*>): ImmutablePair { return ImmutablePair.of( - container.containerInfo.networkSettings.networks.entries - .stream() - .findFirst() - .get() - .value - .ipAddress, - container.exposedPorts.stream().findFirst().get() + container.containerInfo.networkSettings.networks.entries.first().value.ipAddress, + container.exposedPorts.first() ) } diff --git a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/util/HostPortResolver.kt b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/util/HostPortResolver.kt index b0ceaa9101ba..f66896913b02 100644 --- a/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/util/HostPortResolver.kt +++ b/airbyte-cdk/java/airbyte-cdk/core/src/testFixtures/kotlin/io/airbyte/cdk/integrations/util/HostPortResolver.kt @@ -14,7 +14,7 @@ object HostPortResolver { @JvmStatic fun resolvePort(container: GenericContainer<*>): Int { - return container.exposedPorts.stream().findFirst().get() + return container.exposedPorts.first() } fun resolveIpAddress(container: GenericContainer<*>): String { @@ -24,12 +24,7 @@ object HostPortResolver { private fun getIpAddress(container: GenericContainer<*>): String { // Weird double bang here. If I remove the Object.requireNotNull, there's a type error... return Objects.requireNonNull( - container.containerInfo.networkSettings.networks.entries - .stream() - .findFirst() - .get() - .value - .ipAddress + container.containerInfo.networkSettings.networks.entries.first().value.ipAddress )!! } } diff --git a/airbyte-cdk/java/airbyte-cdk/datastore-mongo/src/main/kotlin/io/airbyte/cdk/db/mongodb/MongoUtils.kt b/airbyte-cdk/java/airbyte-cdk/datastore-mongo/src/main/kotlin/io/airbyte/cdk/db/mongodb/MongoUtils.kt index d440ae7fb0f2..e2929a4b9673 100644 --- a/airbyte-cdk/java/airbyte-cdk/datastore-mongo/src/main/kotlin/io/airbyte/cdk/db/mongodb/MongoUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/datastore-mongo/src/main/kotlin/io/airbyte/cdk/db/mongodb/MongoUtils.kt @@ -128,7 +128,6 @@ object MongoUtils { if (node.hasChildren()) { val subFields = node.children!! - .stream() .map { obj: TreeNode> -> nodeToCommonField(obj) } .toList() return CommonField(field.name, field.type, subFields) @@ -296,7 +295,6 @@ object MongoUtils { val allkeys = HashSet(getFieldsName(collection)) return allkeys - .stream() .map { key: String -> val types = getTypes(collection, key) val type = getUniqueType(types) diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/JdbcBufferedConsumerFactory.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/JdbcBufferedConsumerFactory.kt index 22346a11eb2f..c80142ed24d5 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/JdbcBufferedConsumerFactory.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/JdbcBufferedConsumerFactory.kt @@ -96,11 +96,8 @@ object JdbcBufferedConsumerFactory { "jdbc destinations must specify a schema." ) } - return catalog!! - .streams - .stream() - .map(toWriteConfig(namingResolver, config, schemaRequired)) - .toList() + // What is this doing??? + return catalog!!.streams.map(toWriteConfig(namingResolver, config, schemaRequired)::apply) } private fun toWriteConfig( 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 2142408c5917..d9e4e20349c3 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 @@ -27,7 +27,6 @@ import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletionStage import java.util.function.Predicate import lombok.extern.slf4j.Slf4j -import org.jooq.Condition import org.jooq.DSLContext import org.jooq.SQLDialect import org.jooq.conf.ParamType @@ -197,7 +196,6 @@ abstract class JdbcDestinationHandler( val initialStates = streamConfigs - .stream() .map { streamConfig: StreamConfig -> retrieveState(destinationStatesFuture, streamConfig) } @@ -378,26 +376,25 @@ abstract class JdbcDestinationHandler( ) // Filter out Meta columns since they don't exist in stream config. - val actualColumns = - existingTable.columns.entries - .stream() - .filter { column: Map.Entry -> - JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS.stream().noneMatch { - airbyteColumnName: String -> - airbyteColumnName == column.key - } + val actualColumns: LinkedHashMap = LinkedHashMap() + existingTable.columns.entries + .stream() + .filter { column: Map.Entry -> + JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS.none { airbyteColumnName: String + -> + airbyteColumnName == column.key } - .collect( - { LinkedHashMap() }, - { - map: LinkedHashMap, - column: Map.Entry -> - map[column.key] = column.value.type - }, - { obj: LinkedHashMap, m: LinkedHashMap -> - obj.putAll(m) - } - ) + } + .collect( + { LinkedHashMap() }, + { map: LinkedHashMap, column: Map.Entry + -> + map[column.key] = column.value.type + }, + { obj: LinkedHashMap, m: LinkedHashMap -> + obj.putAll(m) + } + ) return actualColumns == intendedColumns } @@ -415,7 +412,6 @@ abstract class JdbcDestinationHandler( .deleteFrom(table(quotedName(rawTableSchemaName, DESTINATION_STATE_TABLE_NAME))) .where( destinationStates.keys - .stream() .map { streamId: StreamId -> field(quotedName(DESTINATION_STATE_TABLE_COLUMN_NAME)) .eq(streamId.originalName) @@ -424,9 +420,7 @@ abstract class JdbcDestinationHandler( .eq(streamId.originalNamespace) ) } - .reduce(DSL.falseCondition()) { obj: Condition, arg2: Condition -> - obj.or(arg2) - } + .reduce { obj, arg2 -> obj.or(arg2) } ) .getSQL(ParamType.INLINED) diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcSqlGenerator.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcSqlGenerator.kt index b9bc5d950c4d..fab24831ab0f 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcSqlGenerator.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcSqlGenerator.kt @@ -15,7 +15,6 @@ import io.airbyte.protocol.models.v0.DestinationSyncMode import java.sql.Timestamp import java.time.Instant import java.util.* -import java.util.stream.Stream import kotlin.Any import kotlin.Boolean import kotlin.IllegalArgumentException @@ -159,19 +158,16 @@ constructor( ): List> { val fields = metaColumns.entries - .stream() .map { metaColumn: Map.Entry?> -> DSL.field(DSL.quotedName(metaColumn.key), metaColumn.value) } .toList() val dataFields = columns.entries - .stream() .map { column: Map.Entry -> DSL.field(DSL.quotedName(column.key!!.name), toDialectType(column.value)) } - .toList() - dataFields.addAll(fields) + .toList() + fields return dataFields } @@ -208,7 +204,6 @@ constructor( ): List> { val fields = metaColumns.entries - .stream() .map { metaColumn: Map.Entry?> -> DSL.field(DSL.quotedName(metaColumn.key), metaColumn.value) } @@ -251,17 +246,8 @@ constructor( val finalTableIdentifier = stream.id.finalName + suffix.lowercase(Locale.getDefault()) if (!force) { return transactionally( - Stream.concat( - Stream.of( - createTableSql( - stream.id.finalNamespace, - finalTableIdentifier, - stream.columns!! - ) - ), - createIndexSql(stream, suffix).stream() - ) - .toList() + createTableSql(stream.id.finalNamespace, finalTableIdentifier, stream.columns!!) + + createIndexSql(stream, suffix) ) } @@ -272,18 +258,10 @@ constructor( } return transactionally( - Stream.concat( - Stream.of( - dropTableStep.getSQL(ParamType.INLINED), - createTableSql( - stream.id.finalNamespace, - finalTableIdentifier, - stream.columns!! - ) - ), - createIndexSql(stream, suffix).stream() - ) - .toList() + listOf( + dropTableStep.getSQL(ParamType.INLINED), + createTableSql(stream.id.finalNamespace, finalTableIdentifier, stream.columns!!) + ) + createIndexSql(stream, suffix) ) } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialFlush.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialFlush.kt index 75744976df5d..cf2316bf351a 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialFlush.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/staging/SerialFlush.kt @@ -15,7 +15,6 @@ import io.airbyte.integrations.base.destination.typing_deduping.TyperDeduper import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.github.oshai.kotlinlogging.KotlinLogging -import java.util.stream.Collectors import org.apache.commons.io.FileUtils private val log = KotlinLogging.logger {} @@ -69,10 +68,9 @@ object SerialFlush { val message = String.format( "You are trying to write multiple streams to the same table. Consider switching to a custom namespace format using \${SOURCE_NAMESPACE}, or moving one of them into a separate connection with a different stream prefix. Affected streams: %s", - conflictingStreams - .stream() - .map { config: WriteConfig -> config.namespace + "." + config.streamName } - .collect(Collectors.joining(", ")) + conflictingStreams.joinToString(", ") { config: WriteConfig -> + config.namespace + "." + config.streamName + } ) throw ConfigErrorException(message) } 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 c4e378a9b0f0..cf18af17a331 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 @@ -125,10 +125,10 @@ open class SerialStagingConsumerFactory { parsedCatalog: ParsedCatalog, useDestinationsV2Columns: Boolean ): List { - return catalog.streams - .stream() - .map(toWriteConfig(namingResolver, config, parsedCatalog, useDestinationsV2Columns)) - .toList() + return catalog.streams.map { + toWriteConfig(namingResolver, config, parsedCatalog, useDestinationsV2Columns) + .apply(it) + } } private fun toWriteConfig( 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 b153b9a688f2..d9ee39dc5ebe 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 @@ -464,11 +464,8 @@ abstract class DestinationAcceptanceTest { val largeNumberRecords = Collections.nCopies(400, messages) - .stream() - .flatMap { obj: List -> - obj.stream() - } // regroup messages per stream - .sorted( + .flatten() // regroup messages per stream + .sortedWith( Comparator.comparing { obj: io.airbyte.protocol.models.v0.AirbyteMessage -> obj.type } @@ -542,7 +539,6 @@ abstract class DestinationAcceptanceTest { val configuredDummyCatalog = CatalogHelpers.toDefaultConfiguredCatalog(dummyCatalog) // update messages to set new dummy stream name firstSyncMessages - .stream() .filter { message: io.airbyte.protocol.models.v0.AirbyteMessage -> message.record != null } @@ -849,7 +845,6 @@ abstract class DestinationAcceptanceTest { // We expect all the of messages to be missing the removed column after normalization. val expectedMessages = messages - .stream() .map { message: io.airbyte.protocol.models.v0.AirbyteMessage -> if (message.record != null) { (message.record.data as ObjectNode).remove("HKD") @@ -1016,7 +1011,6 @@ abstract class DestinationAcceptanceTest { // order) val expectedMessages = expectedMessagesAfterSecondSync - .stream() .filter { it.type == Type.RECORD && it.record != null } .filter { val key = @@ -1594,35 +1588,16 @@ abstract class DestinationAcceptanceTest { val destinationOutput = runSync(config, messages, catalog, runNormalization) val expectedStateMessage = - reversed(messages) - .stream() - .filter { m: io.airbyte.protocol.models.v0.AirbyteMessage -> - m.type == io.airbyte.protocol.models.v0.AirbyteMessage.Type.STATE - } - .findFirst() - .orElseThrow { - IllegalArgumentException( - "All message sets used for testing should include a state record" - ) - }!! + reversed(messages).firstOrNull { m: AirbyteMessage -> m.type == Type.STATE } + ?: throw IllegalArgumentException( + "All message sets used for testing should include a state record" + ) Collections.reverse(destinationOutput) - val actualStateMessage = - destinationOutput - .stream() - .filter { it.type == Type.STATE } - .findFirst() - .map { msg: AirbyteMessage -> - // Modify state message to remove destination stats. - val clone = msg.state - clone.destinationStats = null - msg.state = clone - msg - } - .orElseGet { - Assertions.fail("Destination failed to output state") - null - } + val actualStateMessage = destinationOutput.filter { it.type == Type.STATE }.first() + val clone = actualStateMessage.state + clone.destinationStats = null + actualStateMessage.state = clone Assertions.assertEquals(expectedStateMessage, actualStateMessage) } @@ -1715,7 +1690,6 @@ abstract class DestinationAcceptanceTest { val schema = if (stream.namespace != null) stream.namespace else defaultSchema!! val msgList = retrieveRecords(testEnv, streamName, schema, stream.jsonSchema) - .stream() .map { data: JsonNode -> AirbyteRecordMessage() .withStream(streamName) @@ -1737,12 +1711,11 @@ abstract class DestinationAcceptanceTest { ) { val expectedProcessed = expected - .stream() .filter { message: io.airbyte.protocol.models.v0.AirbyteMessage -> message.type == io.airbyte.protocol.models.v0.AirbyteMessage.Type.RECORD } .map { obj: io.airbyte.protocol.models.v0.AirbyteMessage -> obj.record } - .peek { recordMessage: AirbyteRecordMessage -> recordMessage.emittedAt = null } + .onEach { recordMessage: AirbyteRecordMessage -> recordMessage.emittedAt = null } .map { recordMessage: AirbyteRecordMessage -> if (pruneAirbyteInternalFields) safePrune(recordMessage) else recordMessage } @@ -1751,7 +1724,6 @@ abstract class DestinationAcceptanceTest { val actualProcessed = actual - .stream() .map { recordMessage: AirbyteRecordMessage -> if (pruneAirbyteInternalFields) safePrune(recordMessage) else recordMessage } @@ -1773,7 +1745,6 @@ abstract class DestinationAcceptanceTest { val msgList = retrieveNormalizedRecords(testEnv, streamName, defaultSchema) - .stream() .map { data: JsonNode -> AirbyteRecordMessage().withStream(streamName).withData(data) } @@ -2176,7 +2147,6 @@ abstract class DestinationAcceptanceTest { override fun provideArguments(context: ExtensionContext): Stream { val testCases = Jsons.deserialize(MoreResources.readResource(NAMESPACE_TEST_CASES_JSON)) return MoreIterators.toList(testCases.elements()) - .stream() .filter { testCase: JsonNode -> testCase["enabled"].asBoolean() } .map { testCase: JsonNode -> val namespaceInCatalog = @@ -2195,6 +2165,7 @@ abstract class DestinationAcceptanceTest { namespaceInDst ) } + .stream() } companion object { @@ -2284,7 +2255,7 @@ abstract class DestinationAcceptanceTest { "UNIQUE_KEY" ) if ( - airbyteInternalFields.stream().anyMatch { internalField: String -> + airbyteInternalFields.any { internalField: String -> key.lowercase(Locale.getDefault()) .contains(internalField.lowercase(Locale.getDefault())) } || json[key].isNull 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 7b6587b7394d..47bd62a74eb8 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 @@ -120,14 +120,11 @@ open class AdvancedTestDataComparator : TestDataComparator { } else { for (expectedNode in expectedList) { val sameActualNode = - actualList - .stream() - .filter { actualNode: JsonNode -> - compareJsonNodes(expectedNode, actualNode) - } - .findFirst() - if (sameActualNode.isPresent) { - actualList.remove(sameActualNode.get()) + actualList.firstOrNull { actualNode: JsonNode -> + compareJsonNodes(expectedNode, actualNode) + } + if (sameActualNode != null) { + actualList.remove(sameActualNode) } else { return false } diff --git a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/typing_deduping/JdbcSqlGeneratorIntegrationTest.kt b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/typing_deduping/JdbcSqlGeneratorIntegrationTest.kt index c9edfa5b3762..be7ea6293de8 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/typing_deduping/JdbcSqlGeneratorIntegrationTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/typing_deduping/JdbcSqlGeneratorIntegrationTest.kt @@ -65,7 +65,6 @@ abstract class JdbcSqlGeneratorIntegrationTest DSL.field(DSL.quotedName(columnName)) } .toList() ) @@ -73,7 +72,6 @@ abstract class JdbcSqlGeneratorIntegrationTest // Convert this field to a string. Pretty naive implementation. val column = record[fieldName] diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/AirbyteDebeziumHandler.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/AirbyteDebeziumHandler.kt index d6114b763b88..c4dee9dfcf5f 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/AirbyteDebeziumHandler.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/AirbyteDebeziumHandler.kt @@ -155,9 +155,8 @@ class AirbyteDebeziumHandler( @JvmStatic fun isAnyStreamIncrementalSyncMode(catalog: ConfiguredAirbyteCatalog): Boolean { return catalog.streams - .stream() .map { obj: ConfiguredAirbyteStream -> obj.syncMode } - .anyMatch { syncMode: SyncMode -> syncMode == SyncMode.INCREMENTAL } + .any { syncMode: SyncMode -> syncMode == SyncMode.INCREMENTAL } } } } 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 22a64b5d5763..de8399a0213d 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 @@ -16,8 +16,6 @@ import java.nio.file.NoSuchFileException import java.nio.file.Path import java.util.* import java.util.function.BiFunction -import java.util.function.Function -import java.util.stream.Collectors import org.apache.commons.io.FileUtils import org.apache.kafka.connect.errors.ConnectException import org.apache.kafka.connect.util.SafeObjectInputStream @@ -39,18 +37,7 @@ class AirbyteFileOffsetBackingStore( fun read(): Map { val raw = load() - return raw.entries - .stream() - .collect( - Collectors.toMap( - Function { e: Map.Entry -> - byteBufferToString(e.key) - }, - Function { e: Map.Entry -> - byteBufferToString(e.value) - } - ) - ) + return raw.entries.associate { byteBufferToString(it.key) to byteBufferToString(it.value) } } fun persist(cdcState: JsonNode?) { @@ -62,15 +49,10 @@ class AirbyteFileOffsetBackingStore( val updatedMap = updateStateForDebezium2_1(mapAsString) - val mappedAsStrings = - updatedMap.entries - .stream() - .collect( - Collectors.toMap( - Function { e: Map.Entry -> stringToByteBuffer(e.key) }, - Function { e: Map.Entry -> stringToByteBuffer(e.value) } - ) - ) + val mappedAsStrings: Map = + updatedMap.entries.associate { + stringToByteBuffer(it.key) to stringToByteBuffer(it.value) + } FileUtils.deleteQuietly(offsetFilePath.toFile()) save(mappedAsStrings) @@ -79,7 +61,7 @@ class AirbyteFileOffsetBackingStore( private fun updateStateForDebezium2_1(mapAsString: Map): Map { val updatedMap: MutableMap = LinkedHashMap() if (mapAsString.size > 0) { - val key = mapAsString.keys.stream().toList()[0] + val key = mapAsString.keys.toList()[0] val i = key.indexOf('[') val i1 = key.lastIndexOf(']') diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RelationalDbDebeziumPropertiesManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RelationalDbDebeziumPropertiesManager.kt index b130d4a5fd3c..eba1ab42d338 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RelationalDbDebeziumPropertiesManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/debezium/internals/RelationalDbDebeziumPropertiesManager.kt @@ -66,14 +66,13 @@ class RelationalDbDebeziumPropertiesManager( // } -------> info "schema1.table1, schema2.table2" return catalog.streams - .stream() .filter { s: ConfiguredAirbyteStream -> s.syncMode == SyncMode.INCREMENTAL } .map { obj: ConfiguredAirbyteStream -> obj.stream } - .map { stream: AirbyteStream -> - stream.namespace + "." + stream.name - } // debezium needs commas escaped to split properly - .map { x: String -> StringUtils.escape(Pattern.quote(x), ",".toCharArray(), "\\,") } - .collect(Collectors.joining(",")) + .map { stream: AirbyteStream -> stream.namespace + "." + stream.name } + // debezium needs commas escaped to split properly + .joinToString(",") { x: String -> + StringUtils.escape(Pattern.quote(x), ",".toCharArray(), "\\,") + } } fun getColumnIncludeList(catalog: ConfiguredAirbyteCatalog): String { @@ -91,7 +90,6 @@ class RelationalDbDebeziumPropertiesManager( // } -------> info "schema1.table1.(column1 | column2)" return catalog.streams - .stream() .filter { s: ConfiguredAirbyteStream -> s.syncMode == SyncMode.INCREMENTAL } .map { obj: ConfiguredAirbyteStream -> obj.stream } .map { s: AirbyteStream -> @@ -99,8 +97,7 @@ class RelationalDbDebeziumPropertiesManager( Pattern.quote(s.namespace + "." + s.name) + (if (StringUtils.isNotBlank(fields)) "\\.$fields" else "") } - .map { x: String -> StringUtils.escape(x, ",".toCharArray(), "\\,") } - .collect(Collectors.joining(",")) + .joinToString(",") { x: String -> StringUtils.escape(x, ",".toCharArray(), "\\,") } } private fun parseFields(fieldNames: Iterator?): String { 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 03feabc69b86..7540f838d75f 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 @@ -219,31 +219,25 @@ abstract class AbstractJdbcSource( // each column { resultSet: ResultSet -> this.getColumnMetadata(resultSet) } ) - .stream() .filter( - excludeNotAccessibleTables(internalSchemas, tablesWithSelectGrantPrivilege) + excludeNotAccessibleTables(internalSchemas, tablesWithSelectGrantPrivilege)::test ) // group by schema and table name to handle the case where a table with the same name // exists in // multiple schemas. - .collect( - Collectors.groupingBy>( - Function> { t: JsonNode -> - ImmutablePair.of( - t.get(INTERNAL_SCHEMA_NAME).asText(), - t.get(INTERNAL_TABLE_NAME).asText() - ) - } + .groupBy { t: JsonNode -> + ImmutablePair.of( + t.get(INTERNAL_SCHEMA_NAME).asText(), + t.get(INTERNAL_TABLE_NAME).asText() ) - ) + } .values - .stream() - .map>> { fields: List -> + .map { fields: List -> TableInfo>( nameSpace = fields[0].get(INTERNAL_SCHEMA_NAME).asText(), name = fields[0].get(INTERNAL_TABLE_NAME).asText(), fields = fields - .stream() // read the column metadata Json object, and determine its + // read the column metadata Json object, and determine its // type .map { f: JsonNode -> val datatype = sourceOperations.getDatabaseFieldType(f) @@ -263,25 +257,20 @@ abstract class AbstractJdbcSource( datatype ) {} } - .collect(Collectors.toList>()), + .toList(), cursorFields = extractCursorFields(fields) ) } - .collect(Collectors.toList>>()) + .toList() } private fun extractCursorFields(fields: List): List { return fields - .stream() .filter { field: JsonNode -> isCursorType(sourceOperations.getDatabaseFieldType(field)) } - .map( - Function { field: JsonNode -> - field.get(INTERNAL_COLUMN_NAME).asText() - } - ) - .collect(Collectors.toList()) + .map { it.get(INTERNAL_COLUMN_NAME).asText() } + .toList() } protected fun excludeNotAccessibleTables( @@ -292,10 +281,10 @@ abstract class AbstractJdbcSource( if (tablesWithSelectGrantPrivilege!!.isEmpty()) { return@Predicate isNotInternalSchema(jsonNode, internalSchemas) } - (tablesWithSelectGrantPrivilege.stream().anyMatch { e: JdbcPrivilegeDto -> + (tablesWithSelectGrantPrivilege.any { e: JdbcPrivilegeDto -> e.schemaName == jsonNode.get(INTERNAL_SCHEMA_NAME).asText() } && - tablesWithSelectGrantPrivilege.stream().anyMatch { e: JdbcPrivilegeDto -> + tablesWithSelectGrantPrivilege.any { e: JdbcPrivilegeDto -> e.tableName == jsonNode.get(INTERNAL_TABLE_NAME).asText() } && !internalSchemas.contains(jsonNode.get(INTERNAL_SCHEMA_NAME).asText())) @@ -702,7 +691,6 @@ abstract class AbstractJdbcSource( HashSet(Sets.difference(allStreams, alreadySyncedStreams)) return catalog.streams - .stream() .filter { c: ConfiguredAirbyteStream -> c.syncMode == SyncMode.INCREMENTAL } .filter { stream: ConfiguredAirbyteStream -> newlyAddedStreams.contains( @@ -728,8 +716,7 @@ abstract class AbstractJdbcSource( ): Map> { val result: MutableMap> = HashMap() entries - .stream() - .sorted(Comparator.comparingInt(PrimaryKeyAttributesFromDb::keySequence)) + .sortedWith(Comparator.comparingInt(PrimaryKeyAttributesFromDb::keySequence)) .forEach { entry: PrimaryKeyAttributesFromDb -> if (!result.containsKey(entry.streamName)) { result[entry.streamName] = ArrayList() 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 95df5219dae6..0c8a5ec04657 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 @@ -38,8 +38,6 @@ import java.time.Duration import java.time.Instant import java.util.* import java.util.concurrent.atomic.AtomicLong -import java.util.function.Function -import java.util.stream.Collectors import java.util.stream.Stream import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -137,16 +135,9 @@ protected constructor(driverClassName: String) : logPreSyncDebugData(database, catalog) val fullyQualifiedTableNameToInfo = - discoverWithoutSystemTables(database) - .stream() - .collect( - Collectors.toMap( - Function { t: TableInfo> -> - String.format("%s.%s", t.nameSpace, t.name) - }, - Function.identity() - ) - ) + discoverWithoutSystemTables(database).associateBy { + String.format("%s.%s", it.nameSpace, it.name) + } validateCursorFieldForIncrementalTables(fullyQualifiedTableNameToInfo, catalog, database) @@ -190,7 +181,7 @@ protected constructor(driverClassName: String) : @Throws(SQLException::class) protected fun validateCursorFieldForIncrementalTables( - tableNameToTable: Map>>, + tableNameToTable: Map>>, catalog: ConfiguredAirbyteCatalog, database: Database ) { @@ -218,11 +209,9 @@ protected constructor(driverClassName: String) : } val cursorType = table.fields - .stream() .filter { info: CommonField -> info.name == cursorField.get() } .map { obj: CommonField -> obj.type } - .findFirst() - .orElseThrow() + .first() if (!isCursorType(cursorType)) { tablesWithInvalidCursor.add( @@ -303,7 +292,6 @@ protected constructor(driverClassName: String) : return (if (systemNameSpaces.isEmpty()) discoveredTables else discoveredTables - .stream() .filter { table: TableInfo> -> !systemNameSpaces.contains(table.nameSpace) && !systemViews.contains(table.name) } @@ -313,7 +301,7 @@ protected constructor(driverClassName: String) : protected fun getFullRefreshIterators( database: Database, catalog: ConfiguredAirbyteCatalog, - tableNameToTable: Map>>, + tableNameToTable: Map>>, stateManager: StateManager?, emittedAt: Instant ): List> { @@ -330,7 +318,7 @@ protected constructor(driverClassName: String) : protected open fun getIncrementalIterators( database: Database, catalog: ConfiguredAirbyteCatalog, - tableNameToTable: Map>>, + tableNameToTable: Map>>, stateManager: StateManager?, emittedAt: Instant ): List> { @@ -359,7 +347,7 @@ protected constructor(driverClassName: String) : private fun getSelectedIterators( database: Database, catalog: ConfiguredAirbyteCatalog?, - tableNameToTable: Map>>, + tableNameToTable: Map>>, stateManager: StateManager?, emittedAt: Instant, syncMode: SyncMode @@ -412,7 +400,6 @@ protected constructor(driverClassName: String) : val selectedFieldsInCatalog = CatalogHelpers.getTopLevelFieldNames(airbyteStream) val selectedDatabaseFields = table.fields - .stream() .map { obj: CommonField -> obj.name } .filter { o: String -> selectedFieldsInCatalog.contains(o) } .toList() @@ -532,14 +519,12 @@ protected constructor(driverClassName: String) : val cursorField = getCursorField(airbyteStream) val cursorType = table.fields - .stream() .filter { info: CommonField -> info.name == cursorField } .map { obj: CommonField -> obj.type } - .findFirst() - .orElseThrow() + .first() Preconditions.checkState( - table.fields.stream().anyMatch { f: CommonField -> f.name == cursorField }, + table.fields.any { f: CommonField -> f.name == cursorField }, String.format("Could not find cursor field %s in table %s", cursorField, table.name) ) 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 7b6a023062a2..26d82375e197 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 @@ -12,9 +12,7 @@ import io.airbyte.protocol.models.v0.CatalogHelpers import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.SyncMode import java.util.* -import java.util.function.Consumer import java.util.function.Function -import java.util.stream.Collectors import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -31,7 +29,7 @@ object DbSourceDiscoverUtil { */ @JvmStatic fun logSourceSchemaChange( - fullyQualifiedTableNameToInfo: Map>>, + fullyQualifiedTableNameToInfo: Map>>, catalog: ConfiguredAirbyteCatalog, airbyteTypeConverter: Function ) { @@ -44,7 +42,6 @@ object DbSourceDiscoverUtil { val table = fullyQualifiedTableNameToInfo[fullyQualifiedTableName]!! val fields = table.fields - .stream() .map { commonField: CommonField -> toField(commonField, airbyteTypeConverter) } @@ -89,7 +86,6 @@ object DbSourceDiscoverUtil { ): AirbyteCatalog { val tableInfoFieldList = tableInfos - .stream() .map { t: TableInfo> -> // some databases return multiple copies of the same record for a column (e.g. // redshift) because @@ -100,7 +96,6 @@ object DbSourceDiscoverUtil { assertColumnsWithSameNameAreSame(t.nameSpace, t.name, t.fields) val fields = t.fields - .stream() .map { commonField: CommonField -> toField(commonField, airbyteTypeConverter) } @@ -127,7 +122,6 @@ object DbSourceDiscoverUtil { .map { tableInfo: TableInfo -> val primaryKeys = tableInfo.primaryKeys - .stream() .filter { obj: String -> Objects.nonNull(obj) } .map { listOf(it) } .toList() @@ -164,7 +158,6 @@ object DbSourceDiscoverUtil { ) { val properties = commonField.properties - .stream() .map { commField: CommonField -> toField(commField, airbyteTypeConverter) } @@ -185,28 +178,23 @@ object DbSourceDiscoverUtil { columns: List> ) { columns - .stream() - .collect(Collectors.groupingBy(Function { obj: CommonField -> obj.name })) + .groupBy { obj: CommonField -> obj.name } .values - .forEach( - Consumer { columnsWithSameName: List> -> - val comparisonColumn = columnsWithSameName[0] - columnsWithSameName.forEach( - Consumer { column: CommonField -> - if (column != comparisonColumn) { - throw RuntimeException( - String.format( - "Found multiple columns with same name: %s in table: %s.%s but the columns are not the same. columns: %s", - comparisonColumn.name, - nameSpace, - tableName, - columns - ) - ) - } - } - ) + .forEach { columnsWithSameName: List> -> + val comparisonColumn = columnsWithSameName[0] + columnsWithSameName.forEach { column: CommonField -> + if (column != comparisonColumn) { + throw RuntimeException( + String.format( + "Found multiple columns with same name: %s in table: %s.%s but the columns are not the same. columns: %s", + comparisonColumn.name, + nameSpace, + tableName, + columns + ) + ) + } } - ) + } } } diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/InvalidCursorInfoUtil.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/InvalidCursorInfoUtil.kt index d2c8e2b5ee01..1a9547257bfc 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/InvalidCursorInfoUtil.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/InvalidCursorInfoUtil.kt @@ -3,15 +3,10 @@ */ package io.airbyte.cdk.integrations.source.relationaldb -import java.util.stream.Collectors - object InvalidCursorInfoUtil { fun getInvalidCursorConfigMessage(tablesWithInvalidCursor: List): String { return ("The following tables have invalid columns selected as cursor, please select a column with a well-defined ordering with no null values as a cursor. " + - tablesWithInvalidCursor - .stream() - .map { obj: InvalidCursorInfo -> obj.toString() } - .collect(Collectors.joining(","))) + tablesWithInvalidCursor.joinToString(",") { obj: InvalidCursorInfo -> obj.toString() }) } class InvalidCursorInfo( 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 5bf6fccc78a9..67b79bc3c7d8 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 @@ -10,7 +10,6 @@ import io.airbyte.commons.util.AutoCloseableIterator import io.airbyte.commons.util.AutoCloseableIterators import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import java.util.* -import java.util.stream.Collectors import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -102,10 +101,9 @@ object RelationalDbQueryUtils { } fun prettyPrintConfiguredAirbyteStreamList(streamList: List): String { - return streamList - .stream() - .map { s: ConfiguredAirbyteStream -> "${s.stream.namespace}.${s.stream.name}" } - .collect(Collectors.joining(", ")) + return streamList.joinToString(", ") { s: ConfiguredAirbyteStream -> + "${s.stream.namespace}.${s.stream.name}" + } } class TableSizeInfo(tableSize: Long, avgRowLength: Long) { diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbReadUtil.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbReadUtil.kt index 255ca51c885e..4e1a2a7fdf8d 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbReadUtil.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/RelationalDbReadUtil.kt @@ -19,7 +19,6 @@ object RelationalDbReadUtil { val newlyAddedStreams: Set = HashSet(Sets.difference(allStreams, alreadySyncedStreams)) return catalog.streams - .stream() .filter { c: ConfiguredAirbyteStream -> c.syncMode == SyncMode.INCREMENTAL } .filter { stream: ConfiguredAirbyteStream -> newlyAddedStreams.contains( @@ -42,7 +41,6 @@ object RelationalDbReadUtil { } .toSet() return catalog.streams - .stream() .filter { c: ConfiguredAirbyteStream -> c.syncMode == SyncMode.INCREMENTAL } .filter { stream: ConfiguredAirbyteStream -> !initialLoadStreamsNamespacePairs.contains( diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/AbstractStateManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/AbstractStateManager.kt index 935f8c6d008d..535f29ae112d 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/AbstractStateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/AbstractStateManager.kt @@ -27,7 +27,7 @@ constructor( cursorFunction: Function?, cursorFieldFunction: Function>?, cursorRecordCountFunction: Function?, - namespacePairFunction: Function?, + namespacePairFunction: Function, onlyIncludeIncrementalStreams: Boolean = false ) : StateManager { /** 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 3f78b9726e72..2db0937008da 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 @@ -10,7 +10,6 @@ import java.util.* import java.util.concurrent.* import java.util.function.Function import java.util.function.Supplier -import java.util.stream.Collectors import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -26,7 +25,7 @@ class CursorManager( cursorFunction: Function?, cursorFieldFunction: Function>?, cursorRecordCountFunction: Function?, - namespacePairFunction: Function?, + namespacePairFunction: Function, onlyIncludeIncrementalStreams: Boolean ) { /** @@ -87,7 +86,7 @@ class CursorManager( cursorFunction: Function?, cursorFieldFunction: Function>?, cursorRecordCountFunction: Function?, - namespacePairFunction: Function?, + namespacePairFunction: Function, onlyIncludeIncrementalStreams: Boolean ): Map { val allStreamNames = @@ -106,29 +105,17 @@ class CursorManager( allStreamNames.addAll( streamSupplier .get() - .stream() - .map(namespacePairFunction) + .map { namespacePairFunction.apply(it) } .filter { obj: AirbyteStreamNameNamespacePair? -> Objects.nonNull(obj) } - .collect(Collectors.toSet()) + .toSet() ) val localMap: MutableMap = ConcurrentHashMap() - val pairToState = - streamSupplier - .get() - .stream() - .collect(Collectors.toMap(namespacePairFunction, Function.identity())) + val pairToState = streamSupplier.get().associateBy { namespacePairFunction.apply(it) } val pairToConfiguredAirbyteStream = - catalog.streams - .stream() - .collect( - Collectors.toMap( - Function { stream: ConfiguredAirbyteStream -> - AirbyteStreamNameNamespacePair.fromConfiguredAirbyteSteam(stream) - }, - Function.identity() - ) - ) + catalog.streams.associateBy { + AirbyteStreamNameNamespacePair.fromConfiguredAirbyteSteam(it) + } for (pair in allStreamNames) { val stateOptional: Optional = Optional.ofNullable(pairToState[pair]) diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManager.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManager.kt index 9948a578b5dd..db5d05b22abe 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManager.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/main/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManager.kt @@ -11,7 +11,6 @@ import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.* import java.util.* import java.util.function.Supplier -import java.util.stream.Collectors /** * Global implementation of the [StateManager] interface. @@ -159,8 +158,7 @@ class GlobalStateManager( DbState::class.java )!! .streams - .stream() - .map { s: DbStreamState -> + .map { s: DbStreamState -> AirbyteStreamState() .withStreamState(Jsons.jsonNode(s)) .withStreamDescriptor( @@ -169,7 +167,7 @@ class GlobalStateManager( .withName(s.streamName) ) } - .collect(Collectors.toList()) + .toList() } else { return@Supplier listOf() } 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 0d249b11c46a..5283189c58cb 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 @@ -95,8 +95,7 @@ object StateGeneratorUtils { pairToCursorInfoMap: Map ): List { return pairToCursorInfoMap.entries - .stream() - .sorted(java.util.Map.Entry.comparingByKey()) + .sortedWith(java.util.Map.Entry.comparingByKey()) .map { e: Map.Entry -> generateStreamState(e.key, e.value) } @@ -118,8 +117,7 @@ object StateGeneratorUtils { .withCdc(false) .withStreams( pairToCursorInfoMap.entries - .stream() - .sorted( + .sortedWith( java.util.Map.Entry.comparingByKey() ) // sort by stream name then namespace for sanity. .map { e: Map.Entry -> @@ -206,7 +204,6 @@ object StateGeneratorUtils { .withSharedState(Jsons.jsonNode(dbState.cdcState)) .withStreamStates( dbState.streams - .stream() .map { s: DbStreamState -> AirbyteStreamState() .withStreamDescriptor( @@ -235,7 +232,6 @@ object StateGeneratorUtils { ): List { return Jsons.`object`(airbyteStateMessage.data, DbState::class.java)!! .streams - .stream() .map { s: DbStreamState -> AirbyteStateMessage() .withType(AirbyteStateMessage.AirbyteStateType.STREAM) 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 570f8b805cf4..fdd11857631b 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 @@ -34,7 +34,7 @@ open class StreamStateManager ) : AbstractStateManager( catalog, - Supplier { rawAirbyteStateMessages.stream().map { it.stream }.toList() }, + Supplier { rawAirbyteStateMessages.map { it.stream }.toList() }, StateGeneratorUtils.CURSOR_FUNCTION, StateGeneratorUtils.CURSOR_FIELD_FUNCTION, StateGeneratorUtils.CURSOR_RECORD_COUNT_FUNCTION, diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManagerTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManagerTest.kt index 2fb425949083..63421ca7fe45 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManagerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/GlobalStateManagerTest.kt @@ -98,9 +98,7 @@ class GlobalStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted(Comparator.comparing { obj: DbStreamState -> obj.streamName }) - .toList() + .sortedWith(Comparator.comparing { obj: DbStreamState -> obj.streamName }) ) val stateManager: StateManager = GlobalStateManager(AirbyteStateMessage().withData(Jsons.jsonNode(dbState)), catalog) @@ -126,9 +124,7 @@ class GlobalStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted(Comparator.comparing { obj: DbStreamState -> obj.streamName }) - .toList() + .sortedWith(Comparator.comparing { obj: DbStreamState -> obj.streamName }) ) val expectedGlobalState = @@ -184,13 +180,11 @@ class GlobalStateManagerTest { ) ) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { o: AirbyteStreamState -> o.streamDescriptor.name } ) - .toList() ) val expected = AirbyteStateMessage() @@ -279,9 +273,7 @@ class GlobalStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted(Comparator.comparing { obj: DbStreamState -> obj.streamName }) - .toList() + .sortedWith(Comparator.comparing { obj: DbStreamState -> obj.streamName }) ) val expectedGlobalState = @@ -337,13 +329,11 @@ class GlobalStateManagerTest { ) ) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { o: AirbyteStreamState -> o.streamDescriptor.name } ) - .toList() ) val expected = AirbyteStateMessage() diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManagerTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManagerTest.kt index 62ee96722e53..c88455f87019 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManagerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/LegacyStateManagerTest.kt @@ -150,13 +150,11 @@ class LegacyStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { obj: DbStreamState -> obj.streamName } ) - .toList() ) .withCdc(false) ) @@ -190,13 +188,11 @@ class LegacyStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { obj: DbStreamState -> obj.streamName } ) - .toList() ) .withCdc(false) ) @@ -248,13 +244,11 @@ class LegacyStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME2) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { obj: DbStreamState -> obj.streamName } ) - .toList() ) .withCdc(false) ) @@ -311,13 +305,11 @@ class LegacyStateManagerTest { .withStreamNamespace(StateTestConstants.NAMESPACE) .withCursorField(listOf()) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { obj: DbStreamState -> obj.streamName } ) - .toList() ) .withCdc(true) ) @@ -346,13 +338,11 @@ class LegacyStateManagerTest { .withCursorField(listOf()) .withCursor(null) ) - .stream() - .sorted( + .sortedWith( Comparator.comparing { obj: DbStreamState -> obj.streamName } ) - .toList() ) .withCdc(true) ) diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManagerTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManagerTest.kt index 26a100a9f020..8d05e260d212 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManagerTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/test/kotlin/io/airbyte/cdk/integrations/source/relationaldb/state/StreamStateManagerTest.kt @@ -185,9 +185,7 @@ class StreamStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted(Comparator.comparing { obj: DbStreamState -> obj.streamName }) - .toList() + .sortedWith(Comparator.comparing { obj: DbStreamState -> obj.streamName }) ) val expectedFirstEmission = createStreamState( @@ -228,9 +226,7 @@ class StreamStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME3) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted(Comparator.comparing { obj: DbStreamState -> obj.streamName }) - .toList() + .sortedWith(Comparator.comparing { obj: DbStreamState -> obj.streamName }) ) val expectedSecondEmission = createStreamState( @@ -394,9 +390,7 @@ class StreamStateManagerTest { .withStreamName(StateTestConstants.STREAM_NAME2) .withStreamNamespace(StateTestConstants.NAMESPACE) ) - .stream() - .sorted(Comparator.comparing { obj: DbStreamState -> obj.streamName }) - .toList() + .sortedWith(Comparator.comparing { obj: DbStreamState -> obj.streamName }) ) val expectedFirstEmission = 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 ca5e209b9579..f62439cf0081 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 @@ -287,7 +287,6 @@ abstract class CdcSourceTest> { protected fun extractStateMessages(messages: List): List { return messages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } .map { obj: AirbyteMessage -> obj.state } .toList() @@ -683,14 +682,12 @@ abstract class CdcSourceTest> { val actualCatalog = source().discover(config()!!) Assertions.assertEquals( - expectedCatalog.streams - .stream() - .sorted(Comparator.comparing { obj: AirbyteStream -> obj.name }) - .toList(), - actualCatalog.streams - .stream() - .sorted(Comparator.comparing { obj: AirbyteStream -> obj.name }) - .toList() + expectedCatalog.streams.sortedWith( + Comparator.comparing { obj: AirbyteStream -> obj.name } + ), + actualCatalog.streams.sortedWith( + Comparator.comparing { obj: AirbyteStream -> obj.name } + ) ) } @@ -1054,8 +1051,7 @@ abstract class CdcSourceTest> { @JvmField val MODEL_RECORDS_RANDOM: List = - MODEL_RECORDS.stream() - .map { r: JsonNode -> + MODEL_RECORDS.map { r: JsonNode -> Jsons.jsonNode( ImmutableMap.of( COL_ID + "_random", diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcSourceAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcSourceAcceptanceTest.kt index cf6bd80fd27f..da48b9ecfa88 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcSourceAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/source/jdbc/test/JdbcSourceAcceptanceTest.kt @@ -225,18 +225,15 @@ abstract class JdbcSourceAcceptanceTest> { actual.streams.forEach( Consumer { actualStream: AirbyteStream -> val expectedStream = - expected.streams - .stream() - .filter { stream: AirbyteStream -> - stream.namespace == actualStream.namespace && - stream.name == actualStream.name - } - .findAny() + expected.streams.firstOrNull { stream: AirbyteStream -> + stream.namespace == actualStream.namespace && + stream.name == actualStream.name + } Assertions.assertTrue( - expectedStream.isPresent, + expectedStream != null, String.format("Unexpected stream %s", actualStream.name) ) - Assertions.assertEquals(expectedStream.get(), actualStream) + Assertions.assertEquals(expectedStream, actualStream) } ) } @@ -256,14 +253,9 @@ abstract class JdbcSourceAcceptanceTest> { ) val actual = filterOutOtherSchemas(source().discover(config())) val stream = - actual.streams - .stream() - .filter { s: AirbyteStream -> - s.name.equals(TABLE_NAME_WITHOUT_CURSOR_TYPE, ignoreCase = true) - } - .findFirst() - .orElse(null) - Assertions.assertNotNull(stream) + actual.streams.first { s: AirbyteStream -> + s.name.equals(TABLE_NAME_WITHOUT_CURSOR_TYPE, ignoreCase = true) + } Assertions.assertEquals( TABLE_NAME_WITHOUT_CURSOR_TYPE.lowercase(Locale.getDefault()), stream.name.lowercase(Locale.getDefault()) @@ -288,13 +280,10 @@ abstract class JdbcSourceAcceptanceTest> { val actual = filterOutOtherSchemas(source().discover(config())) val stream = actual.streams - .stream() .filter { s: AirbyteStream -> s.name.equals(TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE, ignoreCase = true) } - .findFirst() - .orElse(null) - Assertions.assertNotNull(stream) + .first() Assertions.assertEquals( TABLE_NAME_WITH_NULLABLE_CURSOR_TYPE.lowercase(Locale.getDefault()), stream.name.lowercase(Locale.getDefault()) @@ -308,14 +297,11 @@ abstract class JdbcSourceAcceptanceTest> { if (supportsSchemas()) { val filteredCatalog = Jsons.clone(catalog) filteredCatalog.streams = - filteredCatalog.streams - .stream() - .filter { stream: AirbyteStream -> - TEST_SCHEMAS.stream().anyMatch { schemaName: String -> - stream.namespace.startsWith(schemaName) - } + filteredCatalog.streams.filter { stream: AirbyteStream -> + TEST_SCHEMAS.any { schemaName: String -> + stream.namespace.startsWith(schemaName) } - .toList() + } return filteredCatalog } else { return catalog @@ -420,9 +406,8 @@ abstract class JdbcSourceAcceptanceTest> { get() { val expectedMessages = testMessages - .stream() .map { `object`: AirbyteMessage -> Jsons.clone(`object`) } - .peek { m: AirbyteMessage -> + .onEach { m: AirbyteMessage -> (m.record.data as ObjectNode).remove(COL_NAME) (m.record.data as ObjectNode).remove(COL_UPDATED_AT) (m.record.data as ObjectNode).replace( @@ -430,7 +415,6 @@ abstract class JdbcSourceAcceptanceTest> { convertIdBasedOnDatabase(m.record.data[COL_ID].asInt()) ) } - .toList() return expectedMessages } @@ -471,9 +455,8 @@ abstract class JdbcSourceAcceptanceTest> { protected open fun getAirbyteMessagesSecondSync(streamName: String?): List { return testMessages - .stream() .map { `object`: AirbyteMessage -> Jsons.clone(`object`) } - .peek { m: AirbyteMessage -> + .onEach { m: AirbyteMessage -> m.record.stream = streamName m.record.namespace = defaultNamespace (m.record.data as ObjectNode).remove(COL_UPDATED_AT) @@ -482,7 +465,6 @@ abstract class JdbcSourceAcceptanceTest> { convertIdBasedOnDatabase(m.record.data[COL_ID].asInt()) ) } - .toList() } @Test @@ -514,9 +496,8 @@ abstract class JdbcSourceAcceptanceTest> { streamForTableWithSpaces: ConfiguredAirbyteStream ): List { return testMessages - .stream() .map { `object`: AirbyteMessage -> Jsons.clone(`object`) } - .peek { m: AirbyteMessage -> + .onEach { m: AirbyteMessage -> m.record.stream = streamForTableWithSpaces.stream.name (m.record.data as ObjectNode).set( COL_LAST_NAME_WITH_SPACE, @@ -528,7 +509,6 @@ abstract class JdbcSourceAcceptanceTest> { convertIdBasedOnDatabase(m.record.data[COL_ID].asInt()) ) } - .toList() } @Test @@ -658,27 +638,19 @@ abstract class JdbcSourceAcceptanceTest> { val stateAfterFirstSyncOptional = actualMessagesFirstSync - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } - .findFirst() - Assertions.assertTrue(stateAfterFirstSyncOptional.isPresent) + .first() executeStatementReadIncrementallyTwice() val actualMessagesSecondSync = MoreIterators.toList( - source() - .read( - config, - configuredCatalog, - extractState(stateAfterFirstSyncOptional.get()) - ) + source().read(config, configuredCatalog, extractState(stateAfterFirstSyncOptional)) ) Assertions.assertEquals( 2, actualMessagesSecondSync - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.RECORD } .count() .toInt() @@ -799,11 +771,9 @@ abstract class JdbcSourceAcceptanceTest> { // get last state message. val stateAfterFirstSyncOptional = - actualMessagesFirstSync - .stream() - .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } - .reduce { first: AirbyteMessage?, second: AirbyteMessage -> second } - Assertions.assertTrue(stateAfterFirstSyncOptional.isPresent) + actualMessagesFirstSync.last { r: AirbyteMessage -> + r.type == AirbyteMessage.Type.STATE + } // we know the second streams messages are the same as the first minus the updated at // column. so we @@ -862,9 +832,8 @@ abstract class JdbcSourceAcceptanceTest> { streamName2: String? ): List { return testMessages - .stream() .map { `object`: AirbyteMessage -> Jsons.clone(`object`) } - .peek { m: AirbyteMessage -> + .onEach { m: AirbyteMessage -> m.record.stream = streamName2 (m.record.data as ObjectNode).remove(COL_UPDATED_AT) (m.record.data as ObjectNode).replace( @@ -872,7 +841,6 @@ abstract class JdbcSourceAcceptanceTest> { convertIdBasedOnDatabase(m.record.data[COL_ID].asInt()) ) } - .toList() } // when initial and final cursor fields are the same. @@ -960,11 +928,9 @@ abstract class JdbcSourceAcceptanceTest> { // cursor after 1st sync: 2021-01-01 00:00:00, count 2 val firstSyncStateOptional = firstSyncActualMessages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } - .findFirst() - Assertions.assertTrue(firstSyncStateOptional.isPresent) - val firstSyncState = getStateData(firstSyncStateOptional.get(), TABLE_NAME_AND_TIMESTAMP) + .first() + val firstSyncState = getStateData(firstSyncStateOptional, TABLE_NAME_AND_TIMESTAMP) Assertions.assertEquals( firstSyncState["cursor_field"].elements().next().asText(), COL_TIMESTAMP @@ -975,7 +941,6 @@ abstract class JdbcSourceAcceptanceTest> { val firstSyncNames = firstSyncActualMessages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.RECORD } .map { r: AirbyteMessage -> r.record.data[COL_NAME].asText() } .toList() @@ -1013,11 +978,9 @@ abstract class JdbcSourceAcceptanceTest> { // cursor after 2nd sync: 2021-01-02 00:00:00, count 1 val secondSyncStateOptional = secondSyncActualMessages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } - .findFirst() - Assertions.assertTrue(secondSyncStateOptional.isPresent) - val secondSyncState = getStateData(secondSyncStateOptional.get(), TABLE_NAME_AND_TIMESTAMP) + .first() + val secondSyncState = getStateData(secondSyncStateOptional, TABLE_NAME_AND_TIMESTAMP) Assertions.assertEquals( secondSyncState["cursor_field"].elements().next().asText(), COL_TIMESTAMP @@ -1028,7 +991,6 @@ abstract class JdbcSourceAcceptanceTest> { val secondSyncNames = secondSyncActualMessages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.RECORD } .map { r: AirbyteMessage -> r.record.data[COL_NAME].asText() } .toList() @@ -1068,11 +1030,9 @@ abstract class JdbcSourceAcceptanceTest> { // Cursor after 3rd sync is: 2021-01-03 00:00:00, count 1. val thirdSyncStateOptional = thirdSyncActualMessages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } - .findFirst() - Assertions.assertTrue(thirdSyncStateOptional.isPresent) - val thirdSyncState = getStateData(thirdSyncStateOptional.get(), TABLE_NAME_AND_TIMESTAMP) + .first() + val thirdSyncState = getStateData(thirdSyncStateOptional, TABLE_NAME_AND_TIMESTAMP) Assertions.assertEquals( thirdSyncState["cursor_field"].elements().next().asText(), COL_TIMESTAMP @@ -1085,7 +1045,6 @@ abstract class JdbcSourceAcceptanceTest> { // record count in the database is different from that in the state. val thirdSyncExpectedNames = thirdSyncActualMessages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.RECORD } .map { r: AirbyteMessage -> r.record.data[COL_NAME].asText() } .toList() @@ -1311,7 +1270,6 @@ abstract class JdbcSourceAcceptanceTest> { numRecords: Long ): List { return states - .stream() .map { s: DbStreamState -> AirbyteMessage() .withType(AirbyteMessage.Type.STATE) @@ -1338,7 +1296,6 @@ abstract class JdbcSourceAcceptanceTest> { protected open fun createState(states: List): List { return states - .stream() .map { s: DbStreamState -> AirbyteStateMessage() .withType(AirbyteStateMessage.AirbyteStateType.STREAM) @@ -1545,7 +1502,6 @@ abstract class JdbcSourceAcceptanceTest> { field: String? ): List { return extractStateMessage(messages) - .stream() .filter { s: AirbyteStateMessage -> s.stream.streamDescriptor.name == streamName } .map { s: AirbyteStateMessage -> if (s.stream.streamState[field] != null) s.stream.streamState[field].asText() @@ -1556,14 +1512,12 @@ abstract class JdbcSourceAcceptanceTest> { protected fun filterRecords(messages: List): List { return messages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.RECORD } .toList() } protected fun extractStateMessage(messages: List): List { return messages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } .map { obj: AirbyteMessage -> obj.state } .toList() @@ -1574,7 +1528,6 @@ abstract class JdbcSourceAcceptanceTest> { streamName: String ): List { return messages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE && r.state.stream.streamDescriptor.name == streamName 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 1af04e601b0f..5846fdcfeaff 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 @@ -13,8 +13,6 @@ import io.airbyte.protocol.models.v0.* import java.io.IOException import java.sql.SQLException import java.util.function.Consumer -import java.util.function.Function -import java.util.stream.Collectors import org.apache.commons.lang3.StringUtils import org.jooq.DSLContext import org.junit.jupiter.api.Assertions @@ -89,15 +87,7 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { fun testDataTypes() { if (testCatalog()) { runDiscover() - val streams = - lastPersistedCatalog.streams - .stream() - .collect( - Collectors.toMap( - Function { obj: AirbyteStream -> obj.name }, - Function { s: AirbyteStream -> s } - ) - ) + val streams = lastPersistedCatalog.streams.associateBy { it.name } // testDataHolders should be initialized using the `addDataTypeTestData` function testDataHolders.forEach( @@ -140,7 +130,6 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { val recordMessages = allMessages - .stream() .filter { m: AirbyteMessage -> m.type == AirbyteMessage.Type.RECORD } .toList() val expectedValues: MutableMap?> = HashMap() @@ -284,7 +273,6 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { ConfiguredAirbyteCatalog() .withStreams( testDataHolders - .stream() .map { test: TestDataHolder -> ConfiguredAirbyteStream() .withSyncMode(SyncMode.INCREMENTAL) @@ -322,10 +310,7 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { fun addDataTypeTestData(test: TestDataHolder) { testDataHolders.add(test) test.setTestNumber( - testDataHolders - .stream() - .filter { t: TestDataHolder -> t.sourceType == test.sourceType } - .count() + testDataHolders.filter { t: TestDataHolder -> t.sourceType == test.sourceType }.count() ) test.nameSpace = nameSpace test.setIdColumnName(idColumnName) @@ -334,7 +319,7 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { } private fun formatCollection(collection: Collection): String { - return collection.stream().map { s: String? -> "`$s`" }.collect(Collectors.joining(", ")) + return collection.joinToString(", ") { s: String? -> "`$s`" } } val markdownTestTable: String @@ -406,7 +391,6 @@ abstract class AbstractSourceDatabaseTypeTest : AbstractSourceConnectorTest() { protected fun extractStateMessages(messages: List): List { return messages - .stream() .filter { r: AirbyteMessage -> r.type == AirbyteMessage.Type.STATE } .map { obj: AirbyteMessage -> obj.state } .toList() 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 1f586965807f..849e0376ba4f 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 @@ -55,18 +55,13 @@ class PythonSourceAcceptanceTest : SourceAcceptanceTest() { .map { obj: JsonNode -> obj.textValue() } .toList() val stringMessages = - allMessages - .stream() - .map { `object`: AirbyteMessage -> Jsons.serialize(`object`) } - .toList() + allMessages.map { `object`: AirbyteMessage -> Jsons.serialize(`object`) }.toList() LOGGER.info("Running " + regexTests.size + " regex tests...") regexTests.forEach( Consumer { regex: String -> LOGGER.info("Looking for [$regex]") Assertions.assertTrue( - stringMessages.stream().anyMatch { line: String -> - line.matches(regex.toRegex()) - }, + stringMessages.any { line: String -> line.matches(regex.toRegex()) }, "Failed to find regex: $regex" ) } 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 031396c85293..0b2c3cfad509 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 @@ -212,7 +212,6 @@ abstract class SourceAcceptanceTest : AbstractSourceConnectorTest() { // only sync incremental streams configuredCatalog.streams = configuredCatalog.streams - .stream() .filter { s: ConfiguredAirbyteStream -> s.syncMode == SyncMode.INCREMENTAL } .toList() @@ -220,7 +219,6 @@ abstract class SourceAcceptanceTest : AbstractSourceConnectorTest() { val recordMessages = filterRecords(airbyteMessages) val stateMessages = airbyteMessages - .stream() .filter { m: AirbyteMessage -> m.type == AirbyteMessage.Type.STATE } .map { obj: AirbyteMessage -> obj.state } .toList() @@ -375,10 +373,9 @@ abstract class SourceAcceptanceTest : AbstractSourceConnectorTest() { message: String ) { val prunedExpected = - expected.stream().map { m: AirbyteRecordMessage -> this.pruneEmittedAt(m) }.toList() + expected.map { m: AirbyteRecordMessage -> this.pruneEmittedAt(m) }.toList() val prunedActual = actual - .stream() .map { m: AirbyteRecordMessage -> this.pruneEmittedAt(m) } .map { m: AirbyteRecordMessage -> this.pruneCdcMetadata(m) } .toList() @@ -419,7 +416,6 @@ abstract class SourceAcceptanceTest : AbstractSourceConnectorTest() { messages: Collection ): List { return messages - .stream() .filter { m: AirbyteMessage -> m.type == AirbyteMessage.Type.RECORD } .map { obj: AirbyteMessage -> obj.record } .toList() diff --git a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestDataHolder.kt b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestDataHolder.kt index 4df587de1382..72acbf38b5fb 100644 --- a/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestDataHolder.kt +++ b/airbyte-cdk/java/airbyte-cdk/db-sources/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/source/TestDataHolder.kt @@ -17,7 +17,7 @@ internal constructor( private val fullSourceDataType: String? ) { var nameSpace: String? = null - private var testNumber: Long = 0 + private var testNumber: Int = 0 private var idColumnName: String? = null private var testColumnName: String? = null @@ -154,7 +154,7 @@ internal constructor( } } - fun setTestNumber(testNumber: Long) { + fun setTestNumber(testNumber: Int) { this.testNumber = testNumber } 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 d4139d9fc7f1..18e56fd1b5c5 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 @@ -9,7 +9,6 @@ import io.airbyte.commons.version.AirbyteVersion import java.util.* import java.util.function.Function import java.util.function.Supplier -import java.util.stream.Collectors import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -69,33 +68,16 @@ class TestEnvConfigs private constructor(envMap: Map) { val jobPrefixedEnvMap = getAllEnvKeys .get() - .stream() - .filter { key: String -> key.startsWith(JOB_DEFAULT_ENV_PREFIX) } - .collect( - Collectors.toMap( - Function { key: String -> key.replace(JOB_DEFAULT_ENV_PREFIX, "") }, - getEnv - ) - ) + .filter { it.startsWith(JOB_DEFAULT_ENV_PREFIX) } + .associate { it.replace(JOB_DEFAULT_ENV_PREFIX, "") to getEnv(it) } + // This method assumes that these shared env variables are not critical to the execution // of the jobs, and only serve as metadata. So any exception is swallowed and default to // an empty string. Change this logic if this assumption no longer holds. val jobSharedEnvMap = - JOB_SHARED_ENVS.entries - .stream() - .collect( - Collectors.toMap( - Function { obj: Map.Entry> -> - obj.key - }, - Function { entry: Map.Entry> -> - Exceptions.swallowWithDefault( - { Objects.requireNonNullElse(entry.value.apply(this), "") }, - "" - ) - } - ) - ) + JOB_SHARED_ENVS.entries.associate { + it.key to Exceptions.swallowWithDefault({ it.value.apply(this) ?: "" }, "") + } return MoreMaps.merge(jobPrefixedEnvMap, jobSharedEnvMap) } 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 6e7848e02d47..cdf43a25adbe 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,8 +9,6 @@ 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 java.util.function.Function -import java.util.stream.Collectors import java.util.stream.Stream import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.TestInstance @@ -81,16 +79,7 @@ abstract class AbstractSourcePerformanceTest : AbstractSourceBasePerformanceTest protected fun validateNumberOfReceivedMsgs(checkStatusMap: Map) { // Iterate through all streams map and check for streams where - val failedStreamsMap = - checkStatusMap.entries - .stream() - .filter { el: Map.Entry -> el.value != 0 } - .collect( - Collectors.toMap( - Function { obj: Map.Entry -> obj.key }, - Function { obj: Map.Entry -> obj.value } - ) - ) + val failedStreamsMap = checkStatusMap.filterValues { it != 0 } if (failedStreamsMap.isNotEmpty()) { Assertions.fail("Non all messages were delivered. $failedStreamsMap") 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 a5a4c97d9159..8553edb6a9ce 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 @@ -168,7 +168,6 @@ object JsonPaths { */ fun getPaths(json: JsonNode?, jsonPath: String): List { return getInternal(GET_PATHS_CONFIGURATION, json, jsonPath) - .stream() .map { obj: JsonNode -> obj.asText() } .toList() } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonSchemas.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonSchemas.kt index 7b1076cfa8be..79bd68f348d5 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonSchemas.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/commons/json/JsonSchemas.kt @@ -150,7 +150,7 @@ object JsonSchemas { traverseJsonSchema(jsonSchema) { node: JsonNode?, path: List -> mapper.apply(node, path).ifPresent { e: T -> collector.add(e) } } - return collector.stream().toList() // make list unmodifiable + return collector.toList() // make list unmodifiable } /** @@ -301,7 +301,6 @@ object JsonSchemas { if (jsonSchema.has(JSON_SCHEMA_TYPE_KEY)) { return if (jsonSchema[JSON_SCHEMA_TYPE_KEY].isArray) { MoreIterators.toList(jsonSchema[JSON_SCHEMA_TYPE_KEY].iterator()) - .stream() .map { obj: JsonNode -> obj.asText() } .toList() } else { diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/helpers/StateMessageHelper.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/helpers/StateMessageHelper.kt index 5945da2c6113..a790a2c39e92 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/helpers/StateMessageHelper.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/main/kotlin/io/airbyte/configoss/helpers/StateMessageHelper.kt @@ -58,14 +58,14 @@ object StateMessageHelper { } } else { if ( - stateMessages.stream().allMatch { stateMessage: AirbyteStateMessage -> + stateMessages.all { stateMessage: AirbyteStateMessage -> stateMessage.type == AirbyteStateMessage.AirbyteStateType.STREAM } ) { return Optional.of(provideStreamState(stateMessages)) } if ( - stateMessages.stream().allMatch { stateMessage: AirbyteStateMessage -> + stateMessages.all { stateMessage: AirbyteStateMessage -> stateMessage.type == null } ) { 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 c06001032560..695ceae76b23 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 @@ -94,14 +94,12 @@ class JsonSchemaValidator @VisibleForTesting constructor(private val baseUri: UR fun getValidationMessageArgs(schemaJson: JsonNode, objectJson: JsonNode): List> { return validateInternal(schemaJson, objectJson) - .stream() .map { obj: ValidationMessage -> obj.arguments } .toList() } fun getValidationMessagePaths(schemaJson: JsonNode, objectJson: JsonNode): List { return validateInternal(schemaJson, objectJson) - .stream() .map { obj: ValidationMessage -> obj.path } .toList() } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/concurrency/CompletableFuturesTest.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/concurrency/CompletableFuturesTest.kt index e2b4a0654666..e2b95daef178 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/concurrency/CompletableFuturesTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/concurrency/CompletableFuturesTest.kt @@ -26,8 +26,7 @@ internal class CompletableFuturesTest { val allOfResult = CompletableFutures.allOf(futures).toCompletableFuture() val result = allOfResult.join() - val success = - result.stream().filter { obj: Either -> obj.isRight() }.toList() + val success = result.filter { obj: Either -> obj.isRight() }.toList() Assertions.assertEquals( success, Arrays.asList( @@ -40,7 +39,6 @@ internal class CompletableFuturesTest { // Extract wrapped CompletionException messages. val failureMessages = result - .stream() .filter { obj: Either -> obj.isLeft() } .map { either: Either -> either.left!!.cause!!.message } .toList() diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/json/JsonPathsTest.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/json/JsonPathsTest.kt index 5c26f9859df1..ca7f6c592613 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/json/JsonPathsTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/test/kotlin/io/airbyte/commons/json/JsonPathsTest.kt @@ -16,27 +16,24 @@ internal class JsonPathsTest { Assertions.assertEquals( listOf(0, 1, 2), JsonPaths.getValues(JSON_NODE, LIST_ALL_QUERY) - .stream() .map { obj: JsonNode -> obj.asInt() } .toList() ) Assertions.assertEquals( listOf(1), JsonPaths.getValues(JSON_NODE, LIST_ONE_QUERY) - .stream() .map { obj: JsonNode -> obj.asInt() } .toList() ) Assertions.assertEquals( listOf(10), JsonPaths.getValues(JSON_NODE, NESTED_FIELD_QUERY) - .stream() .map { obj: JsonNode -> obj.asInt() } .toList() ) Assertions.assertEquals( JSON_NODE["two"], - JsonPaths.getValues(JSON_NODE, JSON_OBJECT_QUERY).stream().findFirst().orElse(null) + JsonPaths.getValues(JSON_NODE, JSON_OBJECT_QUERY).firstOrNull() ) Assertions.assertEquals( emptyList(), 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 8a43668f6711..1375945d605d 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 @@ -117,30 +117,26 @@ object TestHarnessUtils { fun getMostRecentConfigControlMessage( messagesByType: Map> ): Optional { - return messagesByType - .getOrDefault(AirbyteMessage.Type.CONTROL, ArrayList()) - .stream() - .map { obj: AirbyteMessage -> obj.control } - .filter { control: AirbyteControlMessage -> - control.type == AirbyteControlMessage.Type.CONNECTOR_CONFIG - } - .map { obj: AirbyteControlMessage -> obj.connectorConfig } - .reduce { - first: AirbyteControlConnectorConfigMessage?, - second: AirbyteControlConnectorConfigMessage -> - second - } + return Optional.ofNullable( + messagesByType + .getOrDefault(AirbyteMessage.Type.CONTROL, ArrayList()) + .map { obj: AirbyteMessage -> obj.control } + .filter { control: AirbyteControlMessage -> + control.type == AirbyteControlMessage.Type.CONNECTOR_CONFIG + } + .map { obj: AirbyteControlMessage -> obj.connectorConfig } + .lastOrNull() + ) } private fun getTraceMessageFromMessagesByType( messagesByType: Map> - ): Optional { + ): AirbyteTraceMessage? { return messagesByType .getOrDefault(AirbyteMessage.Type.TRACE, ArrayList()) - .stream() .map { obj: AirbyteMessage -> obj.trace } .filter { trace: AirbyteTraceMessage -> trace.type == AirbyteTraceMessage.Type.ERROR } - .findFirst() + .firstOrNull() } fun getDidControlMessageChangeConfig( @@ -174,15 +170,10 @@ object TestHarnessUtils { messagesByType: Map> ): Optional { val traceMessage = getTraceMessageFromMessagesByType(messagesByType) - if (traceMessage.isPresent) { + if (traceMessage != null) { val connectorCommand = getConnectorCommandFromOutputType(outputType) return Optional.of( - FailureHelper.connectorCommandFailure( - traceMessage.get(), - null, - null, - connectorCommand - ) + FailureHelper.connectorCommandFailure(traceMessage, null, null, connectorCommand) ) } else { return Optional.empty() 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 c95da65ed1d1..d40865d35efe 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 @@ -51,9 +51,8 @@ constructor( val connectionStatus = messagesByType .getOrDefault(AirbyteMessage.Type.CONNECTION_STATUS, ArrayList()) - .stream() .map { obj: AirbyteMessage -> obj.connectionStatus } - .findFirst() + .firstOrNull() if (input.actorId != null && input.actorType != null) { val optionalConfigMsg = @@ -95,16 +94,16 @@ constructor( LOGGER.warn("Check connection job subprocess finished with exit code {}", exitCode) } - if (connectionStatus.isPresent) { + if (connectionStatus != null) { val output = StandardCheckConnectionOutput() .withStatus( Enums.convertTo( - connectionStatus.get().status, + connectionStatus.status, StandardCheckConnectionOutput.Status::class.java ) ) - .withMessage(connectionStatus.get().message) + .withMessage(connectionStatus.message) LOGGER.info("Check connection job received output: {}", output) jobOutput.checkConnection = output } else if (failureReason.isEmpty) { 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 61e9884a39c9..fb67ac881dbb 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 @@ -60,9 +60,8 @@ constructor( val catalog = messagesByType .getOrDefault(AirbyteMessage.Type.CATALOG, ArrayList()) - .stream() .map { obj: AirbyteMessage -> obj.catalog } - .findFirst() + .firstOrNull() val optionalConfigMsg = TestHarnessUtils.getMostRecentConfigControlMessage(messagesByType) @@ -92,14 +91,14 @@ constructor( LOGGER.warn("Discover job subprocess finished with exit codee {}", exitCode) } - if (catalog.isPresent) { + if (catalog != null) { val result = AirbyteApiClient.retryWithJitter( { airbyteApiClient.sourceApi.writeDiscoverCatalogResult( buildSourceDiscoverSchemaWriteRequestBody( discoverSchemaInput, - catalog.get() + catalog ) ) }, 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 387c54d3726c..b584a394adcf 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 @@ -39,9 +39,8 @@ constructor( val spec = messagesByType .getOrDefault(AirbyteMessage.Type.SPEC, ArrayList())!! - .stream() .map { obj: AirbyteMessage -> obj.spec } - .findFirst() + .firstOrNull() val failureReason = TestHarnessUtils.getJobFailureReasonFromMessages( @@ -57,8 +56,8 @@ constructor( LOGGER.warn("Spec job subprocess finished with exit code {}", exitCode) } - if (spec.isPresent) { - jobOutput.spec = spec.get() + if (spec != null) { + jobOutput.spec = spec } else if (failureReason.isEmpty) { TestHarnessUtils.throwWorkerException( "Integration failed to output a spec struct and did not output a failure reason", diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/CatalogClientConverters.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/CatalogClientConverters.kt index a480ef7bab05..fbefb4d4b0fa 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/CatalogClientConverters.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/CatalogClientConverters.kt @@ -27,7 +27,6 @@ object CatalogClientConverters { val protoCatalog = io.airbyte.protocol.models.AirbyteCatalog() val airbyteStream = catalog.streams - .stream() .map { stream: AirbyteStreamAndConfiguration -> try { return@map toConfiguredProtocol(stream.stream, stream.config) @@ -133,7 +132,6 @@ object CatalogClientConverters { return AirbyteCatalog() .streams( catalog.streams - .stream() .map { stream: io.airbyte.protocol.models.AirbyteStream -> toAirbyteStreamClientApi(stream) } diff --git a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/FailureHelper.kt b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/FailureHelper.kt index 9ba16639e1fc..1e9d4f401d97 100644 --- a/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/FailureHelper.kt +++ b/airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/helper/FailureHelper.kt @@ -158,7 +158,7 @@ object FailureHelper { } val compareByTimestamp = Comparator.comparing { obj: FailureReason -> obj.timestamp } val compareByTraceAndTimestamp = compareByIsTrace.thenComparing(compareByTimestamp) - return failures.stream().sorted(compareByTraceAndTimestamp).toList() + return failures.sortedWith(compareByTraceAndTimestamp).toList() } enum class ConnectorCommand(private val value: String) { 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 715d6c934274..c5641a2c97fe 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 @@ -255,14 +255,10 @@ class DefaultNormalizationRunner( override val traceMessages: Stream get() { - if ( - airbyteMessagesByType != null && - airbyteMessagesByType!![AirbyteMessage.Type.TRACE] != null - ) { - return airbyteMessagesByType!![AirbyteMessage.Type.TRACE]!!.stream().map { - obj: AirbyteMessage -> - obj.trace - } + if (airbyteMessagesByType[AirbyteMessage.Type.TRACE] != null) { + return airbyteMessagesByType[AirbyteMessage.Type.TRACE]!! + .map { obj: AirbyteMessage -> obj.trace } + .stream() } return Stream.empty() } diff --git a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsAvroParquetDestinationAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsAvroParquetDestinationAcceptanceTest.kt index 05fb1799c1dc..f9f066a5d234 100644 --- a/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsAvroParquetDestinationAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/gcs-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/gcs/GcsAvroParquetDestinationAcceptanceTest.kt @@ -63,16 +63,7 @@ abstract class GcsAvroParquetDestinationAcceptanceTest(fileUploadFormat: FileUpl ) ) - return nameToNode.entries - .stream() - .collect( - Collectors.toMap( - Function { obj: Map.Entry -> obj.key }, - Function { entry: Map.Entry -> - getExpectedSchemaType(entry.value) - } - ) - ) + return nameToNode.entries.associate { it.key to getExpectedSchemaType(it.value) } } private fun getJsonNode(stream: AirbyteStream, name: String): JsonNode { @@ -134,44 +125,29 @@ abstract class GcsAvroParquetDestinationAcceptanceTest(fileUploadFormat: FileUpl protected fun getTypes(record: GenericData.Record): Map> { val fieldList = record.schema.fields - .stream() .filter { field: Schema.Field -> !field.name().startsWith("_airbyte") } .toList() return if (fieldList.size == 1) { - fieldList - .stream() - .collect( - Collectors.toMap( - Function { obj: Schema.Field -> obj.name() }, - Function { field: Schema.Field -> - field - .schema() - .types - .map { obj: Schema -> obj.type } - .filter { type: Schema.Type -> type != Schema.Type.NULL } - .toSet() - } - ) - ) + fieldList.associate { + it.name() to + it.schema() + .types + .map { obj: Schema -> obj.type } + .filter { type: Schema.Type -> type != Schema.Type.NULL } + .toSet() + } } else { - fieldList - .stream() - .collect( - Collectors.toMap( - Function { obj: Schema.Field -> obj.name() }, - Function { field: Schema.Field -> - field - .schema() - .types - .filter { type: Schema -> type.type != Schema.Type.NULL } - .flatMap { type: Schema -> type.elementType.types } - .map { obj: Schema -> obj.type } - .filter { type: Schema.Type -> type != Schema.Type.NULL } - .toSet() - } - ) - ) + fieldList.associate { + it.name() to + it.schema() + .types + .filter { type: Schema -> type.type != Schema.Type.NULL } + .flatMap { type: Schema -> type.elementType.types } + .map { obj: Schema -> obj.type } + .filter { type: Schema.Type -> type != Schema.Type.NULL } + .toSet() + } } } } 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 b286fd92ad9a..78dd8c8c51e5 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 @@ -115,16 +115,13 @@ abstract class GcsDestinationAcceptanceTest(protected val outputFormat: FileUplo s3Client .listObjects(config.bucketName, parentFolder) .objectSummaries - .stream() .filter { o: S3ObjectSummary -> o.key.contains("$streamNameStr/") } - .sorted(Comparator.comparingLong { o: S3ObjectSummary -> o.lastModified.time }) - .toList() + .sortedWith(Comparator.comparingLong { o: S3ObjectSummary -> o.lastModified.time }) LOGGER.info( "All objects: {}", - objectSummaries - .stream() - .map { o: S3ObjectSummary -> String.format("%s/%s", o.bucketName, o.key) } - .toList() + objectSummaries.map { o: S3ObjectSummary -> + String.format("%s/%s", o.bucketName, o.key) + } ) return objectSummaries } 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 5d81dc60670d..fd5a45cc3fbd 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 @@ -18,7 +18,6 @@ import io.airbyte.commons.json.Jsons import io.airbyte.protocol.models.v0.* import java.util.function.Consumer import java.util.function.Function -import java.util.stream.Collectors import org.apache.commons.io.FileUtils import org.joda.time.DateTime import org.joda.time.DateTimeZone @@ -91,15 +90,7 @@ class S3ConsumerFactory { writeConfigs: List, catalog: ConfiguredAirbyteCatalog? ): FlushBufferFunction { - val pairToWriteConfig = - writeConfigs - .stream() - .collect( - Collectors.toUnmodifiableMap( - Function { config: WriteConfig -> toNameNamespacePair(config) }, - Function.identity() - ) - ) + val pairToWriteConfig = writeConfigs.associateBy { toNameNamespacePair(it) } return FlushBufferFunction { pair: AirbyteStreamNameNamespacePair, @@ -164,7 +155,7 @@ class S3ConsumerFactory { config: S3DestinationConfig, catalog: ConfiguredAirbyteCatalog? ): List { - return catalog!!.streams.stream().map(toWriteConfig(storageOperations, config)).toList() + return catalog!!.streams.map { toWriteConfig(storageOperations, config).apply(it) } } private fun toWriteConfig( diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3StorageOperations.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3StorageOperations.kt index 781830c8bc8e..267710949141 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3StorageOperations.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/S3StorageOperations.kt @@ -141,15 +141,13 @@ open class S3StorageOperations( // issue reduces risk of misidentifying errors or reporting a transient error. val areAllExceptionsAuthExceptions: Boolean = exceptionsThrown - .stream() - .filter { e: Exception -> e is AmazonS3Exception } + .filterIsInstance() .map { s3e: Exception -> (s3e as AmazonS3Exception).statusCode } - .filter { o: Int -> + .count { o: Int -> ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains( o, ) - } - .count() == exceptionsThrown.size.toLong() + } == exceptionsThrown.size if (areAllExceptionsAuthExceptions) { throw ConfigErrorException(exceptionsThrown[0].message!!, exceptionsThrown[0]) } else { @@ -310,7 +308,6 @@ open class S3StorageOperations( while (objects.objectSummaries.size > 0) { val keysToDelete: List = objects.objectSummaries - .stream() .filter { obj: S3ObjectSummary -> regexFormat .matcher( @@ -363,7 +360,6 @@ open class S3StorageOperations( while (objects.objectSummaries.size > 0) { val keysToDelete: List = objects.objectSummaries - .stream() .filter { obj: S3ObjectSummary -> stagedFiles.isEmpty() || stagedFiles.contains( @@ -394,7 +390,7 @@ open class S3StorageOperations( ) { if (keysToDelete.isNotEmpty()) { logger.info { - "Deleting objects ${keysToDelete.stream().map { obj: DeleteObjectsRequest.KeyVersion -> obj.key } + "Deleting objects ${keysToDelete.map { obj: DeleteObjectsRequest.KeyVersion -> obj.key } .toList().joinToString(separator = ", ")}" } s3Client.deleteObjects(DeleteObjectsRequest(bucket).withKeys(keysToDelete)) diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/AvroSerializedBuffer.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/AvroSerializedBuffer.kt index 0e3875f47ff8..5f5464a25360 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/AvroSerializedBuffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/AvroSerializedBuffer.kt @@ -94,7 +94,6 @@ class AvroSerializedBuffer( val schema = schemaConverter.getAvroSchema( catalog.streams - .stream() .filter { s: ConfiguredAirbyteStream -> s.stream.name == stream.name && StringUtils.equals( @@ -102,14 +101,12 @@ class AvroSerializedBuffer( stream.namespace, ) } - .findFirst() - .orElseThrow { - RuntimeException( - "No such stream ${stream.namespace}.${stream.name}" - ) - } - .stream - .jsonSchema, + .firstOrNull() + ?.stream + ?.jsonSchema + ?: throw RuntimeException( + "No such stream ${stream.namespace}.${stream.name}" + ), stream.name, stream.namespace, ) diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroSchemaConverter.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroSchemaConverter.kt index a04b3a27ac63..254146e413ce 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroSchemaConverter.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroSchemaConverter.kt @@ -14,8 +14,6 @@ import java.util.LinkedList import java.util.Objects import java.util.Optional import java.util.function.Predicate -import java.util.stream.Collectors -import java.util.stream.Stream import org.apache.avro.LogicalTypes import org.apache.avro.Schema import org.apache.avro.SchemaBuilder @@ -350,7 +348,6 @@ class JsonToAvroSchemaConverter { ): MutableList { val schemas: List = MoreIterators.toList(types.elements()) - .stream() .flatMap { definition: JsonNode -> getSchemas( fieldName = fieldName, @@ -372,8 +369,8 @@ class JsonToAvroSchemaConverter { definition: JsonNode, appendExtraProps: Boolean, addStringToLogicalTypes: Boolean - ): Stream? { - return getNonNullTypes(fieldName, definition).stream().flatMap { type: JsonSchemaType -> + ): List { + return getNonNullTypes(fieldName, definition).flatMap { type: JsonSchemaType -> getSchema( fieldName = fieldName, fieldNamespace = fieldNamespace, @@ -392,7 +389,7 @@ class JsonToAvroSchemaConverter { definition: JsonNode, appendExtraProps: Boolean, addStringToLogicalTypes: Boolean - ): Stream? { + ): List { val namespace: String = if (fieldNamespace == null) fieldName else "$fieldNamespace.$fieldName" val singleFieldSchema: Schema = @@ -405,9 +402,9 @@ class JsonToAvroSchemaConverter { addStringToLogicalTypes, ) if (singleFieldSchema.isUnion) { - return singleFieldSchema.types.stream() + return singleFieldSchema.types } else { - return Stream.of( + return listOf( singleFieldSchema, ) } @@ -438,8 +435,9 @@ class JsonToAvroSchemaConverter { val mergedSchemas: MutableList = schemas - .stream() // gather record schemas to construct a single record schema later on - .peek { schema: Schema -> + + // gather record schemas to construct a single record schema later on + .onEach { schema: Schema -> if (schema.type == Schema.Type.RECORD) { for (field: Schema.Field in schema.fields) { recordFieldSchemas.putIfAbsent( @@ -458,8 +456,7 @@ class JsonToAvroSchemaConverter { } } // remove record schemas because they will be merged into one .filter { schema: Schema -> schema.type != Schema.Type.RECORD } - .collect(Collectors.toList()) - + .toMutableList() // create one record schema from all the record fields if (recordFieldSchemas.isNotEmpty()) { val builder: SchemaBuilder.RecordBuilder = SchemaBuilder.record(fieldName) @@ -486,10 +483,9 @@ class JsonToAvroSchemaConverter { } val subfieldSchemas: List = entry.value - .stream() .flatMap { schema: Schema -> schema.types - .stream() // filter out null and add it later on as the first + // filter out null and add it later on as the first // element .filter { s: Schema -> s != NULL_SCHEMA } } @@ -572,9 +568,9 @@ class JsonToAvroSchemaConverter { // invalid and // cannot be properly processed. if ( - ((nonNullFieldTypes.stream().anyMatch { schema: Schema -> - schema.logicalType != null - }) && (!nonNullFieldTypes.contains(STRING_SCHEMA)) && addStringToLogicalTypes) + ((nonNullFieldTypes.any { schema: Schema -> schema.logicalType != null }) && + (!nonNullFieldTypes.contains(STRING_SCHEMA)) && + addStringToLogicalTypes) ) { nonNullFieldTypes.add(STRING_SCHEMA) } @@ -606,8 +602,8 @@ class JsonToAvroSchemaConverter { ) && ("timestamp-micros" == type.logicalType.name) } - val hasPlainLong: Boolean = unionTypes.stream().anyMatch(isPlainLong) - val hasTimestampMicrosLong: Boolean = unionTypes.stream().anyMatch(isTimestampMicrosLong) + val hasPlainLong: Boolean = unionTypes.any { isPlainLong.test(it) } + val hasTimestampMicrosLong: Boolean = unionTypes.any { isTimestampMicrosLong.test(it) } val removeTimestampType: Predicate = Predicate { type: Schema -> !(hasPlainLong && hasTimestampMicrosLong && @@ -615,14 +611,7 @@ class JsonToAvroSchemaConverter { type, )) } - return Schema.createUnion( - unionTypes - .stream() - .filter(removeTimestampType) - .collect( - Collectors.toList(), - ), - ) + return Schema.createUnion(unionTypes.filter { removeTimestampType.test(it) }.toList()) } companion object { @@ -640,11 +629,8 @@ class JsonToAvroSchemaConverter { @Suppress("DEPRECATION") fun getNonNullTypes(fieldName: String?, fieldDefinition: JsonNode): List { return getTypes(fieldName, fieldDefinition) - .stream() .filter { type: JsonSchemaType -> type != JsonSchemaType.NULL } - .collect( - Collectors.toList(), - ) + .toList() } /** When no type or $ref are specified, it will default to string. */ @@ -660,7 +646,6 @@ class JsonToAvroSchemaConverter { if (typeProperty != null && typeProperty.isArray) { return MoreIterators.toList(typeProperty.elements()) - .stream() .map { s: JsonNode -> JsonSchemaType.fromJsonSchemaType( s.asText(), diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/CsvSerializedBuffer.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/CsvSerializedBuffer.kt index 587c31bc4983..10e2c56d9ae4 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/CsvSerializedBuffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/CsvSerializedBuffer.kt @@ -124,7 +124,6 @@ class CsvSerializedBuffer( val csvSheetGenerator = CsvSheetGenerator.Factory.create( catalog.streams - .stream() .filter { s: ConfiguredAirbyteStream -> s.stream.name == stream.name && StringUtils.equals( @@ -132,18 +131,16 @@ class CsvSerializedBuffer( stream.namespace, ) } - .findFirst() - .orElseThrow { - RuntimeException( - String.format( - "No such stream %s.%s", - stream.namespace, - stream.name, - ), - ) - } - .stream - .jsonSchema, + .firstOrNull() + ?.stream + ?.jsonSchema + ?: throw RuntimeException( + String.format( + "No such stream %s.%s", + stream.namespace, + stream.name, + ), + ), config, ) val csvSettings = diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/RootLevelFlatteningSheetGenerator.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/RootLevelFlatteningSheetGenerator.kt index a77273bd9c42..5a9957463abe 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/RootLevelFlatteningSheetGenerator.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/csv/RootLevelFlatteningSheetGenerator.kt @@ -18,7 +18,6 @@ class RootLevelFlatteningSheetGenerator(jsonSchema: JsonNode) : MoreIterators.toList( jsonSchema["properties"].fieldNames(), ) - .stream() .sorted() .toList() diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/ParquetSerializedBuffer.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/ParquetSerializedBuffer.kt index 91e9adedf6bc..21e465b7e765 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/ParquetSerializedBuffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/s3/parquet/ParquetSerializedBuffer.kt @@ -63,20 +63,16 @@ class ParquetSerializedBuffer( val schema: Schema = schemaConverter.getAvroSchema( catalog.streams - .stream() - .filter { s: ConfiguredAirbyteStream -> + .firstOrNull { s: ConfiguredAirbyteStream -> (s.stream.name == stream.name) && StringUtils.equals( s.stream.namespace, stream.namespace, ) } - .findFirst() - .orElseThrow { - RuntimeException("No such stream ${stream.namespace}.${stream.name}") - } - .stream - .jsonSchema, + ?.stream + ?.jsonSchema + ?: throw RuntimeException("No such stream ${stream.namespace}.${stream.name}"), stream.name, stream.namespace, ) 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 6cc6421c4f09..452772cd63ef 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 @@ -25,7 +25,6 @@ import java.util.* import java.util.concurrent.Executors import java.util.function.Consumer import java.util.function.Function -import java.util.stream.Collectors import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -228,14 +227,9 @@ private constructor( val message = String.format( "You are trying to write multiple streams to the same table. Consider switching to a custom namespace format using \${SOURCE_NAMESPACE}, or moving one of them into a separate connection with a different stream prefix. Affected streams: %s", - conflictingStreams - .stream() - .map( - Function { config: WriteConfig -> - config.namespace + "." + config.streamName - } - ) - .collect(Collectors.joining(", ")) + conflictingStreams.joinToString(", ") { config: WriteConfig -> + config.namespace + "." + config.streamName + } ) throw ConfigErrorException(message) } @@ -265,11 +259,10 @@ private constructor( parsedCatalog: ParsedCatalog?, useDestinationsV2Columns: Boolean ): List { - return catalog!! - .streams - .stream() - .map(toWriteConfig(namingResolver, config, parsedCatalog, useDestinationsV2Columns)) - .toList() + return catalog!!.streams.map { + toWriteConfig(namingResolver, config, parsedCatalog, useDestinationsV2Columns) + .apply(it) + } } private fun toWriteConfig( diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonFieldNameUpdaterTest.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonFieldNameUpdaterTest.kt index 771463d034d0..f2a57715d06d 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonFieldNameUpdaterTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonFieldNameUpdaterTest.kt @@ -3,13 +3,10 @@ */ package io.airbyte.cdk.integrations.destination.s3.avro -import com.fasterxml.jackson.databind.JsonNode import io.airbyte.commons.json.Jsons.deserialize import io.airbyte.commons.resources.MoreResources.readResource import io.airbyte.commons.util.MoreIterators.toList import java.io.IOException -import java.util.function.Function -import java.util.stream.Collectors import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test @@ -22,14 +19,7 @@ internal class JsonFieldNameUpdaterTest { val nameMap = testCase["nameMap"] val nameUpdater = JsonFieldNameUpdater( - toList(nameMap.fields()) - .stream() - .collect( - Collectors.toMap( - Function { obj: Map.Entry -> obj.key }, - Function { e: Map.Entry -> e.value.asText() } - ) - ) + toList(nameMap.fields()).associate { it.key to it.value.asText() } ) val original = testCase["original"] diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroConverterTest.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroConverterTest.kt index 85f82ef078fd..087d6edffba4 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroConverterTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/test/kotlin/io/airbyte/cdk/integrations/destination/s3/avro/JsonToAvroConverterTest.kt @@ -75,13 +75,15 @@ class JsonToAvroConverterTest { "parquet/json_schema_converter/type_conversion_test_cases_v0.json" ) ) - return MoreIterators.toList(testCases.elements()).stream().map { testCase: JsonNode -> - Arguments.of( - testCase["fieldName"].asText(), - testCase["jsonFieldSchema"], - testCase["avroFieldType"], - ) - } + return MoreIterators.toList(testCases.elements()) + .map { testCase: JsonNode -> + Arguments.of( + testCase["fieldName"].asText(), + testCase["jsonFieldSchema"], + testCase["avroFieldType"], + ) + } + .stream() } } @@ -94,13 +96,15 @@ class JsonToAvroConverterTest { "parquet/json_schema_converter/type_conversion_test_cases_v1.json" ) ) - return MoreIterators.toList(testCases.elements()).stream().map { testCase: JsonNode -> - Arguments.of( - testCase["fieldName"].asText(), - testCase["jsonFieldSchema"], - testCase["avroFieldType"], - ) - } + return MoreIterators.toList(testCases.elements()) + .map { testCase: JsonNode -> + Arguments.of( + testCase["fieldName"].asText(), + testCase["jsonFieldSchema"], + testCase["avroFieldType"], + ) + } + .stream() } } @@ -165,17 +169,19 @@ class JsonToAvroConverterTest { "parquet/json_schema_converter/json_conversion_test_cases_v0.json" ) ) - return MoreIterators.toList(testCases.elements()).stream().map { testCase: JsonNode -> - Arguments.of( - testCase["schemaName"].asText(), - testCase["namespace"].asText(), - testCase["appendAirbyteFields"].asBoolean(), - testCase["jsonSchema"], - testCase["jsonObject"], - testCase["avroSchema"], - testCase["avroObject"], - ) - } + return MoreIterators.toList(testCases.elements()) + .map { testCase: JsonNode -> + Arguments.of( + testCase["schemaName"].asText(), + testCase["namespace"].asText(), + testCase["appendAirbyteFields"].asBoolean(), + testCase["jsonSchema"], + testCase["jsonObject"], + testCase["avroSchema"], + testCase["avroObject"], + ) + } + .stream() } } @@ -188,17 +194,19 @@ class JsonToAvroConverterTest { "parquet/json_schema_converter/json_conversion_test_cases_v1.json" ) ) - return MoreIterators.toList(testCases.elements()).stream().map { testCase: JsonNode -> - Arguments.of( - testCase["schemaName"].asText(), - testCase["namespace"].asText(), - testCase["appendAirbyteFields"].asBoolean(), - testCase["jsonSchema"], - testCase["jsonObject"], - testCase["avroSchema"], - testCase["avroObject"], - ) - } + return MoreIterators.toList(testCases.elements()) + .map { testCase: JsonNode -> + Arguments.of( + testCase["schemaName"].asText(), + testCase["namespace"].asText(), + testCase["appendAirbyteFields"].asBoolean(), + testCase["jsonSchema"], + testCase["jsonObject"], + testCase["avroSchema"], + testCase["avroObject"], + ) + } + .stream() } } diff --git a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3AvroParquetDestinationAcceptanceTest.kt b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3AvroParquetDestinationAcceptanceTest.kt index 8e0109311e7f..c6fb4112902e 100644 --- a/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3AvroParquetDestinationAcceptanceTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/s3-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/destination/s3/S3AvroParquetDestinationAcceptanceTest.kt @@ -14,8 +14,6 @@ import io.airbyte.protocol.models.v0.AirbyteStream import io.airbyte.protocol.models.v0.CatalogHelpers import java.io.IOException import java.util.* -import java.util.function.Function -import java.util.stream.Collectors import org.apache.avro.Schema import org.apache.avro.generic.GenericData import org.junit.jupiter.api.Assertions @@ -53,16 +51,7 @@ protected constructor(fileUploadFormat: FileUploadFormat) : val nameToNode: Map = iterableNames.associateWith { name: String -> getJsonNode(stream, name) } - return nameToNode.entries - .stream() - .collect( - Collectors.toMap( - Function { obj: Map.Entry -> obj.key }, - Function { entry: Map.Entry -> - getExpectedSchemaType(entry.value) - } - ) - ) + return nameToNode.entries.associate { it.key to getExpectedSchemaType(it.value) } } private fun getJsonNode(stream: AirbyteStream, name: String): JsonNode { @@ -125,44 +114,29 @@ protected constructor(fileUploadFormat: FileUploadFormat) : protected fun getTypes(record: GenericData.Record): Map> { val fieldList = record.schema.fields - .stream() .filter { field: Schema.Field -> !field.name().startsWith("_airbyte") } .toList() return if (fieldList.size == 1) { - fieldList - .stream() - .collect( - Collectors.toMap( - Function { obj: Schema.Field -> obj.name() }, - Function { field: Schema.Field -> - field - .schema() - .types - .map { obj: Schema -> obj.type } - .filter { type: Schema.Type -> type != Schema.Type.NULL } - .toSet() - } - ) - ) + fieldList.associate { + it.name() to + it.schema() + .types + .map { obj: Schema -> obj.type } + .filter { type: Schema.Type -> type != Schema.Type.NULL } + .toSet() + } } else { - fieldList - .stream() - .collect( - Collectors.toMap( - Function { obj: Schema.Field -> obj.name() }, - Function { field: Schema.Field -> - field - .schema() - .types - .filter { type: Schema -> type.type != Schema.Type.NULL } - .flatMap { type: Schema -> type.elementType.types } - .map { obj: Schema -> obj.type } - .filter { type: Schema.Type -> type != Schema.Type.NULL } - .toSet() - } - ) - ) + fieldList.associate { + it.name() to + it.schema() + .types + .filter { type: Schema -> type.type != Schema.Type.NULL } + .flatMap { type: Schema -> type.elementType.types } + .map { obj: Schema -> obj.type } + .filter { type: Schema.Type -> type != Schema.Type.NULL } + .toSet() + } } } } 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 00597baf306a..f2a6bb1f863b 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 @@ -89,14 +89,12 @@ protected constructor(protected val outputFormat: FileUploadFormat) : Destinatio s3Client!! .listObjects(s3DestinationConfig.bucketName, parentFolder) .objectSummaries - .stream() .filter { o: S3ObjectSummary -> o.key.contains("$streamNameStr/") } - .sorted(Comparator.comparingLong { o: S3ObjectSummary -> o.lastModified.time }) + .sortedWith(Comparator.comparingLong { o: S3ObjectSummary -> o.lastModified.time }) .toList() LOGGER.info( "All objects: {}", objectSummaries - .stream() .map { o: S3ObjectSummary -> String.format("%s/%s", o.bucketName, o.key) } .toList(), ) 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 a89a77f11031..70eb45d0474a 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 @@ -108,7 +108,6 @@ interface AirbyteType { // Recurse into a schema that forces a specific one of each option val options = typeOptions - .stream() .map { typeOption: String -> fromJsonSchema(getTrimmedJsonSchema(schema, typeOption)) } 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 b18d76f21857..f5e2e74e0037 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 @@ -30,10 +30,10 @@ constructor( val actualStreamConfig: StreamConfig // Use empty string quote because we don't really care if ( - streamConfigs.stream().anyMatch { s: StreamConfig -> + streamConfigs.any { s: StreamConfig -> s.id.finalTableId("") == originalStreamConfig.id.finalTableId("") } || - streamConfigs.stream().anyMatch { s: StreamConfig -> + streamConfigs.any { s: StreamConfig -> s.id.rawTableId("") == originalStreamConfig.id.rawTableId("") } ) { @@ -113,12 +113,11 @@ constructor( else -> throw IllegalArgumentException("Top-level schema must be an object") } - require(!stream.primaryKey.stream().anyMatch { key: List -> key.size > 1 }) { + require(!stream.primaryKey.any { key: List -> key.size > 1 }) { "Only top-level primary keys are supported" } val primaryKey = stream.primaryKey - .stream() .map { key: List -> sqlGenerator.buildColumnId(key[0]) } .toList() @@ -156,7 +155,7 @@ constructor( val originalColumnId = sqlGenerator.buildColumnId(key) var columnId: ColumnId if ( - columns.keys.stream().noneMatch { c: ColumnId -> + columns.keys.none { c: ColumnId -> c.canonicalName == originalColumnId.canonicalName } ) { @@ -193,11 +192,7 @@ constructor( } val canonicalName = columnId.canonicalName - if ( - columns.keys.stream().noneMatch { c: ColumnId -> - c.canonicalName == canonicalName - } - ) { + if (columns.keys.none { c: ColumnId -> c.canonicalName == canonicalName }) { break } else { i++ @@ -255,11 +250,7 @@ constructor( newColumnId = sqlGenerator.buildColumnId("$prefix$length$suffix") // if there's _still_ a collision after this, just give up. // we could try to be more clever, but this is already a pretty rare case. - if ( - columns.keys.stream().anyMatch { c: ColumnId -> - c.canonicalName == newColumnId.canonicalName - } - ) { + if (columns.keys.any { c: ColumnId -> c.canonicalName == newColumnId.canonicalName }) { throw IllegalArgumentException( "Cannot solve column name collision: ${newColumnId.originalName}. We recommend removing this column to continue syncing." ) diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CollectionUtils.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CollectionUtils.kt index 9a1a0d3f01d7..45b6bebd52e5 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CollectionUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/CollectionUtils.kt @@ -43,9 +43,7 @@ object CollectionUtils { // searchTerms is empty "Search Terms collection may not be empty" } - return searchTerms.stream().allMatch { term: String -> - containsIgnoreCase(searchCollection, term) - } + return searchTerms.all { term: String -> containsIgnoreCase(searchCollection, term) } } /** @@ -59,9 +57,6 @@ object CollectionUtils { if (collection.contains(search)) { return Optional.of(search) } - return collection - .stream() - .filter { s: String -> s.equals(search, ignoreCase = true) } - .findFirst() + return Optional.ofNullable(collection.firstOrNull { it.equals(search, ignoreCase = true) }) } } 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 387c88d54706..110703b03fff 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 @@ -381,7 +381,6 @@ class DefaultTyperDeduper( LOGGER.info("Typing and deduping all tables") val typeAndDedupeTasks: MutableSet>> = HashSet() parsedCatalog.streams - .stream() .filter { streamConfig: StreamConfig -> // Skip if stream setup failed. if (!streamSetupSucceeded(streamConfig)) { diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/FutureUtils.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/FutureUtils.kt index 75fa0e41552f..0e09e029a9ac 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/FutureUtils.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/FutureUtils.kt @@ -33,7 +33,6 @@ object FutureUtils { ) { val exceptions = potentialExceptions - .stream() .map { obj: CompletableFuture> -> obj.join() } .filter { obj: Optional -> obj.isPresent } .map { obj: Optional -> obj.get() } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/ParsedCatalog.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/ParsedCatalog.kt index 29775ba55adb..58c41a017e0b 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/ParsedCatalog.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/ParsedCatalog.kt @@ -5,7 +5,7 @@ package io.airbyte.integrations.base.destination.typing_deduping import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair -class ParsedCatalog(streams: List) { +class ParsedCatalog(val streams: List) { fun getStream(streamId: AirbyteStreamNameNamespacePair): StreamConfig { return getStream(streamId.namespace, streamId.name) } @@ -15,32 +15,20 @@ class ParsedCatalog(streams: List) { } fun getStream(originalNamespace: String?, originalName: String?): StreamConfig { - return streams - .stream() - .filter { s: StreamConfig -> - s.id.originalNamespace == originalNamespace && s.id.originalName == originalName - } - .findFirst() - .orElseThrow { - IllegalArgumentException( - String.format( - "Could not find stream %s.%s out of streams %s", - originalNamespace, - originalName, - streams - .stream() - .map { stream: StreamConfig -> - stream.id.originalNamespace + "." + stream.id.originalName - } - .toList() - ) + return streams.firstOrNull { s: StreamConfig -> + s.id.originalNamespace == originalNamespace && s.id.originalName == originalName + } + ?: throw IllegalArgumentException( + String.format( + "Could not find stream %s.%s out of streams %s", + originalNamespace, + originalName, + streams + .map { stream: StreamConfig -> + stream.id.originalNamespace + "." + stream.id.originalName + } + .toList() ) - } - } - - val streams: List - - init { - this.streams = streams + ) } } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Sql.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Sql.kt index 4cf947a10071..642c9ac77df0 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Sql.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/Sql.kt @@ -4,7 +4,6 @@ package io.airbyte.integrations.base.destination.typing_deduping import java.util.function.Consumer -import java.util.function.Function import java.util.stream.Stream /** @@ -27,7 +26,6 @@ data class Sql(val transactions: List>) { */ fun asSqlStrings(begin: String?, commit: String?): List { return transactions - .stream() .map { transaction: List -> // If there's only one statement, we don't need to wrap it in a transaction. if (transaction.size == 1) { @@ -54,7 +52,7 @@ data class Sql(val transactions: List>) { transactions.forEach( Consumer { transaction: List -> require(!transaction.isEmpty()) { "Transaction must not be empty" } - require(!transaction.stream().anyMatch { s: String -> s.isNullOrEmpty() }) { + require(!transaction.any { it.isNullOrEmpty() }) { "Transaction must not contain empty statements" } } @@ -76,12 +74,7 @@ data class Sql(val transactions: List>) { /** Execute each statement as its own transaction. */ @JvmStatic fun separately(statements: List): Sql { - return create( - statements - .stream() - .map(Function> { o: String -> listOf(o) }) - .toList() - ) + return create(statements.map { listOf(it) }.toList()) } @JvmStatic @@ -100,14 +93,12 @@ data class Sql(val transactions: List>) { @JvmStatic fun concat(vararg sqls: Sql): Sql { - return create( - Stream.of(*sqls).flatMap { sql: Sql -> sql.transactions.stream() }.toList() - ) + return create(sqls.flatMap { sql: Sql -> sql.transactions }) } @JvmStatic fun concat(sqls: List): Sql { - return create(sqls.stream().flatMap { sql: Sql -> sql.transactions.stream() }.toList()) + return create(sqls.flatMap { sql: Sql -> sql.transactions }) } /** @@ -118,10 +109,8 @@ data class Sql(val transactions: List>) { fun create(transactions: List>): Sql { return Sql( transactions - .stream() .map { transaction: List -> transaction - .stream() .filter { statement: String -> !statement.isNullOrEmpty() } .map internalMap@{ statement: String -> if (!statement.trim { it <= ' ' }.endsWith(";")) { 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 961e1ff5b9e6..4167aa8d25ac 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 @@ -12,7 +12,6 @@ import java.util.* import java.util.concurrent.CompletableFuture import java.util.concurrent.CompletionStage import java.util.concurrent.ExecutorService -import java.util.stream.Collectors.toMap import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -45,21 +44,10 @@ class TyperDeduperUtil { // Execute the migration on all streams in parallel val futures: Map>> = - currentStates - .stream() - .collect( - toMap( - { it.streamConfig.id }, - { initialState -> - runMigrationsAsync( - executorService, - destinationHandler, - migration, - initialState - ) - } - ) - ) + currentStates.associate { + it.streamConfig.id to + runMigrationsAsync(executorService, destinationHandler, migration, it) + } val migrationResultFutures = CompletableFutures.allOf(futures.values.toList()).toCompletableFuture().join() getResultsOrLogAndThrowFirst( 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 5f42e223d1c9..2297a3e25f0b 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 @@ -4,7 +4,6 @@ package io.airbyte.integrations.base.destination.typing_deduping import com.fasterxml.jackson.databind.JsonNode -import com.google.common.collect.Streams import io.airbyte.commons.json.Jsons import io.airbyte.commons.string.Strings import io.airbyte.integrations.base.destination.typing_deduping.TypeAndDedupeTransaction.executeSoftReset @@ -18,9 +17,6 @@ import io.airbyte.protocol.models.v0.SyncMode import java.time.Instant import java.util.* import java.util.function.Consumer -import java.util.function.Function -import java.util.stream.Collectors -import java.util.stream.Stream import org.junit.jupiter.api.AfterEach import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.BeforeEach @@ -776,7 +772,6 @@ abstract class BaseSqlGeneratorIntegrationTest record["_airbyte_loaded_at"] == null } .count(), "Raw table should only have non-null loaded_at on the newer record" @@ -1255,7 +1250,7 @@ abstract class BaseSqlGeneratorIntegrationTest + actualFinalRecords.none { record: JsonNode -> record.has("_ab_cdc_deleted_at") }, "_ab_cdc_deleted_at column was expected to be dropped. Actual final table had: $actualFinalRecords" @@ -1520,15 +1515,7 @@ abstract class BaseSqlGeneratorIntegrationTest, v2RawRecords: List) { - val v2RecordMap = - v2RawRecords - .stream() - .collect( - Collectors.toMap( - { record: JsonNode -> record["_airbyte_raw_id"].asText() }, - Function.identity() - ) - ) + val v2RecordMap = v2RawRecords.associateBy { it["_airbyte_raw_id"].asText() } Assertions.assertAll( Executable { Assertions.assertEquals(6, v1RawRecords.size) }, Executable { Assertions.assertEquals(6, v2RawRecords.size) } @@ -1758,7 +1745,6 @@ abstract class BaseSqlGeneratorIntegrationTest !record.hasNonNull("_airbyte_loaded_at") } .count() ) @@ -1790,7 +1776,6 @@ abstract class BaseSqlGeneratorIntegrationTest !record.hasNonNull("_airbyte_loaded_at") } .count() ) @@ -1838,7 +1823,6 @@ abstract class BaseSqlGeneratorIntegrationTest = - Streams.concat(FINAL_TABLE_COLUMN_NAMES.stream(), Stream.of("_ab_cdc_deleted_at")) - .toList() + FINAL_TABLE_COLUMN_NAMES + "_ab_cdc_deleted_at" } } diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.kt index c948373861f6..9e4aff3fc523 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseTypingDedupingTest.kt @@ -27,7 +27,6 @@ import java.util.concurrent.CompletableFuture import java.util.concurrent.Executors import java.util.function.Consumer import java.util.function.Function -import java.util.stream.Stream import org.apache.commons.lang3.RandomStringUtils import org.junit.jupiter.api.* import org.junit.jupiter.api.function.Executable @@ -570,11 +569,10 @@ abstract class BaseTypingDedupingTest { val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_raw.jsonl") val expectedFinalRecords2 = readRecords("dat/sync2_expectedrecords_fullrefresh_append_final.jsonl") - .stream() - .peek { record: JsonNode -> - (record as ObjectNode).remove(sqlGenerator.buildColumnId("name").name) - } - .toList() + expectedFinalRecords2.forEach { record: JsonNode -> + (record as ObjectNode).remove(sqlGenerator.buildColumnId("name").name) + } + verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison()) } @@ -649,11 +647,8 @@ abstract class BaseTypingDedupingTest { // First sync val messages1 = - Stream.concat( - readMessages("dat/sync1_messages.jsonl", namespace1, streamName).stream(), - readMessages("dat/sync1_messages2.jsonl", namespace2, streamName).stream() - ) - .toList() + readMessages("dat/sync1_messages.jsonl", namespace1, streamName) + + readMessages("dat/sync1_messages2.jsonl", namespace2, streamName) runSync(catalog, messages1) @@ -674,12 +669,8 @@ abstract class BaseTypingDedupingTest { // Second sync val messages2 = - Stream.concat( - readMessages("dat/sync2_messages.jsonl", namespace1, streamName).stream(), - readMessages("dat/sync2_messages2.jsonl", namespace2, streamName).stream() - ) - .toList() - + readMessages("dat/sync2_messages.jsonl", namespace1, streamName) + + readMessages("dat/sync2_messages2.jsonl", namespace2, streamName) runSync(catalog, messages2) verifySyncResult( @@ -897,10 +888,7 @@ abstract class BaseTypingDedupingTest { } private fun repeatList(n: Int, list: List): List { - return Collections.nCopies(n, list) - .stream() - .flatMap { obj: List -> obj.stream() } - .toList() + return Collections.nCopies(n, list).flatMap { obj: List -> obj }.toList() } @Throws(Exception::class) @@ -1102,13 +1090,11 @@ abstract class BaseTypingDedupingTest { streamName: String? ): List { return readRecords(filename) - .stream() .map { record: JsonNode -> Jsons.convertValue(record, AirbyteMessage::class.java) } - .peek { message: AirbyteMessage -> + .onEach { message: AirbyteMessage -> message.record.namespace = streamNamespace message.record.stream = streamName } - .toList() } protected fun pushMessages( diff --git a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.kt b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.kt index 2c4f960213ae..a398faa1a1bb 100644 --- a/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.kt +++ b/airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/RecordDiffer.kt @@ -90,13 +90,9 @@ constructor( val diff = diffRecords( expectedRecords - .stream() - .map { record: JsonNode -> this.copyWithLiftedData(record) } - .toList(), - actualRecords - .stream() .map { record: JsonNode -> this.copyWithLiftedData(record) } .toList(), + actualRecords.map { record: JsonNode -> this.copyWithLiftedData(record) }.toList(), rawRecordIdentityComparator, rawRecordSortComparator, rawRecordIdentityExtractor, @@ -218,8 +214,8 @@ constructor( recordIdExtractor: Function, columnNames: Map ): String { - val expectedRecords = originalExpectedRecords.stream().sorted(sortComparator).toList() - val actualRecords = originalActualRecords.stream().sorted(sortComparator).toList() + val expectedRecords = originalExpectedRecords.sortedWith(sortComparator) + val actualRecords = originalActualRecords.sortedWith(sortComparator) // Iterate through both lists in parallel and compare each record. // Build up an error message listing any incorrect, missing, or unexpected records.