-
Notifications
You must be signed in to change notification settings - Fork 4.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Connector operations by responsibility
- Loading branch information
Showing
23 changed files
with
1,737 additions
and
299 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
31 changes: 31 additions & 0 deletions
31
...k/core/src/main/kotlin/io/airbyte/cdk/integrations/destination/operation/SyncOperation.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,31 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.integrations.destination.operation | ||
|
||
import io.airbyte.cdk.integrations.destination.StreamSyncSummary | ||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage | ||
import io.airbyte.protocol.models.v0.StreamDescriptor | ||
import java.util.stream.Stream | ||
|
||
/** | ||
* Destination Connector sync operations Any initialization required for the connector should be | ||
* done as part of instantiation/init blocks | ||
*/ | ||
interface SyncOperation { | ||
|
||
/** | ||
* This function is a shim for | ||
* [io.airbyte.cdk.integrations.destination.async.function.DestinationFlushFunction] After the | ||
* method control is returned, it should be assumed that the data is committed to a durable | ||
* storage and send back any State message acknowledgements. | ||
*/ | ||
fun flushStream(descriptor: StreamDescriptor, stream: Stream<PartialAirbyteMessage>) | ||
|
||
/** | ||
* Finalize streams which could involve typing deduping or any other housekeeping tasks | ||
* required. | ||
*/ | ||
fun finalizeStreams(streamSyncSummaries: Map<StreamDescriptor, StreamSyncSummary>) | ||
} |
2 changes: 1 addition & 1 deletion
2
airbyte-cdk/java/airbyte-cdk/core/src/main/resources/version.properties
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -1 +1 @@ | ||
version=0.35.5 | ||
version=0.35.6 |
39 changes: 39 additions & 0 deletions
39
.../kotlin/io/airbyte/cdk/integrations/destination/staging/StagingSerializedBufferFactory.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,39 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.integrations.destination.staging | ||
|
||
import io.airbyte.cdk.integrations.base.JavaBaseConstants | ||
import io.airbyte.cdk.integrations.destination.record_buffer.FileBuffer | ||
import io.airbyte.cdk.integrations.destination.record_buffer.SerializableBuffer | ||
import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat | ||
import io.airbyte.cdk.integrations.destination.s3.csv.CsvSerializedBuffer | ||
import io.airbyte.cdk.integrations.destination.s3.csv.StagingDatabaseCsvSheetGenerator | ||
|
||
/** | ||
* Factory which can create an instance of concrete SerializedBuffer for one-time use before buffer | ||
* is closed. [io.airbyte.cdk.integrations.destination.s3.SerializedBufferFactory] is almost similar | ||
* which needs to be unified. That doesn't work well with our DV2 staging destinations, which mostly | ||
* support CSV only. | ||
*/ | ||
object StagingSerializedBufferFactory { | ||
|
||
fun initializeBuffer( | ||
fileUploadFormat: FileUploadFormat, | ||
destinationColumns: JavaBaseConstants.DestinationColumns | ||
): SerializableBuffer { | ||
when (fileUploadFormat) { | ||
FileUploadFormat.CSV -> { | ||
return CsvSerializedBuffer( | ||
FileBuffer(CsvSerializedBuffer.CSV_GZ_SUFFIX), | ||
StagingDatabaseCsvSheetGenerator(destinationColumns), | ||
true, | ||
) | ||
} | ||
else -> { | ||
TODO("Only CSV is supported for Staging format") | ||
} | ||
} | ||
} | ||
} |
55 changes: 55 additions & 0 deletions
55
...tlin/io/airbyte/cdk/integrations/destination/staging/operation/StagingStreamOperations.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.integrations.destination.staging.operation | ||
|
||
import io.airbyte.cdk.integrations.base.JavaBaseConstants | ||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage | ||
import io.airbyte.cdk.integrations.destination.record_buffer.SerializableBuffer | ||
import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat | ||
import io.airbyte.cdk.integrations.destination.staging.StagingSerializedBufferFactory | ||
import io.airbyte.commons.json.Jsons | ||
import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation | ||
import io.airbyte.integrations.base.destination.operation.StorageOperation | ||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus | ||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig | ||
import io.airbyte.integrations.base.destination.typing_deduping.migrators.MinimumDestinationState | ||
import io.github.oshai.kotlinlogging.KotlinLogging | ||
import java.util.stream.Stream | ||
import org.apache.commons.io.FileUtils | ||
|
||
class StagingStreamOperations<DestinationState : MinimumDestinationState>( | ||
private val storageOperation: StorageOperation<SerializableBuffer>, | ||
destinationInitialStatus: DestinationInitialStatus<DestinationState>, | ||
private val fileUploadFormat: FileUploadFormat, | ||
private val destinationColumns: JavaBaseConstants.DestinationColumns, | ||
disableTypeDedupe: Boolean = false | ||
) : | ||
AbstractStreamOperation<DestinationState, SerializableBuffer>( | ||
storageOperation, | ||
destinationInitialStatus, | ||
disableTypeDedupe | ||
) { | ||
|
||
private val log = KotlinLogging.logger {} | ||
override fun writeRecords(streamConfig: StreamConfig, stream: Stream<PartialAirbyteMessage>) { | ||
val writeBuffer = | ||
StagingSerializedBufferFactory.initializeBuffer(fileUploadFormat, destinationColumns) | ||
|
||
writeBuffer.use { | ||
stream.forEach { record: PartialAirbyteMessage -> | ||
it.accept( | ||
record.serialized!!, | ||
Jsons.serialize(record.record!!.meta), | ||
record.record!!.emittedAt | ||
) | ||
} | ||
it.flush() | ||
log.info { | ||
"Buffer flush complete for stream ${streamConfig.id.originalName} (${FileUtils.byteCountToDisplaySize(it.byteCount)}) to staging" | ||
} | ||
storageOperation.writeToStage(streamConfig.id, writeBuffer) | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
165 changes: 165 additions & 0 deletions
165
...main/kotlin/io/airbyte/integrations/base/destination/operation/AbstractStreamOperation.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,165 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.integrations.base.destination.operation | ||
|
||
import io.airbyte.cdk.integrations.destination.StreamSyncSummary | ||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage | ||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus | ||
import io.airbyte.integrations.base.destination.typing_deduping.InitialRawTableStatus | ||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig | ||
import io.airbyte.integrations.base.destination.typing_deduping.migrators.MinimumDestinationState | ||
import io.airbyte.protocol.models.v0.DestinationSyncMode | ||
import io.github.oshai.kotlinlogging.KotlinLogging | ||
import java.util.Optional | ||
import java.util.stream.Stream | ||
|
||
abstract class AbstractStreamOperation<DestinationState : MinimumDestinationState, Data>( | ||
private val storageOperation: StorageOperation<Data>, | ||
destinationInitialStatus: DestinationInitialStatus<DestinationState>, | ||
private val disableTypeDedupe: Boolean = false | ||
) : StreamOperation<DestinationState> { | ||
private val log = KotlinLogging.logger {} | ||
|
||
// State maintained to make decision between async calls | ||
private val finalTmpTableSuffix: String | ||
private val initialRawTableStatus: InitialRawTableStatus = | ||
destinationInitialStatus.initialRawTableStatus | ||
|
||
/** | ||
* After running any sync setup code, we may update the destination state. This field holds that | ||
* updated destination state. | ||
*/ | ||
final override val updatedDestinationState: DestinationState | ||
|
||
init { | ||
val stream = destinationInitialStatus.streamConfig | ||
storageOperation.prepareStage(stream.id, stream.destinationSyncMode) | ||
if (!disableTypeDedupe) { | ||
storageOperation.createFinalNamespace(stream.id) | ||
// Prepare final tables based on sync mode. | ||
finalTmpTableSuffix = prepareFinalTable(destinationInitialStatus) | ||
} else { | ||
log.info { "Typing and deduping disabled, skipping final table initialization" } | ||
finalTmpTableSuffix = NO_SUFFIX | ||
} | ||
updatedDestinationState = destinationInitialStatus.destinationState.withSoftReset(false) | ||
} | ||
|
||
companion object { | ||
private const val NO_SUFFIX = "" | ||
private const val TMP_OVERWRITE_TABLE_SUFFIX = "_airbyte_tmp" | ||
} | ||
|
||
private fun prepareFinalTable( | ||
initialStatus: DestinationInitialStatus<DestinationState> | ||
): String { | ||
val stream = initialStatus.streamConfig | ||
// No special handling if final table doesn't exist, just create and return | ||
if (!initialStatus.isFinalTablePresent) { | ||
log.info { | ||
"Final table does not exist for stream ${initialStatus.streamConfig.id.finalName}, creating." | ||
} | ||
storageOperation.createFinalTable(stream, NO_SUFFIX, false) | ||
return NO_SUFFIX | ||
} | ||
|
||
log.info { "Final Table exists for stream ${stream.id.finalName}" } | ||
// The table already exists. Decide whether we're writing to it directly, or | ||
// using a tmp table. | ||
when (stream.destinationSyncMode) { | ||
DestinationSyncMode.OVERWRITE -> return prepareFinalTableForOverwrite(initialStatus) | ||
DestinationSyncMode.APPEND, | ||
DestinationSyncMode.APPEND_DEDUP -> { | ||
if ( | ||
initialStatus.isSchemaMismatch || | ||
initialStatus.destinationState.needsSoftReset() | ||
) { | ||
// We're loading data directly into the existing table. | ||
// Make sure it has the right schema. | ||
// Also, if a raw table migration wants us to do a soft reset, do that | ||
// here. | ||
log.info { "Executing soft-reset on final table of stream $stream" } | ||
storageOperation.softResetFinalTable(stream) | ||
} | ||
return NO_SUFFIX | ||
} | ||
} | ||
} | ||
|
||
private fun prepareFinalTableForOverwrite( | ||
initialStatus: DestinationInitialStatus<DestinationState> | ||
): String { | ||
val stream = initialStatus.streamConfig | ||
if (!initialStatus.isFinalTableEmpty || initialStatus.isSchemaMismatch) { | ||
// overwrite an existing tmp table if needed. | ||
storageOperation.createFinalTable(stream, TMP_OVERWRITE_TABLE_SUFFIX, true) | ||
log.info { | ||
"Using temp final table for table ${stream.id.finalName}, this will be overwritten at end of sync" | ||
} | ||
// We want to overwrite an existing table. Write into a tmp table. | ||
// We'll overwrite the table at the | ||
// end of the sync. | ||
return TMP_OVERWRITE_TABLE_SUFFIX | ||
} | ||
|
||
log.info { | ||
"Final Table for stream ${stream.id.finalName} is empty and matches the expected v2 format, writing to table directly" | ||
} | ||
return NO_SUFFIX | ||
} | ||
|
||
/** Write records will be destination type specific, Insert vs staging based on format */ | ||
abstract override fun writeRecords( | ||
streamConfig: StreamConfig, | ||
stream: Stream<PartialAirbyteMessage> | ||
) | ||
|
||
override fun finalizeTable(streamConfig: StreamConfig, syncSummary: StreamSyncSummary) { | ||
// Delete staging directory, implementation will handle if it has to do it or not or a No-OP | ||
storageOperation.cleanupStage(streamConfig.id) | ||
if (disableTypeDedupe) { | ||
log.info { | ||
"Typing and deduping disabled, skipping final table finalization. " + | ||
"Raw records can be found at ${streamConfig.id.rawNamespace}.${streamConfig.id.rawName}" | ||
} | ||
return | ||
} | ||
|
||
// Legacy logic that if recordsWritten or not tracked then it could be non-zero | ||
val isNotOverwriteSync = streamConfig.destinationSyncMode != DestinationSyncMode.OVERWRITE | ||
// Legacy logic that if recordsWritten or not tracked then it could be non-zero. | ||
// But for OVERWRITE syncs, we don't need to look at old records. | ||
val shouldRunTypingDeduping = | ||
syncSummary.recordsWritten.map { it > 0 }.orElse(true) || | ||
(initialRawTableStatus.hasUnprocessedRecords && isNotOverwriteSync) | ||
if (!shouldRunTypingDeduping) { | ||
log.info { | ||
"Skipping typing and deduping for stream ${streamConfig.id.originalNamespace}.${streamConfig.id.originalName} " + | ||
"because it had no records during this sync and no unprocessed records from a previous sync." | ||
} | ||
} else { | ||
// In overwrite mode, we want to read all the raw records. Typically, this is equivalent | ||
// to filtering on timestamp, but might as well be explicit. | ||
val timestampFilter = | ||
if (isNotOverwriteSync) { | ||
initialRawTableStatus.maxProcessedTimestamp | ||
} else { | ||
Optional.empty() | ||
} | ||
storageOperation.typeAndDedupe(streamConfig, timestampFilter, finalTmpTableSuffix) | ||
} | ||
|
||
// For overwrite, it's wasteful to do T+D, so we don't do soft-reset in prepare. Instead, we | ||
// do | ||
// type-dedupe | ||
// on a suffixed table and do a swap here when we have to for schema mismatches | ||
if ( | ||
streamConfig.destinationSyncMode == DestinationSyncMode.OVERWRITE && | ||
finalTmpTableSuffix.isNotBlank() | ||
) { | ||
storageOperation.overwriteFinalTable(streamConfig, finalTmpTableSuffix) | ||
} | ||
} | ||
} |
20 changes: 20 additions & 0 deletions
20
...duping/src/main/kotlin/io/airbyte/integrations/base/destination/operation/DefaultFlush.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,20 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.integrations.base.destination.operation | ||
|
||
import io.airbyte.cdk.integrations.destination.async.function.DestinationFlushFunction | ||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage | ||
import io.airbyte.cdk.integrations.destination.operation.SyncOperation | ||
import io.airbyte.protocol.models.v0.StreamDescriptor | ||
import java.util.stream.Stream | ||
|
||
class DefaultFlush( | ||
override val optimalBatchSizeBytes: Long, | ||
private val syncOperation: SyncOperation | ||
) : DestinationFlushFunction { | ||
override fun flush(streamDescriptor: StreamDescriptor, stream: Stream<PartialAirbyteMessage>) { | ||
syncOperation.flushStream(streamDescriptor, stream) | ||
} | ||
} |
Oops, something went wrong.