diff --git a/.github/workflows/auto_merge.yml b/.github/workflows/auto_merge.yml
index 7fa85a7e5ac5d..5df20ff06c9fb 100644
--- a/.github/workflows/auto_merge.yml
+++ b/.github/workflows/auto_merge.yml
@@ -2,8 +2,8 @@ name: Auto merge connector PRs Cron
on:
schedule:
- # 0AM UTC is 2AM CEST, 3AM EEST, 5PM PDT.
- - cron: "0 0 * * *"
+ # Every 2 hours on the hour.
+ - cron: "0 */2 * * *"
workflow_dispatch:
jobs:
run_auto_merge:
diff --git a/.github/workflows/cdk-codeflash.yml b/.github/workflows/cdk-codeflash.yml
deleted file mode 100644
index d76d03dc1f80e..0000000000000
--- a/.github/workflows/cdk-codeflash.yml
+++ /dev/null
@@ -1,44 +0,0 @@
-name: Run Codeflash on CDK PR
-
-on:
- pull_request:
- paths:
- - airbyte-cdk/python/airbyte_cdk/**
- workflow_dispatch:
-
-defaults:
- run:
- working-directory: ./airbyte-cdk/python/airbyte_cdk
-
-jobs:
- optimize:
- name:
- Optimize new code in this PR
- # Only run codeflash if
- # 1. The contributor is not codeflash. To avoid infinite loops
- # 2. The pull request is not from a forked repository
- if: ${{ github.actor != 'codeflash-ai[bot]' && github.event_name == 'pull_request' && github.event.pull_request.head.repo.fork != true}}
- runs-on: ubuntu-latest
- env:
- CODEFLASH_API_KEY: ${{ secrets.CODEFLASH_API_KEY }}
- CODEFLASH_PR_NUMBER: ${{ github.event.number }}
- steps:
- - uses: actions/checkout@v4
- with:
- fetch-depth: 0
- token: ${{ secrets.GITHUB_TOKEN }}
- - name: Set up Python 3.10
- uses: actions/setup-python@v5
- with:
- python-version: "3.10"
- - name: Install Airbyte CDK
- id: install-airbyte-cdk
- run: |
- pip install --upgrade pip
- pip install poetry
- poetry install --all-extras --with dev
- - name: Run Codeflash to optimize code
- id: optimize_code
- run: |
- poetry env use python
- poetry run codeflash
diff --git a/.github/workflows/publish-bulk-cdk.yml b/.github/workflows/publish-bulk-cdk.yml
index a1ef1e716c855..b56bb2e48a023 100644
--- a/.github/workflows/publish-bulk-cdk.yml
+++ b/.github/workflows/publish-bulk-cdk.yml
@@ -76,6 +76,17 @@ jobs:
gradle-distribution-sha-256-sum-warning: false
arguments: --scan :airbyte-cdk:bulk:bulkCdkBuild
+ - name: Integration test Bulk CDK
+ uses: burrunan/gradle-cache-action@v1
+ env:
+ CI: true
+ with:
+ read-only: true
+ job-id: bulk-cdk-publish
+ concurrent: true
+ gradle-distribution-sha-256-sum-warning: false
+ arguments: --scan :airbyte-cdk:bulk:bulkCdkIntegrationTest
+
- name: Publish Poms and Jars to CloudRepo
uses: burrunan/gradle-cache-action@v1
env:
diff --git a/.prettierignore b/.prettierignore
index bd77e4f5a9831..530f275141c88 100644
--- a/.prettierignore
+++ b/.prettierignore
@@ -1,6 +1,7 @@
airbyte-integrations/bases/base-normalization/integration_tests/normalization_test_output
airbyte-ci/connectors/pipelines/tests/test_changelog/result_files
airbyte-integrations/bases/connector-acceptance-test/unit_tests/data/docs
+airbyte-integrations/connectors/destination-*/src/test-integration/resources/expected-spec*.json
# Ignore manifest files in manifest-only connectors
# This is done due to prettier being overly opinionated on the formatting of quotes
diff --git a/airbyte-cdk/bulk/build.gradle b/airbyte-cdk/bulk/build.gradle
index b8474f81a5070..7e32b531c339d 100644
--- a/airbyte-cdk/bulk/build.gradle
+++ b/airbyte-cdk/bulk/build.gradle
@@ -61,6 +61,10 @@ allprojects {
}
}
+tasks.register('bulkCdkIntegrationTest').configure {
+ dependsOn allprojects.collect {it.tasks.matching { it.name == 'integrationTest' }}
+}
+
if (buildNumberFile.exists()) {
tasks.register('bulkCdkBuild').configure {
dependsOn allprojects.collect {it.tasks.named('build')}
diff --git a/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/command/ConnectorCommandLinePropertySource.kt b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/command/ConnectorCommandLinePropertySource.kt
index 0dd3f8bb3a987..1e1515a413b33 100644
--- a/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/command/ConnectorCommandLinePropertySource.kt
+++ b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/command/ConnectorCommandLinePropertySource.kt
@@ -53,7 +53,7 @@ private fun resolveValues(
log.warn { "File '$jsonFile' not found for '$cliOptionKey'." }
continue
}
- values["$prefix.json"] = jsonFile.readText()
+ values["$prefix.json"] = jsonFile.readText().replace("$", "\${:$}")
}
return values
}
diff --git a/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/data/AirbyteType.kt b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/data/AirbyteSchemaType.kt
similarity index 90%
rename from airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/data/AirbyteType.kt
rename to airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/data/AirbyteSchemaType.kt
index e74a74d07bf22..ed7c9426529b1 100644
--- a/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/data/AirbyteType.kt
+++ b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/data/AirbyteSchemaType.kt
@@ -12,7 +12,7 @@ import io.airbyte.protocol.models.JsonSchemaType
* This maps to the subset of [JsonSchemaType] which is used in practice. Its main reason for
* existing is to provide type-safety and convenient comparisons and string representations.
*/
-sealed interface AirbyteType {
+sealed interface AirbyteSchemaType {
/** Unwraps the underlying Airbyte protocol type object. */
fun asJsonSchemaType(): JsonSchemaType
@@ -20,18 +20,18 @@ sealed interface AirbyteType {
fun asJsonSchema(): JsonNode = Jsons.valueToTree(asJsonSchemaType().jsonSchemaTypeMap)
}
-data class ArrayAirbyteType(
- val item: AirbyteType,
-) : AirbyteType {
+data class ArrayAirbyteSchemaType(
+ val item: AirbyteSchemaType,
+) : AirbyteSchemaType {
override fun asJsonSchemaType(): JsonSchemaType =
JsonSchemaType.builder(JsonSchemaPrimitiveUtil.JsonSchemaPrimitive.ARRAY)
.withItems(item.asJsonSchemaType())
.build()
}
-enum class LeafAirbyteType(
+enum class LeafAirbyteSchemaType(
private val jsonSchemaType: JsonSchemaType,
-) : AirbyteType {
+) : AirbyteSchemaType {
BOOLEAN(JsonSchemaType.BOOLEAN),
STRING(JsonSchemaType.STRING),
BINARY(JsonSchemaType.STRING_BASE_64),
diff --git a/airbyte-cdk/bulk/core/base/src/testFixtures/resources/log4j2-test.xml b/airbyte-cdk/bulk/core/base/src/testFixtures/resources/log4j2-test.xml
index e82ce1b1a34db..17ec0e432a371 100644
--- a/airbyte-cdk/bulk/core/base/src/testFixtures/resources/log4j2-test.xml
+++ b/airbyte-cdk/bulk/core/base/src/testFixtures/resources/log4j2-test.xml
@@ -7,7 +7,7 @@
%d{yyyy-MM-dd'T'HH:mm:ss,SSS}{GMT+0}`%replace{%X{log_source}}{^ -}{} > %replace{%m}{$${env:LOG_SCRUB_PATTERN:-\*\*\*\*\*}}{*****}%n
${sys:LOG_LEVEL:-${env:LOG_LEVEL:-INFO}}
- target/test-logs/${date:yyyy-MM-dd'T'HH:mm:ss}
+ build/test-logs/${date:yyyy-MM-dd'T'HH:mm:ss}
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/command/SourceConfiguration.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/command/SourceConfiguration.kt
index 447757f0f46ef..b44451ac18497 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/command/SourceConfiguration.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/command/SourceConfiguration.kt
@@ -10,6 +10,9 @@ interface SourceConfiguration : Configuration, SshTunnelConfiguration {
/** Does READ generate states of type GLOBAL? */
val global: Boolean
+ /** Maximum amount of time may be set to limit overall snapshotting duration */
+ val maxSnapshotReadDuration: Duration?
+
/** During the READ operation, how often a feed should checkpoint, ideally. */
val checkpointTargetInterval: Duration
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/AirbyteStreamFactory.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/AirbyteStreamFactory.kt
index 4dd2b568225ad..b238524f4e96a 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/AirbyteStreamFactory.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/AirbyteStreamFactory.kt
@@ -20,7 +20,7 @@ interface AirbyteStreamFactory {
discoveredStream.id.name,
discoveredStream.id.namespace,
discoveredStream.columns.map {
- AirbyteField.of(it.id, it.type.airbyteType.asJsonSchemaType())
+ AirbyteField.of(it.id, it.type.airbyteSchemaType.asJsonSchemaType())
},
)
}
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/Field.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/Field.kt
index 364c32b43bada..5eec0d556a5b4 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/Field.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/discover/Field.kt
@@ -1,12 +1,12 @@
/* Copyright (c) 2024 Airbyte, Inc., all rights reserved. */
package io.airbyte.cdk.discover
-import io.airbyte.cdk.data.AirbyteType
+import io.airbyte.cdk.data.AirbyteSchemaType
import io.airbyte.cdk.data.IntCodec
import io.airbyte.cdk.data.JsonDecoder
import io.airbyte.cdk.data.JsonEncoder
import io.airbyte.cdk.data.JsonStringCodec
-import io.airbyte.cdk.data.LeafAirbyteType
+import io.airbyte.cdk.data.LeafAirbyteSchemaType
import io.airbyte.cdk.data.OffsetDateTimeCodec
import java.time.OffsetDateTime
@@ -23,7 +23,7 @@ sealed interface FieldOrMetaField {
*/
interface FieldType {
/** maps to [io.airbyte.protocol.models.Field.type] */
- val airbyteType: AirbyteType
+ val airbyteSchemaType: AirbyteSchemaType
val jsonEncoder: JsonEncoder<*>
}
@@ -73,19 +73,20 @@ enum class CommonMetaField(
}
data object CdcStringMetaFieldType : LosslessFieldType {
- override val airbyteType: AirbyteType = LeafAirbyteType.STRING
+ override val airbyteSchemaType: AirbyteSchemaType = LeafAirbyteSchemaType.STRING
override val jsonEncoder: JsonEncoder = JsonStringCodec
override val jsonDecoder: JsonDecoder = JsonStringCodec
}
data object CdcIntegerMetaFieldType : LosslessFieldType {
- override val airbyteType: AirbyteType = LeafAirbyteType.INTEGER
+ override val airbyteSchemaType: AirbyteSchemaType = LeafAirbyteSchemaType.INTEGER
override val jsonEncoder: JsonEncoder = IntCodec
override val jsonDecoder: JsonDecoder = IntCodec
}
data object CdcOffsetDateTimeMetaFieldType : LosslessFieldType {
- override val airbyteType: AirbyteType = LeafAirbyteType.TIMESTAMP_WITH_TIMEZONE
+ override val airbyteSchemaType: AirbyteSchemaType =
+ LeafAirbyteSchemaType.TIMESTAMP_WITH_TIMEZONE
override val jsonEncoder: JsonEncoder = OffsetDateTimeCodec
override val jsonDecoder: JsonDecoder = OffsetDateTimeCodec
}
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/output/CatalogValidationFailureHandler.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/output/CatalogValidationFailureHandler.kt
index 5771b07934d28..61e30795488b0 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/output/CatalogValidationFailureHandler.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/output/CatalogValidationFailureHandler.kt
@@ -2,7 +2,7 @@
package io.airbyte.cdk.output
import io.airbyte.cdk.StreamIdentifier
-import io.airbyte.cdk.data.AirbyteType
+import io.airbyte.cdk.data.AirbyteSchemaType
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.DefaultImplementation
import jakarta.inject.Singleton
@@ -43,8 +43,8 @@ data class FieldNotFound(
data class FieldTypeMismatch(
override val streamID: StreamIdentifier,
val fieldName: String,
- val expected: AirbyteType,
- val actual: AirbyteType,
+ val expected: AirbyteSchemaType,
+ val actual: AirbyteSchemaType,
) : CatalogValidationFailure
data class InvalidPrimaryKey(
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/Feed.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/Feed.kt
index 396e9c7b5d498..64c9b66e4473a 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/Feed.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/Feed.kt
@@ -44,3 +44,11 @@ data class Stream(
override val label: String
get() = id.toString()
}
+
+/** List of [Stream]s this [Feed] emits records for. */
+val Feed.streams
+ get() =
+ when (this) {
+ is Global -> streams
+ is Stream -> listOf(this)
+ }
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/FeedReader.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/FeedReader.kt
index 966b88e34366b..380fc4852909c 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/FeedReader.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/FeedReader.kt
@@ -2,11 +2,9 @@
package io.airbyte.cdk.read
import io.airbyte.cdk.SystemErrorException
-import io.airbyte.cdk.asProtocolStreamDescriptor
import io.airbyte.cdk.command.OpaqueStateValue
import io.airbyte.cdk.util.ThreadRenamingCoroutineName
import io.airbyte.protocol.models.v0.AirbyteStateMessage
-import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage
import io.github.oshai.kotlinlogging.KotlinLogging
import kotlin.coroutines.CoroutineContext
import kotlin.coroutines.coroutineContext
@@ -46,7 +44,7 @@ class FeedReader(
// Publish a checkpoint if applicable.
maybeCheckpoint()
// Publish stream completion.
- emitStreamStatus(AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE)
+ root.streamStatusManager.notifyComplete(feed)
break
}
// Launch coroutines which read from each partition.
@@ -85,7 +83,7 @@ class FeedReader(
acquirePartitionsCreatorResources(partitionsCreatorID, partitionsCreator)
}
if (1L == partitionsCreatorID) {
- emitStreamStatus(AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.STARTED)
+ root.streamStatusManager.notifyStarting(feed)
}
return withContext(ctx("round-$partitionsCreatorID-create-partitions")) {
createPartitionsWithResources(partitionsCreatorID, partitionsCreator)
@@ -309,14 +307,4 @@ class FeedReader(
root.outputConsumer.accept(stateMessage)
}
}
-
- private fun emitStreamStatus(status: AirbyteStreamStatusTraceMessage.AirbyteStreamStatus) {
- if (feed is Stream) {
- root.outputConsumer.accept(
- AirbyteStreamStatusTraceMessage()
- .withStreamDescriptor(feed.id.asProtocolStreamDescriptor())
- .withStatus(status),
- )
- }
- }
}
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/RootReader.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/RootReader.kt
index 9c120b15e1ea4..93ee1095efc87 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/RootReader.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/RootReader.kt
@@ -1,6 +1,7 @@
/* Copyright (c) 2024 Airbyte, Inc., all rights reserved. */
package io.airbyte.cdk.read
+import io.airbyte.cdk.ConfigErrorException
import io.airbyte.cdk.output.OutputConsumer
import io.airbyte.cdk.util.ThreadRenamingCoroutineName
import io.github.oshai.kotlinlogging.KotlinLogging
@@ -8,6 +9,7 @@ import java.time.Duration
import java.util.concurrent.ConcurrentHashMap
import kotlin.coroutines.CoroutineContext
import kotlin.time.toKotlinDuration
+import kotlinx.coroutines.CancellationException
import kotlinx.coroutines.CoroutineExceptionHandler
import kotlinx.coroutines.Job
import kotlinx.coroutines.cancel
@@ -51,6 +53,8 @@ class RootReader(
}
}
+ val streamStatusManager = StreamStatusManager(stateManager.feeds, outputConsumer::accept)
+
/** Reads records from all [Feed]s. */
suspend fun read(listener: suspend (Map) -> Unit = {}) {
supervisorScope {
@@ -60,42 +64,37 @@ class RootReader(
val feedJobs: Map =
feeds.associateWith { feed: Feed ->
val coroutineName = ThreadRenamingCoroutineName(feed.label)
- val handler = FeedExceptionHandler(feed, exceptions)
+ val handler = FeedExceptionHandler(feed, streamStatusManager, exceptions)
launch(coroutineName + handler) { FeedReader(this@RootReader, feed).read() }
}
// Call listener hook.
listener(feedJobs)
- // Join on all stream feeds and collect caught exceptions.
- val streamExceptions: Map =
- feeds.filterIsInstance().associateWith {
- feedJobs[it]?.join()
- exceptions[it]
- }
- // Cancel any incomplete global feed job whose stream feed jobs have not all succeeded.
- for ((global, globalJob) in feedJobs) {
- if (global !is Global) continue
- if (globalJob.isCompleted) continue
- val globalStreamExceptions: List =
- global.streams.mapNotNull { streamExceptions[it] }
- if (globalStreamExceptions.isNotEmpty()) {
- val cause: Throwable =
- globalStreamExceptions.reduce { acc: Throwable, exception: Throwable ->
- acc.addSuppressed(exception)
- acc
- }
- globalJob.cancel("at least one stream did non complete", cause)
- }
- }
// Join on all global feeds and collect caught exceptions.
val globalExceptions: Map =
feeds.filterIsInstance().associateWith {
feedJobs[it]?.join()
exceptions[it]
}
+
+ // Certain errors on the global feed cause a full stop to all stream reads
+ if (globalExceptions.values.filterIsInstance().isNotEmpty()) {
+ this@supervisorScope.cancel()
+ }
+
+ // Join on all stream feeds and collect caught exceptions.
+ val streamExceptions: Map =
+ feeds.filterIsInstance().associateWith {
+ try {
+ feedJobs[it]?.join()
+ exceptions[it]
+ } catch (_: CancellationException) {
+ null
+ }
+ }
// Reduce and throw any caught exceptions.
val caughtExceptions: List =
- streamExceptions.values.mapNotNull { it } +
- globalExceptions.values.mapNotNull { it }
+ globalExceptions.values.mapNotNull { it } +
+ streamExceptions.values.mapNotNull { it }
if (caughtExceptions.isNotEmpty()) {
val cause: Throwable =
caughtExceptions.reduce { acc: Throwable, exception: Throwable ->
@@ -109,6 +108,7 @@ class RootReader(
class FeedExceptionHandler(
val feed: Feed,
+ val streamStatusManager: StreamStatusManager,
private val exceptions: ConcurrentHashMap,
) : CoroutineExceptionHandler {
private val log = KotlinLogging.logger {}
@@ -121,6 +121,7 @@ class RootReader(
exception: Throwable,
) {
log.warn(exception) { "canceled feed '${feed.label}' due to thrown exception" }
+ streamStatusManager.notifyFailure(feed)
exceptions[feed] = exception
}
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StateManagerFactory.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StateManagerFactory.kt
index c142c92d307a9..95140bcc9724b 100644
--- a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StateManagerFactory.kt
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StateManagerFactory.kt
@@ -10,9 +10,9 @@ import io.airbyte.cdk.command.GlobalInputState
import io.airbyte.cdk.command.InputState
import io.airbyte.cdk.command.SourceConfiguration
import io.airbyte.cdk.command.StreamInputState
-import io.airbyte.cdk.data.AirbyteType
-import io.airbyte.cdk.data.ArrayAirbyteType
-import io.airbyte.cdk.data.LeafAirbyteType
+import io.airbyte.cdk.data.AirbyteSchemaType
+import io.airbyte.cdk.data.ArrayAirbyteSchemaType
+import io.airbyte.cdk.data.LeafAirbyteSchemaType
import io.airbyte.cdk.discover.CommonMetaField
import io.airbyte.cdk.discover.Field
import io.airbyte.cdk.discover.FieldOrMetaField
@@ -132,7 +132,7 @@ class StateManagerFactory(
}
}
- val expectedSchema: Map =
+ val expectedSchema: Map =
jsonSchemaProperties.properties().associate { (id: String, schema: JsonNode) ->
id to airbyteTypeFromJsonSchema(schema)
}
@@ -150,15 +150,15 @@ class StateManagerFactory(
handler.accept(FieldNotFound(streamID, id))
return null
}
- val expectedAirbyteType: AirbyteType = expectedSchema[id] ?: return null
- val actualAirbyteType: AirbyteType = actualColumn.type.airbyteType
- if (expectedAirbyteType != actualAirbyteType) {
+ val expectedAirbyteSchemaType: AirbyteSchemaType = expectedSchema[id] ?: return null
+ val actualAirbyteSchemaType: AirbyteSchemaType = actualColumn.type.airbyteSchemaType
+ if (expectedAirbyteSchemaType != actualAirbyteSchemaType) {
handler.accept(
FieldTypeMismatch(
streamID,
id,
- expectedAirbyteType,
- actualAirbyteType,
+ expectedAirbyteSchemaType,
+ actualAirbyteSchemaType,
),
)
return null
@@ -229,44 +229,44 @@ class StateManagerFactory(
}
/**
- * Recursively re-generates the original [AirbyteType] from a catalog stream field's JSON
+ * Recursively re-generates the original [AirbyteSchemaType] from a catalog stream field's JSON
* schema.
*/
- private fun airbyteTypeFromJsonSchema(jsonSchema: JsonNode): AirbyteType {
+ private fun airbyteTypeFromJsonSchema(jsonSchema: JsonNode): AirbyteSchemaType {
fun value(key: String): String = jsonSchema[key]?.asText() ?: ""
return when (value("type")) {
- "array" -> ArrayAirbyteType(airbyteTypeFromJsonSchema(jsonSchema["items"]))
- "null" -> LeafAirbyteType.NULL
- "boolean" -> LeafAirbyteType.BOOLEAN
+ "array" -> ArrayAirbyteSchemaType(airbyteTypeFromJsonSchema(jsonSchema["items"]))
+ "null" -> LeafAirbyteSchemaType.NULL
+ "boolean" -> LeafAirbyteSchemaType.BOOLEAN
"number" ->
when (value("airbyte_type")) {
"integer",
- "big_integer", -> LeafAirbyteType.INTEGER
- else -> LeafAirbyteType.NUMBER
+ "big_integer", -> LeafAirbyteSchemaType.INTEGER
+ else -> LeafAirbyteSchemaType.NUMBER
}
"string" ->
when (value("format")) {
- "date" -> LeafAirbyteType.DATE
+ "date" -> LeafAirbyteSchemaType.DATE
"date-time" ->
if (value("airbyte_type") == "timestamp_with_timezone") {
- LeafAirbyteType.TIMESTAMP_WITH_TIMEZONE
+ LeafAirbyteSchemaType.TIMESTAMP_WITH_TIMEZONE
} else {
- LeafAirbyteType.TIMESTAMP_WITHOUT_TIMEZONE
+ LeafAirbyteSchemaType.TIMESTAMP_WITHOUT_TIMEZONE
}
"time" ->
if (value("airbyte_type") == "time_with_timezone") {
- LeafAirbyteType.TIME_WITH_TIMEZONE
+ LeafAirbyteSchemaType.TIME_WITH_TIMEZONE
} else {
- LeafAirbyteType.TIME_WITHOUT_TIMEZONE
+ LeafAirbyteSchemaType.TIME_WITHOUT_TIMEZONE
}
else ->
if (value("contentEncoding") == "base64") {
- LeafAirbyteType.BINARY
+ LeafAirbyteSchemaType.BINARY
} else {
- LeafAirbyteType.STRING
+ LeafAirbyteSchemaType.STRING
}
}
- else -> LeafAirbyteType.JSONB
+ else -> LeafAirbyteSchemaType.JSONB
}
}
}
diff --git a/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StreamStatusManager.kt b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StreamStatusManager.kt
new file mode 100644
index 0000000000000..52abb4c82982e
--- /dev/null
+++ b/airbyte-cdk/bulk/core/extract/src/main/kotlin/io/airbyte/cdk/read/StreamStatusManager.kt
@@ -0,0 +1,137 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.read
+
+import io.airbyte.cdk.StreamIdentifier
+import io.airbyte.cdk.asProtocolStreamDescriptor
+import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage
+import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage.AirbyteStreamStatus
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.concurrent.atomic.AtomicReference
+import org.apache.mina.util.ConcurrentHashSet
+
+/**
+ * [StreamStatusManager] emits [AirbyteStreamStatusTraceMessage]s in response to [Feed] activity
+ * events, via [notifyStarting], [notifyComplete] and [notifyFailure].
+ */
+class StreamStatusManager(
+ feeds: List,
+ private val emit: (AirbyteStreamStatusTraceMessage) -> Unit,
+) {
+ private val streamStates: Map =
+ feeds
+ .flatMap { feed: Feed -> feed.streams.map { it.id to feed } }
+ .groupBy({ it.first }, { it.second })
+ .mapValues { (id: StreamIdentifier, feeds: List) ->
+ StreamState(id, feeds.toSet())
+ }
+
+ /**
+ * Notify that the [feed] is about to start running.
+ *
+ * Emits Airbyte TRACE messages of type STATUS accordingly. Safe to call even if
+ * [notifyStarting], [notifyComplete] or [notifyFailure] have been called before.
+ */
+ fun notifyStarting(feed: Feed) {
+ handle(feed) { it.onStarting() }
+ }
+
+ /**
+ * Notify that the [feed] has completed running.
+ *
+ * Emits Airbyte TRACE messages of type STATUS accordingly. Idempotent. Safe to call even if
+ * [notifyStarting] hasn't been called previously.
+ */
+ fun notifyComplete(feed: Feed) {
+ handle(feed) { it.onComplete(feed) }
+ }
+
+ /**
+ * Notify that the [feed] has stopped running due to a failure.
+ *
+ * Emits Airbyte TRACE messages of type STATUS accordingly. Idempotent. Safe to call even if
+ * [notifyStarting] hasn't been called previously.
+ */
+ fun notifyFailure(feed: Feed) {
+ handle(feed) { it.onFailure(feed) }
+ }
+
+ private fun handle(feed: Feed, notification: (StreamState) -> List) {
+ for (stream in feed.streams) {
+ val streamState: StreamState = streamStates[stream.id] ?: continue
+ for (statusToEmit: AirbyteStreamStatus in notification(streamState)) {
+ emit(
+ AirbyteStreamStatusTraceMessage()
+ .withStreamDescriptor(stream.id.asProtocolStreamDescriptor())
+ .withStatus(statusToEmit)
+ )
+ }
+ }
+ }
+
+ data class StreamState(
+ val id: StreamIdentifier,
+ val feeds: Set,
+ val state: AtomicReference = AtomicReference(State.PENDING),
+ val stoppedFeeds: ConcurrentHashSet = ConcurrentHashSet(),
+ val numStoppedFeeds: AtomicInteger = AtomicInteger()
+ ) {
+ fun onStarting(): List =
+ if (state.compareAndSet(State.PENDING, State.SUCCESS)) {
+ listOf(AirbyteStreamStatus.STARTED)
+ } else {
+ emptyList()
+ }
+
+ fun onComplete(feed: Feed): List =
+ onStarting() + // ensure the state is not PENDING
+ run {
+ if (!finalStop(feed)) {
+ return@run emptyList()
+ }
+ // At this point, we just stopped the last feed for this stream.
+ // Transition to DONE.
+ if (state.compareAndSet(State.SUCCESS, State.DONE)) {
+ listOf(AirbyteStreamStatus.COMPLETE)
+ } else if (state.compareAndSet(State.FAILURE, State.DONE)) {
+ listOf(AirbyteStreamStatus.INCOMPLETE)
+ } else {
+ emptyList() // this should never happen
+ }
+ }
+
+ fun onFailure(feed: Feed): List =
+ onStarting() + // ensure the state is not PENDING
+ run {
+ state.compareAndSet(State.SUCCESS, State.FAILURE)
+ if (!finalStop(feed)) {
+ return@run emptyList()
+ }
+ // At this point, we just stopped the last feed for this stream.
+ // Transition from FAILURE to DONE.
+ if (state.compareAndSet(State.FAILURE, State.DONE)) {
+ listOf(AirbyteStreamStatus.INCOMPLETE)
+ } else {
+ emptyList() // this should never happen
+ }
+ }
+
+ private fun finalStop(feed: Feed): Boolean {
+ if (!stoppedFeeds.add(feed)) {
+ // This feed was stopped before.
+ return false
+ }
+ // True if and only if this feed was stopped and all others were already stopped.
+ return numStoppedFeeds.incrementAndGet() == feeds.size
+ }
+ }
+
+ enum class State {
+ PENDING,
+ SUCCESS,
+ FAILURE,
+ DONE,
+ }
+}
diff --git a/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/fakesource/FakeSourceConfiguration.kt b/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/fakesource/FakeSourceConfiguration.kt
index 4947654a81e63..b8da868967f86 100644
--- a/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/fakesource/FakeSourceConfiguration.kt
+++ b/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/fakesource/FakeSourceConfiguration.kt
@@ -21,6 +21,7 @@ data class FakeSourceConfiguration(
val cursor: CursorConfiguration,
override val maxConcurrency: Int,
override val checkpointTargetInterval: Duration,
+ override val maxSnapshotReadDuration: Duration? = null,
) : SourceConfiguration {
override val global: Boolean = cursor is CdcCursor
diff --git a/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/RootReaderIntegrationTest.kt b/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/RootReaderIntegrationTest.kt
index 209683a07176c..1932910e8dfab 100644
--- a/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/RootReaderIntegrationTest.kt
+++ b/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/RootReaderIntegrationTest.kt
@@ -4,6 +4,7 @@ package io.airbyte.cdk.read
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.node.ArrayNode
import io.airbyte.cdk.ClockFactory
+import io.airbyte.cdk.ConfigErrorException
import io.airbyte.cdk.StreamIdentifier
import io.airbyte.cdk.command.OpaqueStateValue
import io.airbyte.cdk.output.BufferingOutputConsumer
@@ -194,6 +195,51 @@ class RootReaderIntegrationTest {
Assertions.assertFalse(globalStateMessages.isEmpty())
}
+ @Test
+ fun testAllStreamsGlobalConfigError() {
+ val stateManager =
+ StateManager(
+ global = Global(testCases.map { it.stream }),
+ initialGlobalState = null,
+ initialStreamStates = testCases.associate { it.stream to null },
+ )
+ val testOutputConsumer = BufferingOutputConsumer(ClockFactory().fixed())
+ val rootReader =
+ RootReader(
+ stateManager,
+ slowHeartbeat,
+ excessiveTimeout,
+ testOutputConsumer,
+ listOf(
+ ConfigErrorThrowingGlobalPartitionsCreatorFactory(
+ Semaphore(CONSTRAINED),
+ *testCases.toTypedArray()
+ )
+ ),
+ )
+ Assertions.assertThrows(ConfigErrorException::class.java) {
+ runBlocking(Dispatchers.Default) { rootReader.read() }
+ }
+ val log = KotlinLogging.logger {}
+ for (msg in testOutputConsumer.messages()) {
+ log.info { Jsons.writeValueAsString(msg) }
+ }
+ for (testCase in testCases) {
+ log.info { "checking stream feed for ${testCase.name}" }
+ val streamStateMessages: List =
+ testOutputConsumer.states().filter {
+ it.stream?.streamDescriptor?.name == testCase.name
+ }
+ Assertions.assertTrue(streamStateMessages.isEmpty())
+ }
+ log.info { "checking global feed" }
+ val globalStateMessages: List =
+ testOutputConsumer.states().filter {
+ it.type == AirbyteStateMessage.AirbyteStateType.GLOBAL
+ }
+ Assertions.assertTrue(globalStateMessages.isEmpty())
+ }
+
companion object {
const val CONSTRAINED = 2
}
@@ -273,6 +319,7 @@ data class TestCase(
fun verifyTraces(traceMessages: List) {
var hasStarted = false
var hasCompleted = false
+ var hasIncompleted = false
for (trace in traceMessages) {
when (trace.type) {
AirbyteTraceMessage.Type.STREAM_STATUS -> {
@@ -282,14 +329,29 @@ data class TestCase(
hasStarted = true
Assertions.assertFalse(
hasCompleted,
- "Case $name cannot emit a STARTED trace message because it already emitted a COMPLETE."
+ "Case $name cannot emit a STARTED trace " +
+ "message because it already emitted a COMPLETE."
+ )
+ Assertions.assertFalse(
+ hasIncompleted,
+ "Case $name cannot emit a STARTED trace " +
+ "message because it already emitted an INCOMPLETE."
)
}
AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE -> {
hasCompleted = true
Assertions.assertTrue(
hasStarted,
- "Case $name cannot emit a COMPLETE trace message because it hasn't emitted a STARTED yet."
+ "Case $name cannot emit a COMPLETE trace " +
+ "message because it hasn't emitted a STARTED yet."
+ )
+ }
+ AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.INCOMPLETE -> {
+ hasIncompleted = true
+ Assertions.assertTrue(
+ hasStarted,
+ "Case $name cannot emit an INCOMPLETE trace " +
+ "message because it hasn't emitted a STARTED yet."
)
}
else ->
@@ -310,15 +372,25 @@ data class TestCase(
"Case $name should have emitted a STARTED trace message, but hasn't."
)
if (isSuccessful) {
- Assertions.assertTrue(
- hasCompleted,
- "Case $name should have emitted a COMPLETE trace message, but hasn't."
+ if (!hasCompleted) {
+ Assertions.assertTrue(
+ hasCompleted,
+ "Case $name should have emitted a COMPLETE trace message, but hasn't."
+ )
+ }
+ Assertions.assertFalse(
+ hasIncompleted,
+ "Case $name should not have emitted an INCOMPLETE trace message, but did anyway."
)
} else {
Assertions.assertFalse(
hasCompleted,
"Case $name should not have emitted a COMPLETE trace message, but did anyway."
)
+ Assertions.assertTrue(
+ hasIncompleted,
+ "Case $name should have emitted an INCOMPLETE trace message, but hasn't."
+ )
}
}
@@ -545,7 +617,7 @@ class TestPartitionReader(
)
}
-class TestPartitionsCreatorFactory(
+open class TestPartitionsCreatorFactory(
val resource: Semaphore,
vararg val testCases: TestCase,
) : PartitionsCreatorFactory {
@@ -556,22 +628,7 @@ class TestPartitionsCreatorFactory(
feed: Feed,
): PartitionsCreator {
if (feed is Global) {
- // For a global feed, return a bogus PartitionsCreator which backs off forever.
- // This tests that the corresponding coroutine gets canceled properly.
- return object : PartitionsCreator {
- override fun tryAcquireResources(): PartitionsCreator.TryAcquireResourcesStatus {
- log.info { "failed to acquire resources for global feed, as always" }
- return PartitionsCreator.TryAcquireResourcesStatus.RETRY_LATER
- }
-
- override suspend fun run(): List {
- TODO("unreachable code")
- }
-
- override fun releaseResources() {
- TODO("unreachable code")
- }
- }
+ return makeGlobalPartitionsCreator()
}
// For a stream feed, pick the CreatorCase in the corresponding TestCase
// which is the successor of the one whose corresponding state is in the StateQuerier.
@@ -591,6 +648,40 @@ class TestPartitionsCreatorFactory(
resource,
)
}
+
+ protected open fun makeGlobalPartitionsCreator(): PartitionsCreator {
+ return object : PartitionsCreator {
+ override fun tryAcquireResources(): PartitionsCreator.TryAcquireResourcesStatus {
+ return PartitionsCreator.TryAcquireResourcesStatus.READY_TO_RUN
+ }
+
+ override suspend fun run(): List {
+ // Do nothing.
+ return emptyList()
+ }
+
+ override fun releaseResources() {}
+ }
+ }
+}
+
+class ConfigErrorThrowingGlobalPartitionsCreatorFactory(
+ resource: Semaphore,
+ vararg testCases: TestCase,
+) : TestPartitionsCreatorFactory(resource, *testCases) {
+ override fun makeGlobalPartitionsCreator(): PartitionsCreator {
+ return object : PartitionsCreator {
+ override fun tryAcquireResources(): PartitionsCreator.TryAcquireResourcesStatus {
+ return PartitionsCreator.TryAcquireResourcesStatus.READY_TO_RUN
+ }
+
+ override suspend fun run(): List {
+ throw ConfigErrorException("some config error")
+ }
+
+ override fun releaseResources() {}
+ }
+ }
}
/** Tests should succeed and not timeout. */
diff --git a/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/StreamStatusManagerTest.kt b/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/StreamStatusManagerTest.kt
new file mode 100644
index 0000000000000..0eced8069717b
--- /dev/null
+++ b/airbyte-cdk/bulk/core/extract/src/test/kotlin/io/airbyte/cdk/read/StreamStatusManagerTest.kt
@@ -0,0 +1,228 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.read
+
+import io.airbyte.cdk.StreamIdentifier
+import io.airbyte.cdk.discover.Field
+import io.airbyte.cdk.discover.IntFieldType
+import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage
+import io.airbyte.protocol.models.v0.StreamDescriptor
+import org.junit.jupiter.api.Assertions
+import org.junit.jupiter.api.Test
+
+class StreamStatusManagerTest {
+
+ val streamIncremental =
+ Stream(
+ id = StreamIdentifier.from(StreamDescriptor().withName("streamIncremental")),
+ fields = listOf(Field("v", IntFieldType)),
+ configuredSyncMode = ConfiguredSyncMode.INCREMENTAL,
+ configuredPrimaryKey = null,
+ configuredCursor = null,
+ )
+ val streamFullRefresh =
+ Stream(
+ id = StreamIdentifier.from(StreamDescriptor().withName("streamFullRefresh")),
+ fields = listOf(Field("v", IntFieldType)),
+ configuredSyncMode = ConfiguredSyncMode.FULL_REFRESH,
+ configuredPrimaryKey = null,
+ configuredCursor = null,
+ )
+
+ val allStreams: Set = setOf(streamFullRefresh, streamIncremental)
+
+ val global: Global
+ get() = Global(listOf(streamIncremental))
+
+ val allFeeds: List = listOf(global) + allStreams
+
+ @Test
+ fun testNothing() {
+ TestCase(allFeeds).runTest {}
+ }
+
+ @Test
+ fun testRunningStream() {
+ val testCase = TestCase(listOf(streamFullRefresh), started = setOf(streamFullRefresh))
+ testCase.runTest { it.notifyStarting(streamFullRefresh) }
+ // Check that the outcome is the same if we call notifyStarting multiple times.
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyStarting(streamFullRefresh)
+ it.notifyStarting(streamFullRefresh)
+ }
+ }
+
+ @Test
+ fun testRunningAndCompleteStream() {
+ val testCase =
+ TestCase(
+ feeds = listOf(streamFullRefresh),
+ started = setOf(streamFullRefresh),
+ success = setOf(streamFullRefresh),
+ )
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ }
+ // Check that the outcome is the same if we forget to call notifyStarting.
+ testCase.runTest { it.notifyComplete(streamFullRefresh) }
+ // Check that the outcome is the same if we call notifyComplete many times.
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ }
+ // Check that the outcome is the same if we call notifyFailure afterwards.
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ it.notifyFailure(streamFullRefresh)
+ }
+ }
+
+ @Test
+ fun testRunningAndIncompleteStream() {
+ val testCase =
+ TestCase(
+ feeds = listOf(streamFullRefresh),
+ started = setOf(streamFullRefresh),
+ failure = setOf(streamFullRefresh),
+ )
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyFailure(streamFullRefresh)
+ }
+ // Check that the outcome is the same if we forget to call notifyStarting.
+ testCase.runTest { it.notifyFailure(streamFullRefresh) }
+ // Check that the outcome is the same if we call notifyFailure many times.
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyFailure(streamFullRefresh)
+ it.notifyFailure(streamFullRefresh)
+ it.notifyFailure(streamFullRefresh)
+ }
+ // Check that the outcome is the same if we call notifyComplete afterwards.
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyFailure(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ }
+ }
+
+ @Test
+ fun testRunningStreamWithGlobal() {
+ val testCase = TestCase(allFeeds, started = setOf(streamIncremental))
+ testCase.runTest { it.notifyStarting(streamIncremental) }
+ // Check that the outcome is the same if we call notifyStarting with the global feed.
+ testCase.runTest { it.notifyStarting(global) }
+ testCase.runTest {
+ it.notifyStarting(global)
+ it.notifyStarting(streamIncremental)
+ }
+ }
+
+ @Test
+ fun testRunningAndCompleteWithGlobal() {
+ val testCase =
+ TestCase(
+ feeds = allFeeds,
+ started = setOf(streamIncremental),
+ success = setOf(streamIncremental),
+ )
+ testCase.runTest {
+ it.notifyStarting(global)
+ it.notifyComplete(global)
+ it.notifyStarting(streamIncremental)
+ it.notifyComplete(streamIncremental)
+ }
+ // Check that the outcome is the same if we mix things up a bit.
+ testCase.runTest {
+ it.notifyStarting(global)
+ it.notifyStarting(streamIncremental)
+ it.notifyComplete(global)
+ it.notifyComplete(streamIncremental)
+ }
+ testCase.runTest {
+ it.notifyStarting(streamIncremental)
+ it.notifyStarting(global)
+ it.notifyComplete(global)
+ it.notifyComplete(streamIncremental)
+ }
+ }
+
+ @Test
+ fun testRunningAndIncompleteAll() {
+ val testCase =
+ TestCase(
+ feeds = allFeeds,
+ started = allStreams,
+ success = setOf(streamFullRefresh),
+ failure = setOf(streamIncremental),
+ )
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyComplete(streamFullRefresh)
+ it.notifyStarting(global)
+ it.notifyFailure(global)
+ it.notifyStarting(streamIncremental)
+ it.notifyComplete(streamIncremental)
+ }
+ // Check that the outcome is the same if we mix things up a bit.
+ testCase.runTest {
+ it.notifyStarting(streamFullRefresh)
+ it.notifyStarting(global)
+ it.notifyStarting(streamIncremental)
+ it.notifyComplete(streamIncremental)
+ it.notifyFailure(global)
+ it.notifyComplete(streamFullRefresh)
+ it.notifyComplete(global)
+ }
+ }
+
+ data class TestCase
+ private constructor(
+ val started: Set,
+ val success: Set,
+ val failure: Set,
+ val feeds: List,
+ ) {
+ constructor(
+ feeds: List,
+ started: Set = emptySet(),
+ success: Set = emptySet(),
+ failure: Set = emptySet(),
+ ) : this(
+ started.map { it.id }.toSet(),
+ success.map { it.id }.toSet(),
+ failure.map { it.id }.toSet(),
+ feeds,
+ )
+
+ fun runTest(fn: (StreamStatusManager) -> Unit) {
+ val started = mutableSetOf()
+ val success = mutableSetOf()
+ val failure = mutableSetOf()
+ val streamStatusManager =
+ StreamStatusManager(feeds) {
+ val streamID = StreamIdentifier.from(it.streamDescriptor)
+ when (it.status) {
+ AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.STARTED ->
+ Assertions.assertTrue(started.add(streamID))
+ AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE ->
+ Assertions.assertTrue(success.add(streamID))
+ AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.INCOMPLETE ->
+ Assertions.assertTrue(failure.add(streamID))
+ else -> throw RuntimeException("unexpected status ${it.status}")
+ }
+ }
+ fn(streamStatusManager)
+ Assertions.assertEquals(this.started, started)
+ Assertions.assertEquals(this.success, success)
+ Assertions.assertEquals(this.failure, failure)
+ }
+ }
+}
diff --git a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/IntFieldType.kt b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/IntFieldType.kt
index 821082136bd26..c1bf7c442f9ef 100644
--- a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/IntFieldType.kt
+++ b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/IntFieldType.kt
@@ -4,12 +4,12 @@
package io.airbyte.cdk.discover
-import io.airbyte.cdk.data.AirbyteType
+import io.airbyte.cdk.data.AirbyteSchemaType
import io.airbyte.cdk.data.IntCodec
import io.airbyte.cdk.data.JsonEncoder
-import io.airbyte.cdk.data.LeafAirbyteType
+import io.airbyte.cdk.data.LeafAirbyteSchemaType
data object IntFieldType : FieldType {
- override val airbyteType: AirbyteType = LeafAirbyteType.INTEGER
+ override val airbyteSchemaType: AirbyteSchemaType = LeafAirbyteSchemaType.INTEGER
override val jsonEncoder: JsonEncoder<*> = IntCodec
}
diff --git a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/OffsetDateTimeFieldType.kt b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/OffsetDateTimeFieldType.kt
index ecf3c2035395b..fc25e61939fe6 100644
--- a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/OffsetDateTimeFieldType.kt
+++ b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/OffsetDateTimeFieldType.kt
@@ -4,12 +4,13 @@
package io.airbyte.cdk.discover
-import io.airbyte.cdk.data.AirbyteType
+import io.airbyte.cdk.data.AirbyteSchemaType
import io.airbyte.cdk.data.JsonEncoder
-import io.airbyte.cdk.data.LeafAirbyteType
+import io.airbyte.cdk.data.LeafAirbyteSchemaType
import io.airbyte.cdk.data.OffsetDateTimeCodec
data object OffsetDateTimeFieldType : FieldType {
- override val airbyteType: AirbyteType = LeafAirbyteType.TIMESTAMP_WITH_TIMEZONE
+ override val airbyteSchemaType: AirbyteSchemaType =
+ LeafAirbyteSchemaType.TIMESTAMP_WITH_TIMEZONE
override val jsonEncoder: JsonEncoder<*> = OffsetDateTimeCodec
}
diff --git a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/StringFieldType.kt b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/StringFieldType.kt
index 8f1d386cf2ef2..e5097c873e23a 100644
--- a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/StringFieldType.kt
+++ b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/StringFieldType.kt
@@ -4,12 +4,12 @@
package io.airbyte.cdk.discover
-import io.airbyte.cdk.data.AirbyteType
+import io.airbyte.cdk.data.AirbyteSchemaType
import io.airbyte.cdk.data.JsonEncoder
-import io.airbyte.cdk.data.LeafAirbyteType
+import io.airbyte.cdk.data.LeafAirbyteSchemaType
import io.airbyte.cdk.data.TextCodec
data object StringFieldType : FieldType {
- override val airbyteType: AirbyteType = LeafAirbyteType.STRING
+ override val airbyteSchemaType: AirbyteSchemaType = LeafAirbyteSchemaType.STRING
override val jsonEncoder: JsonEncoder<*> = TextCodec
}
diff --git a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/TestAirbyteStreamFactory.kt b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/TestAirbyteStreamFactory.kt
index 5b84e88c2faa9..c7d9c566460a8 100644
--- a/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/TestAirbyteStreamFactory.kt
+++ b/airbyte-cdk/bulk/core/extract/src/testFixtures/kotlin/io/airbyte/cdk/discover/TestAirbyteStreamFactory.kt
@@ -21,7 +21,7 @@ class TestAirbyteStreamFactory : AirbyteStreamFactory {
supportedSyncModes = listOf(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)
(jsonSchema["properties"] as ObjectNode).apply {
for (metaField in CommonMetaField.entries) {
- set(metaField.id, metaField.type.airbyteType.asJsonSchema())
+ set(metaField.id, metaField.type.airbyteSchemaType.asJsonSchema())
}
}
defaultCursorField = listOf(CommonMetaField.CDC_LSN.id)
diff --git a/airbyte-cdk/bulk/core/load/build.gradle b/airbyte-cdk/bulk/core/load/build.gradle
index e670f0cbe3284..c08fc533c4b7c 100644
--- a/airbyte-cdk/bulk/core/load/build.gradle
+++ b/airbyte-cdk/bulk/core/load/build.gradle
@@ -1,3 +1,16 @@
+// simply declaring the source sets is sufficient to populate them with
+// src/integrationTest/java+resources + src/integrationTest/kotlin.
+sourceSets {
+ integrationTest {
+ }
+}
+kotlin {
+ sourceSets {
+ testIntegration {
+ }
+ }
+}
+
dependencies {
implementation project(':airbyte-cdk:bulk:core:bulk-cdk-core-base')
implementation 'org.apache.commons:commons-lang3:3.17.0'
@@ -9,4 +22,19 @@ dependencies {
testImplementation("org.jetbrains.kotlinx:kotlinx-coroutines-test:1.8.1")
implementation "org.jetbrains.kotlin:kotlin-reflect:2.0.20"
+ testFixturesImplementation "uk.org.webcompere:system-stubs-jupiter:2.1.7"
+}
+
+task integrationTest(type: Test) {
+ description = 'Runs the integration tests.'
+ group = 'verification'
+ testClassesDirs = sourceSets.integrationTest.output.classesDirs
+ classpath = sourceSets.integrationTest.runtimeClasspath
+ useJUnitPlatform()
+}
+configurations {
+ integrationTestImplementation.extendsFrom testImplementation
+ integrationTestRuntimeOnly.extendsFrom testRuntimeOnly
}
+// These tests are lightweight enough to run on every PR.
+rootProject.check.dependsOn(integrationTest)
diff --git a/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockBasicFunctionalityIntegrationTest.kt b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockBasicFunctionalityIntegrationTest.kt
new file mode 100644
index 0000000000000..ae93391b9a1da
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockBasicFunctionalityIntegrationTest.kt
@@ -0,0 +1,25 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.mock_integration_test
+
+import io.airbyte.cdk.load.test.util.NoopDestinationCleaner
+import io.airbyte.cdk.load.test.util.NoopExpectedRecordMapper
+import io.airbyte.cdk.load.test.util.NoopNameMapper
+import io.airbyte.cdk.load.write.BasicFunctionalityIntegrationTest
+import org.junit.jupiter.api.Test
+
+class MockBasicFunctionalityIntegrationTest :
+ BasicFunctionalityIntegrationTest(
+ MockDestinationSpecification(),
+ MockDestinationDataDumper,
+ NoopDestinationCleaner,
+ NoopExpectedRecordMapper,
+ NoopNameMapper
+ ) {
+ @Test
+ override fun testBasicWrite() {
+ super.testBasicWrite()
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationBackend.kt b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationBackend.kt
new file mode 100644
index 0000000000000..fb7720bac2fc0
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationBackend.kt
@@ -0,0 +1,33 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.mock_integration_test
+
+import io.airbyte.cdk.load.test.util.DestinationDataDumper
+import io.airbyte.cdk.load.test.util.OutputRecord
+import java.util.concurrent.ConcurrentHashMap
+
+object MockDestinationBackend {
+ private val files: MutableMap> = ConcurrentHashMap()
+
+ fun insert(filename: String, vararg records: OutputRecord) {
+ getFile(filename).addAll(records)
+ }
+
+ fun readFile(filename: String): List {
+ return getFile(filename)
+ }
+
+ private fun getFile(filename: String): MutableList {
+ return files.getOrPut(filename) { mutableListOf() }
+ }
+}
+
+object MockDestinationDataDumper : DestinationDataDumper {
+ override fun dumpRecords(streamName: String, streamNamespace: String?): List {
+ return MockDestinationBackend.readFile(
+ MockStreamLoader.getFilename(streamNamespace, streamName)
+ )
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationChecker.kt b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationChecker.kt
new file mode 100644
index 0000000000000..9972357704dbd
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationChecker.kt
@@ -0,0 +1,13 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.mock_integration_test
+
+import io.airbyte.cdk.load.check.DestinationChecker
+import javax.inject.Singleton
+
+@Singleton
+class MockDestinationChecker : DestinationChecker {
+ override fun check(config: MockDestinationConfiguration) {}
+}
diff --git a/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationConfiguration.kt b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationConfiguration.kt
new file mode 100644
index 0000000000000..c582e5e463dfb
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationConfiguration.kt
@@ -0,0 +1,34 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.mock_integration_test
+
+import io.airbyte.cdk.command.ConfigurationSpecification
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationConfigurationFactory
+import io.micronaut.context.annotation.Factory
+import jakarta.inject.Singleton
+
+class MockDestinationConfiguration : DestinationConfiguration()
+
+@Singleton class MockDestinationSpecification : ConfigurationSpecification()
+
+@Singleton
+class MockDestinationConfigurationFactory :
+ DestinationConfigurationFactory {
+
+ override fun makeWithoutExceptionHandling(
+ pojo: MockDestinationSpecification
+ ): MockDestinationConfiguration {
+ return MockDestinationConfiguration()
+ }
+}
+
+@Factory
+class MockDestinationConfigurationProvider(private val config: DestinationConfiguration) {
+ @Singleton
+ fun get(): MockDestinationConfiguration {
+ return config as MockDestinationConfiguration
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationWriter.kt b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationWriter.kt
new file mode 100644
index 0000000000000..c49115cb2c1f1
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/integrationTest/kotlin/io/airbyte/cdk/load/mock_integration_test/MockDestinationWriter.kt
@@ -0,0 +1,69 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.mock_integration_test
+
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.data.ObjectValue
+import io.airbyte.cdk.load.message.Batch
+import io.airbyte.cdk.load.message.DestinationRecord
+import io.airbyte.cdk.load.message.SimpleBatch
+import io.airbyte.cdk.load.test.util.OutputRecord
+import io.airbyte.cdk.load.write.DestinationWriter
+import io.airbyte.cdk.load.write.StreamLoader
+import java.time.Instant
+import java.util.UUID
+import javax.inject.Singleton
+
+@Singleton
+class MockDestinationWriter : DestinationWriter {
+ override fun createStreamLoader(stream: DestinationStream): StreamLoader {
+ return MockStreamLoader(stream)
+ }
+}
+
+class MockStreamLoader(override val stream: DestinationStream) : StreamLoader {
+ data class LocalBatch(val records: List) : Batch {
+ override val state = Batch.State.LOCAL
+ }
+ data class PersistedBatch(val records: List) : Batch {
+ override val state = Batch.State.PERSISTED
+ }
+
+ override suspend fun processRecords(
+ records: Iterator,
+ totalSizeBytes: Long
+ ): Batch {
+ return LocalBatch(records.asSequence().toList())
+ }
+
+ override suspend fun processBatch(batch: Batch): Batch {
+ return when (batch) {
+ is LocalBatch -> {
+ batch.records.forEach {
+ MockDestinationBackend.insert(
+ getFilename(it.stream),
+ OutputRecord(
+ UUID.randomUUID(),
+ Instant.ofEpochMilli(it.emittedAtMs),
+ Instant.ofEpochMilli(System.currentTimeMillis()),
+ stream.generationId,
+ it.data as ObjectValue,
+ OutputRecord.Meta(changes = it.meta?.changes, syncId = stream.syncId),
+ )
+ )
+ }
+ PersistedBatch(batch.records)
+ }
+ is PersistedBatch -> SimpleBatch(state = Batch.State.COMPLETE)
+ else -> throw IllegalStateException("Unexpected batch type: $batch")
+ }
+ }
+
+ companion object {
+ fun getFilename(stream: DestinationStream.Descriptor) =
+ getFilename(stream.namespace, stream.name)
+ fun getFilename(namespace: String?, name: String) = "(${namespace},${name})"
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/integrationTest/resources/metadata.yaml b/airbyte-cdk/bulk/core/load/src/integrationTest/resources/metadata.yaml
new file mode 100644
index 0000000000000..3424155343d1d
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/integrationTest/resources/metadata.yaml
@@ -0,0 +1,5 @@
+# This is a minimal metadata.yaml that allows a destination connector to run.
+# A real metadata.yaml obviously contains much more stuff, but we don't strictly
+# need any of it at runtime.
+data:
+ dockerRepository: "airbyte/fake-destination"
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationConfiguration.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationConfiguration.kt
deleted file mode 100644
index e0a0a0e4b987e..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationConfiguration.kt
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.command
-
-import io.micronaut.context.annotation.Factory
-import jakarta.inject.Singleton
-import java.nio.file.Path
-
-abstract class DestinationConfiguration : Configuration {
- open val recordBatchSizeBytes: Long = 200L * 1024L * 1024L
- open val tmpFileDirectory: Path = Path.of("airbyte-cdk-load")
- open val firstStageTmpFilePrefix: String = "staged-raw-records"
- open val firstStageTmpFileSuffix: String = ".jsonl"
-
- /** Memory queue settings */
- open val maxMessageQueueMemoryUsageRatio: Double = 0.2 // 0 => No limit, 1.0 => 100% of JVM heap
- open val estimatedRecordMemoryOverheadRatio: Double =
- 0.1 // 0 => No overhead, 1.0 => 100% overhead
-
- /**
- * Micronaut factory which glues [ConfigurationSpecificationSupplier] and
- * [DestinationConfigurationFactory] together to produce a [DestinationConfiguration] singleton.
- */
- @Factory
- private class MicronautFactory {
- @Singleton
- fun destinationConfig(
- specificationSupplier: ConfigurationSpecificationSupplier,
- factory: DestinationConfigurationFactory,
- ): DestinationConfiguration = factory.make(specificationSupplier.get())
- }
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/check/CheckOperation.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/check/CheckOperation.kt
similarity index 92%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/check/CheckOperation.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/check/CheckOperation.kt
index eae3f262a7899..86b1c6a1befb5 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/check/CheckOperation.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/check/CheckOperation.kt
@@ -2,13 +2,13 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.check
+package io.airbyte.cdk.load.check
import io.airbyte.cdk.Operation
import io.airbyte.cdk.command.ConfigurationSpecification
import io.airbyte.cdk.command.ConfigurationSpecificationSupplier
-import io.airbyte.cdk.command.DestinationConfiguration
-import io.airbyte.cdk.command.DestinationConfigurationFactory
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationConfigurationFactory
import io.airbyte.cdk.output.ExceptionHandler
import io.airbyte.cdk.output.OutputConsumer
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/check/DestinationChecker.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/check/DestinationChecker.kt
similarity index 89%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/check/DestinationChecker.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/check/DestinationChecker.kt
index d3dffe060154a..bbe65d621e8ab 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/check/DestinationChecker.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/check/DestinationChecker.kt
@@ -2,9 +2,9 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.check
+package io.airbyte.cdk.load.check
-import io.airbyte.cdk.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationConfiguration
/**
* A check operation that is run before the destination is used.
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationCatalog.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationCatalog.kt
similarity index 97%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationCatalog.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationCatalog.kt
index 9bf46fd67c0de..b771893960aa4 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationCatalog.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationCatalog.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.command
+package io.airbyte.cdk.load.command
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import io.micronaut.context.annotation.Factory
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationConfiguration.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationConfiguration.kt
new file mode 100644
index 0000000000000..404e53cfda11a
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationConfiguration.kt
@@ -0,0 +1,98 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.command
+
+import io.airbyte.cdk.command.Configuration
+import io.airbyte.cdk.command.ConfigurationSpecification
+import io.airbyte.cdk.command.ConfigurationSpecificationSupplier
+import io.micronaut.context.annotation.Factory
+import jakarta.inject.Singleton
+import java.nio.file.Path
+
+/**
+ * To implement a [DestinationConfiguration]:
+ *
+ * - Create a class `{MyDestination}Specification` extending [ConfigurationSpecification]
+ *
+ * - Add any mixin `...Specification`s from this package (the jackson annotations will be inherited)
+ *
+ * - Add any required custom fields to the spec w/ jackson annotations
+ *
+ * - Create a class `{MyDestination}Configuration` extending [DestinationConfiguration]
+ *
+ * - Add the corresponding mixin `...ConfigurationProvider`s for any added spec mixins
+ *
+ * - (Add overrides for any fields provided by the providers)
+ *
+ * - Add custom config to the configuration as needed
+ *
+ * - Implement `DestinationConfigurationFactory` as a @[Singleton], using the `to...Configuration`
+ * methods from the specs to map to the provided configuration fields
+ *
+ * - (Set your custom fields as needed.)
+ *
+ * - Add a @[Factory] injected with [DestinationConfiguration], returning a @[Singleton] downcast to
+ * your implementation; ie,
+ *
+ * ```
+ * @Factory
+ * class MyDestinationConfigurationProvider(
+ * private val config: DestinationConfiguration
+ * ){
+ * @Singleton
+ * fun destinationConfig(): MyDestinationConfiguration =
+ * config as MyDestinationConfiguration
+ * }
+ * ```
+ *
+ * Now your configuration will be automatically parsed and available for injection. ie,
+ *
+ * ```
+ * @Singleton
+ * class MyDestinationWriter(
+ * private val config: MyDestinationConfiguration // <- automatically injected by micronaut
+ * ): DestinationWriter {
+ * // ...
+ * ```
+ */
+abstract class DestinationConfiguration : Configuration {
+ open val recordBatchSizeBytes: Long = 200L * 1024L * 1024L
+ open val tmpFileDirectory: Path = Path.of("airbyte-cdk-load")
+ open val firstStageTmpFilePrefix: String = "staged-raw-records"
+ open val firstStageTmpFileSuffix: String = ".jsonl"
+
+ /** Memory queue settings */
+ open val maxMessageQueueMemoryUsageRatio: Double = 0.2 // 0 => No limit, 1.0 => 100% of JVM heap
+ open val estimatedRecordMemoryOverheadRatio: Double =
+ 0.1 // 0 => No overhead, 1.0 => 100% overhead
+
+ /**
+ * If we have not flushed state checkpoints in this amount of time, make a best-effort attempt
+ * to force a flush.
+ */
+ open val maxCheckpointFlushTimeMs: Long = 15 * 60 * 1000L // 15 minutes
+
+ /** The max number of threads to use for implementor tasks (e.g. open, processBatch). */
+ open val maxNumImplementorTaskThreads = 64
+
+ /**
+ * The amount of time given to implementor tasks (e.g. open, processBatch) to complete their
+ * current work after a failure.
+ */
+ open val gracefulCancellationTimeoutMs: Long = 60 * 1000L // 1 minutes
+
+ /**
+ * Micronaut factory which glues [ConfigurationSpecificationSupplier] and
+ * [DestinationConfigurationFactory] together to produce a [DestinationConfiguration] singleton.
+ */
+ @Factory
+ private class MicronautFactory {
+ @Singleton
+ fun destinationConfig(
+ specificationSupplier: ConfigurationSpecificationSupplier,
+ factory: DestinationConfigurationFactory,
+ ): DestinationConfiguration = factory.make(specificationSupplier.get())
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationConfigurationFactory.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationConfigurationFactory.kt
similarity index 87%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationConfigurationFactory.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationConfigurationFactory.kt
index 937635acfdbc9..3dbf9e5999e3d 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationConfigurationFactory.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationConfigurationFactory.kt
@@ -2,9 +2,10 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.command
+package io.airbyte.cdk.load.command
import io.airbyte.cdk.ConfigErrorException
+import io.airbyte.cdk.command.ConfigurationSpecification
interface DestinationConfigurationFactory<
I : ConfigurationSpecification, O : DestinationConfiguration> {
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationStream.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt
similarity index 95%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationStream.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt
index cf48050cc4ff2..83fedeb2d95c8 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/command/DestinationStream.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt
@@ -2,11 +2,11 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.command
+package io.airbyte.cdk.load.command
-import io.airbyte.cdk.data.AirbyteType
-import io.airbyte.cdk.data.AirbyteTypeToJsonSchema
-import io.airbyte.cdk.data.JsonSchemaToAirbyteType
+import io.airbyte.cdk.load.data.AirbyteType
+import io.airbyte.cdk.load.data.AirbyteTypeToJsonSchema
+import io.airbyte.cdk.load.data.JsonSchemaToAirbyteType
import io.airbyte.protocol.models.v0.AirbyteStream
import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream
import io.airbyte.protocol.models.v0.DestinationSyncMode
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt
similarity index 96%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteType.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt
index 8824d1699f030..7d0ff0ec224c5 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteType.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
sealed interface AirbyteType
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteTypeToJsonSchema.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteTypeToJsonSchema.kt
similarity index 98%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteTypeToJsonSchema.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteTypeToJsonSchema.kt
index 7fa89feba4474..93f2c5b2aa06e 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteTypeToJsonSchema.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteTypeToJsonSchema.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.node.JsonNodeFactory
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteValue.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteValue.kt
similarity index 99%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteValue.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteValue.kt
index 4f6040bc599ee..fe8215ddb09f4 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteValue.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteValue.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import java.math.BigDecimal
import java.time.LocalDate
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteValueToJson.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteValueToJson.kt
similarity index 97%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteValueToJson.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteValueToJson.kt
index 2d79157c54a4a..d4e688dbda605 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/AirbyteValueToJson.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteValueToJson.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.node.JsonNodeFactory
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/JsonSchemaToAirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/JsonSchemaToAirbyteType.kt
similarity index 99%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/JsonSchemaToAirbyteType.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/JsonSchemaToAirbyteType.kt
index 14ae1967e1df5..b6c67a6ee8a74 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/JsonSchemaToAirbyteType.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/JsonSchemaToAirbyteType.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.node.JsonNodeFactory
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/JsonToAirbyteValue.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/JsonToAirbyteValue.kt
similarity index 99%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/JsonToAirbyteValue.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/JsonToAirbyteValue.kt
index bcce2ea1e1cc1..a0872b4e493c7 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/data/JsonToAirbyteValue.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/JsonToAirbyteValue.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.JsonNode
import java.math.BigDecimal
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/file/LocalFile.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/LocalFile.kt
similarity index 97%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/file/LocalFile.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/LocalFile.kt
index 67ced63db7013..0701e350b499b 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/file/LocalFile.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/LocalFile.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.file
+package io.airbyte.cdk.load.file
import io.micronaut.context.annotation.DefaultImplementation
import java.io.Closeable
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/file/TempFileProvider.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/TempFileProvider.kt
similarity index 94%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/file/TempFileProvider.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/TempFileProvider.kt
index 0455ec323db3d..cdb0f353cf344 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/file/TempFileProvider.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/TempFileProvider.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.file
+package io.airbyte.cdk.load.file
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/TimeProvider.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/TimeProvider.kt
new file mode 100644
index 0000000000000..054bd6074b295
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/file/TimeProvider.kt
@@ -0,0 +1,25 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.file
+
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+
+interface TimeProvider {
+ fun currentTimeMillis(): Long
+ suspend fun delay(ms: Long)
+}
+
+@Singleton
+@Secondary
+class DefaultTimeProvider : TimeProvider {
+ override fun currentTimeMillis(): Long {
+ return System.currentTimeMillis()
+ }
+
+ override suspend fun delay(ms: Long) {
+ kotlinx.coroutines.delay(ms)
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/Batch.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/Batch.kt
similarity index 93%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/Batch.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/Batch.kt
index b3f87e19054e6..41e4f38ce7370 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/Batch.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/Batch.kt
@@ -2,12 +2,12 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.message
+package io.airbyte.cdk.load.message
import com.google.common.collect.Range
import com.google.common.collect.RangeSet
import com.google.common.collect.TreeRangeSet
-import io.airbyte.cdk.file.LocalFile
+import io.airbyte.cdk.load.file.LocalFile
/**
* Represents an accumulated batch of records in some stage of processing.
@@ -53,6 +53,13 @@ interface Batch {
COMPLETE
}
+ fun isPersisted(): Boolean =
+ when (state) {
+ State.PERSISTED,
+ State.COMPLETE -> true
+ else -> false
+ }
+
val state: State
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessage.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt
similarity index 94%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessage.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt
index 73d0ad3f907de..2fd484bdb0ab4 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessage.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt
@@ -2,17 +2,17 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.message
+package io.airbyte.cdk.load.message
import com.fasterxml.jackson.databind.JsonNode
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.data.AirbyteValue
-import io.airbyte.cdk.data.AirbyteValueToJson
-import io.airbyte.cdk.data.JsonToAirbyteValue
-import io.airbyte.cdk.data.ObjectTypeWithoutSchema
-import io.airbyte.cdk.message.CheckpointMessage.Checkpoint
-import io.airbyte.cdk.message.CheckpointMessage.Stats
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.data.AirbyteValue
+import io.airbyte.cdk.load.data.AirbyteValueToJson
+import io.airbyte.cdk.load.data.JsonToAirbyteValue
+import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema
+import io.airbyte.cdk.load.message.CheckpointMessage.Checkpoint
+import io.airbyte.cdk.load.message.CheckpointMessage.Stats
import io.airbyte.protocol.models.Jsons
import io.airbyte.protocol.models.v0.AirbyteGlobalState
import io.airbyte.protocol.models.v0.AirbyteMessage
@@ -193,6 +193,9 @@ data class StreamCheckpoint(
AirbyteStateStats()
.withRecordCount(destinationStats.recordCount.toDouble())
}
+ additionalProperties.forEach { (key, value) ->
+ it.withAdditionalProperty(key, value)
+ }
}
return AirbyteMessage().withType(AirbyteMessage.Type.STATE).withState(stateMessage)
}
@@ -232,6 +235,9 @@ data class GlobalCheckpoint(
AirbyteStateStats()
.withRecordCount(destinationStats.recordCount.toDouble())
}
+ it.additionalProperties.forEach { (key, value) ->
+ it.withAdditionalProperty(key, value)
+ }
}
return AirbyteMessage().withType(AirbyteMessage.Type.STATE).withState(stateMessage)
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessageDeserializer.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessageDeserializer.kt
similarity index 96%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessageDeserializer.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessageDeserializer.kt
index d052dd388cc33..81a7f5a4fd811 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessageDeserializer.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessageDeserializer.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.message
+package io.airbyte.cdk.load.message
import io.airbyte.cdk.util.Jsons
import io.airbyte.protocol.models.v0.AirbyteMessage
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessageQueues.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessageQueues.kt
new file mode 100644
index 0000000000000..aa9099044b246
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessageQueues.kt
@@ -0,0 +1,85 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.message
+
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.state.MemoryManager
+import io.airbyte.cdk.load.state.Reserved
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.util.concurrent.ConcurrentHashMap
+
+interface Sized {
+ val sizeBytes: Long
+}
+
+/**
+ * Wrapper for record messages published to the message queue, containing metadata like index and
+ * size.
+ *
+ * In a future where we deserialize only the info necessary for routing, this could include a dumb
+ * container for the serialized, and deserialization could be deferred until the spooled records
+ * were recovered from disk.
+ */
+sealed class DestinationRecordWrapped : Sized
+
+data class StreamRecordWrapped(
+ val index: Long,
+ override val sizeBytes: Long,
+ val record: DestinationRecord
+) : DestinationRecordWrapped()
+
+data class StreamCompleteWrapped(
+ val index: Long,
+) : DestinationRecordWrapped() {
+ override val sizeBytes: Long = 0L
+}
+
+class DestinationRecordQueue : ChannelMessageQueue>()
+
+/**
+ * A supplier of message queues to which ([MemoryManager.reserveBlocking]'d) @
+ * [DestinationRecordWrapped] messages can be published on a @ [DestinationStream] key. The queues
+ * themselves do not manage memory.
+ */
+@Singleton
+@Secondary
+class DestinationRecordQueueSupplier(catalog: DestinationCatalog) :
+ MessageQueueSupplier> {
+ private val queues = ConcurrentHashMap()
+
+ init {
+ catalog.streams.forEach { queues[it.descriptor] = DestinationRecordQueue() }
+ }
+
+ override fun get(key: DestinationStream.Descriptor): DestinationRecordQueue {
+ return queues[key]
+ ?: throw IllegalArgumentException("Reading from non-existent record stream: $key")
+ }
+}
+
+sealed interface CheckpointMessageWrapped : Sized
+
+data class StreamCheckpointWrapped(
+ override val sizeBytes: Long,
+ val stream: DestinationStream.Descriptor,
+ val index: Long,
+ val checkpoint: CheckpointMessage
+) : CheckpointMessageWrapped
+
+data class GlobalCheckpointWrapped(
+ override val sizeBytes: Long,
+ val streamIndexes: List>,
+ val checkpoint: CheckpointMessage
+) : CheckpointMessageWrapped
+
+/**
+ * A single-channel queue for checkpoint messages. This is so updating the checkpoint manager never
+ * blocks reading from stdin.
+ */
+@Singleton
+@Secondary
+class CheckpointMessageQueue : ChannelMessageQueue>()
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/MessageQueue.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/MessageQueue.kt
new file mode 100644
index 0000000000000..2fe9488d03a4b
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/MessageQueue.kt
@@ -0,0 +1,36 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.message
+
+import io.airbyte.cdk.load.util.CloseableCoroutine
+import kotlinx.coroutines.channels.Channel
+import kotlinx.coroutines.flow.Flow
+import kotlinx.coroutines.flow.receiveAsFlow
+
+interface QueueReader {
+ suspend fun consume(): Flow
+ suspend fun poll(): T?
+}
+
+interface QueueWriter : CloseableCoroutine {
+ suspend fun publish(message: T)
+}
+
+interface MessageQueue : QueueReader, QueueWriter
+
+abstract class ChannelMessageQueue : MessageQueue {
+ val channel = Channel(Channel.UNLIMITED)
+
+ override suspend fun publish(message: T) = channel.send(message)
+ override suspend fun consume(): Flow = channel.receiveAsFlow()
+ override suspend fun poll(): T? = channel.tryReceive().getOrNull()
+ override suspend fun close() {
+ channel.close()
+ }
+}
+
+interface MessageQueueSupplier {
+ fun get(key: K): MessageQueue
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/spec/DestinationSpecificationInternal.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/spec/DestinationSpecificationInternal.kt
similarity index 86%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/spec/DestinationSpecificationInternal.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/spec/DestinationSpecificationInternal.kt
index 944a05a131f0c..780b4b51681c4 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/spec/DestinationSpecificationInternal.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/spec/DestinationSpecificationInternal.kt
@@ -2,8 +2,10 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.spec
+package io.airbyte.cdk.load.spec
+import io.airbyte.cdk.spec.IdentitySpecificationExtender
+import io.airbyte.cdk.spec.SpecificationExtender
import io.airbyte.protocol.models.v0.ConnectorSpecification
import io.airbyte.protocol.models.v0.DestinationSyncMode
import io.micronaut.context.annotation.Replaces
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/CheckpointManager.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/CheckpointManager.kt
new file mode 100644
index 0000000000000..3f155cd5de726
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/CheckpointManager.kt
@@ -0,0 +1,273 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.state
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.file.TimeProvider
+import io.airbyte.cdk.load.message.CheckpointMessage
+import io.airbyte.cdk.load.util.use
+import io.airbyte.protocol.models.v0.AirbyteMessage
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.util.concurrent.ConcurrentHashMap
+import java.util.concurrent.ConcurrentLinkedQueue
+import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent.atomic.AtomicReference
+import java.util.function.Consumer
+import kotlinx.coroutines.delay
+import kotlinx.coroutines.sync.Mutex
+import kotlinx.coroutines.sync.withLock
+
+/**
+ * Interface for checkpoint management. Should accept stream and global checkpoints, as well as
+ * requests to flush all data-sufficient checkpoints.
+ */
+interface CheckpointManager {
+ suspend fun addStreamCheckpoint(key: K, index: Long, checkpointMessage: T)
+ suspend fun addGlobalCheckpoint(keyIndexes: List>, checkpointMessage: T)
+ suspend fun flushReadyCheckpointMessages()
+ suspend fun getLastSuccessfulFlushTimeMs(): Long
+ suspend fun getNextCheckpointIndexes(): Map
+ suspend fun awaitAllCheckpointsFlushed()
+}
+
+/**
+ * Message-type agnostic streams checkpoint manager.
+ *
+ * Accepts global and stream checkpoints, and enforces that stream and global checkpoints are not
+ * mixed. Determines ready checkpoints by querying the StreamsManager for the checkpoint of the
+ * record index range associated with each checkpoint message.
+ *
+ * TODO: Force flush on a configured schedule
+ *
+ * TODO: Ensure that checkpoint is flushed at the end, and require that all checkpoints be flushed
+ * before the destination can succeed.
+ */
+abstract class StreamsCheckpointManager : CheckpointManager {
+
+ private val log = KotlinLogging.logger {}
+ private val flushLock = Mutex()
+ protected val lastFlushTimeMs = AtomicLong(0L)
+
+ abstract val catalog: DestinationCatalog
+ abstract val syncManager: SyncManager
+ abstract val outputConsumer: suspend (T) -> Unit
+ abstract val timeProvider: TimeProvider
+
+ data class GlobalCheckpoint(
+ val streamIndexes: List>,
+ val checkpointMessage: T
+ )
+
+ private val checkpointsAreGlobal: AtomicReference = AtomicReference(null)
+ private val streamCheckpoints:
+ ConcurrentHashMap>> =
+ ConcurrentHashMap()
+ private val globalCheckpoints: ConcurrentLinkedQueue> =
+ ConcurrentLinkedQueue()
+ private val lastIndexEmitted = ConcurrentHashMap()
+
+ override suspend fun addStreamCheckpoint(
+ key: DestinationStream.Descriptor,
+ index: Long,
+ checkpointMessage: T
+ ) {
+ flushLock.withLock {
+ if (checkpointsAreGlobal.updateAndGet { it == true } != false) {
+ throw IllegalStateException(
+ "Global checkpoints cannot be mixed with non-global checkpoints"
+ )
+ }
+
+ val indexedMessages: ConcurrentLinkedQueue> =
+ streamCheckpoints.getOrPut(key) { ConcurrentLinkedQueue() }
+ if (indexedMessages.isNotEmpty()) {
+ // Make sure the messages are coming in order
+ val (latestIndex, _) = indexedMessages.last()!!
+ if (latestIndex > index) {
+ throw IllegalStateException(
+ "Checkpoint message received out of order ($latestIndex before $index)"
+ )
+ }
+ }
+ indexedMessages.add(index to checkpointMessage)
+
+ log.info { "Added checkpoint for stream: $key at index: $index" }
+ }
+ }
+
+ // TODO: Is it an error if we don't get all the streams every time?
+ override suspend fun addGlobalCheckpoint(
+ keyIndexes: List>,
+ checkpointMessage: T
+ ) {
+ flushLock.withLock {
+ if (checkpointsAreGlobal.updateAndGet { it != false } != true) {
+ throw IllegalStateException(
+ "Global checkpoint cannot be mixed with non-global checkpoints"
+ )
+ }
+
+ val head = globalCheckpoints.peek()
+ if (head != null) {
+ val keyIndexesByStream = keyIndexes.associate { it.first to it.second }
+ head.streamIndexes.forEach {
+ if (keyIndexesByStream[it.first]!! < it.second) {
+ throw IllegalStateException(
+ "Global checkpoint message received out of order"
+ )
+ }
+ }
+ }
+
+ globalCheckpoints.add(GlobalCheckpoint(keyIndexes, checkpointMessage))
+ log.info { "Added global checkpoint with stream indexes: $keyIndexes" }
+ }
+ }
+
+ override suspend fun flushReadyCheckpointMessages() {
+ flushLock.withLock {
+ /*
+ Iterate over the checkpoints in order, evicting each that passes
+ the persistence check. If a checkpoint is not persisted, then
+ we can break the loop since the checkpoints are ordered. For global
+ checkpoints, all streams must be persisted up to the checkpoint.
+ */
+ when (checkpointsAreGlobal.get()) {
+ null -> log.info { "No checkpoints to flush" }
+ true -> flushGlobalCheckpoints()
+ false -> flushStreamCheckpoints()
+ }
+ }
+ }
+
+ private suspend fun flushGlobalCheckpoints() {
+ while (!globalCheckpoints.isEmpty()) {
+ val head = globalCheckpoints.peek()
+ val allStreamsPersisted =
+ head.streamIndexes.all { (stream, index) ->
+ syncManager.getStreamManager(stream).areRecordsPersistedUntil(index)
+ }
+ if (allStreamsPersisted) {
+ log.info { "Flushing global checkpoint with stream indexes: ${head.streamIndexes}" }
+ validateAndSendMessage(head.checkpointMessage, head.streamIndexes)
+ globalCheckpoints.poll() // don't remove until after we've successfully sent
+ } else {
+ break
+ }
+ }
+ }
+
+ private suspend fun flushStreamCheckpoints() {
+ for (stream in catalog.streams) {
+ val manager = syncManager.getStreamManager(stream.descriptor)
+ val streamCheckpoints = streamCheckpoints[stream.descriptor] ?: return
+ while (true) {
+ val (nextIndex, nextMessage) = streamCheckpoints.peek() ?: break
+ if (manager.areRecordsPersistedUntil(nextIndex)) {
+ log.info {
+ "Flushing checkpoint for stream: ${stream.descriptor} at index: $nextIndex"
+ }
+ validateAndSendMessage(nextMessage, listOf(stream.descriptor to nextIndex))
+ streamCheckpoints.poll() // don't remove until after we've successfully sent
+ } else {
+ break
+ }
+ }
+ }
+ }
+
+ private suspend fun validateAndSendMessage(
+ checkpointMessage: T,
+ streamIndexes: List>
+ ) {
+ streamIndexes.forEach { (stream, index) ->
+ val lastIndex = lastIndexEmitted[stream]
+ if (lastIndex != null && index < lastIndex) {
+ throw IllegalStateException(
+ "Checkpoint message for $stream emitted out of order (emitting $index after $lastIndex)"
+ )
+ }
+ lastIndexEmitted[stream] = index
+ }
+
+ lastFlushTimeMs.set(timeProvider.currentTimeMillis())
+ outputConsumer.invoke(checkpointMessage)
+ }
+
+ override suspend fun getLastSuccessfulFlushTimeMs(): Long =
+ // Return inside the lock to ensure the value reflects flushes in progress
+ flushLock.withLock { lastFlushTimeMs.get() }
+
+ override suspend fun getNextCheckpointIndexes(): Map {
+ flushLock.withLock {
+ return when (checkpointsAreGlobal.get()) {
+ null -> {
+ emptyMap()
+ }
+ true -> {
+ val head = globalCheckpoints.peek()
+ head?.streamIndexes?.associate { it } ?: emptyMap()
+ }
+ false -> {
+ streamCheckpoints
+ .mapValues { it.value.firstOrNull()?.first }
+ .filterValues { it != null }
+ .mapValues { it.value!! }
+ }
+ }
+ }
+ }
+
+ override suspend fun awaitAllCheckpointsFlushed() {
+ while (true) {
+ val allCheckpointsFlushed =
+ flushLock.withLock {
+ globalCheckpoints.isEmpty() && streamCheckpoints.all { it.value.isEmpty() }
+ }
+ if (allCheckpointsFlushed) {
+ log.info { "All checkpoints flushed" }
+ break
+ }
+ log.info { "Waiting for all checkpoints to flush" }
+ // Not usually a fan of busywaiting, but it's extremely unlikely we
+ // get here without more than a handful of stragglers
+ delay(1000L)
+ flushReadyCheckpointMessages()
+ }
+ }
+}
+
+@Singleton
+@Secondary
+class DefaultCheckpointManager(
+ override val catalog: DestinationCatalog,
+ override val syncManager: SyncManager,
+ override val outputConsumer: suspend (Reserved) -> Unit,
+ override val timeProvider: TimeProvider
+) : StreamsCheckpointManager>() {
+ init {
+ lastFlushTimeMs.set(timeProvider.currentTimeMillis())
+ }
+}
+
+@SuppressFBWarnings(
+ "NP_NONNULL_PARAM_VIOLATION",
+ justification = "message is guaranteed to be non-null by Kotlin's type system"
+)
+@Singleton
+@Secondary
+class FreeingCheckpointConsumer(private val consumer: Consumer) :
+ suspend (Reserved) -> Unit {
+ override suspend fun invoke(message: Reserved) {
+ message.use {
+ val outMessage = it.value.asProtocolMessage()
+ consumer.accept(outMessage)
+ }
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/FlushStrategy.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/FlushStrategy.kt
new file mode 100644
index 0000000000000..a6177ae8d88f6
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/FlushStrategy.kt
@@ -0,0 +1,63 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.state
+
+import com.google.common.collect.Range
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.QueueReader
+import io.airbyte.cdk.load.task.internal.ForceFlushEvent
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.util.concurrent.ConcurrentHashMap
+
+interface FlushStrategy {
+ suspend fun shouldFlush(
+ stream: DestinationStream,
+ rangeRead: Range,
+ bytesProcessed: Long
+ ): Boolean
+}
+
+/**
+ * Flush whenever
+ * - bytes consumed >= the configured batch size
+ * - the current range of indexes being consumed encloses a force flush index
+ */
+@SuppressFBWarnings(
+ "NP_NONNULL_PARAM_VIOLATION",
+ justification = "message is guaranteed to be non-null by Kotlin's type system"
+)
+@Singleton
+@Secondary
+class DefaultFlushStrategy(
+ private val config: DestinationConfiguration,
+ private val eventQueue: QueueReader
+) : FlushStrategy {
+ private val forceFlushIndexes = ConcurrentHashMap()
+
+ override suspend fun shouldFlush(
+ stream: DestinationStream,
+ rangeRead: Range,
+ bytesProcessed: Long
+ ): Boolean {
+ if (bytesProcessed >= config.recordBatchSizeBytes) {
+ return true
+ }
+
+ // Listen to the event stream for a new force flush index
+ val nextFlushIndex = eventQueue.poll()?.indexes?.get(stream.descriptor)
+
+ // Always update the index if the new one is not null
+ return when (
+ val testIndex =
+ forceFlushIndexes.compute(stream.descriptor) { _, v -> nextFlushIndex ?: v }
+ ) {
+ null -> false
+ else -> rangeRead.contains(testIndex)
+ }
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/MemoryManager.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/MemoryManager.kt
similarity index 52%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/MemoryManager.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/MemoryManager.kt
index f5511f58fcc1e..99ed41d73cefd 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/MemoryManager.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/MemoryManager.kt
@@ -2,15 +2,44 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.state
+package io.airbyte.cdk.load.state
+import io.airbyte.cdk.load.util.CloseableCoroutine
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
+import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicLong
import kotlinx.coroutines.channels.Channel
import kotlinx.coroutines.sync.Mutex
import kotlinx.coroutines.sync.withLock
+/**
+ * Releasable reservation of memory. For large blocks (ie, from [MemoryManager.reserveRatio],
+ * provides a submanager that can be used to manage allocating the reservation).
+ */
+class Reserved(
+ private val memoryManager: MemoryManager,
+ val bytesReserved: Long,
+ val value: T,
+) : CloseableCoroutine {
+ private var released = AtomicBoolean(false)
+
+ suspend fun release() {
+ if (!released.compareAndSet(false, true)) {
+ return
+ }
+ memoryManager.release(bytesReserved)
+ }
+
+ fun getReservationManager(): MemoryManager = MemoryManager(bytesReserved)
+
+ fun replace(value: U): Reserved = Reserved(memoryManager, bytesReserved, value)
+
+ override suspend fun close() {
+ release()
+ }
+}
+
/**
* Manages memory usage for the destination.
*
@@ -19,8 +48,17 @@ import kotlinx.coroutines.sync.withLock
* TODO: Some degree of logging/monitoring around how accurate we're actually being?
*/
@Singleton
-class MemoryManager(private val availableMemoryProvider: AvailableMemoryProvider) {
- private val totalMemoryBytes: Long = availableMemoryProvider.availableMemoryBytes
+class MemoryManager(availableMemoryProvider: AvailableMemoryProvider) {
+ // This is slightly awkward, but Micronaut only injects the primary constructor
+ constructor(
+ availableMemory: Long
+ ) : this(
+ object : AvailableMemoryProvider {
+ override val availableMemoryBytes: Long = availableMemory
+ }
+ )
+
+ private val totalMemoryBytes = availableMemoryProvider.availableMemoryBytes
private var usedMemoryBytes = AtomicLong(0L)
private val mutex = Mutex()
private val syncChannel = Channel(Channel.UNLIMITED)
@@ -29,7 +67,7 @@ class MemoryManager(private val availableMemoryProvider: AvailableMemoryProvider
get() = totalMemoryBytes - usedMemoryBytes.get()
/* Attempt to reserve memory. If enough memory is not available, waits until it is, then reserves. */
- suspend fun reserveBlocking(memoryBytes: Long) {
+ suspend fun reserveBlocking(memoryBytes: Long, reservedFor: T): Reserved {
if (memoryBytes > totalMemoryBytes) {
throw IllegalArgumentException(
"Requested ${memoryBytes}b memory exceeds ${totalMemoryBytes}b total"
@@ -41,13 +79,14 @@ class MemoryManager(private val availableMemoryProvider: AvailableMemoryProvider
syncChannel.receive()
}
usedMemoryBytes.addAndGet(memoryBytes)
+
+ return Reserved(this, memoryBytes, reservedFor)
}
}
- suspend fun reserveRatio(ratio: Double): Long {
+ suspend fun reserveRatio(ratio: Double, reservedFor: T): Reserved {
val estimatedSize = (totalMemoryBytes.toDouble() * ratio).toLong()
- reserveBlocking(estimatedSize)
- return estimatedSize
+ return reserveBlocking(estimatedSize, reservedFor)
}
suspend fun release(memoryBytes: Long) {
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/SyncManager.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/StreamManager.kt
similarity index 55%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/SyncManager.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/StreamManager.kt
index 10d46bd667b23..80769f1e8cb18 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/SyncManager.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/StreamManager.kt
@@ -2,75 +2,48 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.state
+package io.airbyte.cdk.load.state
import com.google.common.collect.Range
import com.google.common.collect.RangeSet
import com.google.common.collect.TreeRangeSet
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.message.Batch
-import io.airbyte.cdk.message.BatchEnvelope
-import io.airbyte.cdk.write.StreamLoader
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.Batch
+import io.airbyte.cdk.load.message.BatchEnvelope
import io.github.oshai.kotlinlogging.KotlinLogging
-import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
import java.util.concurrent.ConcurrentHashMap
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicLong
import kotlinx.coroutines.CompletableDeferred
-import kotlinx.coroutines.channels.Channel
-/** Manages the state of all streams in the destination. */
-interface SyncManager {
- /** Get the manager for the given stream. Throws an exception if the stream is not found. */
- fun getStreamManager(stream: DestinationStream.Descriptor): StreamManager
+sealed interface StreamResult
- fun registerStartedStreamLoader(streamLoader: StreamLoader)
- suspend fun getOrAwaitStreamLoader(stream: DestinationStream.Descriptor): StreamLoader
+sealed interface StreamIncompleteResult : StreamResult
- /** Suspend until all streams are closed. */
- suspend fun awaitAllStreamsClosed()
-}
-
-class DefaultSyncManager(
- private val streamManagers: ConcurrentHashMap
-) : SyncManager {
- private val streamLoaders =
- ConcurrentHashMap>()
-
- override fun getStreamManager(stream: DestinationStream.Descriptor): StreamManager {
- return streamManagers[stream] ?: throw IllegalArgumentException("Stream not found: $stream")
- }
+data class StreamFailed(val streamException: Exception) : StreamIncompleteResult
- override fun registerStartedStreamLoader(streamLoader: StreamLoader) {
- streamLoaders
- .getOrPut(streamLoader.stream.descriptor) { CompletableDeferred() }
- .complete(streamLoader)
- }
+data class StreamKilled(val syncException: Exception) : StreamIncompleteResult
- override suspend fun getOrAwaitStreamLoader(
- stream: DestinationStream.Descriptor
- ): StreamLoader {
- return streamLoaders.getOrPut(stream) { CompletableDeferred() }.await()
- }
-
- override suspend fun awaitAllStreamsClosed() {
- streamManagers.forEach { (_, manager) -> manager.awaitStreamClosed() }
- }
-}
+data object StreamSucceeded : StreamResult
/** Manages the state of a single stream. */
interface StreamManager {
- /** Count incoming record and return the record's *index*. */
+ /**
+ * Count incoming record and return the record's *index*. If [markEndOfStream] has been called,
+ * this should throw an exception.
+ */
fun countRecordIn(): Long
+ fun recordCount(): Long
/**
- * Count the end-of-stream. Expect this exactly once. Expect no further `countRecordIn`, and
- * expect that `markClosed` will always occur after this.
+ * Mark the end-of-stream and return the record count. Expect this exactly once. Expect no
+ * further `countRecordIn`, and expect that [markSucceeded] or [markFailed] or [markKilled] will
+ * alway occur after this.
*/
- fun countEndOfStream(): Long
+ fun markEndOfStream(): Long
+ fun endOfStreamRead(): Boolean
/**
* Mark a checkpoint in the stream and return the current index and the number of records since
@@ -100,25 +73,35 @@ interface StreamManager {
fun areRecordsPersistedUntil(index: Long): Boolean
/** Mark the stream as closed. This should only be called after all records have been read. */
- fun markClosed()
+ fun markSucceeded()
+
+ /**
+ * Mark that the stream was killed due to failure elsewhere. Returns false if task was already
+ * complete.
+ */
+ fun markKilled(causedBy: Exception): Boolean
+
+ /** Mark that the stream itself failed. Return false if task was already complete */
+ fun markFailed(causedBy: Exception): Boolean
- /** True if the stream has been marked as closed. */
- fun streamIsClosed(): Boolean
+ /** Suspend until the stream completes, returning the result. */
+ suspend fun awaitStreamResult(): StreamResult
- /** Suspend until the stream is closed. */
- suspend fun awaitStreamClosed()
+ /** True if the stream has not yet been marked successful, failed, or killed. */
+ fun isActive(): Boolean
}
class DefaultStreamManager(
val stream: DestinationStream,
) : StreamManager {
+ private val streamResult = CompletableDeferred()
+
private val log = KotlinLogging.logger {}
private val recordCount = AtomicLong(0)
private val lastCheckpoint = AtomicLong(0L)
- private val readIsClosed = AtomicBoolean(false)
- private val streamIsClosed = AtomicBoolean(false)
- private val closedLock = Channel()
+
+ private val markedEndOfStream = AtomicBoolean(false)
private val rangesState: ConcurrentHashMap> = ConcurrentHashMap()
@@ -127,21 +110,29 @@ class DefaultStreamManager(
}
override fun countRecordIn(): Long {
- if (readIsClosed.get()) {
+ if (markedEndOfStream.get()) {
throw IllegalStateException("Stream is closed for reading")
}
return recordCount.getAndIncrement()
}
- override fun countEndOfStream(): Long {
- if (readIsClosed.getAndSet(true)) {
+ override fun recordCount(): Long {
+ return recordCount.get()
+ }
+
+ override fun markEndOfStream(): Long {
+ if (markedEndOfStream.getAndSet(true)) {
throw IllegalStateException("Stream is closed for reading")
}
return recordCount.get()
}
+ override fun endOfStreamRead(): Boolean {
+ return markedEndOfStream.get()
+ }
+
override fun markCheckpoint(): Pair {
val index = recordCount.get()
val lastCheckpoint = lastCheckpoint.getAndSet(index)
@@ -162,18 +153,19 @@ class DefaultStreamManager(
batch.ranges.asRanges().map { it.span(Range.singleton(it.upperEndpoint() + 1)) }
stateRanges.addAll(expanded)
- log.info { "Updated ranges for $stream[${batch.batch.state}]: $stateRanges" }
+ log.info { "Updated ranges for ${stream.descriptor}[${batch.batch.state}]: $stateRanges" }
}
/** True if all records in `[0, index)` have reached the given state. */
private fun isProcessingCompleteForState(index: Long, state: Batch.State): Boolean {
val completeRanges = rangesState[state]!!
+
return completeRanges.encloses(Range.closedOpen(0L, index))
}
override fun isBatchProcessingComplete(): Boolean {
/* If the stream hasn't been fully read, it can't be done. */
- if (!readIsClosed.get()) {
+ if (!markedEndOfStream.get()) {
return false
}
@@ -186,36 +178,38 @@ class DefaultStreamManager(
isProcessingCompleteForState(index, Batch.State.COMPLETE) // complete => persisted
}
- override fun markClosed() {
- if (!readIsClosed.get()) {
- throw IllegalStateException("Stream must be fully read before it can be closed")
+ override fun markSucceeded() {
+ if (!markedEndOfStream.get()) {
+ throw IllegalStateException("Stream is not closed for reading")
}
+ streamResult.complete(StreamSucceeded)
+ }
- if (streamIsClosed.compareAndSet(false, true)) {
- closedLock.trySend(Unit)
- }
+ override fun markKilled(causedBy: Exception): Boolean {
+ return streamResult.complete(StreamKilled(causedBy))
}
- override fun streamIsClosed(): Boolean {
- return streamIsClosed.get()
+ override fun markFailed(causedBy: Exception): Boolean {
+ return streamResult.complete(StreamFailed(causedBy))
}
- override suspend fun awaitStreamClosed() {
- if (!streamIsClosed.get()) {
- closedLock.receive()
- }
+ override suspend fun awaitStreamResult(): StreamResult {
+ return streamResult.await()
}
+
+ override fun isActive(): Boolean {
+ return streamResult.isActive
+ }
+}
+
+interface StreamManagerFactory {
+ fun create(stream: DestinationStream): StreamManager
}
-@Factory
-class SyncManagerFactory(
- private val catalog: DestinationCatalog,
-) {
- @Singleton
- @Secondary
- fun make(): SyncManager {
- val hashMap = ConcurrentHashMap()
- catalog.streams.forEach { hashMap[it.descriptor] = DefaultStreamManager(it) }
- return DefaultSyncManager(hashMap)
+@Singleton
+@Secondary
+class DefaultStreamManagerFactory : StreamManagerFactory {
+ override fun create(stream: DestinationStream): StreamManager {
+ return DefaultStreamManager(stream)
}
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/SyncManager.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/SyncManager.kt
new file mode 100644
index 0000000000000..066de1796b519
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/state/SyncManager.kt
@@ -0,0 +1,139 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.state
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.write.StreamLoader
+import io.micronaut.context.annotation.Factory
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.util.concurrent.ConcurrentHashMap
+import kotlinx.coroutines.CompletableDeferred
+
+sealed interface SyncResult
+
+data object SyncSuccess : SyncResult
+
+data class SyncFailure(
+ val syncFailure: Exception,
+ val streamResults: Map
+) : SyncResult
+
+/** Manages the state of all streams in the destination. */
+interface SyncManager {
+ /** Get the manager for the given stream. Throws an exception if the stream is not found. */
+ fun getStreamManager(stream: DestinationStream.Descriptor): StreamManager
+
+ fun registerStartedStreamLoader(streamLoader: StreamLoader)
+ suspend fun getOrAwaitStreamLoader(stream: DestinationStream.Descriptor): StreamLoader
+ suspend fun getStreamLoaderOrNull(stream: DestinationStream.Descriptor): StreamLoader?
+
+ /** Suspend until all streams are complete. Returns false if any stream was failed/killed. */
+ suspend fun awaitAllStreamsCompletedSuccessfully(): Boolean
+
+ suspend fun markInputConsumed()
+ suspend fun markCheckpointsProcessed()
+ suspend fun markFailed(causedBy: Exception): SyncFailure
+ suspend fun markSucceeded()
+
+ fun isActive(): Boolean
+
+ suspend fun awaitInputProcessingComplete(): Unit
+ suspend fun awaitSyncResult(): SyncResult
+}
+
+@SuppressFBWarnings(
+ "NP_NONNULL_PARAM_VIOLATION",
+ justification = "exception is guaranteed to be non-null by Kotlin's type system"
+)
+class DefaultSyncManager(
+ private val streamManagers: ConcurrentHashMap
+) : SyncManager {
+ private val syncResult = CompletableDeferred()
+ private val streamLoaders =
+ ConcurrentHashMap>()
+ private val inputConsumed = CompletableDeferred()
+ private val checkpointsProcessed = CompletableDeferred()
+
+ override fun getStreamManager(stream: DestinationStream.Descriptor): StreamManager {
+ return streamManagers[stream] ?: throw IllegalArgumentException("Stream not found: $stream")
+ }
+
+ override fun registerStartedStreamLoader(streamLoader: StreamLoader) {
+ streamLoaders
+ .getOrPut(streamLoader.stream.descriptor) { CompletableDeferred() }
+ .complete(streamLoader)
+ }
+
+ override suspend fun getOrAwaitStreamLoader(
+ stream: DestinationStream.Descriptor
+ ): StreamLoader {
+ return streamLoaders.getOrPut(stream) { CompletableDeferred() }.await()
+ }
+
+ override suspend fun getStreamLoaderOrNull(
+ stream: DestinationStream.Descriptor
+ ): StreamLoader? {
+ val completable = streamLoaders[stream]
+ return completable?.let { if (it.isCompleted) it.await() else null }
+ }
+
+ override suspend fun awaitAllStreamsCompletedSuccessfully(): Boolean {
+ return streamManagers.all { (_, manager) -> manager.awaitStreamResult() is StreamSucceeded }
+ }
+
+ override suspend fun markFailed(causedBy: Exception): SyncFailure {
+ val result =
+ SyncFailure(causedBy, streamManagers.mapValues { it.value.awaitStreamResult() })
+ syncResult.complete(result)
+ return result
+ }
+
+ override suspend fun markSucceeded() {
+ if (streamManagers.values.any { it.isActive() }) {
+ throw IllegalStateException(
+ "Cannot mark sync as succeeded until all streams are complete"
+ )
+ }
+ syncResult.complete(SyncSuccess)
+ }
+
+ override fun isActive(): Boolean {
+ return syncResult.isActive
+ }
+
+ override suspend fun awaitSyncResult(): SyncResult {
+ return syncResult.await()
+ }
+
+ override suspend fun awaitInputProcessingComplete() {
+ inputConsumed.await()
+ checkpointsProcessed.await()
+ }
+
+ override suspend fun markInputConsumed() {
+ inputConsumed.complete(true)
+ }
+
+ override suspend fun markCheckpointsProcessed() {
+ checkpointsProcessed.complete(true)
+ }
+}
+
+@Factory
+class SyncManagerFactory(
+ private val catalog: DestinationCatalog,
+ private val streamManagerFactory: StreamManagerFactory
+) {
+ @Singleton
+ @Secondary
+ fun make(): SyncManager {
+ val hashMap = ConcurrentHashMap()
+ catalog.streams.forEach { hashMap[it.descriptor] = streamManagerFactory.create(it) }
+ return DefaultSyncManager(hashMap)
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskExceptionHandler.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskExceptionHandler.kt
new file mode 100644
index 0000000000000..c4ac938dc4806
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskExceptionHandler.kt
@@ -0,0 +1,172 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.state.StreamSucceeded
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.state.SyncSuccess
+import io.airbyte.cdk.load.task.implementor.FailStreamTaskFactory
+import io.airbyte.cdk.load.task.implementor.FailSyncTaskFactory
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import kotlinx.coroutines.CancellationException
+
+/**
+ * The level at which a task operates:
+ * - SyncTask: global across all streams
+ * - StreamTask: affined to a single stream
+ */
+sealed interface LeveledTask : Task
+
+interface SyncLevel : LeveledTask
+
+interface StreamLevel : LeveledTask {
+ val stream: DestinationStream
+}
+
+interface DestinationTaskExceptionHandler : TaskExceptionHandler {
+ suspend fun handleSyncFailure(e: Exception)
+ suspend fun handleStreamFailure(stream: DestinationStream, e: Exception)
+ suspend fun handleSyncFailed()
+}
+
+interface WrappedTask : Task {
+ val innerTask: T
+}
+
+/**
+ * The exception handler takes over the workflow in the event of an exception. Its contract is
+ * * provide a wrapper that directs exceptions to the correct handler (by task type)
+ * * close the task runner when the cleanup workflow is complete
+ *
+ * Handling works as follows:
+ * * a failure in a sync-level task (setup/teardown) triggers a fail sync task
+ * * a failure in a stream task triggers a fails stream task THEN a fail sync task
+ * * the wrappers will skip tasks if the sync/stream has already failed
+ */
+@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION", justification = "Kotlin async continuation")
+@Singleton
+@Secondary
+class DefaultDestinationTaskExceptionHandler(
+ private val taskScopeProvider: TaskScopeProvider>,
+ private val catalog: DestinationCatalog,
+ private val syncManager: SyncManager,
+ private val failStreamTaskFactory: FailStreamTaskFactory,
+ private val failSyncTaskFactory: FailSyncTaskFactory,
+) : DestinationTaskExceptionHandler> where
+T : LeveledTask,
+T : ScopedTask {
+ val log = KotlinLogging.logger {}
+
+ inner class SyncTaskWrapper(
+ private val syncManager: SyncManager,
+ override val innerTask: ScopedTask,
+ ) : WrappedTask {
+ override suspend fun execute() {
+ if (!syncManager.isActive()) {
+ val result = syncManager.awaitSyncResult()
+ if (result is SyncSuccess) {
+ throw IllegalStateException(
+ "Task $innerTask run after sync has succeeded. This should not happen."
+ )
+ }
+ log.info { "Sync task $innerTask skipped because sync has already failed." }
+ return
+ }
+
+ try {
+ innerTask.execute()
+ } catch (e: CancellationException) {
+ log.warn { "Sync task $innerTask was cancelled." }
+ throw e
+ } catch (e: Exception) {
+ handleSyncFailure(e)
+ }
+ }
+
+ override fun toString(): String {
+ return "SyncTaskWrapper(innerTask=$innerTask)"
+ }
+ }
+
+ inner class StreamTaskWrapper(
+ private val stream: DestinationStream,
+ private val syncManager: SyncManager,
+ override val innerTask: ScopedTask,
+ ) : WrappedTask {
+ override suspend fun execute() {
+ // Stop dispatching tasks if the stream has been killed by a failure elsewhere.
+ // Specifically fail if the stream was marked succeeded: we should not be in this state.
+ val streamManager = syncManager.getStreamManager(stream.descriptor)
+ if (!streamManager.isActive()) {
+ val result = streamManager.awaitStreamResult()
+ if (result is StreamSucceeded) {
+ throw IllegalStateException(
+ "Task $innerTask run after its stream ${stream.descriptor} has succeeded. This should not happen."
+ )
+ }
+ log.info { "Stream task $innerTask skipped because stream has already failed." }
+ return
+ }
+
+ try {
+ innerTask.execute()
+ } catch (e: CancellationException) {
+ log.warn { "Stream task $innerTask was cancelled." }
+ throw e
+ } catch (e: Exception) {
+ handleStreamFailure(stream, e)
+ }
+ }
+
+ override fun toString(): String {
+ return "StreamTaskWrapper(innerTask=$innerTask)"
+ }
+ }
+
+ inner class NoHandlingWrapper(
+ override val innerTask: ScopedTask,
+ ) : WrappedTask {
+ override suspend fun execute() {
+ innerTask.execute()
+ }
+
+ override fun toString(): String {
+ return "NoHandlingWrapper(innerTask=$innerTask)"
+ }
+ }
+
+ override fun withExceptionHandling(task: T): WrappedTask {
+ return when (task) {
+ is SyncLevel -> SyncTaskWrapper(syncManager, task)
+ is StreamLevel -> StreamTaskWrapper(task.stream, syncManager, task)
+ else -> throw IllegalArgumentException("Task without level: $task")
+ }
+ }
+
+ override suspend fun handleSyncFailure(e: Exception) {
+ log.error { "Sync failed: $e: killing remaining streams" }
+ catalog.streams.forEach {
+ val task = failStreamTaskFactory.make(this, e, it, kill = true)
+ taskScopeProvider.launch(NoHandlingWrapper(task))
+ }
+ val failSyncTask = failSyncTaskFactory.make(this, e)
+ taskScopeProvider.launch(NoHandlingWrapper(failSyncTask))
+ }
+
+ override suspend fun handleStreamFailure(stream: DestinationStream, e: Exception) {
+ log.error { "Caught failure in stream task: $e for ${stream.descriptor}, failing stream" }
+ val failStreamTask = failStreamTaskFactory.make(this, e, stream, kill = false)
+ taskScopeProvider.launch(NoHandlingWrapper(failStreamTask))
+ }
+
+ override suspend fun handleSyncFailed() {
+ taskScopeProvider.kill()
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/DestinationTaskLauncher.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskLauncher.kt
similarity index 67%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/DestinationTaskLauncher.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskLauncher.kt
index ad105982d7064..0857721b8674d 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/DestinationTaskLauncher.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskLauncher.kt
@@ -2,49 +2,45 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.message.Batch
-import io.airbyte.cdk.message.BatchEnvelope
-import io.airbyte.cdk.message.SpilledRawMessagesLocalFile
-import io.airbyte.cdk.state.SyncManager
-import io.airbyte.cdk.write.StreamLoader
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.Batch
+import io.airbyte.cdk.load.message.BatchEnvelope
+import io.airbyte.cdk.load.message.SpilledRawMessagesLocalFile
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.implementor.CloseStreamTaskFactory
+import io.airbyte.cdk.load.task.implementor.OpenStreamTaskFactory
+import io.airbyte.cdk.load.task.implementor.ProcessBatchTaskFactory
+import io.airbyte.cdk.load.task.implementor.ProcessRecordsTaskFactory
+import io.airbyte.cdk.load.task.implementor.SetupTaskFactory
+import io.airbyte.cdk.load.task.implementor.TeardownTaskFactory
+import io.airbyte.cdk.load.task.internal.FlushCheckpointsTaskFactory
+import io.airbyte.cdk.load.task.internal.InputConsumerTask
+import io.airbyte.cdk.load.task.internal.SpillToDiskTaskFactory
+import io.airbyte.cdk.load.task.internal.TimedForcedCheckpointFlushTask
+import io.airbyte.cdk.load.task.internal.UpdateCheckpointsTask
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicBoolean
-import kotlinx.coroutines.CompletableDeferred
import kotlinx.coroutines.sync.Mutex
import kotlinx.coroutines.sync.withLock
-sealed interface DestinationWriteTask : Task
-
-interface SyncTask : DestinationWriteTask
-
-interface StreamTask : DestinationWriteTask
-
interface DestinationTaskLauncher : TaskLauncher {
suspend fun handleSetupComplete()
suspend fun handleStreamStarted(stream: DestinationStream)
suspend fun handleNewSpilledFile(
stream: DestinationStream,
- wrapped: BatchEnvelope
+ wrapped: BatchEnvelope,
+ endOfStream: Boolean
)
suspend fun handleNewBatch(stream: DestinationStream, wrapped: BatchEnvelope<*>)
suspend fun handleStreamClosed(stream: DestinationStream)
suspend fun handleTeardownComplete()
}
-interface DestinationTaskLauncherExceptionHandler :
- TaskLauncherExceptionHandler {
- suspend fun handleSyncFailure(e: Exception)
- suspend fun handleStreamFailure(e: Exception)
-}
-
/**
* Governs the task workflow for the entire destination life-cycle.
*
@@ -76,49 +72,66 @@ interface DestinationTaskLauncherExceptionHandler :
* // TODO: Capture failures, retry, and call into close(failure=true) if can't recover.
*/
@Singleton
+@Secondary
@SuppressFBWarnings(
"NP_NONNULL_PARAM_VIOLATION",
justification = "arguments are guaranteed to be non-null by Kotlin's type system"
)
class DefaultDestinationTaskLauncher(
+ private val taskScopeProvider: TaskScopeProvider>,
private val catalog: DestinationCatalog,
private val syncManager: SyncManager,
- override val taskRunner: TaskRunner,
+
+ // Internal Tasks
+ private val inputConsumerTask: InputConsumerTask,
+ private val spillToDiskTaskFactory: SpillToDiskTaskFactory,
+
+ // Implementor Tasks
private val setupTaskFactory: SetupTaskFactory,
private val openStreamTaskFactory: OpenStreamTaskFactory,
- private val spillToDiskTaskFactory: SpillToDiskTaskFactory,
private val processRecordsTaskFactory: ProcessRecordsTaskFactory,
private val processBatchTaskFactory: ProcessBatchTaskFactory,
private val closeStreamTaskFactory: CloseStreamTaskFactory,
private val teardownTaskFactory: TeardownTaskFactory,
- private val exceptionHandler: TaskLauncherExceptionHandler
+
+ // Checkpoint Tasks
+ private val flushCheckpointsTaskFactory: FlushCheckpointsTaskFactory,
+ private val timedFlushTask: TimedForcedCheckpointFlushTask,
+ private val updateCheckpointsTask: UpdateCheckpointsTask,
+
+ // Exception handling
+ private val exceptionHandler: TaskExceptionHandler>
) : DestinationTaskLauncher {
private val log = KotlinLogging.logger {}
- private val runTeardownOnce = AtomicBoolean(false)
private val batchUpdateLock = Mutex()
- private suspend fun enqueue(task: DestinationWriteTask) {
- taskRunner.enqueue(exceptionHandler.withExceptionHandling(task))
- }
-
- private val streamLoaders:
- ConcurrentHashMap> =
- ConcurrentHashMap()
-
- init {
- catalog.streams.forEach { streamLoaders[it.descriptor] = CompletableDeferred() }
+ private suspend fun enqueue(task: LeveledTask) {
+ taskScopeProvider.launch(exceptionHandler.withExceptionHandling(task))
}
override suspend fun start() {
+ // Start the input consumer ASAP
+ log.info { "Starting input consumer task" }
+ enqueue(inputConsumerTask)
+
+ // Launch the client interface setup task
log.info { "Starting startup task" }
val setupTask = setupTaskFactory.make(this)
enqueue(setupTask)
+
+ // Start a spill-to-disk task for each record stream
catalog.streams.forEach { stream ->
log.info { "Starting spill-to-disk task for $stream" }
val spillTask = spillToDiskTaskFactory.make(this, stream)
enqueue(spillTask)
}
+
+ log.info { "Starting timed flush task" }
+ enqueue(timedFlushTask)
+
+ log.info { "Starting checkpoint update task" }
+ enqueue(updateCheckpointsTask)
}
/** Called when the initial destination setup completes. */
@@ -139,11 +152,17 @@ class DefaultDestinationTaskLauncher(
/** Called for each new spilled file. */
override suspend fun handleNewSpilledFile(
stream: DestinationStream,
- wrapped: BatchEnvelope
+ wrapped: BatchEnvelope,
+ endOfStream: Boolean
) {
log.info { "Starting process records task for ${stream.descriptor}, file ${wrapped.batch}" }
val task = processRecordsTaskFactory.make(this, stream, wrapped)
enqueue(task)
+ if (!endOfStream) {
+ log.info { "End-of-stream not reached, restarting spill-to-disk task for $stream" }
+ val spillTask = spillToDiskTaskFactory.make(this, stream)
+ enqueue(spillTask)
+ }
}
/**
@@ -155,6 +174,10 @@ class DefaultDestinationTaskLauncher(
val streamManager = syncManager.getStreamManager(stream.descriptor)
streamManager.updateBatchState(wrapped)
+ if (wrapped.batch.isPersisted()) {
+ enqueue(flushCheckpointsTaskFactory.make())
+ }
+
if (wrapped.batch.state != Batch.State.COMPLETE) {
log.info {
"Batch not complete: Starting process batch task for ${stream.descriptor}, batch $wrapped"
@@ -184,51 +207,6 @@ class DefaultDestinationTaskLauncher(
/** Called exactly once when all streams are closed. */
override suspend fun handleTeardownComplete() {
- stop()
- }
-}
-
-@Singleton
-@Secondary
-class DefaultDestinationTaskLauncherExceptionHandler : DestinationTaskLauncherExceptionHandler {
- class SyncTaskWrapper(
- private val exceptionHandler: DestinationTaskLauncherExceptionHandler,
- private val innerTask: SyncTask,
- ) : Task {
- override suspend fun execute() {
- try {
- innerTask.execute()
- } catch (e: Exception) {
- exceptionHandler.handleSyncFailure(e)
- }
- }
- }
-
- class StreamTaskWrapper(
- private val exceptionHandler: DestinationTaskLauncherExceptionHandler,
- private val innerTask: StreamTask,
- ) : SyncTask {
- override suspend fun execute() {
- try {
- innerTask.execute()
- } catch (e: Exception) {
- exceptionHandler.handleStreamFailure(e)
- }
- }
- }
-
- override fun withExceptionHandling(task: DestinationWriteTask): Task {
- return when (task) {
- is SyncTask -> SyncTaskWrapper(this, task)
- is StreamTask -> SyncTaskWrapper(this, StreamTaskWrapper(this, task))
- }
- }
-
- override suspend fun handleSyncFailure(e: Exception) {
- // TODO: Do stuff
- }
-
- override suspend fun handleStreamFailure(e: Exception) {
- // TODO: Do stuff
+ taskScopeProvider.close()
}
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskScopeProvider.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskScopeProvider.kt
new file mode 100644
index 0000000000000..b8c52bd1b961e
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/DestinationTaskScopeProvider.kt
@@ -0,0 +1,118 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task
+
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.util.concurrent.Executors
+import kotlin.system.measureTimeMillis
+import kotlinx.coroutines.CoroutineDispatcher
+import kotlinx.coroutines.CoroutineScope
+import kotlinx.coroutines.Dispatchers
+import kotlinx.coroutines.Job
+import kotlinx.coroutines.SupervisorJob
+import kotlinx.coroutines.asCoroutineDispatcher
+import kotlinx.coroutines.launch
+import kotlinx.coroutines.withTimeoutOrNull
+
+/**
+ * The scope in which a task should run
+ * - [InternalScope]:
+ * ```
+ * - internal to the task launcher
+ * - should not be blockable by implementor errors
+ * - killable w/o side effects
+ * ```
+ * - [ImplementorScope]: implemented by the destination
+ * ```
+ * - calls implementor interface
+ * - should not block internal tasks (esp reading from stdin)
+ * - should complete if possible even when failing the sync
+ * ```
+ * - [ShutdownScope]: special case of [ImplementorScope]
+ * ```
+ * - tasks that should run during shutdown
+ * - handles canceling/joining other tasks
+ * - (and so should not cancel themselves)
+ * ```
+ */
+sealed interface ScopedTask : Task
+
+interface InternalScope : ScopedTask
+
+interface ImplementorScope : ScopedTask
+
+interface ShutdownScope : ScopedTask
+
+@Singleton
+@Secondary
+class DestinationTaskScopeProvider(config: DestinationConfiguration) :
+ TaskScopeProvider> {
+ private val log = KotlinLogging.logger {}
+
+ private val timeoutMs = config.gracefulCancellationTimeoutMs
+
+ data class ControlScope(
+ val job: Job,
+ val dispatcher: CoroutineDispatcher,
+ val scope: CoroutineScope = CoroutineScope(dispatcher + job)
+ )
+
+ private val internalScope = ControlScope(Job(), Dispatchers.IO)
+
+ private val implementorScope =
+ ControlScope(
+ SupervisorJob(),
+ Executors.newFixedThreadPool(config.maxNumImplementorTaskThreads)
+ .asCoroutineDispatcher()
+ )
+
+ override suspend fun launch(task: WrappedTask) {
+ when (task.innerTask) {
+ is InternalScope -> internalScope.scope.launch { execute(task, "internal") }
+ is ImplementorScope -> implementorScope.scope.launch { execute(task, "implementor") }
+ is ShutdownScope -> implementorScope.scope.launch { execute(task, "shutdown") }
+ }
+ }
+
+ private suspend fun execute(task: WrappedTask, scope: String) {
+ log.info { "Launching task $task in scope $scope" }
+ val elapsed = measureTimeMillis { task.execute() }
+ log.info { "Task $task completed in $elapsed ms" }
+ }
+
+ override suspend fun close() {
+ log.info { "Closing task scopes" }
+ // Under normal operation, all tasks should be complete
+ // (except things like force flush, which loop). So
+ // - it's safe to force cancel the internal tasks
+ // - implementor scope should join immediately
+ implementorScope.job.join()
+ log.info { "Implementor tasks completed, cancelling internal tasks." }
+ internalScope.job.cancel()
+ }
+
+ override suspend fun kill() {
+ log.info { "Killing task scopes" }
+
+ // Give the implementor tasks a chance to fail gracefully
+ withTimeoutOrNull(timeoutMs) {
+ log.info {
+ "Cancelled internal tasks, waiting ${timeoutMs}ms for implementor tasks to complete"
+ }
+ implementorScope.job.join()
+ log.info { "Implementor tasks completed" }
+ }
+ ?: run {
+ log.error { "Implementor tasks did not complete within ${timeoutMs}ms, cancelling" }
+ implementorScope.job.cancel()
+ }
+
+ log.info { "Cancelling internal tasks" }
+ internalScope.job.cancel()
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/Task.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/Task.kt
new file mode 100644
index 0000000000000..492f084fabeb3
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/Task.kt
@@ -0,0 +1,36 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task
+
+import io.airbyte.cdk.load.util.CloseableCoroutine
+
+interface Task {
+ suspend fun execute()
+}
+
+/**
+ * A TaskLauncher is responsible for starting and stopping the task workflow, and for managing
+ * transitions between tasks.
+ */
+interface TaskLauncher {
+ suspend fun start()
+}
+
+/**
+ * Wraps tasks with exception handling. It should provide an exception handling workflow and take
+ * responsibility for closing scopes, etc.
+ */
+interface TaskExceptionHandler {
+ fun withExceptionHandling(task: T): U
+}
+
+/** Provides the scope(s) in which tasks run. */
+interface TaskScopeProvider : CloseableCoroutine {
+ /** Launch a task in the correct scope. */
+ suspend fun launch(task: T)
+
+ /** Unliked close, may attempt to fail gracefully, but should guarantee return. */
+ suspend fun kill()
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/CloseStreamTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/CloseStreamTask.kt
similarity index 68%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/CloseStreamTask.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/CloseStreamTask.kt
index ee786d583c4da..8fcd18c161980 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/CloseStreamTask.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/CloseStreamTask.kt
@@ -2,15 +2,18 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task.implementor
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.state.SyncManager
-import io.airbyte.cdk.write.StreamLoader
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.ImplementorScope
+import io.airbyte.cdk.load.task.StreamLevel
+import io.airbyte.cdk.load.write.StreamLoader
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
-interface CloseStreamTask : StreamTask
+interface CloseStreamTask : StreamLevel, ImplementorScope
/**
* Wraps @[StreamLoader.close] and marks the stream as closed in the stream manager. Also starts the
@@ -18,14 +21,14 @@ interface CloseStreamTask : StreamTask
*/
class DefaultCloseStreamTask(
private val syncManager: SyncManager,
- private val stream: DestinationStream,
+ override val stream: DestinationStream,
private val taskLauncher: DestinationTaskLauncher
) : CloseStreamTask {
override suspend fun execute() {
val streamLoader = syncManager.getOrAwaitStreamLoader(stream.descriptor)
streamLoader.close()
- syncManager.getStreamManager(stream.descriptor).markClosed()
+ syncManager.getStreamManager(stream.descriptor).markSucceeded()
taskLauncher.handleStreamClosed(streamLoader.stream)
}
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/FailStreamTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/FailStreamTask.kt
new file mode 100644
index 0000000000000..bfeceda8b50b3
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/FailStreamTask.kt
@@ -0,0 +1,85 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.implementor
+
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.state.StreamIncompleteResult
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskExceptionHandler
+import io.airbyte.cdk.load.task.ShutdownScope
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+
+interface FailStreamTask : ShutdownScope
+
+/**
+ * FailStreamTask is a task that is executed when a stream fails. It is responsible for cleaning up
+ * resources and reporting the failure.
+ */
+class DefaultFailStreamTask(
+ private val exceptionHandler: DestinationTaskExceptionHandler<*, *>,
+ private val exception: Exception,
+ private val syncManager: SyncManager,
+ private val stream: DestinationStream,
+ private val kill: Boolean,
+) : FailStreamTask {
+ val log = KotlinLogging.logger {}
+
+ override suspend fun execute() {
+ val streamManager = syncManager.getStreamManager(stream.descriptor)
+ if (kill) {
+ if (!streamManager.markKilled(exception)) {
+ log.info { "Stream ${stream.descriptor} already complete, skipping kill." }
+ return
+ }
+ } else {
+ if (!streamManager.markFailed(exception)) {
+ throw IllegalStateException(
+ "Cannot fail stream ${stream.descriptor}, which is already complete."
+ )
+ }
+ // Stream failure implies sync failure
+ exceptionHandler.handleSyncFailure(exception)
+ }
+
+ val streamResult = streamManager.awaitStreamResult()
+ val incompleteResult =
+ if (streamResult is StreamIncompleteResult) {
+ streamResult
+ } else {
+ null
+ }
+ // TODO: Bit of smell here, suggests we should be fetching the StreamLoader
+ // lazily+unconditionally
+ // through the DestinationWriter (via an injected wrapper?)
+ syncManager.getStreamLoaderOrNull(stream.descriptor)?.close(incompleteResult)
+ ?: log.warn {
+ "StreamLoader not found for stream ${stream.descriptor}, cannot call close."
+ }
+ }
+}
+
+interface FailStreamTaskFactory {
+ fun make(
+ exceptionHandler: DestinationTaskExceptionHandler<*, *>,
+ exception: Exception,
+ stream: DestinationStream,
+ kill: Boolean,
+ ): FailStreamTask
+}
+
+@Singleton
+@Secondary
+class DefaultFailStreamTaskFactory(private val syncManager: SyncManager) : FailStreamTaskFactory {
+ override fun make(
+ exceptionHandler: DestinationTaskExceptionHandler<*, *>,
+ exception: Exception,
+ stream: DestinationStream,
+ kill: Boolean,
+ ): FailStreamTask {
+ return DefaultFailStreamTask(exceptionHandler, exception, syncManager, stream, kill)
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/FailSyncTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/FailSyncTask.kt
new file mode 100644
index 0000000000000..5865ef25c66eb
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/FailSyncTask.kt
@@ -0,0 +1,79 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.implementor
+
+import io.airbyte.cdk.load.state.CheckpointManager
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskExceptionHandler
+import io.airbyte.cdk.load.task.ShutdownScope
+import io.airbyte.cdk.load.util.setOnce
+import io.airbyte.cdk.load.write.DestinationWriter
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.util.concurrent.atomic.AtomicBoolean
+
+interface FailSyncTask : ShutdownScope
+
+/**
+ * FailSyncTask is a task that is executed when a sync fails. It is responsible for cleaning up
+ * resources and reporting the failure.
+ */
+class DefaultFailSyncTask(
+ private val exceptionHandler: DestinationTaskExceptionHandler<*, *>,
+ private val destinationWriter: DestinationWriter,
+ private val exception: Exception,
+ private val syncManager: SyncManager,
+ private val checkpointManager: CheckpointManager<*, *>
+) : FailSyncTask {
+ private val log = KotlinLogging.logger {}
+
+ companion object {
+ private val syncFailedHasRun = AtomicBoolean(false)
+ }
+
+ override suspend fun execute() {
+ if (syncFailedHasRun.setOnce()) {
+ // Ensure any remaining ready state gets captured: don't waste work!
+ checkpointManager.flushReadyCheckpointMessages()
+ val result = syncManager.markFailed(exception) // awaits stream completion
+ log.info { "Calling teardown with failure result $result" }
+ exceptionHandler.handleSyncFailed()
+ // Do this cleanup last, after all the tasks have had a decent chance to finish.
+ destinationWriter.teardown(result)
+ } else {
+ log.info { "Fail sync task already initiated, doing nothing." }
+ }
+ }
+}
+
+interface FailSyncTaskFactory {
+ fun make(
+ exceptionHandler: DestinationTaskExceptionHandler<*, *>,
+ exception: Exception
+ ): FailSyncTask
+}
+
+@Singleton
+@Secondary
+class DefaultFailSyncTaskFactory(
+ private val syncManager: SyncManager,
+ private val checkpointManager: CheckpointManager<*, *>,
+ private val destinationWriter: DestinationWriter
+) : FailSyncTaskFactory {
+
+ override fun make(
+ exceptionHandler: DestinationTaskExceptionHandler<*, *>,
+ exception: Exception
+ ): FailSyncTask {
+ return DefaultFailSyncTask(
+ exceptionHandler,
+ destinationWriter,
+ exception,
+ syncManager,
+ checkpointManager
+ )
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/OpenStreamTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/OpenStreamTask.kt
similarity index 72%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/OpenStreamTask.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/OpenStreamTask.kt
index 5c55464507306..4a65fefd19fce 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/OpenStreamTask.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/OpenStreamTask.kt
@@ -2,16 +2,19 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task.implementor
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.state.SyncManager
-import io.airbyte.cdk.write.DestinationWriter
-import io.airbyte.cdk.write.StreamLoader
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.ImplementorScope
+import io.airbyte.cdk.load.task.StreamLevel
+import io.airbyte.cdk.load.write.DestinationWriter
+import io.airbyte.cdk.load.write.StreamLoader
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
-interface OpenStreamTask : StreamTask
+interface OpenStreamTask : StreamLevel, ImplementorScope
/**
* Wraps @[StreamLoader.start] and starts the spill-to-disk tasks.
@@ -21,7 +24,7 @@ interface OpenStreamTask : StreamTask
class DefaultOpenStreamTask(
private val destinationWriter: DestinationWriter,
private val syncManager: SyncManager,
- private val stream: DestinationStream,
+ override val stream: DestinationStream,
private val taskLauncher: DestinationTaskLauncher
) : OpenStreamTask {
override suspend fun execute() {
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/ProcessBatchTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/ProcessBatchTask.kt
similarity index 73%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/ProcessBatchTask.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/ProcessBatchTask.kt
index 370de18e97b2f..c884a1c3c72e8 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/ProcessBatchTask.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/ProcessBatchTask.kt
@@ -2,22 +2,25 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task.implementor
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.message.BatchEnvelope
-import io.airbyte.cdk.state.SyncManager
-import io.airbyte.cdk.write.StreamLoader
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.BatchEnvelope
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.ImplementorScope
+import io.airbyte.cdk.load.task.StreamLevel
+import io.airbyte.cdk.load.write.StreamLoader
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
-interface ProcessBatchTask : StreamTask
+interface ProcessBatchTask : StreamLevel, ImplementorScope
/** Wraps @[StreamLoader.processBatch] and handles the resulting batch. */
class DefaultProcessBatchTask(
private val syncManager: SyncManager,
private val batchEnvelope: BatchEnvelope<*>,
- private val stream: DestinationStream,
+ override val stream: DestinationStream,
private val taskLauncher: DestinationTaskLauncher
) : ProcessBatchTask {
override suspend fun execute() {
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/ProcessRecordsTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/ProcessRecordsTask.kt
similarity index 54%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/ProcessRecordsTask.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/ProcessRecordsTask.kt
index 86e03a0a51042..05830995a764b 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/ProcessRecordsTask.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/ProcessRecordsTask.kt
@@ -2,26 +2,27 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task.implementor
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.message.BatchEnvelope
-import io.airbyte.cdk.message.Deserializer
-import io.airbyte.cdk.message.DestinationMessage
-import io.airbyte.cdk.message.DestinationRecord
-import io.airbyte.cdk.message.DestinationStreamAffinedMessage
-import io.airbyte.cdk.message.DestinationStreamComplete
-import io.airbyte.cdk.message.DestinationStreamIncomplete
-import io.airbyte.cdk.message.SpilledRawMessagesLocalFile
-import io.airbyte.cdk.state.SyncManager
-import io.airbyte.cdk.write.StreamLoader
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.BatchEnvelope
+import io.airbyte.cdk.load.message.Deserializer
+import io.airbyte.cdk.load.message.DestinationMessage
+import io.airbyte.cdk.load.message.DestinationRecord
+import io.airbyte.cdk.load.message.DestinationStreamAffinedMessage
+import io.airbyte.cdk.load.message.DestinationStreamComplete
+import io.airbyte.cdk.load.message.DestinationStreamIncomplete
+import io.airbyte.cdk.load.message.SpilledRawMessagesLocalFile
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.ImplementorScope
+import io.airbyte.cdk.load.task.StreamLevel
+import io.airbyte.cdk.load.write.StreamLoader
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
-import kotlinx.coroutines.Dispatchers
-import kotlinx.coroutines.withContext
-interface ProcessRecordsTask : StreamTask
+interface ProcessRecordsTask : StreamLevel, ImplementorScope
/**
* Wraps @[StreamLoader.processRecords] and feeds it a lazy iterator over the last batch of spooled
@@ -32,7 +33,7 @@ interface ProcessRecordsTask : StreamTask
* moved to the task launcher.
*/
class DefaultProcessRecordsTask(
- private val stream: DestinationStream,
+ override val stream: DestinationStream,
private val taskLauncher: DestinationTaskLauncher,
private val fileEnvelope: BatchEnvelope,
private val deserializer: Deserializer,
@@ -47,28 +48,26 @@ class DefaultProcessRecordsTask(
log.info { "Processing records from ${fileEnvelope.batch.localFile}" }
val nextBatch =
try {
- withContext(Dispatchers.IO) {
- fileEnvelope.batch.localFile.toFileReader().use { reader ->
- val records =
- reader
- .lines()
- .map {
- when (val message = deserializer.deserialize(it)) {
- is DestinationStreamAffinedMessage -> message
- else ->
- throw IllegalStateException(
- "Expected record message, got ${message::class}"
- )
- }
+ fileEnvelope.batch.localFile.toFileReader().use { reader ->
+ val records =
+ reader
+ .lines()
+ .map {
+ when (val message = deserializer.deserialize(it)) {
+ is DestinationStreamAffinedMessage -> message
+ else ->
+ throw IllegalStateException(
+ "Expected record message, got ${message::class}"
+ )
}
- .takeWhile {
- it !is DestinationStreamComplete &&
- it !is DestinationStreamIncomplete
- }
- .map { it as DestinationRecord }
- .iterator()
- streamLoader.processRecords(records, fileEnvelope.batch.totalSizeBytes)
- }
+ }
+ .takeWhile {
+ it !is DestinationStreamComplete &&
+ it !is DestinationStreamIncomplete
+ }
+ .map { it as DestinationRecord }
+ .iterator()
+ streamLoader.processRecords(records, fileEnvelope.batch.totalSizeBytes)
}
} finally {
log.info { "Processing completed, deleting ${fileEnvelope.batch.localFile}" }
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/SetupTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/SetupTask.kt
similarity index 76%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/SetupTask.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/SetupTask.kt
index 5fecd72b16cd5..d55b4acb6c43e 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/SetupTask.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/SetupTask.kt
@@ -2,13 +2,16 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task.implementor
-import io.airbyte.cdk.write.DestinationWriter
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.ImplementorScope
+import io.airbyte.cdk.load.task.SyncLevel
+import io.airbyte.cdk.load.write.DestinationWriter
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
-interface SetupTask : SyncTask
+interface SetupTask : SyncLevel, ImplementorScope
/**
* Wraps @[DestinationWriter.setup] and starts the open stream tasks.
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TeardownTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/TeardownTask.kt
similarity index 56%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TeardownTask.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/TeardownTask.kt
index 83f9fbf65af38..7e897327d1b1d 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TeardownTask.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/implementor/TeardownTask.kt
@@ -2,17 +2,21 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.task
+package io.airbyte.cdk.load.task.implementor
-import io.airbyte.cdk.state.CheckpointManager
-import io.airbyte.cdk.state.SyncManager
-import io.airbyte.cdk.write.DestinationWriter
+import io.airbyte.cdk.load.state.CheckpointManager
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.ShutdownScope
+import io.airbyte.cdk.load.task.SyncLevel
+import io.airbyte.cdk.load.util.setOnce
+import io.airbyte.cdk.load.write.DestinationWriter
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
import java.util.concurrent.atomic.AtomicBoolean
-interface TeardownTask : SyncTask
+interface TeardownTask : SyncLevel, ShutdownScope
/**
* Wraps @[DestinationWriter.teardown] and stops the task launcher.
@@ -27,19 +31,29 @@ class DefaultTeardownTask(
) : TeardownTask {
val log = KotlinLogging.logger {}
- private val teardownHasRun = AtomicBoolean(false)
+ companion object {
+ private val teardownHasRun = AtomicBoolean(false)
+ }
override suspend fun execute() {
- // TODO: This should be its own task, dispatched on a timer or something.
- checkpointManager.flushReadyCheckpointMessages()
-
// Run the task exactly once, and only after all streams have closed.
- if (teardownHasRun.compareAndSet(false, true)) {
- syncManager.awaitAllStreamsClosed()
- log.info { "Starting teardown task" }
+ if (teardownHasRun.setOnce()) {
+ syncManager.awaitInputProcessingComplete()
+ checkpointManager.awaitAllCheckpointsFlushed()
+ log.info { "Teardown task awaiting stream completion" }
+ if (!syncManager.awaitAllStreamsCompletedSuccessfully()) {
+ log.info { "Streams failed to complete successfully, doing nothing." }
+ return
+ }
+
+ log.info { "Starting teardown task" }
destination.teardown()
+ log.info { "Teardown task complete, marking sync succeeded." }
+ syncManager.markSucceeded()
taskLauncher.handleTeardownComplete()
+ } else {
+ log.info { "Teardown already initiated, doing nothing." }
}
}
}
@@ -55,6 +69,7 @@ class DefaultTeardownTaskFactory(
private val syncManager: SyncManager,
private val destination: DestinationWriter,
) : TeardownTaskFactory {
+
override fun make(taskLauncher: DestinationTaskLauncher): TeardownTask {
return DefaultTeardownTask(checkpointManager, syncManager, destination, taskLauncher)
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/FlushCheckpointsTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/FlushCheckpointsTask.kt
new file mode 100644
index 0000000000000..251416da46e21
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/FlushCheckpointsTask.kt
@@ -0,0 +1,35 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.internal
+
+import io.airbyte.cdk.load.state.CheckpointManager
+import io.airbyte.cdk.load.task.InternalScope
+import io.airbyte.cdk.load.task.SyncLevel
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+
+interface FlushCheckpointsTask : SyncLevel, InternalScope
+
+class DefaultFlushCheckpointsTask(
+ private val checkpointManager: CheckpointManager<*, *>,
+) : FlushCheckpointsTask {
+ override suspend fun execute() {
+ checkpointManager.flushReadyCheckpointMessages()
+ }
+}
+
+interface FlushCheckpointsTaskFactory {
+ fun make(): FlushCheckpointsTask
+}
+
+@Singleton
+@Secondary
+class DefaultFlushCheckpointsTaskFactory(
+ private val checkpointManager: CheckpointManager<*, *>,
+) : FlushCheckpointsTaskFactory {
+ override fun make(): FlushCheckpointsTask {
+ return DefaultFlushCheckpointsTask(checkpointManager)
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/InputConsumerTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/InputConsumerTask.kt
new file mode 100644
index 0000000000000..268ae746e7a6b
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/InputConsumerTask.kt
@@ -0,0 +1,214 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.internal
+
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.CheckpointMessage
+import io.airbyte.cdk.load.message.CheckpointMessageWrapped
+import io.airbyte.cdk.load.message.Deserializer
+import io.airbyte.cdk.load.message.DestinationMessage
+import io.airbyte.cdk.load.message.DestinationRecord
+import io.airbyte.cdk.load.message.DestinationRecordWrapped
+import io.airbyte.cdk.load.message.DestinationStreamAffinedMessage
+import io.airbyte.cdk.load.message.DestinationStreamComplete
+import io.airbyte.cdk.load.message.DestinationStreamIncomplete
+import io.airbyte.cdk.load.message.GlobalCheckpoint
+import io.airbyte.cdk.load.message.GlobalCheckpointWrapped
+import io.airbyte.cdk.load.message.MessageQueueSupplier
+import io.airbyte.cdk.load.message.QueueWriter
+import io.airbyte.cdk.load.message.StreamCheckpoint
+import io.airbyte.cdk.load.message.StreamCheckpointWrapped
+import io.airbyte.cdk.load.message.StreamCompleteWrapped
+import io.airbyte.cdk.load.message.StreamRecordWrapped
+import io.airbyte.cdk.load.message.Undefined
+import io.airbyte.cdk.load.state.MemoryManager
+import io.airbyte.cdk.load.state.Reserved
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.InternalScope
+import io.airbyte.cdk.load.task.SyncLevel
+import io.airbyte.cdk.load.util.use
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+import java.io.InputStream
+import kotlinx.coroutines.flow.Flow
+import kotlinx.coroutines.flow.FlowCollector
+
+interface InputConsumerTask : SyncLevel, InternalScope
+
+/**
+ * Routes @[DestinationStreamAffinedMessage]s by stream to the appropriate channel and @
+ * [CheckpointMessage]s to the state manager.
+ *
+ * TODO: Handle other message types.
+ */
+@SuppressFBWarnings(
+ "NP_NONNULL_PARAM_VIOLATION",
+ justification = "message is guaranteed to be non-null by Kotlin's type system"
+)
+@Singleton
+@Secondary
+class DefaultInputConsumerTask(
+ private val catalog: DestinationCatalog,
+ private val inputFlow: SizedInputFlow>,
+ private val recordQueueSupplier:
+ MessageQueueSupplier>,
+ private val checkpointQueue: QueueWriter>,
+ private val syncManager: SyncManager,
+) : InputConsumerTask {
+ private val log = KotlinLogging.logger {}
+
+ private suspend fun handleRecord(
+ reserved: Reserved,
+ sizeBytes: Long
+ ) {
+ val stream = reserved.value.stream
+ val manager = syncManager.getStreamManager(stream)
+ val queue = recordQueueSupplier.get(stream)
+ when (val message = reserved.value) {
+ is DestinationRecord -> {
+ val wrapped =
+ StreamRecordWrapped(
+ index = manager.countRecordIn(),
+ sizeBytes = sizeBytes,
+ record = message
+ )
+ queue.publish(reserved.replace(wrapped))
+ }
+ is DestinationStreamComplete -> {
+ reserved.release() // safe because multiple calls conflate
+ val wrapped = StreamCompleteWrapped(index = manager.markEndOfStream())
+ queue.publish(reserved.replace(wrapped))
+ queue.close()
+ }
+ is DestinationStreamIncomplete ->
+ throw IllegalStateException("Stream $stream failed upstream, cannot continue.")
+ }
+ }
+
+ private suspend fun handleCheckpoint(
+ reservation: Reserved,
+ sizeBytes: Long
+ ) {
+ when (val checkpoint = reservation.value) {
+ /**
+ * For a stream state message, mark the checkpoint and add the message with index and
+ * count to the state manager. Also, add the count to the destination stats.
+ */
+ is StreamCheckpoint -> {
+ val stream = checkpoint.checkpoint.stream
+ val manager = syncManager.getStreamManager(stream)
+ val (currentIndex, countSinceLast) = manager.markCheckpoint()
+ val messageWithCount =
+ checkpoint.withDestinationStats(CheckpointMessage.Stats(countSinceLast))
+ checkpointQueue.publish(
+ reservation.replace(
+ StreamCheckpointWrapped(sizeBytes, stream, currentIndex, messageWithCount)
+ )
+ )
+ }
+
+ /**
+ * For a global state message, collect the index per stream, but add the total count to
+ * the destination stats.
+ */
+ is GlobalCheckpoint -> {
+ val streamWithIndexAndCount =
+ catalog.streams.map { stream ->
+ val manager = syncManager.getStreamManager(stream.descriptor)
+ val (currentIndex, countSinceLast) = manager.markCheckpoint()
+ Triple(stream, currentIndex, countSinceLast)
+ }
+ val totalCount = streamWithIndexAndCount.sumOf { it.third }
+ val messageWithCount =
+ checkpoint.withDestinationStats(CheckpointMessage.Stats(totalCount))
+ val streamIndexes = streamWithIndexAndCount.map { it.first.descriptor to it.second }
+ checkpointQueue.publish(
+ reservation.replace(
+ GlobalCheckpointWrapped(sizeBytes, streamIndexes, messageWithCount)
+ )
+ )
+ }
+ }
+ }
+
+ /**
+ * Deserialize and route the message to the appropriate channel.
+ *
+ * NOTE: Not thread-safe! Only a single writer should publish to the queue.
+ */
+ override suspend fun execute() {
+ log.info { "Starting consuming messages from the input flow" }
+ try {
+ checkpointQueue.use {
+ inputFlow.collect { (sizeBytes, reserved) ->
+ when (val message = reserved.value) {
+ /* If the input message represents a record. */
+ is DestinationStreamAffinedMessage ->
+ handleRecord(reserved.replace(message), sizeBytes)
+ is CheckpointMessage ->
+ handleCheckpoint(reserved.replace(message), sizeBytes)
+ is Undefined -> {
+ log.warn { "Unhandled message: $message" }
+ }
+ }
+ }
+ }
+ syncManager.markInputConsumed()
+ } finally {
+ log.info { "Closing record queues" }
+ catalog.streams.forEach { recordQueueSupplier.get(it.descriptor).close() }
+ }
+ }
+}
+
+interface SizedInputFlow : Flow>
+
+abstract class ReservingDeserializingInputFlow : SizedInputFlow> {
+ val log = KotlinLogging.logger {}
+
+ abstract val config: DestinationConfiguration
+ abstract val deserializer: Deserializer
+ abstract val memoryManager: MemoryManager
+ abstract val inputStream: InputStream
+
+ override suspend fun collect(collector: FlowCollector>>) {
+ val reservation = memoryManager.reserveRatio(config.maxMessageQueueMemoryUsageRatio, this)
+ val reservationManager = reservation.getReservationManager()
+
+ log.info { "Reserved ${reservation.bytesReserved/1024}mb memory for input processing" }
+
+ reservation.use { _ ->
+ inputStream.bufferedReader().lineSequence().forEachIndexed { index, line ->
+ if (line.isEmpty()) {
+ return@forEachIndexed
+ }
+
+ val lineSize = line.length.toLong()
+ val estimatedSize = lineSize * config.estimatedRecordMemoryOverheadRatio
+ val reserved = reservationManager.reserveBlocking(estimatedSize.toLong(), line)
+ val message = deserializer.deserialize(line)
+ collector.emit(Pair(lineSize, reserved.replace(message)))
+
+ if (index % 10_000 == 0) {
+ log.info { "Processed $index lines" }
+ }
+ }
+ }
+
+ log.info { "Finished processing input" }
+ }
+}
+
+@Singleton
+class DefaultInputFlow(
+ override val config: DestinationConfiguration,
+ override val deserializer: Deserializer,
+ override val memoryManager: MemoryManager,
+ override val inputStream: InputStream
+) : ReservingDeserializingInputFlow()
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/SpillToDiskTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/SpillToDiskTask.kt
new file mode 100644
index 0000000000000..4e1950e5466c9
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/SpillToDiskTask.kt
@@ -0,0 +1,132 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.internal
+
+import com.google.common.collect.Range
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.file.TempFileProvider
+import io.airbyte.cdk.load.message.BatchEnvelope
+import io.airbyte.cdk.load.message.DestinationRecordWrapped
+import io.airbyte.cdk.load.message.MessageQueueSupplier
+import io.airbyte.cdk.load.message.QueueReader
+import io.airbyte.cdk.load.message.SpilledRawMessagesLocalFile
+import io.airbyte.cdk.load.message.StreamCompleteWrapped
+import io.airbyte.cdk.load.message.StreamRecordWrapped
+import io.airbyte.cdk.load.state.FlushStrategy
+import io.airbyte.cdk.load.state.Reserved
+import io.airbyte.cdk.load.task.DestinationTaskLauncher
+import io.airbyte.cdk.load.task.InternalScope
+import io.airbyte.cdk.load.task.StreamLevel
+import io.airbyte.cdk.load.util.takeUntilInclusive
+import io.airbyte.cdk.load.util.use
+import io.airbyte.cdk.load.util.withNextAdjacentValue
+import io.github.oshai.kotlinlogging.KotlinLogging
+import jakarta.inject.Singleton
+import kotlinx.coroutines.flow.last
+import kotlinx.coroutines.flow.runningFold
+
+interface SpillToDiskTask : StreamLevel, InternalScope
+
+/**
+ * Reads records from the message queue and writes them to disk. This task is internal and is not
+ * exposed to the implementor.
+ *
+ * TODO: Allow for the record batch size to be supplied per-stream. (Needed?)
+ */
+class DefaultSpillToDiskTask(
+ private val config: DestinationConfiguration,
+ private val tmpFileProvider: TempFileProvider,
+ private val queue: QueueReader>,
+ private val flushStrategy: FlushStrategy,
+ override val stream: DestinationStream,
+ private val launcher: DestinationTaskLauncher,
+) : SpillToDiskTask {
+ private val log = KotlinLogging.logger {}
+
+ data class ReadResult(
+ val range: Range? = null,
+ val sizeBytes: Long = 0,
+ val hasReadEndOfStream: Boolean = false,
+ val forceFlush: Boolean = false,
+ )
+
+ override suspend fun execute() {
+ val tmpFile =
+ tmpFileProvider.createTempFile(
+ config.tmpFileDirectory,
+ config.firstStageTmpFilePrefix,
+ config.firstStageTmpFileSuffix
+ )
+ val result =
+ tmpFile.toFileWriter().use { writer ->
+ queue
+ .consume()
+ .runningFold(ReadResult()) { (range, sizeBytes, _), reserved ->
+ reserved.use {
+ when (val wrapped = it.value) {
+ is StreamRecordWrapped -> {
+ writer.write(wrapped.record.serialized)
+ writer.write("\n")
+ val nextRange = range.withNextAdjacentValue(wrapped.index)
+ val nextSize = sizeBytes + wrapped.sizeBytes
+ val forceFlush =
+ flushStrategy.shouldFlush(stream, nextRange, nextSize)
+ ReadResult(nextRange, nextSize, forceFlush = forceFlush)
+ }
+ is StreamCompleteWrapped -> {
+ val nextRange = range.withNextAdjacentValue(wrapped.index)
+ ReadResult(nextRange, sizeBytes, hasReadEndOfStream = true)
+ }
+ }
+ }
+ }
+ .takeUntilInclusive { it.hasReadEndOfStream || it.forceFlush }
+ .last()
+ }
+
+ /** Handle the result */
+ val (range, sizeBytes, endOfStream) = result
+
+ log.info { "Finished writing $range records (${sizeBytes}b) to $tmpFile" }
+
+ // This could happen if the chunk only contained end-of-stream
+ if (range == null) {
+ // We read 0 records, do nothing
+ return
+ }
+
+ val batch = SpilledRawMessagesLocalFile(tmpFile, sizeBytes)
+ val wrapped = BatchEnvelope(batch, range)
+ launcher.handleNewSpilledFile(stream, wrapped, endOfStream)
+ }
+}
+
+interface SpillToDiskTaskFactory {
+ fun make(taskLauncher: DestinationTaskLauncher, stream: DestinationStream): SpillToDiskTask
+}
+
+@Singleton
+class DefaultSpillToDiskTaskFactory(
+ private val config: DestinationConfiguration,
+ private val tmpFileProvider: TempFileProvider,
+ private val queueSupplier:
+ MessageQueueSupplier>,
+ private val flushStrategy: FlushStrategy,
+) : SpillToDiskTaskFactory {
+ override fun make(
+ taskLauncher: DestinationTaskLauncher,
+ stream: DestinationStream
+ ): SpillToDiskTask {
+ return DefaultSpillToDiskTask(
+ config,
+ tmpFileProvider,
+ queueSupplier.get(stream.descriptor),
+ flushStrategy,
+ stream,
+ taskLauncher,
+ )
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/TimedForcedCheckpointFlushTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/TimedForcedCheckpointFlushTask.kt
new file mode 100644
index 0000000000000..ff3053c80f4ef
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/TimedForcedCheckpointFlushTask.kt
@@ -0,0 +1,72 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.internal
+
+import io.airbyte.cdk.load.command.DestinationConfiguration
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.file.TimeProvider
+import io.airbyte.cdk.load.message.ChannelMessageQueue
+import io.airbyte.cdk.load.message.QueueWriter
+import io.airbyte.cdk.load.state.CheckpointManager
+import io.airbyte.cdk.load.task.InternalScope
+import io.airbyte.cdk.load.task.SyncLevel
+import io.airbyte.cdk.load.util.use
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+
+interface TimedForcedCheckpointFlushTask : SyncLevel, InternalScope
+
+@Singleton
+@Secondary
+class DefaultTimedForcedCheckpointFlushTask(
+ private val config: DestinationConfiguration,
+ private val checkpointManager: CheckpointManager,
+ private val eventQueue: QueueWriter,
+ private val timeProvider: TimeProvider,
+) : TimedForcedCheckpointFlushTask {
+ private val log = KotlinLogging.logger {}
+
+ override suspend fun execute() {
+ val cadenceMs = config.maxCheckpointFlushTimeMs
+ // Wait for the configured time
+ log.info { "Sleeping for ${cadenceMs}ms" }
+ timeProvider.delay(cadenceMs)
+
+ eventQueue.use {
+ while (true) {
+ // Flush whatever is handy
+ checkpointManager.flushReadyCheckpointMessages()
+
+ // Compare the time since the last successful flush to the configured interval
+ val lastFlushTimeMs = checkpointManager.getLastSuccessfulFlushTimeMs()
+ val nowMs = timeProvider.currentTimeMillis()
+ val timeSinceLastFlushMs = nowMs - lastFlushTimeMs
+
+ if (timeSinceLastFlushMs >= cadenceMs) {
+ val nextIndexes = checkpointManager.getNextCheckpointIndexes()
+ log.info {
+ "${timeSinceLastFlushMs}ms since last flush, forcing flush at $nextIndexes"
+ }
+ it.publish(ForceFlushEvent(nextIndexes))
+ timeProvider.delay(cadenceMs)
+ log.info { "Flush event published; sleeping for ${cadenceMs}ms" }
+ } else {
+ val timeUntilNextAttempt = cadenceMs - timeSinceLastFlushMs
+ log.info {
+ "$timeSinceLastFlushMs < $cadenceMs ms elapsed, sleeping for $timeUntilNextAttempt"
+ }
+ timeProvider.delay(timeUntilNextAttempt)
+ }
+ }
+ }
+ }
+}
+
+data class ForceFlushEvent(val indexes: Map)
+
+@Singleton
+@Secondary
+class DefaultForceFlushEventMessageQueue : ChannelMessageQueue()
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/UpdateCheckpointsTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/UpdateCheckpointsTask.kt
new file mode 100644
index 0000000000000..3b0bc8be50e78
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/task/internal/UpdateCheckpointsTask.kt
@@ -0,0 +1,50 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.task.internal
+
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.CheckpointMessage
+import io.airbyte.cdk.load.message.CheckpointMessageWrapped
+import io.airbyte.cdk.load.message.GlobalCheckpointWrapped
+import io.airbyte.cdk.load.message.MessageQueue
+import io.airbyte.cdk.load.message.StreamCheckpointWrapped
+import io.airbyte.cdk.load.state.CheckpointManager
+import io.airbyte.cdk.load.state.Reserved
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.task.InternalScope
+import io.airbyte.cdk.load.task.SyncLevel
+import io.github.oshai.kotlinlogging.KotlinLogging
+import io.micronaut.context.annotation.Secondary
+import jakarta.inject.Singleton
+
+interface UpdateCheckpointsTask : SyncLevel, InternalScope
+
+@Singleton
+@Secondary
+class DefaultUpdateCheckpointsTask(
+ private val syncManager: SyncManager,
+ private val checkpointManager:
+ CheckpointManager>,
+ private val checkpointMessageQueue: MessageQueue>
+) : UpdateCheckpointsTask {
+ val log = KotlinLogging.logger {}
+ override suspend fun execute() {
+ log.info { "Starting to consume checkpoint messages (state) for updating" }
+ checkpointMessageQueue.consume().collect {
+ when (it.value) {
+ is StreamCheckpointWrapped -> {
+ val (_, stream, index, message) = it.value
+ checkpointManager.addStreamCheckpoint(stream, index, it.replace(message))
+ }
+ is GlobalCheckpointWrapped -> {
+ val (_, streamIndexes, message) = it.value
+ checkpointManager.addGlobalCheckpoint(streamIndexes, it.replace(message))
+ }
+ }
+ }
+ syncManager.markCheckpointsProcessed()
+ log.info { "All checkpoints (state) updated" }
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/util/CoroutineUtils.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/util/CoroutineUtils.kt
new file mode 100644
index 0000000000000..4644b29f8868e
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/util/CoroutineUtils.kt
@@ -0,0 +1,28 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.util
+
+import java.util.concurrent.atomic.AtomicBoolean
+import kotlinx.coroutines.flow.Flow
+import kotlinx.coroutines.flow.transformWhile
+
+fun Flow.takeUntilInclusive(predicate: (T) -> Boolean): Flow = transformWhile { value ->
+ emit(value)
+ !predicate(value)
+}
+
+interface CloseableCoroutine {
+ suspend fun close()
+}
+
+suspend fun T.use(block: suspend (T) -> R) =
+ try {
+ block(this)
+ } finally {
+ close()
+ }
+
+/** Set the latch exactly once. Return true iff this is the first time we've set it. */
+fun AtomicBoolean.setOnce() = compareAndSet(false, true)
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/util/RangeUtils.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/util/RangeUtils.kt
new file mode 100644
index 0000000000000..e4d7ec1c85816
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/util/RangeUtils.kt
@@ -0,0 +1,18 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.util
+
+import com.google.common.collect.Range
+
+// Necessary because Guava's Range/sets have no "empty" range
+fun Range?.withNextAdjacentValue(index: Long): Range {
+ return if (this == null) {
+ Range.singleton(index)
+ } else if (index != this.upperEndpoint() + 1L) {
+ throw IllegalStateException("Expected index ${this.upperEndpoint() + 1}, got $index")
+ } else {
+ this.span(Range.singleton(index))
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/DestinationWriter.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/DestinationWriter.kt
similarity index 76%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/DestinationWriter.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/DestinationWriter.kt
index 5d8f9c737e59b..7ad369a3623dc 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/DestinationWriter.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/DestinationWriter.kt
@@ -2,9 +2,10 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.write
+package io.airbyte.cdk.load.write
-import io.airbyte.cdk.command.DestinationStream
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.state.SyncFailure
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
@@ -20,7 +21,8 @@ interface DestinationWriter {
fun createStreamLoader(stream: DestinationStream): StreamLoader
// Called once at the end of the job, unconditionally.
- suspend fun teardown(succeeded: Boolean = true) {}
+ // NOTE: we don't pass Success here, because it depends on this completing successfully.
+ suspend fun teardown(syncFailure: SyncFailure? = null) {}
}
@Singleton
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/StreamLoader.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/StreamLoader.kt
similarity index 80%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/StreamLoader.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/StreamLoader.kt
index 6e35c0cea99f1..d7b0581590df7 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/StreamLoader.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/StreamLoader.kt
@@ -2,12 +2,13 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.write
+package io.airbyte.cdk.load.write
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.message.Batch
-import io.airbyte.cdk.message.DestinationRecord
-import io.airbyte.cdk.message.SimpleBatch
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.message.Batch
+import io.airbyte.cdk.load.message.DestinationRecord
+import io.airbyte.cdk.load.message.SimpleBatch
+import io.airbyte.cdk.load.state.StreamIncompleteResult
/**
* Implementor interface. The framework calls open and close once per stream at the beginning and
@@ -36,7 +37,5 @@ interface StreamLoader {
suspend fun start() {}
suspend fun processRecords(records: Iterator, totalSizeBytes: Long): Batch
suspend fun processBatch(batch: Batch): Batch = SimpleBatch(Batch.State.COMPLETE)
- suspend fun finalize() {}
-
- suspend fun close(failedBatches: List = emptyList()) {}
+ suspend fun close(streamFailure: StreamIncompleteResult? = null) {}
}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/WriteOperation.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/WriteOperation.kt
similarity index 52%
rename from airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/WriteOperation.kt
rename to airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/WriteOperation.kt
index 5c17e796004c5..6955da322fb1e 100644
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/WriteOperation.kt
+++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/write/WriteOperation.kt
@@ -2,18 +2,19 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.write
+package io.airbyte.cdk.load.write
import io.airbyte.cdk.Operation
-import io.airbyte.cdk.message.DestinationMessage
-import io.airbyte.cdk.task.TaskLauncher
-import io.airbyte.cdk.task.TaskRunner
+import io.airbyte.cdk.load.state.SyncFailure
+import io.airbyte.cdk.load.state.SyncManager
+import io.airbyte.cdk.load.state.SyncSuccess
+import io.airbyte.cdk.load.task.TaskLauncher
+import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Requires
import io.micronaut.context.annotation.Secondary
import java.io.InputStream
import javax.inject.Singleton
-import kotlinx.coroutines.launch
import kotlinx.coroutines.runBlocking
/**
@@ -23,24 +24,29 @@ import kotlinx.coroutines.runBlocking
@Singleton
@Requires(property = Operation.PROPERTY, value = "write")
class WriteOperation(
- private val inputConsumer: InputConsumer,
private val taskLauncher: TaskLauncher,
- private val taskRunner: TaskRunner,
+ private val syncManager: SyncManager,
) : Operation {
- override fun execute() {
- runBlocking {
- launch { inputConsumer.run() }
+ val log = KotlinLogging.logger {}
- launch { taskLauncher.start() }
+ override fun execute() = runBlocking {
+ taskLauncher.start()
- launch { taskRunner.run() }
+ when (val result = syncManager.awaitSyncResult()) {
+ is SyncSuccess -> {
+ log.info { "Sync completed successfully" }
+ }
+ is SyncFailure -> {
+ log.info { "Sync failed with stream results ${result.streamResults}" }
+ throw result.syncFailure
+ }
}
}
}
/** Override to provide a custom input stream. */
@Factory
-class InputStreamFactory {
+class InputStreamProvider {
@Singleton
@Secondary
@Requires(property = Operation.PROPERTY, value = "write")
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessageQueue.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessageQueue.kt
deleted file mode 100644
index 694d1d29e1bcc..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/DestinationMessageQueue.kt
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.message
-
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationConfiguration
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.state.MemoryManager
-import io.github.oshai.kotlinlogging.KotlinLogging
-import jakarta.inject.Singleton
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.atomic.AtomicLong
-import java.util.concurrent.locks.ReentrantLock
-import kotlin.concurrent.withLock
-import kotlinx.coroutines.runBlocking
-
-/**
- * Wrapper for record messages published to the message queue, containing metadata like index and
- * size.
- *
- * In a future where we deserialize only the info necessary for routing, this could include a dumb
- * container for the serialized, and deserialization could be deferred until the spooled records
- * were recovered from disk.
- */
-sealed class DestinationRecordWrapped : Sized
-
-data class StreamRecordWrapped(
- val index: Long,
- override val sizeBytes: Long,
- val record: DestinationRecord
-) : DestinationRecordWrapped()
-
-data class StreamCompleteWrapped(
- val index: Long,
-) : DestinationRecordWrapped() {
- override val sizeBytes: Long = 0L
-}
-
-/**
- * Message queue to which @[DestinationRecordWrapped] messages can be published on a @
- * [DestinationStream] key.
- *
- * It maintains a map of @[QueueChannel]s by stream, and tracks the memory usage across all
- * channels, blocking when the maximum is reached.
- *
- * This maximum is expected to be low, as the assumption is that data will be spooled to disk as
- * quickly as possible.
- */
-@Singleton
-class DestinationMessageQueue(
- catalog: DestinationCatalog,
- config: DestinationConfiguration,
- private val memoryManager: MemoryManager,
- private val queueChannelFactory: QueueChannelFactory
-) : MessageQueue {
- private val channels:
- ConcurrentHashMap> =
- ConcurrentHashMap()
-
- private val totalQueueSizeBytes = AtomicLong(0L)
- private val maxQueueSizeBytes: Long
- private val memoryLock = ReentrantLock()
- private val memoryLockCondition = memoryLock.newCondition()
-
- init {
- catalog.streams.forEach { channels[it.descriptor] = queueChannelFactory.make(this) }
- val adjustedRatio =
- config.maxMessageQueueMemoryUsageRatio /
- (1.0 + config.estimatedRecordMemoryOverheadRatio)
- maxQueueSizeBytes = runBlocking { memoryManager.reserveRatio(adjustedRatio) }
- }
-
- override suspend fun acquireQueueBytesBlocking(bytes: Long) {
- memoryLock.withLock {
- while (totalQueueSizeBytes.get() + bytes > maxQueueSizeBytes) {
- memoryLockCondition.await()
- }
- totalQueueSizeBytes.addAndGet(bytes)
- }
- }
-
- override suspend fun releaseQueueBytes(bytes: Long) {
- memoryLock.withLock {
- totalQueueSizeBytes.addAndGet(-bytes)
- memoryLockCondition.signalAll()
- }
- }
-
- override suspend fun getChannel(
- key: DestinationStream.Descriptor,
- ): QueueChannel {
- return channels[key]
- ?: throw IllegalArgumentException("Reading from non-existent QueueChannel: ${key}")
- }
-
- private val log = KotlinLogging.logger {}
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageConverter.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageConverter.kt
deleted file mode 100644
index 5ba878606ea30..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageConverter.kt
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.message
-
-import io.airbyte.protocol.models.v0.AirbyteGlobalState
-import io.airbyte.protocol.models.v0.AirbyteMessage
-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 jakarta.inject.Singleton
-
-/**
- * Converts the internal @[CheckpointMessage] case class to the Protocol state messages required by
- * @[io.airbyte.cdk.output.OutputConsumer]
- */
-interface MessageConverter {
- fun from(message: T): U
-}
-
-@Singleton
-class DefaultMessageConverter : MessageConverter {
- override fun from(message: CheckpointMessage): AirbyteMessage {
- val state =
- when (message) {
- is StreamCheckpoint ->
- AirbyteStateMessage()
- .also {
- if (message.sourceStats != null) {
- it.sourceStats =
- AirbyteStateStats()
- .withRecordCount(
- message.sourceStats!!.recordCount.toDouble()
- )
- }
- }
- .withDestinationStats(
- message.destinationStats?.let {
- AirbyteStateStats().withRecordCount(it.recordCount.toDouble())
- }
- ?: throw IllegalStateException(
- "Destination stats must be provided for DestinationStreamState"
- )
- )
- .withType(AirbyteStateMessage.AirbyteStateType.STREAM)
- .withStream(fromStreamState(message.checkpoint))
- .also {
- message.additionalProperties.forEach { (key, value) ->
- it.withAdditionalProperty(key, value)
- }
- }
- is GlobalCheckpoint ->
- AirbyteStateMessage()
- .also {
- if (message.sourceStats != null) {
- it.sourceStats =
- AirbyteStateStats()
- .withRecordCount(
- message.sourceStats!!.recordCount.toDouble()
- )
- }
- }
- .withDestinationStats(
- message.destinationStats?.let {
- AirbyteStateStats().withRecordCount(it.recordCount.toDouble())
- }
- )
- .withType(AirbyteStateMessage.AirbyteStateType.GLOBAL)
- .withGlobal(
- AirbyteGlobalState()
- .withSharedState(message.state)
- .withStreamStates(message.checkpoints.map { fromStreamState(it) })
- )
- .also {
- message.additionalProperties.forEach { (key, value) ->
- it.withAdditionalProperty(key, value)
- }
- }
- }
- return AirbyteMessage().withType(AirbyteMessage.Type.STATE).withState(state)
- }
-
- private fun fromStreamState(checkpoint: CheckpointMessage.Checkpoint): AirbyteStreamState {
- return AirbyteStreamState()
- .withStreamDescriptor(
- StreamDescriptor()
- .withNamespace(checkpoint.stream.namespace)
- .withName(checkpoint.stream.name)
- )
- .withStreamState(checkpoint.state)
- }
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueue.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueue.kt
deleted file mode 100644
index 8d392f8afa3d5..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueue.kt
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.message
-
-import jakarta.inject.Singleton
-import java.util.concurrent.atomic.AtomicBoolean
-import kotlinx.coroutines.channels.Channel
-
-interface Sized {
- val sizeBytes: Long
-}
-
-interface MessageQueue {
- suspend fun acquireQueueBytesBlocking(bytes: Long)
- suspend fun releaseQueueBytes(bytes: Long)
- suspend fun getChannel(key: K): QueueChannel
-}
-
-interface QueueChannel {
- suspend fun close()
- suspend fun isClosed(): Boolean
- suspend fun send(message: T)
- suspend fun receive(): T
-}
-
-/** A channel that blocks when its parent queue has no available memory. */
-interface BlockingQueueChannel : QueueChannel {
- val messageQueue: MessageQueue<*, T>
- val channel: Channel
-
- override suspend fun send(message: T) {
- if (isClosed()) {
- throw IllegalStateException("Send to closed QueueChannel")
- }
- val estimatedSize = message.sizeBytes
- messageQueue.acquireQueueBytesBlocking(estimatedSize)
- channel.send(message)
- }
-
- override suspend fun receive(): T {
- if (isClosed()) {
- throw IllegalStateException("Receive from closed QueueChannel")
- }
- val message = channel.receive()
- val estimatedSize = message.sizeBytes
- messageQueue.releaseQueueBytes(estimatedSize)
- return message
- }
-}
-
-interface QueueChannelFactory {
- fun make(messageQueue: MessageQueue<*, T>): QueueChannel
-}
-
-/**
- * The default queue channel is just a dumb wrapper around an unlimited kotlin channel of wrapped
- * records.
- *
- * Note: we wrap channel closedness in an atomic boolean because the @[Channel.isClosedForSend] and
- * @[Channel.isClosedForReceive] apis are marked as delicate/experimental.
- */
-class DefaultQueueChannel(override val messageQueue: MessageQueue<*, DestinationRecordWrapped>) :
- BlockingQueueChannel {
- override val channel = Channel(Channel.UNLIMITED)
- private val closed = AtomicBoolean(false)
-
- override suspend fun close() {
- if (closed.compareAndSet(false, true)) {
- channel.close()
- }
- }
-
- override suspend fun isClosed(): Boolean = closed.get()
-}
-
-@Singleton
-class DefaultQueueChannelFactory : QueueChannelFactory {
- override fun make(
- messageQueue: MessageQueue<*, DestinationRecordWrapped>
- ): QueueChannel = DefaultQueueChannel(messageQueue)
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueueReader.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueueReader.kt
deleted file mode 100644
index 4f8c40a9f4da6..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueueReader.kt
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.message
-
-import io.airbyte.cdk.command.DestinationStream
-import io.github.oshai.kotlinlogging.KotlinLogging
-import io.micronaut.context.annotation.Secondary
-import jakarta.inject.Singleton
-import kotlinx.coroutines.flow.Flow
-import kotlinx.coroutines.flow.flow
-
-/**
- * A reader should provide a byte-limited flow of messages of the underlying type. The flow should
- * terminate when maxBytes has been read, or when the stream is complete.
- */
-interface MessageQueueReader {
- suspend fun readChunk(key: K, maxBytes: Long): Flow
-}
-
-@Singleton
-@Secondary
-class DestinationMessageQueueReader(
- private val messageQueue: DestinationMessageQueue,
-) : MessageQueueReader {
- private val log = KotlinLogging.logger {}
-
- override suspend fun readChunk(
- key: DestinationStream.Descriptor,
- maxBytes: Long
- ): Flow = flow {
- log.info { "Reading chunk of $maxBytes bytes from stream $key" }
-
- var totalBytesRead = 0L
- var recordsRead = 0L
- while (totalBytesRead < maxBytes) {
- when (val wrapped = messageQueue.getChannel(key).receive()) {
- is StreamRecordWrapped -> {
- totalBytesRead += wrapped.sizeBytes
- emit(wrapped)
- }
- is StreamCompleteWrapped -> {
- messageQueue.getChannel(key).close()
- emit(wrapped)
- log.info { "Read end-of-stream for $key" }
- return@flow
- }
- }
- recordsRead++
- }
-
- log.info { "Read $recordsRead records (${totalBytesRead}b) from stream $key" }
-
- return@flow
- }
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueueWriter.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueueWriter.kt
deleted file mode 100644
index 9a8202d0fc830..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/message/MessageQueueWriter.kt
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.message
-
-import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.state.CheckpointManager
-import io.airbyte.cdk.state.SyncManager
-import jakarta.inject.Singleton
-
-/** A publishing interface for writing messages to a message queue. */
-interface MessageQueueWriter {
- suspend fun publish(message: T, sizeBytes: Long)
-}
-
-/**
- * Routes @[DestinationStreamAffinedMessage]s by stream to the appropriate channel and @
- * [CheckpointMessage]s to the state manager.
- *
- * TODO: Handle other message types.
- */
-@SuppressFBWarnings(
- "NP_NONNULL_PARAM_VIOLATION",
- justification = "message is guaranteed to be non-null by Kotlin's type system"
-)
-@Singleton
-class DestinationMessageQueueWriter(
- private val catalog: DestinationCatalog,
- private val messageQueue: MessageQueue,
- private val syncManager: SyncManager,
- private val checkpointManager:
- CheckpointManager
-) : MessageQueueWriter {
- /**
- * Deserialize and route the message to the appropriate channel.
- *
- * NOTE: Not thread-safe! Only a single writer should publish to the queue.
- */
- override suspend fun publish(message: DestinationMessage, sizeBytes: Long) {
- when (message) {
- /* If the input message represents a record. */
- is DestinationStreamAffinedMessage -> {
- val manager = syncManager.getStreamManager(message.stream)
- when (message) {
- /* If a data record */
- is DestinationRecord -> {
- val wrapped =
- StreamRecordWrapped(
- index = manager.countRecordIn(),
- sizeBytes = sizeBytes,
- record = message
- )
- messageQueue.getChannel(message.stream).send(wrapped)
- }
-
- /* If an end-of-stream marker. */
- is DestinationStreamComplete,
- is DestinationStreamIncomplete -> {
- val wrapped = StreamCompleteWrapped(index = manager.countEndOfStream())
- messageQueue.getChannel(message.stream).send(wrapped)
- }
- }
- }
- is CheckpointMessage -> {
- when (message) {
- /**
- * For a stream state message, mark the checkpoint and add the message with
- * index and count to the state manager. Also, add the count to the destination
- * stats.
- */
- is StreamCheckpoint -> {
- val stream = message.checkpoint.stream
- val manager = syncManager.getStreamManager(stream)
- val (currentIndex, countSinceLast) = manager.markCheckpoint()
- val messageWithCount =
- message.withDestinationStats(CheckpointMessage.Stats(countSinceLast))
- checkpointManager.addStreamCheckpoint(
- stream,
- currentIndex,
- messageWithCount
- )
- }
- /**
- * For a global state message, collect the index per stream, but add the total
- * count to the destination stats.
- */
- is GlobalCheckpoint -> {
- val streamWithIndexAndCount =
- catalog.streams.map { stream ->
- val manager = syncManager.getStreamManager(stream.descriptor)
- val (currentIndex, countSinceLast) = manager.markCheckpoint()
- Triple(stream, currentIndex, countSinceLast)
- }
- val totalCount = streamWithIndexAndCount.sumOf { it.third }
- val messageWithCount =
- message.withDestinationStats(CheckpointMessage.Stats(totalCount))
- val streamIndexes =
- streamWithIndexAndCount.map { it.first.descriptor to it.second }
- checkpointManager.addGlobalCheckpoint(streamIndexes, messageWithCount)
- }
- }
- }
- is Undefined -> {} // Do nothing
- }
- }
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/CheckpointManager.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/CheckpointManager.kt
deleted file mode 100644
index a038317ae4ea0..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/state/CheckpointManager.kt
+++ /dev/null
@@ -1,183 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.state
-
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.message.CheckpointMessage
-import io.airbyte.cdk.message.MessageConverter
-import io.airbyte.protocol.models.v0.AirbyteMessage
-import io.github.oshai.kotlinlogging.KotlinLogging
-import io.micronaut.context.annotation.Secondary
-import io.micronaut.core.util.clhm.ConcurrentLinkedHashMap
-import jakarta.inject.Singleton
-import java.util.concurrent.ConcurrentHashMap
-import java.util.concurrent.ConcurrentLinkedQueue
-import java.util.concurrent.atomic.AtomicReference
-import java.util.function.Consumer
-
-/**
- * Interface for checkpoint management. Should accept stream and global checkpoints, as well as
- * requests to flush all data-sufficient checkpoints.
- */
-interface CheckpointManager {
- fun addStreamCheckpoint(key: K, index: Long, checkpointMessage: T)
- fun addGlobalCheckpoint(keyIndexes: List>, checkpointMessage: T)
- suspend fun flushReadyCheckpointMessages()
-}
-
-/**
- * Message-type agnostic streams checkpoint manager.
- *
- * Accepts global and stream checkpoints, and enforces that stream and global checkpoints are not
- * mixed. Determines ready checkpoints by querying the StreamsManager for the checkpoint of the
- * record index range associated with each checkpoint message.
- *
- * TODO: Force flush on a configured schedule
- *
- * TODO: Ensure that checkpoint is flushed at the end, and require that all checkpoints be flushed
- * before the destination can succeed.
- */
-abstract class StreamsCheckpointManager() :
- CheckpointManager {
- private val log = KotlinLogging.logger {}
-
- abstract val catalog: DestinationCatalog
- abstract val syncManager: SyncManager
- abstract val outputFactory: MessageConverter
- abstract val outputConsumer: Consumer
-
- data class GlobalCheckpoint(
- val streamIndexes: List>,
- val checkpointMessage: T
- )
-
- private val checkpointsAreGlobal: AtomicReference = AtomicReference(null)
- private val streamCheckpoints:
- ConcurrentHashMap> =
- ConcurrentHashMap()
- private val globalCheckpoints: ConcurrentLinkedQueue> =
- ConcurrentLinkedQueue()
-
- override fun addStreamCheckpoint(
- key: DestinationStream.Descriptor,
- index: Long,
- checkpointMessage: T
- ) {
- if (checkpointsAreGlobal.updateAndGet { it == true } != false) {
- throw IllegalStateException(
- "Global checkpoints cannot be mixed with non-global checkpoints"
- )
- }
-
- streamCheckpoints.compute(key) { _, indexToMessage ->
- val map =
- if (indexToMessage == null) {
- // If the map doesn't exist yet, build it.
- ConcurrentLinkedHashMap.Builder().maximumWeightedCapacity(1000).build()
- } else {
- if (indexToMessage.isNotEmpty()) {
- // Make sure the messages are coming in order
- val oldestIndex = indexToMessage.ascendingKeySet().first()
- if (oldestIndex > index) {
- throw IllegalStateException(
- "Checkpoint message received out of order ($oldestIndex before $index)"
- )
- }
- }
- indexToMessage
- }
- // Actually add the message
- map[index] = checkpointMessage
- map
- }
-
- log.info { "Added checkpoint for stream: $key at index: $index" }
- }
-
- // TODO: Is it an error if we don't get all the streams every time?
- override fun addGlobalCheckpoint(
- keyIndexes: List>,
- checkpointMessage: T
- ) {
- if (checkpointsAreGlobal.updateAndGet { it != false } != true) {
- throw IllegalStateException(
- "Global checkpoint cannot be mixed with non-global checkpoints"
- )
- }
-
- val head = globalCheckpoints.peek()
- if (head != null) {
- val keyIndexesByStream = keyIndexes.associate { it.first to it.second }
- head.streamIndexes.forEach {
- if (keyIndexesByStream[it.first]!! < it.second) {
- throw IllegalStateException("Global checkpoint message received out of order")
- }
- }
- }
-
- globalCheckpoints.add(GlobalCheckpoint(keyIndexes, checkpointMessage))
- log.info { "Added global checkpoint with stream indexes: $keyIndexes" }
- }
-
- override suspend fun flushReadyCheckpointMessages() {
- /*
- Iterate over the checkpoints in order, evicting each that passes
- the persistence check. If a checkpoint is not persisted, then
- we can break the loop since the checkpoints are ordered. For global
- checkpoints, all streams must be persisted up to the checkpoint.
- */
- when (checkpointsAreGlobal.get()) {
- null -> log.info { "No checkpoints to flush" }
- true -> flushGlobalCheckpoints()
- false -> flushStreamCheckpoints()
- }
- }
-
- private fun flushGlobalCheckpoints() {
- while (!globalCheckpoints.isEmpty()) {
- val head = globalCheckpoints.peek()
- val allStreamsPersisted =
- head.streamIndexes.all { (stream, index) ->
- syncManager.getStreamManager(stream).areRecordsPersistedUntil(index)
- }
- if (allStreamsPersisted) {
- globalCheckpoints.poll()
- val outMessage = outputFactory.from(head.checkpointMessage)
- outputConsumer.accept(outMessage)
- } else {
- break
- }
- }
- }
-
- private fun flushStreamCheckpoints() {
- for (stream in catalog.streams) {
- val manager = syncManager.getStreamManager(stream.descriptor)
- val streamCheckpoints = streamCheckpoints[stream.descriptor] ?: return
- for (index in streamCheckpoints.keys) {
- if (manager.areRecordsPersistedUntil(index)) {
- val checkpointMessage =
- streamCheckpoints.remove(index)
- ?: throw IllegalStateException("Checkpoint not found for index: $index")
- log.info { "Flushing checkpoint for stream: $stream at index: $index" }
- val outMessage = outputFactory.from(checkpointMessage)
- outputConsumer.accept(outMessage)
- } else {
- break
- }
- }
- }
- }
-}
-
-@Singleton
-@Secondary
-class DefaultCheckpointManager(
- override val catalog: DestinationCatalog,
- override val syncManager: SyncManager,
- override val outputFactory: MessageConverter,
- override val outputConsumer: Consumer
-) : StreamsCheckpointManager()
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/SpillToDiskTask.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/SpillToDiskTask.kt
deleted file mode 100644
index acc8ecbac7ca5..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/SpillToDiskTask.kt
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.task
-
-import com.google.common.collect.Range
-import io.airbyte.cdk.command.DestinationConfiguration
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.file.TempFileProvider
-import io.airbyte.cdk.message.BatchEnvelope
-import io.airbyte.cdk.message.DestinationRecordWrapped
-import io.airbyte.cdk.message.MessageQueueReader
-import io.airbyte.cdk.message.SpilledRawMessagesLocalFile
-import io.airbyte.cdk.message.StreamCompleteWrapped
-import io.airbyte.cdk.message.StreamRecordWrapped
-import io.github.oshai.kotlinlogging.KotlinLogging
-import jakarta.inject.Singleton
-import kotlinx.coroutines.Dispatchers
-import kotlinx.coroutines.flow.flowOn
-import kotlinx.coroutines.flow.runningFold
-import kotlinx.coroutines.flow.toList
-import kotlinx.coroutines.withContext
-import kotlinx.coroutines.yield
-
-interface SpillToDiskTask : StreamTask
-
-/**
- * Reads records from the message queue and writes them to disk. This task is internal and is not
- * exposed to the implementor.
- *
- * TODO: Allow for the record batch size to be supplied per-stream. (Needed?)
- */
-class DefaultSpillToDiskTask(
- private val config: DestinationConfiguration,
- private val tmpFileProvider: TempFileProvider,
- private val queueReader:
- MessageQueueReader,
- private val stream: DestinationStream,
- private val launcher: DestinationTaskLauncher
-) : SpillToDiskTask {
- private val log = KotlinLogging.logger {}
-
- data class ReadResult(
- val range: Range? = null,
- val sizeBytes: Long = 0,
- val hasReadEndOfStream: Boolean = false,
- )
-
- // Necessary because Guava's Range/sets have no "empty" range
- private fun withIndex(range: Range?, index: Long): Range {
- return if (range == null) {
- Range.singleton(index)
- } else if (index != range.upperEndpoint() + 1) {
- throw IllegalStateException("Expected index ${range.upperEndpoint() + 1}, got $index")
- } else {
- range.span(Range.singleton(index))
- }
- }
-
- override suspend fun execute() {
- do {
- val (path, result) =
- withContext(Dispatchers.IO) {
- val tmpFile =
- tmpFileProvider.createTempFile(
- config.tmpFileDirectory,
- config.firstStageTmpFilePrefix,
- config.firstStageTmpFileSuffix
- )
- val result =
- tmpFile.toFileWriter().use {
- queueReader
- .readChunk(stream.descriptor, config.recordBatchSizeBytes)
- .runningFold(ReadResult()) { (range, sizeBytes, _), wrapped ->
- when (wrapped) {
- is StreamRecordWrapped -> {
- val nextRange = withIndex(range, wrapped.index)
- it.write(wrapped.record.serialized)
- it.write("\n")
- ReadResult(nextRange, sizeBytes + wrapped.sizeBytes)
- }
- is StreamCompleteWrapped -> {
- val nextRange = withIndex(range, wrapped.index)
- return@runningFold ReadResult(
- nextRange,
- sizeBytes,
- true
- )
- }
- }
- }
- .flowOn(Dispatchers.IO)
- .toList()
- }
- Pair(tmpFile, result.last())
- }
-
- /** Handle the result */
- val (range, sizeBytes, endOfStream) = result
-
- log.info { "Finished writing $range records (${sizeBytes}b) to $path" }
-
- // This could happen if the chunk only contained end-of-stream
- if (range == null) {
- // We read 0 records, do nothing
- return
- }
-
- val batch = SpilledRawMessagesLocalFile(path, sizeBytes)
- val wrapped = BatchEnvelope(batch, range)
- launcher.handleNewSpilledFile(stream, wrapped)
-
- yield()
- } while (!endOfStream)
- }
-}
-
-interface SpillToDiskTaskFactory {
- fun make(taskLauncher: DestinationTaskLauncher, stream: DestinationStream): SpillToDiskTask
-}
-
-@Singleton
-class DefaultSpillToDiskTaskFactory(
- private val config: DestinationConfiguration,
- private val tmpFileProvider: TempFileProvider,
- private val queueReader:
- MessageQueueReader
-) : SpillToDiskTaskFactory {
- override fun make(
- taskLauncher: DestinationTaskLauncher,
- stream: DestinationStream
- ): SpillToDiskTask {
- return DefaultSpillToDiskTask(config, tmpFileProvider, queueReader, stream, taskLauncher)
- }
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TaskLauncher.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TaskLauncher.kt
deleted file mode 100644
index f919a9c67e886..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TaskLauncher.kt
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.task
-
-interface Task {
- suspend fun execute()
-}
-
-/**
- * A TaskLauncher is responsible for starting and stopping the task workflow, and for managing
- * transitions between tasks.
- */
-interface TaskLauncher {
- val taskRunner: TaskRunner
-
- suspend fun start()
- suspend fun stop() {
- taskRunner.close()
- }
-}
-
-interface TaskLauncherExceptionHandler {
- fun withExceptionHandling(task: T): Task
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TaskRunner.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TaskRunner.kt
deleted file mode 100644
index 5c93a1447085d..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/task/TaskRunner.kt
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.task
-
-import io.github.oshai.kotlinlogging.KotlinLogging
-import jakarta.inject.Singleton
-import kotlinx.coroutines.channels.Channel
-import kotlinx.coroutines.coroutineScope
-import kotlinx.coroutines.flow.consumeAsFlow
-import kotlinx.coroutines.launch
-
-/**
- * A Task is a unit of work that can be executed concurrently. Even though we aren't scheduling
- * threads or enforcing concurrency limits here, launching tasks from a queue in a dedicated scope
- * frees the caller not to have to await completion.
- *
- * TODO: Extend this to collect and report task completion.
- *
- * TODO: Set concurrency for this scope from the configuration.
- */
-@Singleton
-class TaskRunner {
- val log = KotlinLogging.logger {}
-
- private val queue = Channel(Channel.UNLIMITED)
-
- suspend fun enqueue(task: Task) {
- queue.send(task)
- }
-
- suspend fun run() = coroutineScope {
- queue.consumeAsFlow().collect { task ->
- launch {
- log.info { "Executing task: $task" }
- task.execute()
- }
- }
- }
-
- fun close() {
- queue.close()
- }
-}
diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/InputConsumer.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/InputConsumer.kt
deleted file mode 100644
index 898d9889692ff..0000000000000
--- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/write/InputConsumer.kt
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
- */
-
-package io.airbyte.cdk.write
-
-import io.airbyte.cdk.message.Deserializer
-import io.airbyte.cdk.message.DestinationMessage
-import io.airbyte.cdk.message.MessageQueueWriter
-import io.github.oshai.kotlinlogging.KLogger
-import io.github.oshai.kotlinlogging.KotlinLogging
-import jakarta.inject.Singleton
-import java.io.InputStream
-import java.nio.charset.StandardCharsets
-import kotlinx.coroutines.Dispatchers
-import kotlinx.coroutines.withContext
-
-/** Runnable input consumer. */
-interface InputConsumer {
- suspend fun run()
-}
-
-/** Input consumer that deserializes and publishes to a queue. */
-interface DeserializingInputStreamConsumer : InputConsumer {
- val log: KLogger
- val inputStream: InputStream
- val deserializer: Deserializer
- val messageQueue: MessageQueueWriter
-
- override suspend fun run() =
- withContext(Dispatchers.IO) {
- val log = KotlinLogging.logger {}
-
- log.info { "Starting consuming messages from the input stream" }
-
- var index = 0L
- var bytes = 0L
- inputStream.bufferedReader(StandardCharsets.UTF_8).lineSequence().forEach { line ->
- val lineSize = line.length.toLong()
- if (lineSize > 0L) {
- val deserialized = deserializer.deserialize(line)
- messageQueue.publish(deserialized, lineSize)
-
- bytes += lineSize
- if (++index % 10_000L == 0L) {
- log.info {
- "Consumed $index messages (${bytes / 1024L}mb) from the input stream"
- }
- }
- }
- }
-
- log.info { "Finished consuming $index messages (${bytes}b) from the input stream" }
- }
-}
-
-@Singleton
-class DefaultInputConsumer(
- override val inputStream: InputStream,
- override val deserializer: Deserializer,
- override val messageQueue: MessageQueueWriter
-) : DeserializingInputStreamConsumer {
- override val log = KotlinLogging.logger {}
-}
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/command/DestinationCatalogTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/DestinationCatalogTest.kt
similarity index 98%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/command/DestinationCatalogTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/DestinationCatalogTest.kt
index d6db2d9601c94..c21f0775ffe0c 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/command/DestinationCatalogTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/DestinationCatalogTest.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.command
+package io.airbyte.cdk.load.command
import io.airbyte.protocol.models.Jsons
import io.airbyte.protocol.models.v0.AirbyteStream
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/command/MockDestinationCatalogFactory.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/MockDestinationCatalogFactory.kt
similarity index 92%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/command/MockDestinationCatalogFactory.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/MockDestinationCatalogFactory.kt
index bb57923822c63..7076f46b81dff 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/command/MockDestinationCatalogFactory.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/MockDestinationCatalogFactory.kt
@@ -2,12 +2,12 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.command
+package io.airbyte.cdk.load.command
-import io.airbyte.cdk.data.FieldType
-import io.airbyte.cdk.data.IntegerType
-import io.airbyte.cdk.data.ObjectType
-import io.airbyte.cdk.data.StringType
+import io.airbyte.cdk.load.data.FieldType
+import io.airbyte.cdk.load.data.IntegerType
+import io.airbyte.cdk.load.data.ObjectType
+import io.airbyte.cdk.load.data.StringType
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Primary
import io.micronaut.context.annotation.Requires
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/MockDestinationConfiguration.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/MockDestinationConfiguration.kt
new file mode 100644
index 0000000000000..a6e0c39c2349c
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/command/MockDestinationConfiguration.kt
@@ -0,0 +1,24 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.command
+
+import io.micronaut.context.annotation.Primary
+import io.micronaut.context.annotation.Requires
+import jakarta.inject.Singleton
+import java.nio.file.Path
+
+@Singleton
+@Primary
+@Requires(env = ["MockDestinationConfiguration"])
+class MockDestinationConfiguration : DestinationConfiguration() {
+ override val recordBatchSizeBytes: Long = 1024L
+ override val tmpFileDirectory: Path = Path.of("/tmp-test")
+ override val firstStageTmpFilePrefix: String = "spilled"
+ override val firstStageTmpFileSuffix: String = ".jsonl"
+
+ override val maxCheckpointFlushTimeMs: Long = 1000L
+ override val maxMessageQueueMemoryUsageRatio: Double = 0.1
+ override val estimatedRecordMemoryOverheadRatio: Double = 1.1
+}
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/AirbyteTypeToJsonSchemaTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/AirbyteSchemaTypeToJsonSchemaTest.kt
similarity index 98%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/AirbyteTypeToJsonSchemaTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/AirbyteSchemaTypeToJsonSchemaTest.kt
index 5005755fcda7b..52011496f2ba8 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/AirbyteTypeToJsonSchemaTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/AirbyteSchemaTypeToJsonSchemaTest.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.node.JsonNodeFactory
@@ -10,7 +10,7 @@ import com.fasterxml.jackson.databind.node.ObjectNode
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test
-class AirbyteTypeToJsonSchemaTest {
+class AirbyteSchemaTypeToJsonSchemaTest {
@Test
fun testRoundTrip() {
val schema = JsonNodeFactory.instance.objectNode()
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/AirbyteValueToJsonTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/AirbyteValueToJsonTest.kt
similarity index 99%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/AirbyteValueToJsonTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/AirbyteValueToJsonTest.kt
index dedc0997a8d55..e74922f2430f2 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/AirbyteValueToJsonTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/AirbyteValueToJsonTest.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/JsonSchemaToAirbyteTypeTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/JsonSchemaToAirbyteSchemaTypeTest.kt
similarity index 99%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/JsonSchemaToAirbyteTypeTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/JsonSchemaToAirbyteSchemaTypeTest.kt
index bea239ed17f7a..24847b151e0c6 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/JsonSchemaToAirbyteTypeTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/JsonSchemaToAirbyteSchemaTypeTest.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.node.JsonNodeFactory
import com.fasterxml.jackson.databind.node.ObjectNode
@@ -10,7 +10,7 @@ import io.airbyte.cdk.util.Jsons
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test
-class JsonSchemaToAirbyteTypeTest {
+class JsonSchemaToAirbyteSchemaTypeTest {
private fun ofType(type: String): ObjectNode {
return JsonNodeFactory.instance.objectNode().put("type", type)
}
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/JsonToAirbyteValueTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/JsonToAirbyteValueTest.kt
similarity index 99%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/JsonToAirbyteValueTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/JsonToAirbyteValueTest.kt
index 38b3259652b62..225b8d12720bf 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/data/JsonToAirbyteValueTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/JsonToAirbyteValueTest.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.data
+package io.airbyte.cdk.load.data
import com.fasterxml.jackson.databind.node.JsonNodeFactory
import java.math.BigDecimal
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/file/MockTempFileProvider.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/file/MockTempFileProvider.kt
similarity index 98%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/file/MockTempFileProvider.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/file/MockTempFileProvider.kt
index ff844ec767075..093e160f3124a 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/file/MockTempFileProvider.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/file/MockTempFileProvider.kt
@@ -2,7 +2,7 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.file
+package io.airbyte.cdk.load.file
import io.micronaut.context.annotation.Requires
import jakarta.inject.Singleton
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/file/MockTimeProvider.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/file/MockTimeProvider.kt
new file mode 100644
index 0000000000000..7909f2bfdf24e
--- /dev/null
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/file/MockTimeProvider.kt
@@ -0,0 +1,29 @@
+/*
+ * Copyright (c) 2024 Airbyte, Inc., all rights reserved.
+ */
+
+package io.airbyte.cdk.load.file
+
+import io.micronaut.context.annotation.Primary
+import io.micronaut.context.annotation.Requires
+import jakarta.inject.Singleton
+import java.util.concurrent.atomic.AtomicLong
+
+@Singleton
+@Primary
+@Requires(env = ["MockTimeProvider"])
+class MockTimeProvider : TimeProvider {
+ private var currentTime = AtomicLong(0)
+
+ override fun currentTimeMillis(): Long {
+ return currentTime.get()
+ }
+
+ fun setCurrentTime(currentTime: Long) {
+ this.currentTime.set(currentTime)
+ }
+
+ override suspend fun delay(ms: Long) {
+ currentTime.addAndGet(ms)
+ }
+}
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/message/DestinationMessageTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/message/DestinationMessageTest.kt
similarity index 95%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/message/DestinationMessageTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/message/DestinationMessageTest.kt
index c7ba1edb0009c..10b0ea1d89f61 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/message/DestinationMessageTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/message/DestinationMessageTest.kt
@@ -2,12 +2,12 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.message
+package io.airbyte.cdk.load.message
-import io.airbyte.cdk.command.Append
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.data.ObjectTypeWithEmptySchema
+import io.airbyte.cdk.load.command.Append
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.data.ObjectTypeWithEmptySchema
import io.airbyte.protocol.models.Jsons
import io.airbyte.protocol.models.v0.AirbyteGlobalState
import io.airbyte.protocol.models.v0.AirbyteMessage
@@ -19,7 +19,7 @@ import io.airbyte.protocol.models.v0.AirbyteStateStats
import io.airbyte.protocol.models.v0.AirbyteStreamState
import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage
import io.airbyte.protocol.models.v0.AirbyteTraceMessage
-import org.junit.jupiter.api.Assertions.assertEquals
+import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.Arguments
@@ -47,7 +47,7 @@ class DestinationMessageTest {
fun testRoundTrip(message: AirbyteMessage) {
val roundTripped =
factory.fromAirbyteMessage(message, Jsons.serialize(message)).asProtocolMessage()
- assertEquals(message, roundTripped)
+ Assertions.assertEquals(message, roundTripped)
}
// Checkpoint messages aren't round-trippable.
@@ -73,7 +73,7 @@ class DestinationMessageTest {
factory.fromAirbyteMessage(inputMessage, Jsons.serialize(inputMessage))
as StreamCheckpoint
- assertEquals(
+ Assertions.assertEquals(
inputMessage.also {
it.state.destinationStats = AirbyteStateStats().withRecordCount(3.0)
},
@@ -110,7 +110,7 @@ class DestinationMessageTest {
Jsons.serialize(inputMessage),
) as GlobalCheckpoint
- assertEquals(
+ Assertions.assertEquals(
inputMessage.also {
it.state.destinationStats = AirbyteStateStats().withRecordCount(3.0)
},
diff --git a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/state/CheckpointManagerTest.kt b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/state/CheckpointManagerTest.kt
similarity index 66%
rename from airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/state/CheckpointManagerTest.kt
rename to airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/state/CheckpointManagerTest.kt
index a192257bc9a41..246f96a8120db 100644
--- a/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/state/CheckpointManagerTest.kt
+++ b/airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/state/CheckpointManagerTest.kt
@@ -2,22 +2,26 @@
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
-package io.airbyte.cdk.state
+package io.airbyte.cdk.load.state
import com.google.common.collect.Range
-import io.airbyte.cdk.command.DestinationCatalog
-import io.airbyte.cdk.command.DestinationStream
-import io.airbyte.cdk.command.MockDestinationCatalogFactory.Companion.stream1
-import io.airbyte.cdk.command.MockDestinationCatalogFactory.Companion.stream2
-import io.airbyte.cdk.message.MessageConverter
-import io.micronaut.context.annotation.Prototype
+import com.google.common.collect.TreeRangeSet
+import io.airbyte.cdk.load.command.DestinationCatalog
+import io.airbyte.cdk.load.command.DestinationStream
+import io.airbyte.cdk.load.command.MockDestinationCatalogFactory.Companion.stream1
+import io.airbyte.cdk.load.command.MockDestinationCatalogFactory.Companion.stream2
+import io.airbyte.cdk.load.file.TimeProvider
+import io.airbyte.cdk.load.message.Batch
+import io.airbyte.cdk.load.message.BatchEnvelope
+import io.airbyte.cdk.load.message.SimpleBatch
+import io.micronaut.context.annotation.Requires
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
import jakarta.inject.Inject
import jakarta.inject.Singleton
-import java.util.function.Consumer
import java.util.stream.Stream
import kotlinx.coroutines.test.runTest
import org.junit.jupiter.api.Assertions
+import org.junit.jupiter.api.Test
import org.junit.jupiter.api.extension.ExtensionContext
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.Arguments
@@ -25,73 +29,63 @@ import org.junit.jupiter.params.provider.ArgumentsProvider
import org.junit.jupiter.params.provider.ArgumentsSource
@MicronautTest(
+ rebuildContext = true,
environments =
[
"CheckpointManagerTest",
- "MockSyncManager",
"MockDestinationCatalog",
]
)
class CheckpointManagerTest {
@Inject lateinit var checkpointManager: TestCheckpointManager
+ @Inject lateinit var syncManager: SyncManager
/**
* Test state messages.
*
* StateIn: What is passed to the manager. StateOut: What is sent from the manager to the output
* consumer.
*/
- sealed class MockCheckpointIn
- data class MockStreamCheckpointIn(val stream: DestinationStream, val payload: Int) :
- MockCheckpointIn()
- data class MockGlobalCheckpointIn(val payload: Int) : MockCheckpointIn()
-
- sealed class MockCheckpointOut
- data class MockStreamCheckpointOut(val stream: DestinationStream, val payload: String) :
- MockCheckpointOut()
- data class MockGlobalCheckpointOut(val payload: String) : MockCheckpointOut()
+ sealed class MockCheckpoint
+ data class MockStreamCheckpoint(val stream: DestinationStream, val payload: Int) :
+ MockCheckpoint()
+ data class MockGlobalCheckpoint(val payload: Int) : MockCheckpoint()
@Singleton
- class MockStateMessageFactory : MessageConverter {
- override fun from(message: MockCheckpointIn): MockCheckpointOut {
- return when (message) {
- is MockStreamCheckpointIn ->
- MockStreamCheckpointOut(message.stream, message.payload.toString())
- is MockGlobalCheckpointIn -> MockGlobalCheckpointOut(message.payload.toString())
- }
- }
- }
-
- @Prototype
- class MockOutputConsumer : Consumer {
- val collectedStreamOutput = mutableMapOf>()
+ @Requires(env = ["CheckpointManagerTest"])
+ class MockOutputConsumer : suspend (MockCheckpoint) -> Unit {
+ val collectedStreamOutput =
+ mutableMapOf>()
val collectedGlobalOutput = mutableListOf()
- override fun accept(t: MockCheckpointOut) {
+ override suspend fun invoke(t: MockCheckpoint) {
when (t) {
- is MockStreamCheckpointOut ->
- collectedStreamOutput.getOrPut(t.stream) { mutableListOf() }.add(t.payload)
- is MockGlobalCheckpointOut -> collectedGlobalOutput.add(t.payload)
+ is MockStreamCheckpoint ->
+ collectedStreamOutput
+ .getOrPut(t.stream.descriptor) { mutableListOf() }
+ .add(t.payload.toString())
+ is MockGlobalCheckpoint -> collectedGlobalOutput.add(t.payload.toString())
}
}
}
- @Prototype
+ @Singleton
+ @Requires(env = ["CheckpointManagerTest"])
class TestCheckpointManager(
override val catalog: DestinationCatalog,
- override val syncManager: MockSyncManager,
- override val outputFactory: MessageConverter,
- override val outputConsumer: MockOutputConsumer
- ) : StreamsCheckpointManager()
+ override val syncManager: SyncManager,
+ override val outputConsumer: MockOutputConsumer,
+ override val timeProvider: TimeProvider
+ ) : StreamsCheckpointManager()
sealed class TestEvent
data class TestStreamMessage(val stream: DestinationStream, val index: Long, val message: Int) :
TestEvent() {
- fun toMockCheckpointIn() = MockStreamCheckpointIn(stream, message)
+ fun toMockCheckpointIn() = MockStreamCheckpoint(stream, message)
}
data class TestGlobalMessage(
val streamIndexes: List>,
val message: Int
) : TestEvent() {
- fun toMockCheckpointIn() = MockGlobalCheckpointIn(message)
+ fun toMockCheckpointIn() = MockGlobalCheckpoint(message)
}
data class FlushPoint(
val persistedRanges: Map>> = mapOf()
@@ -101,7 +95,7 @@ class CheckpointManagerTest {
val name: String,
val events: List