Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

chore: dont need state manager in iceberg #48577

Merged
merged 3 commits into from
Nov 20, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -10,54 +10,35 @@ import io.airbyte.cdk.load.data.MapperPipeline
import io.airbyte.cdk.load.message.Batch
import io.airbyte.cdk.load.message.DestinationFile
import io.airbyte.cdk.load.message.DestinationRecord
import io.airbyte.cdk.load.state.DestinationStateManager
import io.airbyte.cdk.load.state.object_storage.ObjectStorageDestinationState
import io.airbyte.cdk.load.message.SimpleBatch
import io.airbyte.cdk.load.state.StreamIncompleteResult
import io.airbyte.cdk.load.write.StreamLoader
import io.airbyte.cdk.load.write.object_storage.ObjectStorageStreamLoader
import io.airbyte.integrations.destination.iceberg.v2.io.IcebergTableCleaner
import io.airbyte.integrations.destination.iceberg.v2.io.IcebergTableWriterFactory
import io.airbyte.integrations.destination.iceberg.v2.io.IcebergUtil
import io.airbyte.integrations.destination.iceberg.v2.io.IcebergUtil.constructGenerationIdSuffix
import io.github.oshai.kotlinlogging.KotlinLogging
import java.util.concurrent.atomic.AtomicLong
import org.apache.iceberg.Table

@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION", justification = "Kotlin async continuation")
class IcebergStreamLoader(
override val stream: DestinationStream,
private val table: Table,
private val icebergTableWriterFactory: IcebergTableWriterFactory,
private val destinationStateManager: DestinationStateManager<ObjectStorageDestinationState>,
private val pipeline: MapperPipeline,
private val stagingBranchName: String,
private val mainBranchName: String
) : StreamLoader {
private val log = KotlinLogging.logger {}

private val partNumber = AtomicLong(0L)

override suspend fun start() {
val state = destinationStateManager.getState(stream)
val maxPartNumber =
state.generations
.filter { it.generationId >= stream.minimumGenerationId }
.mapNotNull { it.objects.maxOfOrNull { obj -> obj.partNumber } }
.maxOrNull()
log.info { "Got max part number from destination state: $maxPartNumber" }
maxPartNumber?.let { partNumber.set(it + 1L) }
}

override suspend fun processRecords(
records: Iterator<DestinationRecord>,
totalSizeBytes: Long
): Batch {
val partNumber = partNumber.getAndIncrement()
val state = destinationStateManager.getState(stream)

icebergTableWriterFactory
.create(table = table, generationId = constructGenerationIdSuffix(stream))
.use { writer ->
log.info { "Writing records to branch $stagingBranchName" }
state.addObject(stream.generationId, stagingBranchName, partNumber)
records.forEach { record ->
val icebergRecord =
IcebergUtil.toRecord(
Expand All @@ -82,27 +63,25 @@ class IcebergStreamLoader(
log.info { "Finished writing records to $stagingBranchName" }
}

return ObjectStorageStreamLoader.StagedObject(
remoteObject = stagingBranchName,
partNumber = partNumber
)
return SimpleBatch(Batch.State.PERSISTED)
}

override suspend fun processFile(file: DestinationFile): Batch {
throw NotImplementedError("Destination Iceberg does not support universal file transfer.")
}

override suspend fun processBatch(batch: Batch): Batch {
val stagedObject = batch as ObjectStorageStreamLoader.StagedObject<*>
log.info { "Moving staged object from $stagingBranchName to $mainBranchName" }
table.manageSnapshots().fastForwardBranch(mainBranchName, stagingBranchName).commit()

val state = destinationStateManager.getState(stream)
state.removeObject(stream.generationId, stagingBranchName)
state.addObject(stream.generationId, mainBranchName, stagedObject.partNumber)

val finalizedObject =
ObjectStorageStreamLoader.FinalizedObject(remoteObject = mainBranchName)
return finalizedObject
override suspend fun close(streamFailure: StreamIncompleteResult?) {
if (streamFailure == null) {
table.manageSnapshots().fastForwardBranch(mainBranchName, stagingBranchName).commit()
if (stream.minimumGenerationId > 0) {
val icebergTableCleaner = IcebergTableCleaner()
for (i in 0 until stream.minimumGenerationId) {
icebergTableCleaner.deleteGenerationId(
table,
constructGenerationIdSuffix(i),
)
}
}
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,8 +8,6 @@ import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.data.iceberg.parquet.toIcebergSchema
import io.airbyte.cdk.load.data.parquet.ParquetMapperPipelineFactory
import io.airbyte.cdk.load.data.withAirbyteMeta
import io.airbyte.cdk.load.state.DestinationStateManager
import io.airbyte.cdk.load.state.object_storage.ObjectStorageDestinationState
import io.airbyte.cdk.load.write.DestinationWriter
import io.airbyte.cdk.load.write.StreamLoader
import io.airbyte.integrations.destination.iceberg.v2.io.IcebergTableWriterFactory
Expand All @@ -21,7 +19,6 @@ import org.apache.iceberg.catalog.TableIdentifier

@Singleton
class IcebergV2Writer(
private val destinationStateManager: DestinationStateManager<ObjectStorageDestinationState>,
private val icebergTableWriterFactory: IcebergTableWriterFactory,
private val icebergConfiguration: IcebergV2Configuration,
) : DestinationWriter {
Expand All @@ -48,7 +45,6 @@ class IcebergV2Writer(
stream = stream,
table = table,
icebergTableWriterFactory = icebergTableWriterFactory,
destinationStateManager = destinationStateManager,
pipeline = pipeline,
stagingBranchName = DEFAULT_STAGING_BRANCH,
mainBranchName = icebergConfiguration.nessieServerConfiguration.mainBranchName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -45,12 +45,16 @@ object IcebergUtil {
}

fun constructGenerationIdSuffix(stream: DestinationStream): String {
if (stream.generationId < 0) {
return constructGenerationIdSuffix(stream.generationId)
}

fun constructGenerationIdSuffix(generationId: Long): String {
if (generationId < 0) {
throw IllegalArgumentException(
"GenerationId must be non-negative. Provided: ${stream.generationId}",
"GenerationId must be non-negative. Provided: ${generationId}",
)
}
return "ab-generation-id-${stream.generationId}"
return "ab-generation-id-${generationId}"
}
/**
* Builds an Iceberg [Catalog].
Expand Down
Loading