1
0
mirror of synced 2025-12-19 18:14:56 -05:00

Load CDK E2ETest Destination w/o tests (#45147)

This commit is contained in:
Johnny Schmidt
2024-09-26 14:24:49 -07:00
committed by GitHub
parent adea83d8e2
commit 7cdd998aa5
84 changed files with 2505 additions and 1265 deletions

View File

@@ -34,6 +34,7 @@ class BufferingOutputConsumer(
private val catalogs = mutableListOf<AirbyteCatalog>()
private val traces = mutableListOf<AirbyteTraceMessage>()
private val messages = mutableListOf<AirbyteMessage>()
private var messagesIndex: Int = 0
var callback: (AirbyteMessage) -> Unit = {}
set(value) {
@@ -79,4 +80,15 @@ class BufferingOutputConsumer(
fun traces(): List<AirbyteTraceMessage> = synchronized(this) { listOf(*traces.toTypedArray()) }
fun messages(): List<AirbyteMessage> = synchronized(this) { listOf(*messages.toTypedArray()) }
fun newMessages(): List<AirbyteMessage> =
synchronized(this) {
val newMessages = messages.subList(messagesIndex, messages.size)
messagesIndex = messages.size
newMessages
}
fun resetNewMessagesCursor() {
synchronized(this) { messagesIndex = 0 }
}
}

View File

@@ -22,15 +22,15 @@ import jakarta.inject.Singleton
class CheckOperation<T : ConfigurationJsonObjectBase, C : DestinationConfiguration>(
val configJsonObjectSupplier: ConfigurationJsonObjectSupplier<T>,
val configFactory: DestinationConfigurationFactory<T, C>,
private val destinationCheckOperation: DestinationCheckOperation<C>,
private val destinationChecker: DestinationChecker<C>,
private val exceptionHandler: ExceptionHandler,
private val outputConsumer: OutputConsumer,
) : Operation {
override fun execute() {
try {
val pojo: T = configJsonObjectSupplier.get()
val config: C = configFactory.make(pojo)
destinationCheckOperation.check(config)
val pojo = configJsonObjectSupplier.get()
val config = configFactory.make(pojo)
destinationChecker.check(config)
val successMessage =
AirbyteMessage()
.withType(AirbyteMessage.Type.CONNECTION_STATUS)
@@ -44,7 +44,7 @@ class CheckOperation<T : ConfigurationJsonObjectBase, C : DestinationConfigurati
outputConsumer.accept(traceMessage)
outputConsumer.accept(statusMessage)
} finally {
destinationCheckOperation.cleanup()
destinationChecker.cleanup()
}
}
}

View File

@@ -17,7 +17,7 @@ import io.airbyte.cdk.command.DestinationConfiguration
* * Implementors should not throw exceptions in the constructor.
* * Implementors should not inject configuration; only use the config passed in [check].
*/
interface DestinationCheckOperation<C : DestinationConfiguration> {
interface DestinationChecker<C : DestinationConfiguration> {
fun check(config: C)
fun cleanup() {}
}

View File

@@ -16,7 +16,7 @@ data class DestinationCatalog(val streams: List<DestinationStream> = emptyList()
private val byDescriptor: Map<DestinationStream.Descriptor, DestinationStream> =
streams.associateBy { it.descriptor }
fun getStream(name: String, namespace: String): DestinationStream {
fun getStream(name: String, namespace: String?): DestinationStream {
val descriptor = DestinationStream.Descriptor(namespace = namespace, name = name)
return byDescriptor[descriptor]
?: throw IllegalArgumentException("Stream not found: namespace=$namespace, name=$name")

View File

@@ -4,12 +4,21 @@
package io.airbyte.cdk.command
import io.micronaut.context.annotation.ConfigurationProperties
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
@ConfigurationProperties("destination.config")
interface DestinationConfiguration : Configuration {
/**
* Micronaut factory which glues [ConfigurationJsonObjectSupplier] and
* [DestinationConfigurationFactory] together to produce a [DestinationConfiguration] singleton.
@@ -17,9 +26,11 @@ interface DestinationConfiguration : Configuration {
@Factory
private class MicronautFactory {
@Singleton
fun <I : ConfigurationJsonObjectBase> sourceConfig(
fun <I : ConfigurationJsonObjectBase> destinationConfig(
pojoSupplier: ConfigurationJsonObjectSupplier<I>,
factory: DestinationConfigurationFactory<I, out DestinationConfiguration>,
): DestinationConfiguration = factory.make(pojoSupplier.get())
): DestinationConfiguration {
return factory.make(pojoSupplier.get())
}
}
}

View File

@@ -29,9 +29,13 @@ data class DestinationStream(
val minimumGenerationId: Long,
val syncId: Long,
) {
data class Descriptor(val namespace: String, val name: String) {
data class Descriptor(val namespace: String?, val name: String) {
fun asProtocolObject(): StreamDescriptor =
StreamDescriptor().withNamespace(namespace).withName(name)
StreamDescriptor().withName(name).also {
if (namespace != null) {
it.namespace = namespace
}
}
}
/**

View File

@@ -1,37 +0,0 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.command
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
import java.nio.file.Path
/**
* General configuration for the write operation. The implementor can override this to tweak runtime
* behavior.
*/
interface WriteConfiguration {
/** Batch accumulation settings. */
val recordBatchSizeBytes: Long
val tmpFileDirectory: Path
val firstStageTmpFilePrefix: String
val firstStageTmpFileSuffix: String
/** Memory queue settings */
val maxMessageQueueMemoryUsageRatio: Double // as fraction of available memory
val estimatedRecordMemoryOverheadRatio: Double // 0 => No overhead, 1.0 => 2x overhead
}
@Singleton
@Secondary
open class DefaultWriteConfiguration : WriteConfiguration {
override val recordBatchSizeBytes: Long = 200L * 1024L * 1024L
override val tmpFileDirectory: Path = Path.of("/airbyte-cdk-load")
override val firstStageTmpFilePrefix = "staged-raw-records"
override val firstStageTmpFileSuffix = ".jsonl"
override val maxMessageQueueMemoryUsageRatio: Double = 0.2
override val estimatedRecordMemoryOverheadRatio: Double = 0.1
}

View File

@@ -5,27 +5,154 @@
package io.airbyte.cdk.data
import java.math.BigDecimal
import java.time.LocalDate
import java.time.LocalDateTime
import java.time.LocalTime
import java.time.OffsetDateTime
import java.time.OffsetTime
import java.time.ZoneOffset
sealed interface AirbyteValue
sealed interface AirbyteValue {
companion object {
fun from(value: Any?): AirbyteValue =
when (value) {
null -> NullValue
is String -> StringValue(value)
is Boolean -> BooleanValue(value)
is Int -> IntegerValue(value.toLong())
is Long -> IntegerValue(value)
is Double -> NumberValue(BigDecimal.valueOf(value))
is BigDecimal -> NumberValue(value)
is LocalDate -> DateValue(value.toString())
is OffsetDateTime,
is LocalDateTime -> TimestampValue(value.toString())
is OffsetTime,
is LocalTime -> TimeValue(value.toString())
is Map<*, *> ->
ObjectValue.from(@Suppress("UNCHECKED_CAST") (value as Map<String, Any?>))
is List<*> -> ArrayValue.from(value)
else ->
throw IllegalArgumentException(
"Unrecognized value (${value.javaClass.name}: $value"
)
}
}
}
data object NullValue : AirbyteValue
// Comparable implementations are intended for use in tests.
// They're not particularly robust, and probably shouldn't be relied on
// for actual sync-time logic.
// (mostly the date/timestamp/time types - everything else is fine)
data object NullValue : AirbyteValue, Comparable<NullValue> {
override fun compareTo(other: NullValue): Int = 0
}
@JvmInline value class StringValue(val value: String) : AirbyteValue
@JvmInline
value class StringValue(val value: String) : AirbyteValue, Comparable<StringValue> {
override fun compareTo(other: StringValue): Int = value.compareTo(other.value)
}
@JvmInline value class BooleanValue(val value: Boolean) : AirbyteValue
@JvmInline
value class BooleanValue(val value: Boolean) : AirbyteValue, Comparable<BooleanValue> {
override fun compareTo(other: BooleanValue): Int = value.compareTo(other.value)
}
@JvmInline value class IntegerValue(val value: Long) : AirbyteValue
@JvmInline
value class IntegerValue(val value: Long) : AirbyteValue, Comparable<IntegerValue> {
override fun compareTo(other: IntegerValue): Int = value.compareTo(other.value)
}
@JvmInline value class NumberValue(val value: BigDecimal) : AirbyteValue
@JvmInline
value class NumberValue(val value: BigDecimal) : AirbyteValue, Comparable<NumberValue> {
override fun compareTo(other: NumberValue): Int = value.compareTo(other.value)
}
@JvmInline value class DateValue(val value: String) : AirbyteValue
@JvmInline
value class DateValue(val value: String) : AirbyteValue, Comparable<DateValue> {
override fun compareTo(other: DateValue): Int {
val thisDate =
try {
LocalDate.parse(value)
} catch (e: Exception) {
LocalDate.MIN
}
val otherDate =
try {
LocalDate.parse(other.value)
} catch (e: Exception) {
LocalDate.MIN
}
return thisDate.compareTo(otherDate)
}
}
@JvmInline value class TimestampValue(val value: String) : AirbyteValue
@JvmInline
value class TimestampValue(val value: String) : AirbyteValue, Comparable<TimestampValue> {
override fun compareTo(other: TimestampValue): Int {
// Do all comparisons using OffsetDateTime for convenience.
// First, try directly parsing as OffsetDateTime.
// If that fails, try parsing as LocalDateTime and assume UTC.
// We could maybe have separate value classes for these cases,
// but that comes with its own set of problems
// (mostly around sources declaring bad schemas).
val thisTimestamp =
try {
OffsetDateTime.parse(value)
} catch (e: Exception) {
LocalDateTime.parse(value).atOffset(ZoneOffset.UTC)
} catch (e: Exception) {
LocalDateTime.MIN.atOffset(ZoneOffset.UTC)
}
val otherTimestamp =
try {
OffsetDateTime.parse(other.value)
} catch (e: Exception) {
LocalDateTime.parse(other.value).atOffset(ZoneOffset.UTC)
} catch (e: Exception) {
LocalDateTime.MIN.atOffset(ZoneOffset.UTC)
}
return thisTimestamp.compareTo(otherTimestamp)
}
}
@JvmInline value class TimeValue(val value: String) : AirbyteValue
@JvmInline
value class TimeValue(val value: String) : AirbyteValue, Comparable<TimeValue> {
override fun compareTo(other: TimeValue): Int {
// Similar to TimestampValue, try parsing with/without timezone,
// and do all comparisons using OffsetTime.
val thisTime =
try {
OffsetTime.parse(value)
} catch (e: Exception) {
LocalTime.parse(value).atOffset(ZoneOffset.UTC)
} catch (e: Exception) {
LocalTime.MIN.atOffset(ZoneOffset.UTC)
}
val otherTime =
try {
OffsetTime.parse(other.value)
} catch (e: Exception) {
LocalTime.parse(other.value).atOffset(ZoneOffset.UTC)
} catch (e: Exception) {
LocalTime.MIN.atOffset(ZoneOffset.UTC)
}
return thisTime.compareTo(otherTime)
}
}
@JvmInline value class ArrayValue(val values: List<AirbyteValue>) : AirbyteValue
@JvmInline
value class ArrayValue(val values: List<AirbyteValue>) : AirbyteValue {
companion object {
fun from(list: List<Any?>): ArrayValue = ArrayValue(list.map { it as AirbyteValue })
}
}
@JvmInline value class ObjectValue(val values: LinkedHashMap<String, AirbyteValue>) : AirbyteValue
@JvmInline
value class ObjectValue(val values: LinkedHashMap<String, AirbyteValue>) : AirbyteValue {
companion object {
fun from(map: Map<String, Any?>): ObjectValue =
ObjectValue(map.mapValuesTo(linkedMapOf()) { (_, v) -> AirbyteValue.from(v) })
}
}
@JvmInline value class UnknownValue(val what: String) : AirbyteValue

View File

@@ -4,17 +4,20 @@
package io.airbyte.cdk.file
import io.micronaut.context.annotation.DefaultImplementation
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
import java.nio.file.Files
import java.nio.file.Path
@DefaultImplementation(DefaultTempFileProvider::class)
interface TempFileProvider {
fun createTempFile(directory: Path, prefix: String, suffix: String): LocalFile
}
@Singleton
@Secondary
class DefaultTempFileProvider : TempFileProvider {
override fun createTempFile(directory: Path, prefix: String, suffix: String): LocalFile {
Files.createDirectories(directory)
return DefaultLocalFile(Files.createTempFile(directory, prefix, suffix))
}
}

View File

@@ -10,8 +10,10 @@ 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.protocol.models.Jsons
import io.airbyte.protocol.models.v0.AirbyteGlobalState
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
@@ -35,16 +37,31 @@ sealed interface DestinationMessage {
/** Records. */
sealed interface DestinationStreamAffinedMessage : DestinationMessage {
val stream: DestinationStream
val stream: DestinationStream.Descriptor
}
data class DestinationRecord(
override val stream: DestinationStream,
override val stream: DestinationStream.Descriptor,
val data: AirbyteValue,
val emittedAtMs: Long,
val meta: Meta?,
val serialized: String,
) : DestinationStreamAffinedMessage {
/** Convenience constructor, primarily intended for use in tests. */
constructor(
namespace: String?,
name: String,
data: String,
emittedAtMs: Long,
changes: List<Change>? = null,
) : this(
stream = DestinationStream.Descriptor(namespace, name),
data = JsonToAirbyteValue().convert(Jsons.deserialize(data), ObjectTypeWithoutSchema),
emittedAtMs = emittedAtMs,
meta = Meta(changes),
serialized = "",
)
data class Meta(val changes: List<Change>?) {
fun asProtocolObject(): AirbyteRecordMessageMeta =
AirbyteRecordMessageMeta().also {
@@ -70,8 +87,8 @@ data class DestinationRecord(
.withType(AirbyteMessage.Type.RECORD)
.withRecord(
AirbyteRecordMessage()
.withStream(stream.descriptor.name)
.withNamespace(stream.descriptor.namespace)
.withStream(stream.name)
.withNamespace(stream.namespace)
.withEmittedAt(emittedAtMs)
.withData(AirbyteValueToJson().convert(data))
.also {
@@ -83,7 +100,7 @@ data class DestinationRecord(
}
private fun statusToProtocolMessage(
stream: DestinationStream,
stream: DestinationStream.Descriptor,
emittedAtMs: Long,
status: AirbyteStreamStatus,
): AirbyteMessage =
@@ -95,13 +112,13 @@ private fun statusToProtocolMessage(
.withEmittedAt(emittedAtMs.toDouble())
.withStreamStatus(
AirbyteStreamStatusTraceMessage()
.withStreamDescriptor(stream.descriptor.asProtocolObject())
.withStreamDescriptor(stream.asProtocolObject())
.withStatus(status)
)
)
data class DestinationStreamComplete(
override val stream: DestinationStream,
override val stream: DestinationStream.Descriptor,
val emittedAtMs: Long,
) : DestinationStreamAffinedMessage {
override fun asProtocolMessage(): AirbyteMessage =
@@ -109,7 +126,7 @@ data class DestinationStreamComplete(
}
data class DestinationStreamIncomplete(
override val stream: DestinationStream,
override val stream: DestinationStream.Descriptor,
val emittedAtMs: Long,
) : DestinationStreamAffinedMessage {
override fun asProtocolMessage(): AirbyteMessage =
@@ -120,12 +137,12 @@ data class DestinationStreamIncomplete(
sealed interface CheckpointMessage : DestinationMessage {
data class Stats(val recordCount: Long)
data class Checkpoint(
val stream: DestinationStream,
val stream: DestinationStream.Descriptor,
val state: JsonNode,
) {
fun asProtocolObject(): AirbyteStreamState =
AirbyteStreamState()
.withStreamDescriptor(stream.descriptor.asProtocolObject())
.withStreamDescriptor(stream.asProtocolObject())
.withStreamState(state)
}
@@ -138,10 +155,28 @@ sealed interface CheckpointMessage : DestinationMessage {
data class StreamCheckpoint(
val checkpoint: Checkpoint,
override val sourceStats: Stats,
override val destinationStats: Stats? = null
override val destinationStats: Stats? = null,
val additionalProperties: Map<String, Any>
) : CheckpointMessage {
/** Convenience constructor, intended for use in tests. */
constructor(
streamNamespace: String?,
streamName: String,
blob: String,
sourceRecordCount: Long,
destinationRecordCount: Long? = null,
) : this(
Checkpoint(
DestinationStream.Descriptor(streamNamespace, streamName),
state = Jsons.deserialize(blob)
),
Stats(sourceRecordCount),
destinationRecordCount?.let { Stats(it) },
additionalProperties = mutableMapOf(),
)
override fun withDestinationStats(stats: Stats) =
StreamCheckpoint(checkpoint, sourceStats, stats)
StreamCheckpoint(checkpoint, sourceStats, stats, additionalProperties)
override fun asProtocolMessage(): AirbyteMessage {
val stateMessage =
@@ -166,10 +201,16 @@ data class GlobalCheckpoint(
val state: JsonNode,
override val sourceStats: Stats,
override val destinationStats: Stats? = null,
val checkpoints: List<Checkpoint> = emptyList()
val checkpoints: List<Checkpoint> = emptyList(),
val additionalProperties: MutableMap<String, Any> = mutableMapOf()
) : CheckpointMessage {
/** Convenience constructor, primarily intended for use in tests. */
constructor(
blob: String,
sourceRecordCount: Long,
) : this(state = Jsons.deserialize(blob), Stats(sourceRecordCount))
override fun withDestinationStats(stats: Stats) =
GlobalCheckpoint(state, sourceStats, stats, checkpoints)
GlobalCheckpoint(state, sourceStats, stats, checkpoints, additionalProperties)
override fun asProtocolMessage(): AirbyteMessage {
val stateMessage =
@@ -216,7 +257,7 @@ class DestinationMessageFactory(private val catalog: DestinationCatalog) {
name = message.record.stream,
)
DestinationRecord(
stream = stream,
stream = stream.descriptor,
data = JsonToAirbyteValue().convert(message.record.data, stream.schema),
emittedAtMs = message.record.emittedAt,
meta =
@@ -244,9 +285,15 @@ class DestinationMessageFactory(private val catalog: DestinationCatalog) {
if (message.trace.type == AirbyteTraceMessage.Type.STREAM_STATUS) {
when (status.status) {
AirbyteStreamStatus.COMPLETE ->
DestinationStreamComplete(stream, message.trace.emittedAt.toLong())
DestinationStreamComplete(
stream.descriptor,
message.trace.emittedAt.toLong()
)
AirbyteStreamStatus.INCOMPLETE ->
DestinationStreamIncomplete(stream, message.trace.emittedAt.toLong())
DestinationStreamIncomplete(
stream.descriptor,
message.trace.emittedAt.toLong()
)
else -> Undefined
}
} else {
@@ -259,7 +306,8 @@ class DestinationMessageFactory(private val catalog: DestinationCatalog) {
StreamCheckpoint(
checkpoint = fromAirbyteStreamState(message.state.stream),
sourceStats =
Stats(recordCount = message.state.sourceStats.recordCount.toLong())
Stats(recordCount = message.state.sourceStats.recordCount.toLong()),
additionalProperties = message.state.additionalProperties
)
AirbyteStateMessage.AirbyteStateType.GLOBAL ->
GlobalCheckpoint(
@@ -267,7 +315,10 @@ class DestinationMessageFactory(private val catalog: DestinationCatalog) {
Stats(recordCount = message.state.sourceStats.recordCount.toLong()),
state = message.state.global.sharedState,
checkpoints =
message.state.global.streamStates.map { fromAirbyteStreamState(it) }
message.state.global.streamStates.map {
fromAirbyteStreamState(it)
},
additionalProperties = message.state.additionalProperties
)
else -> // TODO: Do we still need to handle LEGACY?
Undefined
@@ -280,7 +331,7 @@ class DestinationMessageFactory(private val catalog: DestinationCatalog) {
private fun fromAirbyteStreamState(streamState: AirbyteStreamState): Checkpoint {
val descriptor = streamState.streamDescriptor
return Checkpoint(
stream = catalog.getStream(namespace = descriptor.namespace, name = descriptor.name),
stream = DestinationStream.Descriptor(descriptor.namespace, descriptor.name),
state = streamState.streamState
)
}

View File

@@ -22,11 +22,10 @@ class DefaultDestinationMessageDeserializer(private val messageFactory: Destinat
override fun deserialize(serialized: String): DestinationMessage {
try {
val node = Jsons.readTree(serialized)
val airbyteMessage = Jsons.treeToValue(node, AirbyteMessage::class.java)
val airbyteMessage = Jsons.readValue(serialized, AirbyteMessage::class.java)
return messageFactory.fromAirbyteMessage(airbyteMessage, serialized)
} catch (t: Throwable) {
throw RuntimeException("Failed to deserialize AirbyteMessage")
throw RuntimeException("Failed to deserialize AirbyteMessage: $serialized")
}
}
}

View File

@@ -5,8 +5,8 @@
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.command.WriteConfiguration
import io.airbyte.cdk.state.MemoryManager
import io.github.oshai.kotlinlogging.KotlinLogging
import jakarta.inject.Singleton
@@ -51,10 +51,10 @@ data class StreamCompleteWrapped(
@Singleton
class DestinationMessageQueue(
catalog: DestinationCatalog,
config: WriteConfiguration,
config: DestinationConfiguration,
private val memoryManager: MemoryManager,
private val queueChannelFactory: QueueChannelFactory<DestinationRecordWrapped>
) : MessageQueue<DestinationStream, DestinationRecordWrapped> {
) : MessageQueue<DestinationStream.Descriptor, DestinationRecordWrapped> {
private val channels:
ConcurrentHashMap<DestinationStream.Descriptor, QueueChannel<DestinationRecordWrapped>> =
ConcurrentHashMap()
@@ -89,12 +89,10 @@ class DestinationMessageQueue(
}
override suspend fun getChannel(
key: DestinationStream,
key: DestinationStream.Descriptor,
): QueueChannel<DestinationRecordWrapped> {
return channels[key.descriptor]
?: throw IllegalArgumentException(
"Reading from non-existent QueueChannel: ${key.descriptor}"
)
return channels[key]
?: throw IllegalArgumentException("Reading from non-existent QueueChannel: ${key}")
}
private val log = KotlinLogging.logger {}

View File

@@ -41,6 +41,11 @@ class DefaultMessageConverter : MessageConverter<CheckpointMessage, AirbyteMessa
)
.withType(AirbyteStateMessage.AirbyteStateType.STREAM)
.withStream(fromStreamState(message.checkpoint))
.also {
message.additionalProperties.forEach { (key, value) ->
it.withAdditionalProperty(key, value)
}
}
is GlobalCheckpoint ->
AirbyteStateMessage()
.withSourceStats(
@@ -58,6 +63,11 @@ class DefaultMessageConverter : MessageConverter<CheckpointMessage, AirbyteMessa
.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)
}
@@ -66,8 +76,8 @@ class DefaultMessageConverter : MessageConverter<CheckpointMessage, AirbyteMessa
return AirbyteStreamState()
.withStreamDescriptor(
StreamDescriptor()
.withNamespace(checkpoint.stream.descriptor.namespace)
.withName(checkpoint.stream.descriptor.name)
.withNamespace(checkpoint.stream.namespace)
.withName(checkpoint.stream.name)
)
.withStreamState(checkpoint.state)
}

View File

@@ -6,7 +6,8 @@ package io.airbyte.cdk.message
import io.airbyte.cdk.command.DestinationStream
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.DefaultImplementation
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
import kotlinx.coroutines.flow.Flow
import kotlinx.coroutines.flow.flow
@@ -14,18 +15,19 @@ 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.
*/
@DefaultImplementation(DestinationMessageQueueReader::class)
interface MessageQueueReader<K, T> {
suspend fun readChunk(key: K, maxBytes: Long): Flow<T>
}
@Singleton
@Secondary
class DestinationMessageQueueReader(
private val messageQueue: DestinationMessageQueue,
) : MessageQueueReader<DestinationStream, DestinationRecordWrapped> {
) : MessageQueueReader<DestinationStream.Descriptor, DestinationRecordWrapped> {
private val log = KotlinLogging.logger {}
override suspend fun readChunk(
key: DestinationStream,
key: DestinationStream.Descriptor,
maxBytes: Long
): Flow<DestinationRecordWrapped> = flow {
log.info { "Reading chunk of $maxBytes bytes from stream $key" }

View File

@@ -29,9 +29,10 @@ interface MessageQueueWriter<T : Any> {
@Singleton
class DestinationMessageQueueWriter(
private val catalog: DestinationCatalog,
private val messageQueue: MessageQueue<DestinationStream, DestinationRecordWrapped>,
private val messageQueue: MessageQueue<DestinationStream.Descriptor, DestinationRecordWrapped>,
private val streamsManager: StreamsManager,
private val checkpointManager: CheckpointManager<DestinationStream, CheckpointMessage>
private val checkpointManager:
CheckpointManager<DestinationStream.Descriptor, CheckpointMessage>
) : MessageQueueWriter<DestinationMessage> {
/**
* Deserialize and route the message to the appropriate channel.
@@ -89,14 +90,15 @@ class DestinationMessageQueueWriter(
is GlobalCheckpoint -> {
val streamWithIndexAndCount =
catalog.streams.map { stream ->
val manager = streamsManager.getManager(stream)
val manager = streamsManager.getManager(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 to it.second }
val streamIndexes =
streamWithIndexAndCount.map { it.first.descriptor to it.second }
checkpointManager.addGlobalCheckpoint(streamIndexes, messageWithCount)
}
}

View File

@@ -13,7 +13,7 @@ import jakarta.inject.Singleton
@Singleton
@Replaces(IdentitySpecificationExtender::class)
@Requires(env = ["destination"])
class DestinationSpecificationExtender(private val spec: DestinationSpecification) :
class DestinationSpecificationExtender(private val spec: DestinationSpecificationInternal) :
SpecificationExtender {
override fun invoke(specification: ConnectorSpecification): ConnectorSpecification {
return specification
@@ -22,7 +22,7 @@ class DestinationSpecificationExtender(private val spec: DestinationSpecificatio
}
}
interface DestinationSpecification {
interface DestinationSpecificationInternal {
val supportedSyncModes: List<DestinationSyncMode>
val supportsIncremental: Boolean
}

View File

@@ -10,6 +10,7 @@ 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
@@ -39,7 +40,8 @@ interface CheckpointManager<K, T> {
* TODO: Ensure that checkpoint is flushed at the end, and require that all checkpoints be flushed
* before the destination can succeed.
*/
abstract class StreamsCheckpointManager<T, U>() : CheckpointManager<DestinationStream, T> {
abstract class StreamsCheckpointManager<T, U>() :
CheckpointManager<DestinationStream.Descriptor, T> {
private val log = KotlinLogging.logger {}
abstract val catalog: DestinationCatalog
@@ -48,18 +50,22 @@ abstract class StreamsCheckpointManager<T, U>() : CheckpointManager<DestinationS
abstract val outputConsumer: Consumer<U>
data class GlobalCheckpoint<T>(
val streamIndexes: List<Pair<DestinationStream, Long>>,
val streamIndexes: List<Pair<DestinationStream.Descriptor, Long>>,
val checkpointMessage: T
)
private val checkpointsAreGlobal: AtomicReference<Boolean?> = AtomicReference(null)
private val streamCheckpoints:
ConcurrentHashMap<DestinationStream, ConcurrentLinkedHashMap<Long, T>> =
ConcurrentHashMap<DestinationStream.Descriptor, ConcurrentLinkedHashMap<Long, T>> =
ConcurrentHashMap()
private val globalCheckpoints: ConcurrentLinkedQueue<GlobalCheckpoint<T>> =
ConcurrentLinkedQueue()
override fun addStreamCheckpoint(key: DestinationStream, index: Long, checkpointMessage: T) {
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"
@@ -93,7 +99,7 @@ abstract class StreamsCheckpointManager<T, U>() : CheckpointManager<DestinationS
// TODO: Is it an error if we don't get all the streams every time?
override fun addGlobalCheckpoint(
keyIndexes: List<Pair<DestinationStream, Long>>,
keyIndexes: List<Pair<DestinationStream.Descriptor, Long>>,
checkpointMessage: T
) {
if (checkpointsAreGlobal.updateAndGet { it != false } != true) {
@@ -149,8 +155,8 @@ abstract class StreamsCheckpointManager<T, U>() : CheckpointManager<DestinationS
private fun flushStreamCheckpoints() {
for (stream in catalog.streams) {
val manager = streamsManager.getManager(stream)
val streamCheckpoints = streamCheckpoints[stream] ?: return
val manager = streamsManager.getManager(stream.descriptor)
val streamCheckpoints = streamCheckpoints[stream.descriptor] ?: return
for (index in streamCheckpoints.keys) {
if (manager.areRecordsPersistedUntil(index)) {
val checkpointMessage =
@@ -168,6 +174,7 @@ abstract class StreamsCheckpointManager<T, U>() : CheckpointManager<DestinationS
}
@Singleton
@Secondary
class DefaultCheckpointManager(
override val catalog: DestinationCatalog,
override val streamsManager: StreamsManager,

View File

@@ -22,16 +22,16 @@ import kotlinx.coroutines.channels.Channel
/** Manages the state of all streams in the destination. */
interface StreamsManager {
/** Get the manager for the given stream. Throws an exception if the stream is not found. */
fun getManager(stream: DestinationStream): StreamManager
fun getManager(stream: DestinationStream.Descriptor): StreamManager
/** Suspend until all streams are closed. */
suspend fun awaitAllStreamsClosed()
}
class DefaultStreamsManager(
private val streamManagers: ConcurrentHashMap<DestinationStream, StreamManager>
private val streamManagers: ConcurrentHashMap<DestinationStream.Descriptor, StreamManager>
) : StreamsManager {
override fun getManager(stream: DestinationStream): StreamManager {
override fun getManager(stream: DestinationStream.Descriptor): StreamManager {
return streamManagers[stream] ?: throw IllegalArgumentException("Stream not found: $stream")
}
@@ -192,8 +192,8 @@ class StreamsManagerFactory(
) {
@Singleton
fun make(): StreamsManager {
val hashMap = ConcurrentHashMap<DestinationStream, StreamManager>()
catalog.streams.forEach { hashMap[it] = DefaultStreamManager(it) }
val hashMap = ConcurrentHashMap<DestinationStream.Descriptor, StreamManager>()
catalog.streams.forEach { hashMap[it.descriptor] = DefaultStreamManager(it) }
return DefaultStreamsManager(hashMap)
}
}

View File

@@ -29,7 +29,7 @@ interface DestinationTaskLauncher : TaskLauncher {
suspend fun handleSetupComplete()
suspend fun handleStreamOpen(streamLoader: StreamLoader)
suspend fun handleNewSpilledFile(
stream: DestinationStream,
stream: DestinationStream.Descriptor,
wrapped: BatchEnvelope<SpilledRawMessagesLocalFile>
)
suspend fun handleNewBatch(streamLoader: StreamLoader, wrapped: BatchEnvelope<*>)
@@ -75,7 +75,8 @@ class DefaultDestinationTaskLauncher(
private val catalog: DestinationCatalog,
private val streamsManager: StreamsManager,
override val taskRunner: TaskRunner,
private val checkpointManager: CheckpointManager<DestinationStream, CheckpointMessage>,
private val checkpointManager:
CheckpointManager<DestinationStream.Descriptor, CheckpointMessage>,
private val setupTaskFactory: SetupTaskFactory,
private val openStreamTaskFactory: OpenStreamTaskFactory,
private val spillToDiskTaskFactory: SpillToDiskTaskFactory,
@@ -90,11 +91,11 @@ class DefaultDestinationTaskLauncher(
private val batchUpdateLock = Mutex()
private val streamLoaders:
ConcurrentHashMap<DestinationStream, CompletableDeferred<StreamLoader>> =
ConcurrentHashMap<DestinationStream.Descriptor, CompletableDeferred<StreamLoader>> =
ConcurrentHashMap()
init {
catalog.streams.forEach { streamLoaders[it] = CompletableDeferred() }
catalog.streams.forEach { streamLoaders[it.descriptor] = CompletableDeferred() }
}
override suspend fun start() {
@@ -103,7 +104,7 @@ class DefaultDestinationTaskLauncher(
taskRunner.enqueue(setupTask)
catalog.streams.forEach { stream ->
log.info { "Starting spill-to-disk task for $stream" }
val spillTask = spillToDiskTaskFactory.make(this, stream)
val spillTask = spillToDiskTaskFactory.make(this, stream.descriptor)
taskRunner.enqueue(spillTask)
}
}
@@ -120,12 +121,12 @@ class DefaultDestinationTaskLauncher(
/** Called when a stream is ready for loading. */
override suspend fun handleStreamOpen(streamLoader: StreamLoader) {
log.info { "Registering stream open and loader available for ${streamLoader.stream}" }
streamLoaders[streamLoader.stream]!!.complete(streamLoader)
streamLoaders[streamLoader.stream.descriptor]!!.complete(streamLoader)
}
/** Called for each new spilled file. */
override suspend fun handleNewSpilledFile(
stream: DestinationStream,
stream: DestinationStream.Descriptor,
wrapped: BatchEnvelope<SpilledRawMessagesLocalFile>
) {
val streamLoader = streamLoaders[stream]!!.await()
@@ -142,7 +143,7 @@ class DefaultDestinationTaskLauncher(
*/
override suspend fun handleNewBatch(streamLoader: StreamLoader, wrapped: BatchEnvelope<*>) {
batchUpdateLock.withLock {
val streamManager = streamsManager.getManager(streamLoader.stream)
val streamManager = streamsManager.getManager(streamLoader.stream.descriptor)
streamManager.updateBatchState(wrapped)
if (wrapped.batch.state != Batch.State.COMPLETE) {
@@ -169,7 +170,7 @@ class DefaultDestinationTaskLauncher(
/** Called when a stream is closed. */
override suspend fun handleStreamClosed(stream: DestinationStream) {
streamsManager.getManager(stream).markClosed()
streamsManager.getManager(stream.descriptor).markClosed()
checkpointManager.flushReadyCheckpointMessages()
if (runTeardownOnce.compareAndSet(false, true)) {
streamsManager.awaitAllStreamsClosed()
@@ -190,7 +191,8 @@ class DestinationTaskLauncherFactory(
private val catalog: DestinationCatalog,
private val streamsManager: StreamsManager,
private val taskRunner: TaskRunner,
private val checkpointManager: CheckpointManager<DestinationStream, CheckpointMessage>,
private val checkpointManager:
CheckpointManager<DestinationStream.Descriptor, CheckpointMessage>,
private val setupTaskFactory: SetupTaskFactory,
private val openStreamTaskFactory: OpenStreamTaskFactory,
private val spillToDiskTaskFactory: SpillToDiskTaskFactory,

View File

@@ -5,7 +5,7 @@
package io.airbyte.cdk.task
import io.airbyte.cdk.command.DestinationStream
import io.airbyte.cdk.write.DestinationWriteOperation
import io.airbyte.cdk.write.DestinationWriter
import io.airbyte.cdk.write.StreamLoader
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
@@ -34,7 +34,7 @@ interface OpenStreamTaskFactory {
@Singleton
@Secondary
class DefaultOpenStreamTaskFactory(
private val destination: DestinationWriteOperation,
private val destination: DestinationWriter,
) : OpenStreamTaskFactory {
override fun make(
taskLauncher: DestinationTaskLauncher,

View File

@@ -5,7 +5,6 @@
package io.airbyte.cdk.task
import io.airbyte.cdk.message.BatchEnvelope
import io.airbyte.cdk.state.StreamsManager
import io.airbyte.cdk.write.StreamLoader
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
@@ -35,9 +34,7 @@ interface ProcessBatchTaskFactory {
@Singleton
@Secondary
class DefaultProcessBatchTaskFactory(
private val streamsManager: StreamsManager,
) : ProcessBatchTaskFactory {
class DefaultProcessBatchTaskFactory : ProcessBatchTaskFactory {
override fun make(
taskLauncher: DestinationTaskLauncher,
streamLoader: StreamLoader,

View File

@@ -4,20 +4,20 @@
package io.airbyte.cdk.task
import io.airbyte.cdk.write.DestinationWriteOperation
import io.airbyte.cdk.write.DestinationWriter
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
interface SetupTask : Task
/**
* Wraps @[DestinationWriteOperation.setup] and starts the open stream tasks.
* Wraps @[DestinationWriter.setup] and starts the open stream tasks.
*
* TODO: This should call something like "TaskLauncher.setupComplete" and let it decide what to do
* next.
*/
class DefaultSetupTask(
private val destination: DestinationWriteOperation,
private val destination: DestinationWriter,
private val taskLauncher: DestinationTaskLauncher
) : SetupTask {
override suspend fun execute() {
@@ -33,7 +33,7 @@ interface SetupTaskFactory {
@Singleton
@Secondary
class DefaultSetupTaskFactory(
private val destination: DestinationWriteOperation,
private val destination: DestinationWriter,
) : SetupTaskFactory {
override fun make(taskLauncher: DestinationTaskLauncher): SetupTask {
return DefaultSetupTask(destination, taskLauncher)

View File

@@ -5,8 +5,8 @@
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.command.WriteConfiguration
import io.airbyte.cdk.file.TempFileProvider
import io.airbyte.cdk.message.BatchEnvelope
import io.airbyte.cdk.message.DestinationRecordWrapped
@@ -32,10 +32,11 @@ interface SpillToDiskTask : Task
* TODO: Allow for the record batch size to be supplied per-stream. (Needed?)
*/
class DefaultSpillToDiskTask(
private val config: WriteConfiguration,
private val config: DestinationConfiguration,
private val tmpFileProvider: TempFileProvider,
private val queueReader: MessageQueueReader<DestinationStream, DestinationRecordWrapped>,
private val stream: DestinationStream,
private val queueReader:
MessageQueueReader<DestinationStream.Descriptor, DestinationRecordWrapped>,
private val stream: DestinationStream.Descriptor,
private val launcher: DestinationTaskLauncher
) : SpillToDiskTask {
private val log = KotlinLogging.logger {}
@@ -116,18 +117,22 @@ class DefaultSpillToDiskTask(
}
interface SpillToDiskTaskFactory {
fun make(taskLauncher: DestinationTaskLauncher, stream: DestinationStream): SpillToDiskTask
fun make(
taskLauncher: DestinationTaskLauncher,
stream: DestinationStream.Descriptor
): SpillToDiskTask
}
@Singleton
class DefaultSpillToDiskTaskFactory(
private val config: WriteConfiguration,
private val config: DestinationConfiguration,
private val tmpFileProvider: TempFileProvider,
private val queueReader: MessageQueueReader<DestinationStream, DestinationRecordWrapped>
private val queueReader:
MessageQueueReader<DestinationStream.Descriptor, DestinationRecordWrapped>
) : SpillToDiskTaskFactory {
override fun make(
taskLauncher: DestinationTaskLauncher,
stream: DestinationStream
stream: DestinationStream.Descriptor
): SpillToDiskTask {
return DefaultSpillToDiskTask(config, tmpFileProvider, queueReader, stream, taskLauncher)
}

View File

@@ -4,7 +4,7 @@
package io.airbyte.cdk.task
import io.airbyte.cdk.write.DestinationWriteOperation
import io.airbyte.cdk.write.DestinationWriter
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Secondary
import jakarta.inject.Singleton
@@ -12,12 +12,12 @@ import jakarta.inject.Singleton
interface TeardownTask : Task
/**
* Wraps @[DestinationWriteOperation.teardown] and stops the task launcher.
* Wraps @[DestinationWriter.teardown] and stops the task launcher.
*
* TODO: Report teardown-complete and let the task launcher decide what to do next.
*/
class DefaultTeardownTask(
private val destination: DestinationWriteOperation,
private val destination: DestinationWriter,
private val taskLauncher: DestinationTaskLauncher
) : TeardownTask {
val log = KotlinLogging.logger {}
@@ -35,7 +35,7 @@ interface TeardownTaskFactory {
@Singleton
@Secondary
class DefaultTeardownTaskFactory(
private val destination: DestinationWriteOperation,
private val destination: DestinationWriter,
) : TeardownTaskFactory {
override fun make(taskLauncher: DestinationTaskLauncher): TeardownTask {
return DefaultTeardownTask(destination, taskLauncher)

View File

@@ -12,7 +12,7 @@ import jakarta.inject.Singleton
* Implementor interface. Every Destination must extend this and at least provide an implementation
* of [getStreamLoader].
*/
interface DestinationWriteOperation {
interface DestinationWriter {
// Called once before anything else
suspend fun setup() {}
@@ -25,7 +25,7 @@ interface DestinationWriteOperation {
@Singleton
@Secondary
class DefaultDestinationWriteOperation : DestinationWriteOperation {
class DefaultDestinationWriter : DestinationWriter {
init {
throw NotImplementedError(
"DestinationWrite not implemented. Please create a custom @Singleton implementation."

View File

@@ -25,7 +25,7 @@ import kotlinx.coroutines.runBlocking
class WriteOperation(
private val inputConsumer: InputConsumer<DestinationMessage>,
private val taskLauncher: TaskLauncher,
private val taskRunner: TaskRunner
private val taskRunner: TaskRunner,
) : Operation {
override fun execute() {
runBlocking {

View File

@@ -12,6 +12,7 @@ import io.airbyte.cdk.command.MockCatalogFactory.Companion.stream2
import io.airbyte.cdk.data.NullValue
import io.airbyte.cdk.state.CheckpointManager
import io.micronaut.context.annotation.Prototype
import io.micronaut.context.annotation.Requires
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
import jakarta.inject.Inject
import jakarta.inject.Named
@@ -62,12 +63,12 @@ class DestinationMessageQueueWriterTest {
}
@Prototype
class MockMessageQueue : MessageQueue<DestinationStream, DestinationRecordWrapped> {
class MockMessageQueue : MessageQueue<DestinationStream.Descriptor, DestinationRecordWrapped> {
private val channels =
mutableMapOf<DestinationStream, QueueChannel<DestinationRecordWrapped>>()
mutableMapOf<DestinationStream.Descriptor, QueueChannel<DestinationRecordWrapped>>()
override suspend fun getChannel(
key: DestinationStream
key: DestinationStream.Descriptor
): QueueChannel<DestinationRecordWrapped> {
return channels.getOrPut(key) { MockQueueChannel() }
}
@@ -82,14 +83,16 @@ class DestinationMessageQueueWriterTest {
}
@Prototype
class MockCheckpointManager : CheckpointManager<DestinationStream, CheckpointMessage> {
@Requires(env = ["MockStreamsManager"])
class MockCheckpointManager :
CheckpointManager<DestinationStream.Descriptor, CheckpointMessage> {
val streamStates =
mutableMapOf<DestinationStream, MutableList<Pair<Long, CheckpointMessage>>>()
mutableMapOf<DestinationStream.Descriptor, MutableList<Pair<Long, CheckpointMessage>>>()
val globalStates =
mutableListOf<Pair<List<Pair<DestinationStream, Long>>, CheckpointMessage>>()
mutableListOf<Pair<List<Pair<DestinationStream.Descriptor, Long>>, CheckpointMessage>>()
override fun addStreamCheckpoint(
key: DestinationStream,
key: DestinationStream.Descriptor,
index: Long,
checkpointMessage: CheckpointMessage
) {
@@ -97,20 +100,20 @@ class DestinationMessageQueueWriterTest {
}
override fun addGlobalCheckpoint(
keyIndexes: List<Pair<DestinationStream, Long>>,
keyIndexes: List<Pair<DestinationStream.Descriptor, Long>>,
checkpointMessage: CheckpointMessage
) {
globalStates.add(keyIndexes to checkpointMessage)
}
override suspend fun flushReadyCheckpointMessages() {
TODO("Not yet implemented")
throw NotImplementedError()
}
}
private fun makeRecord(stream: DestinationStream, record: String): DestinationRecord {
return DestinationRecord(
stream = stream,
stream = stream.descriptor,
data = NullValue,
emittedAtMs = 0,
meta = null,
@@ -119,14 +122,18 @@ class DestinationMessageQueueWriterTest {
}
private fun makeStreamComplete(stream: DestinationStream): DestinationStreamComplete {
return DestinationStreamComplete(stream = stream, emittedAtMs = 0)
return DestinationStreamComplete(stream = stream.descriptor, emittedAtMs = 0)
}
private fun makeStreamState(stream: DestinationStream, recordCount: Long): CheckpointMessage {
return StreamCheckpoint(
checkpoint =
CheckpointMessage.Checkpoint(stream, JsonNodeFactory.instance.objectNode()),
sourceStats = CheckpointMessage.Stats(recordCount)
CheckpointMessage.Checkpoint(
stream.descriptor,
JsonNodeFactory.instance.objectNode()
),
sourceStats = CheckpointMessage.Stats(recordCount),
additionalProperties = emptyMap()
)
}
@@ -142,11 +149,15 @@ class DestinationMessageQueueWriterTest {
fun testSendRecords() = runTest {
val writer = queueWriterFactory.make()
val channel1 = queueWriterFactory.messageQueue.getChannel(stream1) as MockQueueChannel
val channel2 = queueWriterFactory.messageQueue.getChannel(stream2) as MockQueueChannel
val channel1 =
queueWriterFactory.messageQueue.getChannel(stream1.descriptor) as MockQueueChannel
val channel2 =
queueWriterFactory.messageQueue.getChannel(stream2.descriptor) as MockQueueChannel
val manager1 = queueWriterFactory.streamsManager.getManager(stream1) as MockStreamManager
val manager2 = queueWriterFactory.streamsManager.getManager(stream2) as MockStreamManager
val manager1 =
queueWriterFactory.streamsManager.getManager(stream1.descriptor) as MockStreamManager
val manager2 =
queueWriterFactory.streamsManager.getManager(stream2.descriptor) as MockStreamManager
(0 until 10).forEach { writer.publish(makeRecord(stream1, "test${it}"), it * 2L) }
Assertions.assertEquals(10, channel1.messages.size)
@@ -198,7 +209,8 @@ class DestinationMessageQueueWriterTest {
repeat(count) { writer.publish(makeRecord(stream, "test"), 1L) }
writer.publish(makeStreamState(stream, count.toLong()), 0L)
val state =
queueWriterFactory.checkpointManager.streamStates[stream]!![stateLookupIndex]
queueWriterFactory.checkpointManager.streamStates[stream.descriptor]!![
stateLookupIndex]
Assertions.assertEquals(expectedCount, state.first)
Assertions.assertEquals(count.toLong(), state.second.destinationStats?.recordCount)
}
@@ -238,8 +250,8 @@ class DestinationMessageQueueWriterTest {
writer.publish(makeGlobalState(event.expectedStream1Count), 0L)
val state =
queueWriterFactory.checkpointManager.globalStates[event.stateLookupIndex]
val stream1State = state.first.find { it.first == stream1 }!!
val stream2State = state.first.find { it.first == stream2 }!!
val stream1State = state.first.find { it.first == stream1.descriptor }!!
val stream2State = state.first.find { it.first == stream2.descriptor }!!
Assertions.assertEquals(event.expectedStream1Count, stream1State.second)
Assertions.assertEquals(event.expectedStream2Count, stream2State.second)
Assertions.assertEquals(

View File

@@ -70,13 +70,13 @@ class MockStreamManager : StreamManager {
@Prototype
@Requires(env = ["MockStreamsManager"])
class MockStreamsManager(@Named("mockCatalog") catalog: DestinationCatalog) : StreamsManager {
private val mockManagers = catalog.streams.associateWith { MockStreamManager() }
private val mockManagers = catalog.streams.associate { it.descriptor to MockStreamManager() }
fun addPersistedRanges(stream: DestinationStream, ranges: List<Range<Long>>) {
mockManagers[stream]!!.persistedRanges.addAll(ranges)
mockManagers[stream.descriptor]!!.persistedRanges.addAll(ranges)
}
override fun getManager(stream: DestinationStream): StreamManager {
override fun getManager(stream: DestinationStream.Descriptor): StreamManager {
return mockManagers[stream] ?: throw IllegalArgumentException("Stream not found: $stream")
}

View File

@@ -83,7 +83,7 @@ class CheckpointManagerTest {
fun toMockCheckpointIn() = MockStreamCheckpointIn(stream, message)
}
data class TestGlobalMessage(
val streamIndexes: List<Pair<DestinationStream, Long>>,
val streamIndexes: List<Pair<DestinationStream.Descriptor, Long>>,
val message: Int
) : TestEvent() {
fun toMockCheckpointIn() = MockGlobalCheckpointIn(message)
@@ -180,8 +180,14 @@ class CheckpointManagerTest {
name = "Global checkpoint, two messages, flush all",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
FlushPoint(
persistedRanges =
mapOf(
@@ -196,8 +202,14 @@ class CheckpointManagerTest {
name = "Global checkpoint, two messages, range only covers the first",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
FlushPoint(
persistedRanges =
mapOf(
@@ -213,8 +225,14 @@ class CheckpointManagerTest {
"Global checkpoint, two messages, where the range only covers *one stream*",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
FlushPoint(
mapOf(
stream1 to listOf(Range.closed(0L, 20L)),
@@ -228,8 +246,14 @@ class CheckpointManagerTest {
name = "Global checkpoint, out of order (should fail)",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
FlushPoint(
mapOf(
stream1 to listOf(Range.closed(0L, 20L)),
@@ -244,7 +268,10 @@ class CheckpointManagerTest {
events =
listOf(
TestStreamMessage(stream1, 10L, 1),
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
FlushPoint(
mapOf(
stream1 to listOf(Range.closed(0L, 20L)),
@@ -258,7 +285,10 @@ class CheckpointManagerTest {
name = "Mixed: first global, then stream checkpoint (should fail)",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
TestStreamMessage(stream1, 20L, 2),
FlushPoint(
persistedRanges =
@@ -303,15 +333,24 @@ class CheckpointManagerTest {
name = "Global checkpoint, multiple flush points, no output",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
FlushPoint(),
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
FlushPoint(
mapOf(
stream1 to listOf(Range.closed(0L, 20L)),
)
),
TestGlobalMessage(listOf(stream1 to 30L, stream2 to 40L), 3),
TestGlobalMessage(
listOf(stream1.descriptor to 30L, stream2.descriptor to 40L),
3
),
FlushPoint(mapOf(stream2 to listOf(Range.closed(20L, 30L))))
),
expectedGlobalOutput = listOf()
@@ -320,15 +359,24 @@ class CheckpointManagerTest {
name = "Global checkpoint, multiple flush points, no output until end",
events =
listOf(
TestGlobalMessage(listOf(stream1 to 10L, stream2 to 20L), 1),
TestGlobalMessage(
listOf(stream1.descriptor to 10L, stream2.descriptor to 20L),
1
),
FlushPoint(),
TestGlobalMessage(listOf(stream1 to 20L, stream2 to 30L), 2),
TestGlobalMessage(
listOf(stream1.descriptor to 20L, stream2.descriptor to 30L),
2
),
FlushPoint(
mapOf(
stream1 to listOf(Range.closed(0L, 20L)),
)
),
TestGlobalMessage(listOf(stream1 to 30L, stream2 to 40L), 3),
TestGlobalMessage(
listOf(stream1.descriptor to 30L, stream2.descriptor to 40L),
3
),
FlushPoint(
mapOf(
stream1 to listOf(Range.closed(20L, 30L)),
@@ -374,7 +422,7 @@ class CheckpointManagerTest {
when (it) {
is TestStreamMessage -> {
checkpointManager.addStreamCheckpoint(
it.stream,
it.stream.descriptor,
it.index,
it.toMockCheckpointIn()
)

View File

@@ -16,13 +16,13 @@ import java.util.concurrent.atomic.AtomicBoolean
import kotlinx.coroutines.CompletableDeferred
class MockStreamsManager(@Named("mockCatalog") catalog: DestinationCatalog) : StreamsManager {
private val mockManagers = catalog.streams.associateWith { MockStreamManager() }
private val mockManagers = catalog.streams.associate { it.descriptor to MockStreamManager() }
fun addPersistedRanges(stream: DestinationStream, ranges: List<Range<Long>>) {
mockManagers[stream]!!.persistedRanges.addAll(ranges)
mockManagers[stream.descriptor]!!.persistedRanges.addAll(ranges)
}
override fun getManager(stream: DestinationStream): StreamManager {
override fun getManager(stream: DestinationStream.Descriptor): StreamManager {
return mockManagers[stream] ?: throw IllegalArgumentException("Stream not found: $stream")
}

View File

@@ -5,12 +5,10 @@
package io.airbyte.cdk.state
import com.google.common.collect.Range
import io.airbyte.cdk.command.Append
import io.airbyte.cdk.command.DestinationCatalog
import io.airbyte.cdk.command.DestinationStream
import io.airbyte.cdk.command.MockCatalogFactory.Companion.stream1
import io.airbyte.cdk.command.MockCatalogFactory.Companion.stream2
import io.airbyte.cdk.data.NullType
import io.airbyte.cdk.message.Batch
import io.airbyte.cdk.message.BatchEnvelope
import io.airbyte.cdk.message.SimpleBatch
@@ -37,8 +35,8 @@ class StreamsManagerTest {
@Test
fun testCountRecordsAndCheckpoint() {
val streamsManager = StreamsManagerFactory(catalog).make()
val manager1 = streamsManager.getManager(stream1)
val manager2 = streamsManager.getManager(stream2)
val manager1 = streamsManager.getManager(stream1.descriptor)
val manager2 = streamsManager.getManager(stream2.descriptor)
// Incrementing once yields (n, n)
repeat(10) { manager1.countRecordIn() }
@@ -71,16 +69,7 @@ class StreamsManagerTest {
fun testGettingNonexistentManagerFails() {
val streamsManager = StreamsManagerFactory(catalog).make()
Assertions.assertThrows(IllegalArgumentException::class.java) {
streamsManager.getManager(
DestinationStream(
DestinationStream.Descriptor("test", "non-existent"),
importType = Append,
schema = NullType,
generationId = 42,
minimumGenerationId = 0,
syncId = 42,
)
)
streamsManager.getManager(DestinationStream.Descriptor("test", "non-existent"))
}
}
@@ -181,7 +170,7 @@ class StreamsManagerTest {
fun testUpdateBatchState(testCase: TestCase) {
val streamsManager = StreamsManagerFactory(catalog).make()
testCase.events.forEach { (stream, event) ->
val manager = streamsManager.getManager(stream)
val manager = streamsManager.getManager(stream.descriptor)
when (event) {
is SetRecordCount -> repeat(event.count.toInt()) { manager.countRecordIn() }
is SetEndOfStream -> manager.countEndOfStream()
@@ -218,7 +207,7 @@ class StreamsManagerTest {
@Test
fun testCannotUpdateOrCloseReadClosedStream() {
val streamsManager = StreamsManagerFactory(catalog).make()
val manager = streamsManager.getManager(stream1)
val manager = streamsManager.getManager(stream1.descriptor)
// Can't close before end-of-stream
Assertions.assertThrows(IllegalStateException::class.java) { manager.markClosed() }
@@ -237,7 +226,7 @@ class StreamsManagerTest {
@Test
fun testAwaitStreamClosed() = runTest {
val streamsManager = StreamsManagerFactory(catalog).make()
val manager = streamsManager.getManager(stream1)
val manager = streamsManager.getManager(stream1.descriptor)
val hasClosed = AtomicBoolean(false)
val job = launch {
@@ -259,8 +248,8 @@ class StreamsManagerTest {
@Test
fun testAwaitAllStreamsClosed() = runTest {
val streamsManager = StreamsManagerFactory(catalog).make()
val manager1 = streamsManager.getManager(stream1)
val manager2 = streamsManager.getManager(stream2)
val manager1 = streamsManager.getManager(stream1.descriptor)
val manager2 = streamsManager.getManager(stream2.descriptor)
val allHaveClosed = AtomicBoolean(false)
val awaitStream1 = launch { manager1.awaitStreamClosed() }

View File

@@ -21,7 +21,7 @@ import io.airbyte.cdk.state.CheckpointManager
import io.airbyte.cdk.state.MockStreamManager
import io.airbyte.cdk.state.MockStreamsManager
import io.airbyte.cdk.state.StreamsManager
import io.airbyte.cdk.write.DestinationWriteOperation
import io.airbyte.cdk.write.DestinationWriter
import io.airbyte.cdk.write.StreamLoader
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Primary
@@ -72,15 +72,15 @@ class DestinationTaskLauncherTest {
@Replaces(DefaultSpillToDiskTaskFactory::class)
@Requires(env = ["DestinationTaskLauncherTest"])
class MockSpillToDiskTaskFactory(catalog: DestinationCatalog) : SpillToDiskTaskFactory {
val streamHasRun = mutableMapOf<DestinationStream, Channel<Unit>>()
val streamHasRun = mutableMapOf<DestinationStream.Descriptor, Channel<Unit>>()
init {
catalog.streams.forEach { streamHasRun[it] = Channel(Channel.UNLIMITED) }
catalog.streams.forEach { streamHasRun[it.descriptor] = Channel(Channel.UNLIMITED) }
}
override fun make(
taskLauncher: DestinationTaskLauncher,
stream: DestinationStream
stream: DestinationStream.Descriptor
): SpillToDiskTask {
return object : SpillToDiskTask {
override suspend fun execute() {
@@ -196,11 +196,12 @@ class DestinationTaskLauncherTest {
@Singleton
@Primary
@Requires(env = ["DestinationTaskLauncherTest"])
class MockCheckpointManager : CheckpointManager<DestinationStream, CheckpointMessage> {
class MockCheckpointManager :
CheckpointManager<DestinationStream.Descriptor, CheckpointMessage> {
val hasBeenFlushed = Channel<Unit>()
override fun addStreamCheckpoint(
key: DestinationStream,
key: DestinationStream.Descriptor,
index: Long,
checkpointMessage: CheckpointMessage
) {
@@ -208,7 +209,7 @@ class DestinationTaskLauncherTest {
}
override fun addGlobalCheckpoint(
keyIndexes: List<Pair<DestinationStream, Long>>,
keyIndexes: List<Pair<DestinationStream.Descriptor, Long>>,
checkpointMessage: CheckpointMessage
) {
TODO("Not needed")
@@ -219,7 +220,7 @@ class DestinationTaskLauncherTest {
}
}
class MockDestinationWrite : DestinationWriteOperation {
class MockDestinationWrite : DestinationWriter {
override fun getStreamLoader(stream: DestinationStream): StreamLoader {
return object : StreamLoader {
override val stream: DestinationStream = stream
@@ -263,7 +264,7 @@ class DestinationTaskLauncherTest {
// This will block until the stream is done opening.
launch {
launcher.handleNewSpilledFile(
stream1,
stream1.descriptor,
BatchEnvelope(
SpilledRawMessagesLocalFile(DefaultLocalFile(Path("not/a/real/file")), 100L)
)
@@ -299,7 +300,9 @@ class DestinationTaskLauncherTest {
// Verify incomplete batch triggers process batch
val incompleteBatch = BatchEnvelope(MockBatch(Batch.State.PERSISTED), range)
launcher.handleNewBatch(streamLoader, incompleteBatch)
Assertions.assertTrue(streamsManager.getManager(stream1).areRecordsPersistedUntil(100L))
Assertions.assertTrue(
streamsManager.getManager(stream1.descriptor).areRecordsPersistedUntil(100L)
)
val batchReceived = processBatchTaskFactory.hasRun.receive()
Assertions.assertEquals(incompleteBatch, batchReceived)
@@ -308,7 +311,8 @@ class DestinationTaskLauncherTest {
launcher.handleNewBatch(streamLoader, completeBatch)
delay(1000)
Assertions.assertTrue(closeStreamTaskFactory.hasRun.tryReceive().isFailure)
(streamsManager.getManager(stream1) as MockStreamManager).mockBatchProcessingComplete(true)
(streamsManager.getManager(stream1.descriptor) as MockStreamManager)
.mockBatchProcessingComplete(true)
// Verify complete batch w/ batch processing complete triggers close stream
launcher.handleNewBatch(streamLoader, completeBatch)
@@ -329,11 +333,11 @@ class DestinationTaskLauncherTest {
val hasRun = teardownTaskFactory.hasRun.tryReceive()
Assertions.assertTrue(hasRun.isFailure)
checkpointManager.hasBeenFlushed.receive() // Stream1 close triggered flush
streamsManager.getManager(stream1).markClosed()
streamsManager.getManager(stream1.descriptor).markClosed()
delay(1000)
val hasRun2 = teardownTaskFactory.hasRun.tryReceive()
Assertions.assertTrue(hasRun2.isFailure)
streamsManager.getManager(stream2).markClosed()
streamsManager.getManager(stream2.descriptor).markClosed()
teardownTaskFactory.hasRun.receive()
Assertions.assertTrue(true)

View File

@@ -22,7 +22,7 @@ class MockTaskLauncher(override val taskRunner: TaskRunner) : DestinationTaskLau
}
override suspend fun handleNewSpilledFile(
stream: DestinationStream,
stream: DestinationStream.Descriptor,
wrapped: BatchEnvelope<SpilledRawMessagesLocalFile>
) {
spilledFiles.add(wrapped)

View File

@@ -68,7 +68,7 @@ class ProcessRecordsTaskTest {
class MockDeserializer : Deserializer<DestinationMessage> {
override fun deserialize(serialized: String): DestinationMessage {
return DestinationRecord(
stream = stream1,
stream = stream1.descriptor,
data = IntegerValue(serialized.toLong()),
emittedAtMs = 0L,
meta = null,

View File

@@ -4,10 +4,9 @@
package io.airbyte.cdk.task
import io.airbyte.cdk.command.DefaultWriteConfiguration
import io.airbyte.cdk.command.DestinationConfiguration
import io.airbyte.cdk.command.DestinationStream
import io.airbyte.cdk.command.MockCatalogFactory.Companion.stream1
import io.airbyte.cdk.command.WriteConfiguration
import io.airbyte.cdk.data.NullValue
import io.airbyte.cdk.file.MockTempFileProvider
import io.airbyte.cdk.message.DestinationRecord
@@ -16,6 +15,7 @@ import io.airbyte.cdk.message.MessageQueueReader
import io.airbyte.cdk.message.StreamCompleteWrapped
import io.airbyte.cdk.message.StreamRecordWrapped
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Primary
import io.micronaut.context.annotation.Requires
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
import jakarta.inject.Inject
@@ -44,8 +44,9 @@ class SpillToDiskTaskTest {
}
@Singleton
@Primary
@Requires(env = ["SpillToDiskTaskTest"])
class MockWriteConfiguration : DefaultWriteConfiguration(), WriteConfiguration {
class MockWriteConfiguration : DestinationConfiguration() {
override val recordBatchSizeBytes: Long = 1024L
override val tmpFileDirectory: Path = Path.of("/tmp-test")
override val firstStageTmpFilePrefix: String = "spilled"
@@ -54,12 +55,13 @@ class SpillToDiskTaskTest {
@Singleton
@Requires(env = ["SpillToDiskTaskTest"])
class MockQueueReader : MessageQueueReader<DestinationStream, DestinationRecordWrapped> {
class MockQueueReader :
MessageQueueReader<DestinationStream.Descriptor, DestinationRecordWrapped> {
// Make enough records for a full batch + half a batch
private val maxRecords = ((1024 * 1.5) / 8).toLong()
private val recordsWritten = AtomicLong(0)
override suspend fun readChunk(
key: DestinationStream,
key: DestinationStream.Descriptor,
maxBytes: Long
): Flow<DestinationRecordWrapped> = flow {
var totalBytes = 0
@@ -71,7 +73,7 @@ class SpillToDiskTaskTest {
sizeBytes = 8,
record =
DestinationRecord(
stream = stream1,
stream = stream1.descriptor,
data = NullValue,
emittedAtMs = 0,
meta = null,
@@ -91,7 +93,7 @@ class SpillToDiskTaskTest {
@Test
fun testSpillToDiskTask() = runTest {
val mockTaskLauncher = MockTaskLauncher(taskRunner)
spillToDiskTaskFactory.make(mockTaskLauncher, stream1).execute()
spillToDiskTaskFactory.make(mockTaskLauncher, stream1.descriptor).execute()
Assertions.assertEquals(2, mockTaskLauncher.spilledFiles.size)
Assertions.assertEquals(1024, mockTaskLauncher.spilledFiles[0].batch.totalSizeBytes)
Assertions.assertEquals(512, mockTaskLauncher.spilledFiles[1].batch.totalSizeBytes)

View File

@@ -0,0 +1,121 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
import java.time.OffsetDateTime
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test
class RecordDifferTest {
@Test
fun testBasicBehavior() {
val differ =
RecordDiffer(
primaryKey = listOf(listOf("id1"), listOf("id2")),
cursor = listOf("updated_at"),
)
val diff =
differ.diffRecords(
expectedRecords =
listOf(
// Extra expected record
OutputRecord(
extractedAt = 1234,
generationId = 42,
mapOf(
"id1" to 1,
"id2" to 100,
"updated_at" to OffsetDateTime.parse("1970-01-01T00:00:00Z"),
"name" to "alice",
"phone" to "1234"
),
airbyteMeta = null
),
// Matching records
OutputRecord(
extractedAt = 1234,
generationId = 42,
mapOf(
"id1" to 1,
"id2" to 100,
"updated_at" to OffsetDateTime.parse("1970-01-01T00:00:01Z"),
"name" to "bob",
),
airbyteMeta = null
),
// Different records
OutputRecord(
extractedAt = 1234,
generationId = 42,
mapOf(
"id1" to 1,
"id2" to 100,
"updated_at" to OffsetDateTime.parse("1970-01-01T00:00:02Z"),
"name" to "charlie",
"phone" to "1234",
"email" to "charlie@example.com"
),
airbyteMeta = """{"sync_id": 12}""",
),
),
actualRecords =
listOf(
// Matching records
OutputRecord(
extractedAt = 1234,
generationId = 42,
mapOf(
"id1" to 1,
"id2" to 100,
"updated_at" to OffsetDateTime.parse("1970-01-01T00:00:01Z"),
"name" to "bob",
),
airbyteMeta = null
),
// Different records
OutputRecord(
extractedAt = 1234,
generationId = 41,
mapOf(
"id1" to 1,
"id2" to 100,
"updated_at" to OffsetDateTime.parse("1970-01-01T00:00:02Z"),
"name" to "charlie",
"phone" to "5678",
"address" to "1234 charlie street"
),
airbyteMeta = null
),
// Extra actual record
OutputRecord(
extractedAt = 1234,
generationId = 42,
mapOf(
"id1" to 1,
"id2" to 100,
"updated_at" to OffsetDateTime.parse("1970-01-01T00:00:03Z"),
"name" to "dana",
),
airbyteMeta = null
),
),
)
Assertions.assertEquals(
"""
Missing record (pk=[IntegerValue(value=1), IntegerValue(value=100)], cursor=TimestampValue(value=1970-01-01T00:00Z)): OutputRecord(rawId=null, extractedAt=1970-01-01T00:00:01.234Z, loadedAt=null, generationId=42, data=ObjectValue(values={id1=IntegerValue(value=1), id2=IntegerValue(value=100), updated_at=TimestampValue(value=1970-01-01T00:00Z), name=StringValue(value=alice), phone=StringValue(value=1234)}), airbyteMeta=null)
Incorrect record (pk=[IntegerValue(value=1), IntegerValue(value=100)], cursor=TimestampValue(value=1970-01-01T00:00:02Z)):
generationId: Expected 42, got 41
airbyteMeta: Expected {"sync_id":12}, got null
phone: Expected StringValue(value=1234), but was StringValue(value=5678)
email: Expected StringValue(value=charlie@example.com), but was <unset>
address: Expected <unset>, but was StringValue(value=1234 charlie street)
Unexpected record (pk=[IntegerValue(value=1), IntegerValue(value=100)], cursor=TimestampValue(value=1970-01-01T00:00:03Z)): OutputRecord(rawId=null, extractedAt=1970-01-01T00:00:01.234Z, loadedAt=null, generationId=42, data=ObjectValue(values={id1=IntegerValue(value=1), id2=IntegerValue(value=100), updated_at=TimestampValue(value=1970-01-01T00:00:03Z), name=StringValue(value=dana)}), airbyteMeta=null)
""".trimIndent(),
diff
)
}
}

View File

@@ -0,0 +1,86 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.check
import io.airbyte.cdk.command.ConfigurationJsonObjectBase
import io.airbyte.cdk.command.ValidatedJsonUtils
import io.airbyte.cdk.test.util.FakeDataDumper
import io.airbyte.cdk.test.util.IntegrationTest
import io.airbyte.cdk.test.util.NoopDestinationCleaner
import io.airbyte.cdk.test.util.NoopExpectedRecordMapper
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import java.nio.charset.StandardCharsets
import java.nio.file.Files
import java.nio.file.Path
import java.util.regex.Pattern
import kotlin.test.assertEquals
import org.junit.jupiter.api.Assertions.assertTrue
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.assertAll
open class CheckIntegrationTest<T : ConfigurationJsonObjectBase>(
val configurationClass: Class<T>,
val successConfigFilenames: List<String>,
val failConfigFilenamesAndFailureReasons: Map<String, Pattern>,
) :
IntegrationTest(
FakeDataDumper,
NoopDestinationCleaner,
NoopExpectedRecordMapper,
) {
@Test
open fun testSuccessConfigs() {
for (path in successConfigFilenames) {
val fileContents = Files.readString(Path.of(path), StandardCharsets.UTF_8)
val config = ValidatedJsonUtils.parseOne(configurationClass, fileContents)
val process =
destinationProcessFactory.createDestinationProcess("check", config = config)
process.run()
val messages = process.readMessages()
val checkMessages = messages.filter { it.type == AirbyteMessage.Type.CONNECTION_STATUS }
assertEquals(
checkMessages.size,
1,
"Expected to receive exactly one connection status message, but got ${checkMessages.size}: $checkMessages"
)
assertEquals(
AirbyteConnectionStatus.Status.SUCCEEDED,
checkMessages.first().connectionStatus.status
)
}
}
@Test
open fun testFailConfigs() {
for ((path, failurePattern) in failConfigFilenamesAndFailureReasons) {
val fileContents = Files.readString(Path.of(path))
val config = ValidatedJsonUtils.parseOne(configurationClass, fileContents)
val process =
destinationProcessFactory.createDestinationProcess("check", config = config)
process.run()
val messages = process.readMessages()
val checkMessages = messages.filter { it.type == AirbyteMessage.Type.CONNECTION_STATUS }
assertEquals(
checkMessages.size,
1,
"Expected to receive exactly one connection status message, but got ${checkMessages.size}: $checkMessages"
)
val connectionStatus = checkMessages.first().connectionStatus
assertAll(
{ assertEquals(AirbyteConnectionStatus.Status.FAILED, connectionStatus.status) },
{
assertTrue(
failurePattern.matcher(connectionStatus.message).matches(),
"Expected to match ${failurePattern.pattern()}, but got ${connectionStatus.message}"
)
}
)
}
}
}

View File

@@ -0,0 +1,81 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.spec
import com.deblock.jsondiff.DiffGenerator
import com.deblock.jsondiff.diff.JsonDiff
import com.deblock.jsondiff.matcher.CompositeJsonMatcher
import com.deblock.jsondiff.matcher.JsonMatcher
import com.deblock.jsondiff.matcher.LenientJsonObjectPartialMatcher
import com.deblock.jsondiff.matcher.StrictJsonArrayPartialMatcher
import com.deblock.jsondiff.matcher.StrictPrimitivePartialMatcher
import com.deblock.jsondiff.viewer.OnlyErrorDiffViewer
import io.airbyte.cdk.test.util.DestinationProcessFactory
import io.airbyte.cdk.test.util.FakeDataDumper
import io.airbyte.cdk.test.util.IntegrationTest
import io.airbyte.cdk.test.util.NoopDestinationCleaner
import io.airbyte.cdk.test.util.NoopExpectedRecordMapper
import io.airbyte.cdk.util.Jsons
import io.airbyte.protocol.models.v0.AirbyteMessage
import java.nio.file.Files
import java.nio.file.Path
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.assertAll
private const val EXPECTED_SPEC_FILENAME = "expected-spec.json"
private val expectedSpecPath = Path.of(EXPECTED_SPEC_FILENAME)
/**
* This is largely copied from [io.airbyte.cdk.spec.SpecTest], but adapted to use our
* [DestinationProcessFactory].
*
* It also automatically writes the actual spec back to `expected-spec.json` for easier inspection
* of the diff. This diff is _really messy_ for the initial migration from the old CDK to the new
* one, but after that, it should be pretty readable.
*/
abstract class SpecTest :
IntegrationTest(
FakeDataDumper,
NoopDestinationCleaner,
NoopExpectedRecordMapper,
) {
@Test
fun testSpec() {
if (!Files.exists(expectedSpecPath)) {
Files.createFile(expectedSpecPath)
}
val expectedSpec = Files.readString(expectedSpecPath)
val process = destinationProcessFactory.createDestinationProcess("spec")
process.run()
val messages = process.readMessages()
val specMessages = messages.filter { it.type == AirbyteMessage.Type.SPEC }
assertEquals(
specMessages.size,
1,
"Expected to receive exactly one connection status message, but got ${specMessages.size}: $specMessages"
)
val spec = specMessages.first().spec
val actualSpecPrettyPrint: String =
Jsons.writerWithDefaultPrettyPrinter().writeValueAsString(spec)
Files.write(expectedSpecPath, actualSpecPrettyPrint.toByteArray())
val jsonMatcher: JsonMatcher =
CompositeJsonMatcher(
StrictJsonArrayPartialMatcher(),
LenientJsonObjectPartialMatcher(),
StrictPrimitivePartialMatcher(),
)
val diff: JsonDiff =
DiffGenerator.diff(expectedSpec, Jsons.writeValueAsString(spec), jsonMatcher)
assertAll(
"Spec snapshot test failed. Run this test locally and then `git diff <...>/expected_spec.json` to see what changed, and commit the diff if that change was intentional.",
{ assertEquals("", OnlyErrorDiffViewer.from(diff).toString()) },
{ assertEquals(expectedSpec, actualSpecPrettyPrint) }
)
}
}

View File

@@ -0,0 +1,17 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
fun interface DestinationCleaner {
/**
* Search the test destination for old test data and delete it. This should leave recent data
* (e.g. from the last week) untouched, to avoid causing failures in actively-running tests.
*/
fun cleanup()
}
object NoopDestinationCleaner : DestinationCleaner {
override fun cleanup() {}
}

View File

@@ -0,0 +1,22 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
fun interface DestinationDataDumper {
fun dumpRecords(
streamName: String,
streamNamespace: String?,
): List<OutputRecord>
}
/**
* Some integration tests don't need to actually read records from the destination, and can use this
* implementation to satisfy the compiler.
*/
object FakeDataDumper : DestinationDataDumper {
override fun dumpRecords(streamName: String, streamNamespace: String?): List<OutputRecord> {
throw NotImplementedError()
}
}

View File

@@ -0,0 +1,193 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.cdk.command.CliRunnable
import io.airbyte.cdk.command.CliRunner
import io.airbyte.cdk.command.ConfigurationJsonObjectBase
import io.airbyte.protocol.models.Jsons
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import io.github.oshai.kotlinlogging.KotlinLogging
import java.io.ByteArrayOutputStream
import java.io.InputStream
import java.io.PipedInputStream
import java.io.PipedOutputStream
import java.io.PrintWriter
import javax.inject.Singleton
private val logger = KotlinLogging.logger {}
/**
* Represents a destination process, whether running in-JVM via micronaut, or as a separate Docker
* container. The general lifecycle is:
* 1. `val dest = DestinationProcessFactory.createDestinationProcess(...)`
* 2. `launch { dest.run() }`
* 3. [sendMessage] as many times as you want
* 4. [readMessages] as needed (e.g. to check that state messages are emitted during the sync)
* 5. [shutdown] once you have no more messages to send to the destination
*/
interface DestinationProcess {
/**
* Run the destination process. Callers who want to interact with the destination should
* `launch` this method.
*/
fun run()
fun sendMessage(message: AirbyteMessage)
/** Return all messages the destination emitted since the last call to [readMessages]. */
fun readMessages(): List<AirbyteMessage>
/**
* Wait for the destination to terminate, then return all messages it emitted since the last
* call to [readMessages].
*/
fun shutdown()
}
interface DestinationProcessFactory {
fun createDestinationProcess(
command: String,
config: ConfigurationJsonObjectBase? = null,
catalog: ConfiguredAirbyteCatalog? = null,
): DestinationProcess
}
class NonDockerizedDestination(
command: String,
config: ConfigurationJsonObjectBase?,
catalog: ConfiguredAirbyteCatalog?,
) : DestinationProcess {
private val destinationStdinPipe: PrintWriter
private val destination: CliRunnable
init {
val destinationStdin = PipedInputStream()
// This could probably be a channel, somehow. But given the current structure,
// it's easier to just use the pipe stuff.
destinationStdinPipe =
// spotbugs requires explicitly specifying the charset,
// so we also have to specify autoFlush=false (i.e. the default behavior
// from PrintWriter(outputStream) ).
// Thanks, spotbugs.
PrintWriter(PipedOutputStream(destinationStdin), false, Charsets.UTF_8)
destination =
CliRunner.destination(
command,
config = config,
catalog = catalog,
inputStream = destinationStdin,
)
}
override fun run() {
destination.run()
}
override fun sendMessage(message: AirbyteMessage) {
destinationStdinPipe.println(Jsons.serialize(message))
}
override fun readMessages(): List<AirbyteMessage> = destination.results.newMessages()
override fun shutdown() {
destinationStdinPipe.close()
}
}
// Notably, not actually a Micronaut factory. We want to inject the actual
// factory into our tests, not a pre-instantiated destination, because we want
// to run multiple destination processes per test.
// TODO only inject this when not running in CI, a la @Requires(notEnv = "CI_master_merge")
@Singleton
class NonDockerizedDestinationFactory : DestinationProcessFactory {
override fun createDestinationProcess(
command: String,
config: ConfigurationJsonObjectBase?,
catalog: ConfiguredAirbyteCatalog?
): DestinationProcess {
return NonDockerizedDestination(command, config, catalog)
}
}
// TODO define a factory for this class + @Require(env = CI_master_merge)
// suppress the unused argument warnings in the kotlin compiler
class DockerizedDestination(
val command: String,
val config: JsonNode?,
val catalog: ConfiguredAirbyteCatalog?,
) : DestinationProcess {
override fun run() {
TODO("launch a docker container")
}
override fun sendMessage(message: AirbyteMessage) {
// push a message to the docker process' stdin
TODO("Not yet implemented")
}
override fun readMessages(): List<AirbyteMessage> {
// read everything from the process' stdout
TODO("Not yet implemented")
}
override fun shutdown() {
// close stdin, wait until process exits
TODO("Not yet implemented")
}
}
// This is currently unused, but we'll need it for the Docker version.
// it exists right now b/c I wrote it prior to the CliRunner retooling.
/**
* There doesn't seem to be a built-in equivalent to this? Scanner and BufferedReader both have
* `hasNextLine` methods which block until the stream has data to read, which we don't want to do.
*
* This class simply buffers the next line in-memory until it reaches a newline or EOF.
*/
private class LazyInputStreamReader(private val input: InputStream) {
private val buffer: ByteArrayOutputStream = ByteArrayOutputStream()
private var eof = false
/**
* Returns the next line of data, or null if no line is available. Doesn't block if the
* inputstream has no data.
*/
fun nextLine(): MaybeLine {
if (eof) {
return NoLine.EOF
}
while (input.available() != 0) {
when (val read = input.read()) {
-1 -> {
eof = true
val line = Line(buffer.toByteArray().toString(Charsets.UTF_8))
buffer.reset()
return line
}
'\n'.code -> {
val bytes = buffer.toByteArray()
buffer.reset()
return Line(bytes.toString(Charsets.UTF_8))
}
else -> {
buffer.write(read)
}
}
}
return NoLine.NOT_YET_AVAILABLE
}
companion object {
interface MaybeLine
enum class NoLine : MaybeLine {
EOF,
NOT_YET_AVAILABLE
}
data class Line(val line: String) : MaybeLine
}
}

View File

@@ -0,0 +1,13 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
fun interface ExpectedRecordMapper {
fun mapRecord(expectedRecord: OutputRecord): OutputRecord
}
object NoopExpectedRecordMapper : ExpectedRecordMapper {
override fun mapRecord(expectedRecord: OutputRecord): OutputRecord = expectedRecord
}

View File

@@ -0,0 +1,143 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
import io.airbyte.cdk.command.ConfigurationJsonObjectBase
import io.airbyte.cdk.command.DestinationCatalog
import io.airbyte.cdk.command.DestinationStream
import io.airbyte.cdk.message.DestinationMessage
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage
import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage.AirbyteStreamStatus
import io.airbyte.protocol.models.v0.AirbyteTraceMessage
import io.airbyte.protocol.models.v0.StreamDescriptor
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
import java.time.Instant
import java.time.LocalDateTime
import java.time.ZoneOffset
import java.time.format.DateTimeFormatter
import java.util.concurrent.atomic.AtomicBoolean
import javax.inject.Inject
import kotlin.test.fail
import kotlinx.coroutines.async
import kotlinx.coroutines.runBlocking
import org.apache.commons.lang3.RandomStringUtils
import org.junit.jupiter.api.AfterEach
import org.junit.jupiter.api.parallel.Execution
import org.junit.jupiter.api.parallel.ExecutionMode
@MicronautTest
@Execution(ExecutionMode.CONCURRENT)
// Spotbugs doesn't let you suppress the actual lateinit property,
// so we have to suppress the entire class.
// Thanks, spotbugs.
@SuppressFBWarnings("NP_NONNULL_RETURN_VIOLATION", justification = "Micronaut DI")
abstract class IntegrationTest(
val dataDumper: DestinationDataDumper,
val destinationCleaner: DestinationCleaner,
val recordMangler: ExpectedRecordMapper = NoopExpectedRecordMapper,
val nameMapper: NameMapper = NoopNameMapper,
) {
// Intentionally don't inject the actual destination process - we need a full factory
// because some tests want to run multiple syncs, so we need to run the destination
// multiple times.
@Inject lateinit var destinationProcessFactory: DestinationProcessFactory
@Suppress("DEPRECATION") private val randomSuffix = RandomStringUtils.randomAlphabetic(4)
private val timestampString =
LocalDateTime.ofInstant(Instant.now(), ZoneOffset.UTC)
.format(DateTimeFormatter.ofPattern("YYYYMMDD"))
// stream name doesn't need to be randomized, only the namespace.
val randomizedNamespace = "test$timestampString$randomSuffix"
@AfterEach
fun teardown() {
if (hasRunCleaner.compareAndSet(false, true)) {
destinationCleaner.cleanup()
}
}
fun dumpAndDiffRecords(
canonicalExpectedRecords: List<OutputRecord>,
streamName: String,
streamNamespace: String?,
primaryKey: List<List<String>>,
cursor: List<String>?,
) {
val actualRecords: List<OutputRecord> = dataDumper.dumpRecords(streamName, streamNamespace)
val expectedRecords: List<OutputRecord> =
canonicalExpectedRecords.map { recordMangler.mapRecord(it) }
RecordDiffer(
primaryKey.map { nameMapper.mapFieldName(it) },
cursor?.let { nameMapper.mapFieldName(it) },
)
.diffRecords(expectedRecords, actualRecords)
?.let(::fail)
}
/** Convenience wrapper for [runSync] using a single stream. */
fun runSync(
config: ConfigurationJsonObjectBase,
stream: DestinationStream,
messages: List<DestinationMessage>,
streamStatus: AirbyteStreamStatus? = AirbyteStreamStatus.COMPLETE,
): List<AirbyteMessage> =
runSync(config, DestinationCatalog(listOf(stream)), messages, streamStatus)
/**
* Run a sync with the given config+stream+messages, sending a trace message at the end of the
* sync with the given stream status for every stream. [messages] should not include
* [AirbyteStreamStatus] messages unless [streamStatus] is set to `null` (unless you actually
* want to send multiple stream status messages).
*/
fun runSync(
config: ConfigurationJsonObjectBase,
catalog: DestinationCatalog,
messages: List<DestinationMessage>,
streamStatus: AirbyteStreamStatus? = AirbyteStreamStatus.COMPLETE,
): List<AirbyteMessage> {
val destination =
destinationProcessFactory.createDestinationProcess(
"write",
config,
catalog.asProtocolObject(),
)
return runBlocking {
val destinationCompletion = async { destination.run() }
messages.forEach { destination.sendMessage(it.asProtocolMessage()) }
if (streamStatus != null) {
catalog.streams.forEach {
destination.sendMessage(
AirbyteMessage()
.withType(AirbyteMessage.Type.TRACE)
.withTrace(
AirbyteTraceMessage()
.withType(AirbyteTraceMessage.Type.STREAM_STATUS)
.withEmittedAt(System.currentTimeMillis().toDouble())
.withStreamStatus(
AirbyteStreamStatusTraceMessage()
.withStreamDescriptor(
StreamDescriptor()
.withName(it.descriptor.name)
.withNamespace(it.descriptor.namespace)
)
.withStatus(streamStatus)
)
)
)
}
}
destination.shutdown()
destinationCompletion.await()
destination.readMessages()
}
}
companion object {
private val hasRunCleaner = AtomicBoolean(false)
}
}

View File

@@ -0,0 +1,21 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
fun interface NameMapper {
/**
* Some destinations only need to mangle the top-level names (e.g. Snowflake, where we write
* nested data to a VARIANT column which preserves the nested field names), whereas other
* destinations need to mangle the entire path (e.g. Avro files).
*
* So we need to accept the entire path here, instead of just accepting individual path
* elements.
*/
fun mapFieldName(path: List<String>): List<String>
}
object NoopNameMapper : NameMapper {
override fun mapFieldName(path: List<String>): List<String> = path
}

View File

@@ -0,0 +1,63 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.ObjectMapper
import io.airbyte.cdk.data.ObjectValue
import java.time.Instant
import java.util.UUID
/** A record that we expect to exist in the destination, whether raw _or_ final. */
data class OutputRecord(
val rawId: UUID?,
val extractedAt: Instant,
val loadedAt: Instant?,
val generationId: Long?,
/**
* strongly-typed map, e.g. ZonedDateTime for timestamp_with_timezone. this makes destination
* test implementations easier. values can be null, b/c warehouse destinations with a JSON
* column type can be either SQL null, or JSON null, and we want to distinguish between those.
* Destinations _must_ filter out the airbyte_* fields from this map.
*/
val data: ObjectValue,
val airbyteMeta: JsonNode?,
) {
/** Utility constructor with easier types to write by hand */
constructor(
rawId: String,
extractedAt: Long,
loadedAt: Long?,
generationId: Long?,
data: Map<String, Any?>,
airbyteMeta: String?,
) : this(
UUID.fromString(rawId),
Instant.ofEpochMilli(extractedAt),
loadedAt?.let { Instant.ofEpochMilli(it) },
generationId,
ObjectValue.from(data),
airbyteMeta?.let { ObjectMapper().readTree(it) },
)
/**
* Utility constructor for "expected records". [rawId] and [loadedAt] are generated by the
* destination at runtime, so we don't have those when writing the test. Just generate arbitrary
* values for them.
*/
constructor(
extractedAt: Long,
generationId: Long?,
data: Map<String, Any?>,
airbyteMeta: String?,
) : this(
null,
Instant.ofEpochMilli(extractedAt),
loadedAt = null,
generationId,
ObjectValue.from(data),
airbyteMeta?.let { ObjectMapper().readTree(it) },
)
}

View File

@@ -0,0 +1,264 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.util
import io.airbyte.cdk.data.AirbyteValue
import io.airbyte.cdk.data.IntegerValue
import io.airbyte.cdk.data.NullValue
import io.airbyte.cdk.data.ObjectValue
import kotlin.reflect.jvm.jvmName
class RecordDiffer(
/**
* The path(s) to the primary key fields from a record. Most streams will have some `id`
* field(s), even if they're not running in dedup mode. This comparator lets us match records
* together to generate a more useful diff.
*
* In the rare case that a stream truly has no PK, the default value simply returns an empty
* list.
*/
val primaryKey: List<List<String>> = emptyList(),
/** The path to the cursor from a record, or null if the stream has no cursor. */
val cursor: List<String>? = null,
) {
private fun extract(data: Map<String, AirbyteValue>, path: List<String>): AirbyteValue {
return when (path.size) {
0 -> throw IllegalArgumentException("Empty path")
1 -> data[path.first()] ?: NullValue
else -> {
when (val next = data[path.first()]) {
null -> NullValue
is ObjectValue -> extract(next.values, path.subList(1, path.size))
else ->
throw IllegalArgumentException(
"Encountered non-map entry in path: $next at ${path.first()}"
)
}
}
}
}
// if primaryKey is empty list, this always returns emptyList.
private fun extractPrimaryKey(record: OutputRecord): List<AirbyteValue> {
val pks = mutableListOf<AirbyteValue>()
for (pkField in primaryKey) {
pks.add(extract(record.data.values, pkField))
}
return pks
}
private fun extractCursor(record: OutputRecord): AirbyteValue {
return extract(record.data.values, cursor!!)
}
/** Comparator that sorts records by their primary key */
private val identityComparator: Comparator<OutputRecord> = Comparator { rec1, rec2 ->
val pk1 = extractPrimaryKey(rec1)
val pk2 = extractPrimaryKey(rec2)
if (pk1.size != pk2.size) {
throw IllegalStateException(
"Records must have the same number of primary keys. Got $pk1 and $pk2."
)
}
// Compare each PK field in order, until we find a field that the two records differ in.
// If all the fields are equal, then these two records have the same PK.
pk1.zip(pk2)
.map { (pk1Field, pk2Field) -> valueComparator.compare(pk1Field, pk2Field) }
.firstOrNull { it != 0 }
?: 0
}
/**
* Comparator to sort records by their cursor (if there is one), breaking ties with extractedAt
*/
private val sortComparator: Comparator<OutputRecord> =
Comparator.comparing(
{ it: OutputRecord ->
(if (cursor == null) IntegerValue(0) else extractCursor(it))
},
valueComparator
)
.thenComparing { it -> it.extractedAt }
/**
* The actual comparator we'll use to sort the expected/actual record lists. I.e. group records
* by their PK, then within each PK, sort by cursor/extractedAt.
*/
private val everythingComparator = identityComparator.thenComparing(sortComparator)
/** Returns a pretty-printed diff of the two lists, or null if they were identical */
fun diffRecords(
expectedRecords: List<OutputRecord>,
actualRecords: List<OutputRecord>
): String? {
val expectedRecordsSorted = expectedRecords.sortedWith(everythingComparator)
val actualRecordsSorted = actualRecords.sortedWith(everythingComparator)
// Match up all the records between the expected and actual records,
// or if there's no matching record then detect that also.
// We'll filter this list down to actual differing records later on.
val matches = mutableListOf<MatchingRecords>()
var expectedRecordIndex = 0
var actualRecordIndex = 0
while (
expectedRecordIndex < expectedRecordsSorted.size &&
actualRecordIndex < actualRecordsSorted.size
) {
val expectedRecord = expectedRecords[expectedRecordIndex]
val actualRecord = actualRecords[actualRecordIndex]
val compare = everythingComparator.compare(expectedRecord, actualRecord)
if (compare == 0) {
// These records are the same underlying record
matches.add(MatchingRecords(expectedRecord, actualRecord))
expectedRecordIndex++
actualRecordIndex++
} else if (compare < 0) {
// There's an extra expected record
matches.add(MatchingRecords(expectedRecord, actualRecord = null))
expectedRecordIndex++
} else {
// There's an extra actual record
matches.add(MatchingRecords(expectedRecord = null, actualRecord))
actualRecordIndex++
}
}
// Tail loops in case we reached the end of one list before the other.
while (expectedRecordIndex < expectedRecords.size) {
matches.add(MatchingRecords(expectedRecords[expectedRecordIndex], actualRecord = null))
expectedRecordIndex++
}
while (actualRecordIndex < actualRecords.size) {
matches.add(MatchingRecords(expectedRecord = null, actualRecords[actualRecordIndex]))
actualRecordIndex++
}
// We've paired up all the records, now find just the ones that are wrong.
val diffs = matches.filter { it.isMismatch() }
return if (diffs.isEmpty()) {
null
} else {
diffs.joinToString("\n") { it.prettyPrintMismatch() }
}
}
private inner class MatchingRecords(
val expectedRecord: OutputRecord?,
val actualRecord: OutputRecord?,
) {
fun isMismatch(): Boolean =
(expectedRecord == null && actualRecord != null) ||
(expectedRecord != null && actualRecord == null) ||
!recordsMatch(expectedRecord, actualRecord)
fun prettyPrintMismatch(): String {
return if (expectedRecord == null) {
"Unexpected record (${generateRecordIdentifier(actualRecord!!)}): $actualRecord"
} else if (actualRecord == null) {
"Missing record (${generateRecordIdentifier(expectedRecord)}): $expectedRecord"
} else {
"Incorrect record (${generateRecordIdentifier(actualRecord)}):\n" +
generateDiffString(expectedRecord, actualRecord).prependIndent(" ")
}
}
private fun recordsMatch(
expectedRecord: OutputRecord?,
actualRecord: OutputRecord?,
): Boolean =
(expectedRecord == null && actualRecord == null) ||
(expectedRecord != null &&
actualRecord != null &&
generateDiffString(expectedRecord, actualRecord).isEmpty())
private fun generateRecordIdentifier(record: OutputRecord): String {
// If the PK is an empty list, then don't include it
val pk: List<AirbyteValue> = extractPrimaryKey(record)
val pkString = if (pk.isEmpty()) "" else "pk=$pk"
if (cursor != null) {
val cursor: AirbyteValue = extractCursor(record)
return "$pkString, cursor=$cursor"
} else {
return pkString
}
}
private fun generateDiffString(
expectedRecord: OutputRecord,
actualRecord: OutputRecord,
): String {
val diff: StringBuilder = StringBuilder()
// Intentionally don't diff loadedAt / rawId, since those are generated dynamically by
// the destination.
if (expectedRecord.extractedAt != actualRecord.extractedAt) {
diff.append(
"extractedAt: Expected ${expectedRecord.extractedAt}, got ${actualRecord.extractedAt}\n"
)
}
if (expectedRecord.generationId != actualRecord.generationId) {
diff.append(
"generationId: Expected ${expectedRecord.generationId}, got ${actualRecord.generationId}\n"
)
}
if (expectedRecord.airbyteMeta != actualRecord.airbyteMeta) {
diff.append(
"airbyteMeta: Expected ${expectedRecord.airbyteMeta}, got ${actualRecord.airbyteMeta}\n"
)
}
// Diff the data. Iterate over all keys in the expected/actual records and compare their
// values.
val allDataKeys: Set<String> =
expectedRecord.data.values.keys + actualRecord.data.values.keys
allDataKeys.forEach { key ->
val expectedPresent: Boolean = expectedRecord.data.values.containsKey(key)
val actualPresent: Boolean = actualRecord.data.values.containsKey(key)
if (expectedPresent && !actualPresent) {
// The expected record contained this key, but the actual record was missing
// this key.
diff.append(
"$key: Expected ${expectedRecord.data.values[key]}, but was <unset>\n"
)
} else if (!expectedPresent && actualPresent) {
// The expected record didn't contain this key, but the actual record contained
// this key.
diff.append(
"$key: Expected <unset>, but was ${actualRecord.data.values[key]}\n"
)
} else if (expectedPresent && actualPresent) {
// The expected and actual records both contain this key.
// Compare the values for equality.
// (actualPresent is always true here, but I think the if-tree is more readable
// with it explicitly in the condition)
val expectedValue = expectedRecord.data.values[key]
val actualValue = actualRecord.data.values[key]
if (expectedValue != actualValue) {
diff.append("$key: Expected $expectedValue, but was $actualValue\n")
}
}
}
return diff.toString().trim()
}
}
companion object {
val valueComparator: Comparator<AirbyteValue> =
Comparator.nullsFirst { v1, v2 -> compare(v1!!, v2!!) }
private fun compare(v1: AirbyteValue, v2: AirbyteValue): Int {
// when comparing values of different types, just sort by their class name.
// in theory, we could check for numeric types and handle them smartly...
// that's a lot of work though
return if (v1::class != v2::class) {
v1::class.jvmName.compareTo(v2::class.jvmName)
} else {
// otherwise, just be a terrible person.
// we know these are the same type, so this is safe to do.
@Suppress("UNCHECKED_CAST") (v1 as Comparable<AirbyteValue>).compareTo(v2)
}
}
}
}

View File

@@ -0,0 +1,126 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.test.write
import io.airbyte.cdk.command.Append
import io.airbyte.cdk.command.ConfigurationJsonObjectBase
import io.airbyte.cdk.command.DestinationStream
import io.airbyte.cdk.data.ObjectTypeWithoutSchema
import io.airbyte.cdk.message.DestinationRecord
import io.airbyte.cdk.message.StreamCheckpoint
import io.airbyte.cdk.test.util.DestinationCleaner
import io.airbyte.cdk.test.util.DestinationDataDumper
import io.airbyte.cdk.test.util.ExpectedRecordMapper
import io.airbyte.cdk.test.util.IntegrationTest
import io.airbyte.cdk.test.util.NameMapper
import io.airbyte.cdk.test.util.NoopExpectedRecordMapper
import io.airbyte.cdk.test.util.NoopNameMapper
import io.airbyte.cdk.test.util.OutputRecord
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import kotlin.test.assertEquals
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.assertAll
abstract class BasicFunctionalityIntegrationTest(
val config: ConfigurationJsonObjectBase,
dataDumper: DestinationDataDumper,
destinationCleaner: DestinationCleaner,
recordMangler: ExpectedRecordMapper = NoopExpectedRecordMapper,
nameMapper: NameMapper = NoopNameMapper,
/**
* Whether to actually verify that the connector wrote data to the destination. This should only
* ever be disabled for test destinations (dev-null, etc.).
*/
val verifyDataWriting: Boolean = true,
) : IntegrationTest(dataDumper, destinationCleaner, recordMangler, nameMapper) {
@Test
open fun testCheck() {
val process = destinationProcessFactory.createDestinationProcess("check", config = config)
process.run()
val messages = process.readMessages()
val checkMessages = messages.filter { it.type == AirbyteMessage.Type.CONNECTION_STATUS }
assertEquals(
checkMessages.size,
1,
"Expected to receive exactly one connection status message, but got ${checkMessages.size}: $checkMessages"
)
assertEquals(
AirbyteConnectionStatus.Status.SUCCEEDED,
checkMessages.first().connectionStatus.status
)
}
@Test
open fun testBasicWrite() {
val messages =
runSync(
config,
DestinationStream(
DestinationStream.Descriptor(randomizedNamespace, "test_stream"),
Append,
ObjectTypeWithoutSchema,
generationId = 0,
minimumGenerationId = 0,
syncId = 42,
),
listOf(
DestinationRecord(
namespace = randomizedNamespace,
name = "test_stream",
data = """{"id": 5678}""",
emittedAtMs = 1234,
),
StreamCheckpoint(
streamName = "test_stream",
streamNamespace = randomizedNamespace,
blob = """{"foo": "bar"}""",
sourceRecordCount = 1,
)
)
)
val stateMessages = messages.filter { it.type == AirbyteMessage.Type.STATE }
assertAll(
{
assertEquals(
1,
stateMessages.size,
"Expected to receive exactly one state message, got ${stateMessages.size} ($stateMessages)"
)
assertEquals(
StreamCheckpoint(
streamName = "test_stream",
streamNamespace = randomizedNamespace,
blob = """{"foo": "bar"}""",
sourceRecordCount = 1,
destinationRecordCount = 1,
)
.asProtocolMessage(),
stateMessages.first()
)
},
{
if (verifyDataWriting) {
dumpAndDiffRecords(
listOf(
OutputRecord(
extractedAt = 1234,
generationId = 0,
data = mapOf("id" to 5678),
airbyteMeta = """{"changes": [], "sync_id": 42}"""
)
),
"test_stream",
randomizedNamespace,
primaryKey = listOf(listOf("id")),
cursor = null,
)
}
},
)
}
}

View File

@@ -47,7 +47,7 @@ private val LOGGER = KotlinLogging.logger {}
*/
@Execution(ExecutionMode.CONCURRENT)
abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestinationState> {
protected var DIFFER: RecordDiffer = mock()
protected var DIFFER: LegacyRecordDiffer = mock()
/** Subclasses may use these four StreamConfigs in their tests. */
protected var incrementalDedupStream: StreamConfig = mock()
@@ -200,7 +200,7 @@ abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestina
AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE
DIFFER =
RecordDiffer(
LegacyRecordDiffer(
rawMetadataColumnNames,
finalMetadataColumnNames,
id1 to AirbyteProtocolType.INTEGER,

View File

@@ -60,7 +60,7 @@ private val LOGGER = KotlinLogging.logger {}
// execution.
@Execution(ExecutionMode.CONCURRENT)
abstract class BaseTypingDedupingTest {
protected var DIFFER: RecordDiffer? = null
protected var DIFFER: LegacyRecordDiffer? = null
private var randomSuffix: String? = null
protected var config: JsonNode? = null
@@ -205,7 +205,7 @@ abstract class BaseTypingDedupingTest {
val generator = sqlGenerator
DIFFER =
RecordDiffer(
LegacyRecordDiffer(
rawMetadataColumnNames,
finalMetadataColumnNames,
generator.buildColumnId("id1") to AirbyteProtocolType.INTEGER,

View File

@@ -22,8 +22,13 @@ import org.junit.jupiter.api.function.Executable
/**
* Utility class to generate human-readable diffs between expected and actual records. Assumes 1s1t
* output format.
*
* Prefer [io.airbyte.cdk.test.RecordDiffer], which operates on strongly-typed objects instead of
* JsonNodes. This class is effectively deprecated; we're just keeping it around so that
* [BaseTypingDedupingTest] and [BaseSqlGeneratorIntegrationTest] continue to function. Once those
* classes are using the new RecordDiffer, we should remove this class.
*/
class RecordDiffer
class LegacyRecordDiffer
@SafeVarargs
constructor(
private val rawRecordColumnNames: Map<String, String>,

View File

@@ -1,17 +1,23 @@
plugins {
id 'airbyte-java-connector'
id 'application'
id 'airbyte-bulk-connector'
}
airbyteJavaConnector {
cdkVersionRequired = '0.45.0'
features = ['db-destinations']
useLocalCdk = true
airbyteBulkConnector {
core = 'load'
toolkits = []
cdk = '0.61'
}
application {
mainClass = 'io.airbyte.integrations.destination.dev_null.TestingDestinations'
mainClass = 'io.airbyte.integrations.destination.dev_null.DevNullDestination'
applicationDefaultJvmArgs = ['-XX:+ExitOnOutOfMemoryError', '-XX:MaxRAMPercentage=75.0']
// Uncomment and replace to run locally
//applicationDefaultJvmArgs = ['-XX:+ExitOnOutOfMemoryError', '-XX:MaxRAMPercentage=75.0', '--add-opens', 'java.base/java.lang=ALL-UNNAMED']
}
dependencies {
}
// Uncomment to run locally
//run {
// standardInput = System.in
//}

View File

@@ -0,0 +1,173 @@
{
"documentationUrl" : "https://docs.airbyte.com/integrations/destinations/dev-null",
"connectionSpecification" : {
"$schema" : "http://json-schema.org/draft-07/schema#",
"title" : "E2E Test Destination Spec",
"type" : "object",
"additionalProperties" : true,
"properties" : {
"test_destination" : {
"oneOf" : [ {
"title" : "Logging",
"type" : "object",
"additionalProperties" : true,
"properties" : {
"test_destination_type" : {
"type" : "string",
"enum" : [ "LOGGING" ],
"default" : "LOGGING"
},
"logging_config" : {
"oneOf" : [ {
"title" : "First N Entries",
"type" : "object",
"additionalProperties" : true,
"description" : "Log first N entries per stream.",
"properties" : {
"logging_type" : {
"type" : "string",
"enum" : [ "FirstN" ],
"default" : "FirstN"
},
"max_entry_count" : {
"type" : "number",
"minimum" : 1,
"maximum" : 1000,
"default" : 100,
"description" : "Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries.",
"title" : "N",
"examples" : [ 100 ]
}
},
"required" : [ "logging_type", "max_entry_count" ]
}, {
"title" : "Every N-th Entry",
"type" : "object",
"additionalProperties" : true,
"description" : "For each stream, log every N-th entry with a maximum cap.",
"properties" : {
"logging_type" : {
"type" : "string",
"enum" : [ "EveryNth" ],
"default" : "EveryNth"
},
"nth_entry_to_log" : {
"type" : "integer",
"minimum" : 1,
"maximum" : 1000,
"description" : "The N-th entry to log for each stream. N starts from 1. For example, when N = 1, every entry is logged; when N = 2, every other entry is logged; when N = 3, one out of three entries is logged.",
"title" : "N",
"examples" : [ 3 ]
},
"max_entry_count" : {
"type" : "number",
"minimum" : 1,
"maximum" : 1000,
"default" : 100,
"description" : "Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries.",
"title" : "Max Log Entries",
"examples" : [ 100 ]
}
},
"required" : [ "logging_type", "nth_entry_to_log", "max_entry_count" ]
}, {
"title" : "Random Sampling",
"type" : "object",
"additionalProperties" : true,
"description" : "For each stream, randomly log a percentage of the entries with a maximum cap.",
"properties" : {
"logging_type" : {
"type" : "string",
"enum" : [ "RandomSampling" ],
"default" : "RandomSampling"
},
"sampling_ratio" : {
"type" : "number",
"minimum" : 0,
"maximum" : 1,
"description" : "A positive floating number smaller than 1.",
"title" : "Sampling Ratio",
"examples" : [ 0.001 ],
"default" : 0.001
},
"seed" : {
"type" : "number",
"description" : "When the seed is unspecified, the current time millis will be used as the seed.",
"title" : "Random Number Generator Seed",
"examples" : [ 1900 ]
},
"max_entry_count" : {
"type" : "number",
"minimum" : 1,
"maximum" : 1000,
"default" : 100,
"description" : "Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries.",
"title" : "Max Log Entries",
"examples" : [ 100 ]
}
},
"required" : [ "logging_type", "sampling_ratio", "max_entry_count" ]
} ],
"description" : "Configurate how the messages are logged.",
"title" : "Logging Configuration",
"type" : "object"
}
},
"required" : [ "test_destination_type", "logging_config" ]
}, {
"title" : "Silent",
"type" : "object",
"additionalProperties" : true,
"properties" : {
"test_destination_type" : {
"type" : "string",
"enum" : [ "SILENT" ],
"default" : "SILENT"
}
},
"required" : [ "test_destination_type" ]
}, {
"title" : "Throttled",
"type" : "object",
"additionalProperties" : true,
"properties" : {
"test_destination_type" : {
"type" : "string",
"enum" : [ "THROTTLED" ],
"default" : "THROTTLED"
},
"millis_per_record" : {
"type" : "integer",
"description" : "The number of milliseconds to wait between each record."
}
},
"required" : [ "test_destination_type", "millis_per_record" ]
}, {
"title" : "Failing",
"type" : "object",
"additionalProperties" : true,
"properties" : {
"test_destination_type" : {
"type" : "string",
"enum" : [ "FAILING" ],
"default" : "FAILING"
},
"num_messages" : {
"type" : "integer",
"description" : "Number of messages after which to fail."
}
},
"required" : [ "test_destination_type", "num_messages" ]
} ],
"description" : "The type of destination to be used",
"title" : "Test Destination",
"type" : "object"
}
},
"required" : [ "test_destination" ]
},
"supportsIncremental" : true,
"supportsNormalization" : false,
"supportsDBT" : false,
"supported_destination_sync_modes" : [ "overwrite", "append", "append_dedup" ]
}

View File

@@ -2,7 +2,7 @@ data:
connectorSubtype: file
connectorType: destination
definitionId: a7bcc9d8-13b3-4e49-b80d-d020b90045e3
dockerImageTag: 0.6.1
dockerImageTag: 0.7.0
dockerRepository: airbyte/destination-dev-null
githubIssueLabel: destination-dev-null
icon: airbyte.svg

View File

@@ -0,0 +1,15 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.check.DestinationChecker
import jakarta.inject.Singleton
@Singleton
class DevNullChecker() : DestinationChecker<DevNullConfiguration> {
override fun check(config: DevNullConfiguration) {
// Do nothing
}
}

View File

@@ -0,0 +1,115 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.command.DestinationConfiguration
import io.airbyte.cdk.command.DestinationConfigurationFactory
import io.micronaut.context.annotation.Factory
import jakarta.inject.Singleton
/** This is the simplified configuration object actually used by the implementation. */
sealed interface DevNullType
data class Logging(
val maxEntryCount: Int,
val logEvery: Int = 1,
val sampleRate: Double = 1.0,
val seed: Long? = null,
) : DevNullType
data object Silent : DevNullType
data class Failing(val numMessages: Int) : DevNullType
data class Throttled(val millisPerRecord: Long) : DevNullType
data class DevNullConfiguration(
val type: DevNullType,
) : DestinationConfiguration()
/**
* This factory is injected into the initialization code and used to map from the client-provided
* configuration json to the simplified configuration.
*
* Its role is to hide the complexities imposed by json-schema and the cloud/oss dichotomy from the
* rest of the implementation.
*/
@Singleton
class DevNullConfigurationFactory :
DestinationConfigurationFactory<DevNullSpecification, DevNullConfiguration> {
override fun makeWithoutExceptionHandling(pojo: DevNullSpecification): DevNullConfiguration {
return when (pojo) {
is DevNullSpecificationOss -> {
when (pojo.testDestination) {
is LoggingDestination -> {
when (pojo.testDestination.loggingConfig) {
is FirstNEntriesConfig -> {
DevNullConfiguration(
type =
Logging(
maxEntryCount =
pojo.testDestination.loggingConfig.maxEntryCount
.toInt(),
)
)
}
is EveryNthEntryConfig -> {
DevNullConfiguration(
type =
Logging(
maxEntryCount =
pojo.testDestination.loggingConfig.maxEntryCount
.toInt(),
logEvery =
pojo.testDestination.loggingConfig.nthEntryToLog,
)
)
}
is RandomSamplingConfig -> {
DevNullConfiguration(
type =
Logging(
maxEntryCount =
pojo.testDestination.loggingConfig.maxEntryCount
.toInt(),
sampleRate =
pojo.testDestination.loggingConfig.samplingRatio,
seed = pojo.testDestination.loggingConfig.seed?.toLong()
)
)
}
}
}
is SilentDestination -> {
DevNullConfiguration(type = Silent)
}
is ThrottledDestination -> {
DevNullConfiguration(type = Throttled(pojo.testDestination.millisPerRecord))
}
is FailingDestination -> {
DevNullConfiguration(type = Failing(pojo.testDestination.numMessages))
}
}
}
is DevNullSpecificationCloud -> {
when (pojo.testDestination) {
is SilentDestinationCloud -> {
DevNullConfiguration(type = Silent)
}
}
}
}
}
}
/** This allows micronaut to inject the simplified configuration into the implementation. */
@Factory
class DevNullConfigurationProvider(private val config: DestinationConfiguration) {
@Singleton
fun get(): DevNullConfiguration {
return config as DevNullConfiguration
}
}

View File

@@ -0,0 +1,16 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.AirbyteDestinationRunner
class DevNullDestination {
companion object {
@JvmStatic
fun main(args: Array<String>) {
AirbyteDestinationRunner.run(*args)
}
}
}

View File

@@ -0,0 +1,243 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.annotation.JsonClassDescription
import com.fasterxml.jackson.annotation.JsonProperty
import com.fasterxml.jackson.annotation.JsonPropertyDescription
import com.fasterxml.jackson.annotation.JsonSubTypes
import com.fasterxml.jackson.annotation.JsonTypeInfo
import com.fasterxml.jackson.annotation.JsonValue
import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaInject
import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle
import io.airbyte.cdk.command.ConfigurationJsonObjectBase
import io.airbyte.cdk.spec.DestinationSpecificationInternal
import io.airbyte.protocol.models.v0.DestinationSyncMode
import io.micronaut.context.annotation.Requires
import jakarta.inject.Singleton
import javax.validation.constraints.Max
import javax.validation.constraints.Min
sealed class TestDestination
sealed class DevNullSpecification : ConfigurationJsonObjectBase() {
abstract val testDestination: TestDestination
}
/**
* This doesn't quite conform with the old spec:
* - Some fields that make more sense as integral need to be Double to yield a "number" type
* - Due to https://github.com/mbknor/mbknor-jackson-jsonSchema/issues/184, this causes
* ```
* `@JsonSchemaExamples` to break. Instead, we inject a raw JSON blob to the schema.
* ```
* - Similarly, there are some cases where [JsonSchemaTitle] / [JsonClassDescription]
* ```
* don't work as expected. In these cases, we also inject raw JSON.
* ```
* - Additionally, there are extra fields:
* ```
* - "additionalProperties: true" appears throughout (not helped by @JsonIgnoreProperties)
* - "type": "object" is appended to the case classes
* ```
*/
@JsonSchemaTitle("E2E Test Destination Spec")
@Singleton
@Requires(property = "deployment.mode", pattern = "(?i)oss")
class DevNullSpecificationOss : DevNullSpecification() {
@JsonProperty("test_destination")
@JsonSchemaTitle("Test Destination")
@JsonPropertyDescription("The type of destination to be used")
override val testDestination: TestDestinationOSS = LoggingDestination()
}
@JsonTypeInfo(
use = JsonTypeInfo.Id.NAME,
include = JsonTypeInfo.As.EXISTING_PROPERTY,
property = "test_destination_type"
)
@JsonSubTypes(
JsonSubTypes.Type(value = LoggingDestination::class, name = "LOGGING"),
JsonSubTypes.Type(value = SilentDestination::class, name = "SILENT"),
JsonSubTypes.Type(value = ThrottledDestination::class, name = "THROTTLED"),
JsonSubTypes.Type(value = FailingDestination::class, name = "FAILING")
)
sealed class TestDestinationOSS(
@JsonProperty("test_destination_type") open val testDestinationType: Type
) : TestDestination() {
enum class Type(val typeName: String) {
LOGGING("LOGGING"),
SILENT("SILENT"),
THROTTLED("THROTTLED"),
FAILING("FAILING")
}
}
@JsonSchemaTitle("Logging")
data class LoggingDestination(
@JsonProperty("test_destination_type") override val testDestinationType: Type = Type.LOGGING,
@JsonPropertyDescription("Configurate how the messages are logged.")
@JsonProperty("logging_config")
val loggingConfig: LoggingConfig = FirstNEntriesConfig()
) : TestDestinationOSS(testDestinationType)
@JsonTypeInfo(
use = JsonTypeInfo.Id.NAME,
include = JsonTypeInfo.As.EXISTING_PROPERTY,
property = "logging_type"
)
@JsonSubTypes(
JsonSubTypes.Type(value = FirstNEntriesConfig::class, name = "FirstN"),
JsonSubTypes.Type(value = EveryNthEntryConfig::class, name = "EveryNth"),
JsonSubTypes.Type(value = RandomSamplingConfig::class, name = "RandomSampling")
)
@JsonSchemaInject(json = """{"title":"Logging Configuration"}""")
sealed class LoggingConfig(
@JsonProperty("logging_type") open val loggingType: Type = Type.FIRST_N
) {
enum class Type(@get:JsonValue val typeName: String) {
FIRST_N("FirstN"),
EVERY_NTH("EveryNth"),
RANDOM_SAMPLING("RandomSampling")
}
}
@JsonSchemaTitle("First N Entries")
@JsonSchemaInject(json = """{"description":"Log first N entries per stream."}""")
data class FirstNEntriesConfig(
@JsonProperty("logging_type") override val loggingType: Type = Type.FIRST_N,
@JsonSchemaTitle("N")
@JsonPropertyDescription(
"Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries."
)
@JsonProperty("max_entry_count", defaultValue = "100")
@JsonSchemaInject(json = """{"examples":[100]}""")
@Max(1000)
@Min(1)
val maxEntryCount: Double = 100.0
) : LoggingConfig(loggingType)
@JsonSchemaTitle("Every N-th Entry")
@JsonSchemaInject(
json = """{"description":"For each stream, log every N-th entry with a maximum cap."}"""
)
data class EveryNthEntryConfig(
@JsonProperty("logging_type") override val loggingType: Type = Type.EVERY_NTH,
@JsonSchemaTitle("N")
@JsonPropertyDescription(
"The N-th entry to log for each stream. N starts from 1. For example, when N = 1, every entry is logged; when N = 2, every other entry is logged; when N = 3, one out of three entries is logged."
)
@JsonProperty("nth_entry_to_log")
@JsonSchemaInject(json = """{"examples":[3]}""")
@Max(1000)
@Min(1)
val nthEntryToLog: Int,
@JsonSchemaTitle("Max Log Entries")
@JsonPropertyDescription(
"Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries."
)
@JsonProperty("max_entry_count", defaultValue = "100")
@JsonSchemaInject(json = """{"examples":[100]}""")
@Max(1000)
@Min(1)
val maxEntryCount: Double
) : LoggingConfig(loggingType)
@JsonSchemaTitle("Random Sampling")
@JsonSchemaInject(
json =
"""{"description":"For each stream, randomly log a percentage of the entries with a maximum cap."}"""
)
data class RandomSamplingConfig(
@JsonProperty("logging_type") override val loggingType: Type = Type.RANDOM_SAMPLING,
@JsonSchemaTitle("Sampling Ratio")
@JsonPropertyDescription("A positive floating number smaller than 1.")
@JsonProperty("sampling_ratio")
@JsonSchemaInject(json = """{"examples":[0.001],"default":0.001}""")
@Max(1)
@Min(0)
val samplingRatio: Double = 0.001,
@JsonSchemaTitle("Random Number Generator Seed")
@JsonPropertyDescription(
"When the seed is unspecified, the current time millis will be used as the seed."
)
@JsonSchemaInject(json = """{"examples":[1900]}""")
@JsonProperty("seed")
val seed: Double? = null,
@JsonSchemaTitle("Max Log Entries")
@JsonPropertyDescription(
"Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries."
)
@JsonProperty("max_entry_count", defaultValue = "100")
@JsonSchemaInject(json = """{"examples":[100]}""")
@Max(1000)
@Min(1)
val maxEntryCount: Double = 100.0
) : LoggingConfig(loggingType)
@JsonSchemaTitle("Silent")
data class SilentDestination(
@JsonProperty("test_destination_type") override val testDestinationType: Type = Type.SILENT
) : TestDestinationOSS(testDestinationType)
@JsonSchemaTitle("Throttled")
data class ThrottledDestination(
@JsonProperty("test_destination_type") override val testDestinationType: Type = Type.THROTTLED,
@JsonPropertyDescription("The number of milliseconds to wait between each record.")
@JsonProperty("millis_per_record")
val millisPerRecord: Long
) : TestDestinationOSS(testDestinationType)
@JsonSchemaTitle("Failing")
data class FailingDestination(
@JsonProperty("test_destination_type") override val testDestinationType: Type = Type.FAILING,
@JsonPropertyDescription("Number of messages after which to fail.")
@JsonProperty("num_messages")
val numMessages: Int
) : TestDestinationOSS(testDestinationType)
/** The cloud variant is more restricted: it only allows for a single destination type. */
@JsonSchemaTitle("E2E Test Destination Spec")
@Singleton
@Requires(property = "deployment.mode", pattern = "(?i)cloud")
class DevNullSpecificationCloud : DevNullSpecification() {
@JsonProperty("test_destination")
@JsonSchemaTitle("Test Destination")
@JsonPropertyDescription("The type of destination to be used")
override val testDestination: TestDestinationCloud = SilentDestinationCloud()
}
@JsonTypeInfo(
use = JsonTypeInfo.Id.NAME,
include = JsonTypeInfo.As.EXISTING_PROPERTY,
property = "test_destination_type"
)
@JsonSubTypes(
JsonSubTypes.Type(value = SilentDestinationCloud::class, name = "SILENT"),
)
sealed class TestDestinationCloud(
@JsonProperty("test_destination_type") open val testDestinationType: Type
) : TestDestination() {
enum class Type(val typeName: String) {
SILENT("SILENT"),
}
}
@JsonSchemaTitle("Silent")
data class SilentDestinationCloud(
@JsonProperty("test_destination_type") override val testDestinationType: Type = Type.SILENT
) : TestDestinationCloud(testDestinationType)
/** Non-client-facing configuration. */
@Singleton
class DevNullSpecificationInternal : DestinationSpecificationInternal {
override val supportedSyncModes =
listOf(
DestinationSyncMode.OVERWRITE,
DestinationSyncMode.APPEND,
DestinationSyncMode.APPEND_DEDUP,
)
override val supportsIncremental = true
}

View File

@@ -0,0 +1,145 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
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.write.DestinationWriter
import io.airbyte.cdk.write.StreamLoader
import io.github.oshai.kotlinlogging.KotlinLogging
import jakarta.inject.Singleton
import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.atomic.AtomicLong
import kotlin.random.Random
import kotlinx.coroutines.delay
@Singleton
class DevNullWriter(private val config: DevNullConfiguration) : DestinationWriter {
private val log = KotlinLogging.logger {}
override fun getStreamLoader(stream: DestinationStream): StreamLoader {
return when (config.type) {
is Logging -> {
log.info { "Creating LoggingStreamLoader for LoggingDestination" }
LoggingStreamLoader(stream, config.type)
}
is Silent -> {
log.info { "Creating SilentStreamLoader for SilentDestination" }
SilentStreamLoader(stream)
}
is Throttled -> {
log.info { "Creating ThrottledStreamLoader for ThrottledDestination" }
ThrottledStreamLoader(stream, config.type.millisPerRecord)
}
is Failing -> {
log.info { "Creating FailingStreamLoader for FailingDestination" }
FailingStreamLoader(stream, config.type.numMessages)
}
}
}
}
class LoggingStreamLoader(override val stream: DestinationStream, loggingConfig: Logging) :
StreamLoader {
private val log = KotlinLogging.logger {}
private val maxEntryCount: Int = loggingConfig.maxEntryCount
private val logEvery: Int = loggingConfig.logEvery
private val sampleRate: Double = loggingConfig.sampleRate
private val prng: Random = loggingConfig.seed?.let { Random(it) } ?: Random.Default
companion object {
private val recordCount = AtomicLong()
private val logCount = AtomicLong()
}
override suspend fun processRecords(
records: Iterator<DestinationRecord>,
totalSizeBytes: Long
): Batch {
log.info { "Processing record batch with logging" }
records.forEach { record ->
if (recordCount.getAndIncrement() % logEvery == 0L) {
if (sampleRate == 1.0 || prng.nextDouble() < sampleRate) {
if (logCount.incrementAndGet() < maxEntryCount) {
log.info {
"Logging Destination(stream=${stream.descriptor}, recordIndex=$recordCount, logEntry=$logCount/$maxEntryCount): $record"
}
}
}
}
}
log.info { "Completed record batch." }
return SimpleBatch(state = Batch.State.COMPLETE)
}
}
class SilentStreamLoader(override val stream: DestinationStream) : StreamLoader {
override suspend fun processRecords(
records: Iterator<DestinationRecord>,
totalSizeBytes: Long
): Batch {
return SimpleBatch(state = Batch.State.COMPLETE)
}
}
@SuppressFBWarnings(
"NP_NONNULL_PARAM_VIOLATION",
justification = "message is guaranteed to be non-null by Kotlin's type system"
)
class ThrottledStreamLoader(
override val stream: DestinationStream,
private val millisPerRecord: Long
) : StreamLoader {
private val log = KotlinLogging.logger {}
override suspend fun processRecords(
records: Iterator<DestinationRecord>,
totalSizeBytes: Long
): Batch {
log.info { "Processing record batch with delay of $millisPerRecord per record" }
records.forEach { _ -> delay(millisPerRecord) }
log.info { "Completed record batch." }
return SimpleBatch(state = Batch.State.COMPLETE)
}
}
class FailingStreamLoader(override val stream: DestinationStream, private val numMessages: Int) :
StreamLoader {
private val log = KotlinLogging.logger {}
companion object {
private val messageCount = AtomicInteger(0)
}
override suspend fun processRecords(
records: Iterator<DestinationRecord>,
totalSizeBytes: Long
): Batch {
log.info { "Processing record batch with failure after $numMessages messages" }
records.forEach { record ->
messageCount.getAndIncrement().let { messageCount ->
if (messageCount > numMessages) {
val message =
"Failing Destination(stream=${stream.descriptor}, numMessages=$numMessages: failing at $record"
log.info { message }
throw RuntimeException(message)
}
}
}
log.info { "Completed record batch." }
return SimpleBatch(state = Batch.State.COMPLETE)
}
}

View File

@@ -1,64 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.cdk.integrations.BaseConnector
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
import io.airbyte.cdk.integrations.base.Destination
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import java.util.function.Consumer
import org.slf4j.Logger
import org.slf4j.LoggerFactory
class FailAfterNDestination : BaseConnector(), Destination {
override fun check(config: JsonNode): AirbyteConnectionStatus {
return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
}
override fun getConsumer(
config: JsonNode,
catalog: ConfiguredAirbyteCatalog,
outputRecordCollector: Consumer<AirbyteMessage>
): AirbyteMessageConsumer {
return FailAfterNConsumer(
config["test_destination"]["num_messages"].asLong(),
outputRecordCollector
)
}
class FailAfterNConsumer(
private val numMessagesAfterWhichToFail: Long,
private val outputRecordCollector: Consumer<AirbyteMessage>
) : AirbyteMessageConsumer {
private var numMessagesSoFar: Long = 0
init {
LOGGER.info("Will fail after {} messages", numMessagesAfterWhichToFail)
}
override fun start() {}
override fun accept(message: AirbyteMessage) {
numMessagesSoFar += 1
check(numMessagesSoFar <= numMessagesAfterWhichToFail) {
"Forcing a fail after processing $numMessagesAfterWhichToFail messages."
}
if (message.type == AirbyteMessage.Type.STATE) {
LOGGER.info("Emitting state: {}", message)
outputRecordCollector.accept(message)
}
}
override fun close() {}
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(FailAfterNDestination::class.java)
}
}

View File

@@ -1,36 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.cdk.integrations.BaseConnector
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
import io.airbyte.cdk.integrations.base.Destination
import io.airbyte.integrations.destination.dev_null.logging.LoggingConsumer
import io.airbyte.integrations.destination.dev_null.logging.TestingLoggerFactory
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import java.util.function.Consumer
import org.slf4j.Logger
import org.slf4j.LoggerFactory
/** This destination logs each record it receives. */
class LoggingDestination : BaseConnector(), Destination {
override fun check(config: JsonNode): AirbyteConnectionStatus {
return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
}
override fun getConsumer(
config: JsonNode,
catalog: ConfiguredAirbyteCatalog,
outputRecordCollector: Consumer<AirbyteMessage>
): AirbyteMessageConsumer {
return LoggingConsumer(TestingLoggerFactory(config), catalog, outputRecordCollector)
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(LoggingDestination::class.java)
}
}

View File

@@ -1,41 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.cdk.integrations.BaseConnector
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
import io.airbyte.cdk.integrations.base.Destination
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import java.util.function.Consumer
/** This destination silently receives records. */
class SilentDestination : BaseConnector(), Destination {
override fun check(config: JsonNode): AirbyteConnectionStatus {
return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
}
override fun getConsumer(
config: JsonNode,
catalog: ConfiguredAirbyteCatalog,
outputRecordCollector: Consumer<AirbyteMessage>
): AirbyteMessageConsumer {
return RecordConsumer(outputRecordCollector)
}
class RecordConsumer(private val outputRecordCollector: Consumer<AirbyteMessage>) :
AirbyteMessageConsumer {
override fun start() {}
override fun accept(message: AirbyteMessage) {
if (message.type == AirbyteMessage.Type.STATE) {
outputRecordCollector.accept(message)
}
}
override fun close() {}
}
}

View File

@@ -1,111 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import com.fasterxml.jackson.databind.node.ArrayNode
import com.fasterxml.jackson.databind.node.ObjectNode
import com.google.common.collect.ImmutableMap
import io.airbyte.cdk.integrations.BaseConnector
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
import io.airbyte.cdk.integrations.base.Destination
import io.airbyte.cdk.integrations.base.IntegrationRunner
import io.airbyte.commons.features.EnvVariableFeatureFlags
import io.airbyte.commons.features.FeatureFlags
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import io.airbyte.protocol.models.v0.ConnectorSpecification
import java.util.function.Consumer
import org.slf4j.Logger
import org.slf4j.LoggerFactory
class TestingDestinations
@JvmOverloads
constructor(
override val featureFlags: FeatureFlags = EnvVariableFeatureFlags(),
private val destinationMap: Map<TestDestinationType, Destination> =
ImmutableMap.builder<TestDestinationType, Destination>()
.put(TestDestinationType.LOGGING, LoggingDestination())
.put(TestDestinationType.THROTTLED, ThrottledDestination())
.put(TestDestinationType.SILENT, SilentDestination())
.put(TestDestinationType.FAILING, FailAfterNDestination())
.build(),
) : BaseConnector(), Destination {
enum class TestDestinationType {
LOGGING,
THROTTLED,
SILENT,
FAILING
}
private fun selectDestination(config: JsonNode): Destination? {
return destinationMap[
TestDestinationType.valueOf(
config["test_destination"]["test_destination_type"].asText()
)
]
}
override fun spec(): ConnectorSpecification {
if (!isCloudDeployment) {
return super.spec()
} else {
/** 1. Update the title. 2. Only keep the "silent" mode. */
val spec = super.spec()
(spec.connectionSpecification as ObjectNode).put("title", DEV_NULL_DESTINATION_TITLE)
val properties =
spec.connectionSpecification["properties"]["test_destination"] as ObjectNode
val types = properties["oneOf"] as ArrayNode
val typesIterator = types.elements()
while (typesIterator.hasNext()) {
val typeNode = typesIterator.next()
if (
!typeNode["properties"]["test_destination_type"]["const"]
.asText()
.equals("silent", ignoreCase = true)
) {
typesIterator.remove()
}
}
return spec
}
}
@Throws(Exception::class)
override fun getConsumer(
config: JsonNode,
catalog: ConfiguredAirbyteCatalog,
outputRecordCollector: Consumer<AirbyteMessage>
): AirbyteMessageConsumer? {
return selectDestination(config)!!.getConsumer(config, catalog, outputRecordCollector)
}
@Throws(Exception::class)
override fun check(config: JsonNode): AirbyteConnectionStatus? {
return try {
selectDestination(config)!!.check(config)
} catch (e: Exception) {
AirbyteConnectionStatus()
.withStatus(AirbyteConnectionStatus.Status.FAILED)
.withMessage(e.message)
}
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(TestingDestinations::class.java)
private const val DEV_NULL_DESTINATION_TITLE = "E2E Test (/dev/null) Destination Spec"
@Throws(Exception::class)
@JvmStatic
fun main(args: Array<String>) {
val destination: Destination = TestingDestinations()
LOGGER.info("starting destination: {}", TestingDestinations::class.java)
IntegrationRunner(destination).run(args)
LOGGER.info("completed destination: {}", TestingDestinations::class.java)
}
}
}

View File

@@ -1,63 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.cdk.integrations.BaseConnector
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
import io.airbyte.cdk.integrations.base.Destination
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import java.util.function.Consumer
import org.slf4j.Logger
import org.slf4j.LoggerFactory
/**
* This destination logs each record it receives. It sleeps for millis_per_record between accepting
* each record. Useful for simulating backpressure / slow destination writes.
*/
class ThrottledDestination() : BaseConnector(), Destination {
override fun check(config: JsonNode): AirbyteConnectionStatus {
return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
}
override fun getConsumer(
config: JsonNode,
catalog: ConfiguredAirbyteCatalog,
outputRecordCollector: Consumer<AirbyteMessage>
): AirbyteMessageConsumer {
return ThrottledConsumer(
config["test_destination"]["millis_per_record"].asLong(),
outputRecordCollector
)
}
class ThrottledConsumer(
private val millisPerRecord: Long,
private val outputRecordCollector: Consumer<AirbyteMessage>
) : AirbyteMessageConsumer {
init {
LOGGER.info("Will sleep {} millis before processing every record", millisPerRecord)
}
override fun start() {}
@Throws(Exception::class)
override fun accept(message: AirbyteMessage) {
Thread.sleep(millisPerRecord)
if (message.type == AirbyteMessage.Type.STATE) {
LOGGER.info("Emitting state: {}", message)
outputRecordCollector.accept(message)
}
}
override fun close() {}
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(ThrottledDestination::class.java)
}
}

View File

@@ -1,43 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair
import java.time.Instant
import java.time.OffsetDateTime
import java.time.ZoneId
import java.time.format.DateTimeFormatter
abstract class BaseLogger(
protected val streamNamePair: AirbyteStreamNameNamespacePair,
protected val maxEntryCount: Int
) : TestingLogger {
protected var loggedEntryCount: Int = 0
protected fun entryMessage(recordMessage: AirbyteRecordMessage): String {
return String.format(
"[%s] %s #%04d: %s",
emissionTimestamp(recordMessage.emittedAt),
streamName(streamNamePair),
loggedEntryCount,
recordMessage.data
)
}
companion object {
protected fun streamName(pair: AirbyteStreamNameNamespacePair): String {
return if (pair.namespace == null) {
pair.name
} else {
String.format("%s.%s", pair.namespace, pair.name)
}
}
protected fun emissionTimestamp(emittedAt: Long): String {
return OffsetDateTime.ofInstant(Instant.ofEpochMilli(emittedAt), ZoneId.systemDefault())
.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME)
}
}
}

View File

@@ -1,33 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair
import org.slf4j.Logger
import org.slf4j.LoggerFactory
class EveryNthLogger(
streamNamePair: AirbyteStreamNameNamespacePair,
private val nthEntryToLog: Int,
maxEntryCount: Int
) : BaseLogger(streamNamePair, maxEntryCount), TestingLogger {
private var currentEntry = 0
override fun log(recordMessage: AirbyteRecordMessage?) {
if (loggedEntryCount >= maxEntryCount) {
return
}
currentEntry += 1
if (currentEntry % nthEntryToLog == 0) {
loggedEntryCount += 1
LOGGER.info(entryMessage(recordMessage!!))
}
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(EveryNthLogger::class.java)
}
}

View File

@@ -1,25 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair
import org.slf4j.Logger
import org.slf4j.LoggerFactory
class FirstNLogger(streamNamePair: AirbyteStreamNameNamespacePair, maxEntryCount: Int) :
BaseLogger(streamNamePair, maxEntryCount), TestingLogger {
override fun log(recordMessage: AirbyteRecordMessage?) {
if (loggedEntryCount >= maxEntryCount) {
return
}
loggedEntryCount += 1
LOGGER.info(entryMessage(recordMessage!!))
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(FirstNLogger::class.java)
}
}

View File

@@ -1,58 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
import io.airbyte.commons.json.Jsons
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
import java.util.function.Consumer
import org.slf4j.Logger
import org.slf4j.LoggerFactory
class LoggingConsumer(
private val loggerFactory: TestingLoggerFactory,
private val configuredCatalog: ConfiguredAirbyteCatalog,
private val outputRecordCollector: Consumer<AirbyteMessage>
) : AirbyteMessageConsumer {
private val loggers: MutableMap<AirbyteStreamNameNamespacePair, TestingLogger?> = HashMap()
override fun start() {
for (configuredStream in configuredCatalog.streams) {
val stream = configuredStream.stream
val streamNamePair = AirbyteStreamNameNamespacePair.fromAirbyteStream(stream)
val logger = loggerFactory.create(streamNamePair)
loggers[streamNamePair] = logger
}
}
override fun accept(message: AirbyteMessage) {
if (message.type == AirbyteMessage.Type.STATE) {
LOGGER.info("Emitting state: {}", message)
outputRecordCollector.accept(message)
} else if (message.type == AirbyteMessage.Type.TRACE) {
LOGGER.info("Received a trace: {}", message)
} else if (message.type == AirbyteMessage.Type.RECORD) {
val recordMessage = message.record
val pair = AirbyteStreamNameNamespacePair.fromRecordMessage(recordMessage)
require(loggers.containsKey(pair)) {
String.format(
"Message contained record from a stream that was not in the catalog.\n Catalog: %s\n Message: %s",
Jsons.serialize(configuredCatalog),
Jsons.serialize(recordMessage)
)
}
loggers[pair]!!.log(recordMessage)
}
}
override fun close() {}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(LoggingConsumer::class.java)
}
}

View File

@@ -1,34 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair
import java.util.*
import org.slf4j.Logger
import org.slf4j.LoggerFactory
class RandomSamplingLogger(
streamNamePair: AirbyteStreamNameNamespacePair,
private val samplingRatio: Double,
seed: Long,
maxEntryCount: Int
) : BaseLogger(streamNamePair, maxEntryCount), TestingLogger {
private val random = Random(seed)
override fun log(recordMessage: AirbyteRecordMessage?) {
if (loggedEntryCount >= maxEntryCount) {
return
}
if (random.nextDouble() < samplingRatio) {
loggedEntryCount += 1
LOGGER.info(entryMessage(recordMessage!!))
}
}
companion object {
private val LOGGER: Logger = LoggerFactory.getLogger(RandomSamplingLogger::class.java)
}
}

View File

@@ -1,16 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
interface TestingLogger {
enum class LoggingType {
FirstN,
EveryNth,
RandomSampling
}
fun log(recordMessage: AirbyteRecordMessage?)
}

View File

@@ -1,44 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null.logging
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair
class TestingLoggerFactory(private val config: JsonNode) {
fun create(streamNamePair: AirbyteStreamNameNamespacePair): TestingLogger {
require(config.has("test_destination")) {
"Property test_destination is required, but not found"
}
val testDestinationConfig = config["test_destination"]
require(testDestinationConfig.has("logging_config")) {
"Property logging_config is required, but not found"
}
val logConfig = testDestinationConfig["logging_config"]
val loggingType = TestingLogger.LoggingType.valueOf(logConfig["logging_type"].asText())
when (loggingType) {
TestingLogger.LoggingType.FirstN -> {
val maxEntryCount = logConfig["max_entry_count"].asInt()
return FirstNLogger(streamNamePair, maxEntryCount)
}
TestingLogger.LoggingType.EveryNth -> {
val nthEntryToLog = logConfig["nth_entry_to_log"].asInt()
val maxEntryCount = logConfig["max_entry_count"].asInt()
return EveryNthLogger(streamNamePair, nthEntryToLog, maxEntryCount)
}
TestingLogger.LoggingType.RandomSampling -> {
val samplingRatio = logConfig["sampling_ratio"].asDouble()
val seed =
if (logConfig.has("seed")) logConfig["seed"].asLong()
else System.currentTimeMillis()
val maxEntryCount = logConfig["max_entry_count"].asInt()
return RandomSamplingLogger(streamNamePair, samplingRatio, seed, maxEntryCount)
}
else -> throw IllegalArgumentException("Unexpected logging type: $loggingType")
}
}
}

View File

@@ -0,0 +1,3 @@
# Default to OSS in the absence of an env variable
deployment:
mode: oss

View File

@@ -1,179 +0,0 @@
{
"documentationUrl": "https://docs.airbyte.com/integrations/destinations/e2e-test",
"supportsIncremental": true,
"supportsNormalization": false,
"supportsDBT": false,
"supported_destination_sync_modes": ["overwrite", "append", "append_dedup"],
"protocol_version": "0.2.1",
"connectionSpecification": {
"$schema": "http://json-schema.org/draft-07/schema#",
"title": "E2E Test Destination Spec",
"type": "object",
"required": ["test_destination"],
"properties": {
"test_destination": {
"title": "Test Destination",
"type": "object",
"description": "The type of destination to be used",
"oneOf": [
{
"title": "Logging",
"required": ["test_destination_type", "logging_config"],
"properties": {
"test_destination_type": {
"type": "string",
"const": "LOGGING",
"default": "LOGGING"
},
"logging_config": {
"title": "Logging Configuration",
"type": "object",
"description": "Configurate how the messages are logged.",
"oneOf": [
{
"title": "First N Entries",
"description": "Log first N entries per stream.",
"type": "object",
"required": ["logging_type", "max_entry_count"],
"properties": {
"logging_type": {
"type": "string",
"enum": ["FirstN"],
"default": "FirstN"
},
"max_entry_count": {
"title": "N",
"description": "Number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries.",
"type": "number",
"default": 100,
"examples": [100],
"minimum": 1,
"maximum": 1000
}
}
},
{
"title": "Every N-th Entry",
"description": "For each stream, log every N-th entry with a maximum cap.",
"type": "object",
"required": [
"logging_type",
"nth_entry_to_log",
"max_entry_count"
],
"properties": {
"logging_type": {
"type": "string",
"enum": ["EveryNth"],
"default": "EveryNth"
},
"nth_entry_to_log": {
"title": "N",
"description": "The N-th entry to log for each stream. N starts from 1. For example, when N = 1, every entry is logged; when N = 2, every other entry is logged; when N = 3, one out of three entries is logged.",
"type": "number",
"example": [3],
"minimum": 1,
"maximum": 1000
},
"max_entry_count": {
"title": "Max Log Entries",
"description": "Max number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries.",
"type": "number",
"default": 100,
"examples": [100],
"minimum": 1,
"maximum": 1000
}
}
},
{
"title": "Random Sampling",
"description": "For each stream, randomly log a percentage of the entries with a maximum cap.",
"type": "object",
"required": [
"logging_type",
"sampling_ratio",
"max_entry_count"
],
"properties": {
"logging_type": {
"type": "string",
"enum": ["RandomSampling"],
"default": "RandomSampling"
},
"sampling_ratio": {
"title": "Sampling Ratio",
"description": "A positive floating number smaller than 1.",
"type": "number",
"default": 0.001,
"examples": [0.001],
"minimum": 0,
"maximum": 1
},
"seed": {
"title": "Random Number Generator Seed",
"description": "When the seed is unspecified, the current time millis will be used as the seed.",
"type": "number",
"examples": [1900]
},
"max_entry_count": {
"title": "Max Log Entries",
"description": "Max number of entries to log. This destination is for testing only. So it won't make sense to log infinitely. The maximum is 1,000 entries.",
"type": "number",
"default": 100,
"examples": [100],
"minimum": 1,
"maximum": 1000
}
}
}
]
}
}
},
{
"title": "Silent",
"required": ["test_destination_type"],
"properties": {
"test_destination_type": {
"type": "string",
"const": "SILENT",
"default": "SILENT"
}
}
},
{
"title": "Throttled",
"required": ["test_destination_type", "millis_per_record"],
"properties": {
"test_destination_type": {
"type": "string",
"const": "THROTTLED",
"default": "THROTTLED"
},
"millis_per_record": {
"description": "Number of milli-second to pause in between records.",
"type": "integer"
}
}
},
{
"title": "Failing",
"required": ["test_destination_type", "num_messages"],
"properties": {
"test_destination_type": {
"type": "string",
"const": "FAILING",
"default": "FAILING"
},
"num_messages": {
"description": "Number of messages after which to fail.",
"type": "integer"
}
}
}
]
}
}
}
}

View File

@@ -1,69 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.commons.json.Jsons
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
import java.util.*
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Disabled
import org.junit.jupiter.api.Test
class CloudDevNullDestinationAcceptanceTest : TestingSilentDestinationAcceptanceTest() {
override val isCloudTest = true
override fun getConfig(): JsonNode {
return Jsons.jsonNode(
Collections.singletonMap(
"test_destination",
Collections.singletonMap("test_destination_type", "SILENT")
)
)
}
override fun getFailCheckConfig(): JsonNode {
return Jsons.jsonNode(
Collections.singletonMap(
"test_destination",
Collections.singletonMap("test_destination_type", "invalid")
)
)
}
override fun retrieveRecords(
testEnv: TestDestinationEnv?,
streamName: String,
namespace: String,
streamSchema: JsonNode
): List<JsonNode> {
return emptyList()
}
override fun setup(testEnv: TestDestinationEnv, TEST_SCHEMAS: HashSet<String>) {
// do nothing
}
override fun tearDown(testEnv: TestDestinationEnv) {
// do nothing
}
override fun assertSameMessages(
expected: List<AirbyteMessage>,
actual: List<AirbyteRecordMessage>,
pruneAirbyteInternalFields: Boolean
) {
Assertions.assertEquals(0, actual.size)
}
// Skip because `retrieveRecords` returns an empty list at all times.
@Disabled @Test override fun testSyncNotFailsWithNewFields() {}
@Disabled @Test override fun testAirbyteTimeTypes() {}
open override fun getDefaultSchema(config: JsonNode): String? {
return super.getDefaultSchema(config) ?: "default_schema"
}
}

View File

@@ -0,0 +1,33 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.test.spec.SpecTest
import io.airbyte.cdk.test.util.NoopDestinationCleaner
import io.airbyte.cdk.test.util.NoopExpectedRecordMapper
import io.airbyte.cdk.test.write.BasicFunctionalityIntegrationTest
import org.junit.jupiter.api.Test
class DevNullBasicFunctionalityIntegrationTest :
BasicFunctionalityIntegrationTest(
DevNullTestUtils.loggingConfig,
DevNullDestinationDataDumper,
NoopDestinationCleaner,
NoopExpectedRecordMapper,
verifyDataWriting = false,
) {
@Test
override fun testCheck() {
super.testCheck()
}
@Test
override fun testBasicWrite() {
super.testBasicWrite()
}
}
class DevNullSpecTest : SpecTest()

View File

@@ -0,0 +1,21 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.test.check.CheckIntegrationTest
import org.junit.jupiter.api.Test
class DevNullCheckIntegrationTest :
CheckIntegrationTest<DevNullSpecificationOss>(
DevNullSpecificationOss::class.java,
successConfigFilenames = listOf(DevNullTestUtils.LOGGING_CONFIG_PATH),
failConfigFilenamesAndFailureReasons = mapOf(),
) {
@Test
override fun testSuccessConfigs() {
super.testSuccessConfigs()
}
}

View File

@@ -0,0 +1,16 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.test.util.DestinationDataDumper
import io.airbyte.cdk.test.util.OutputRecord
object DevNullDestinationDataDumper : DestinationDataDumper {
override fun dumpRecords(streamName: String, streamNamespace: String?): List<OutputRecord> {
// E2e destination doesn't actually write records, so we shouldn't even
// have tests that try to read back the records
throw NotImplementedError()
}
}

View File

@@ -0,0 +1,32 @@
/*
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.command.ValidatedJsonUtils
import java.nio.file.Files
import java.nio.file.Path
object DevNullTestUtils {
/*
* Most destinations probably want a function to randomize the config:
* fun getS3StagingConfig(randomizedNamespace: String) {
* return baseConfig.withDefaultNamespace(randomizedNamespace)
* }
* but destination-e2e doesn't actually _do_ anything, so we can just
* use a constant config
*/
/*
* destination-e2e-test has no real creds, so we just commit these configs
* directly on git.
* most real destinations will put their configs in GSM,
* so their paths would be `secrets/blah.json`.
*/
const val LOGGING_CONFIG_PATH = "test_configs/logging.json"
val loggingConfig: DevNullSpecification =
ValidatedJsonUtils.parseOne(
DevNullSpecificationOss::class.java,
Files.readString(Path.of(LOGGING_CONFIG_PATH)),
)
}

View File

@@ -1,80 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.fasterxml.jackson.databind.JsonNode
import io.airbyte.cdk.integrations.standardtest.destination.DestinationAcceptanceTest
import io.airbyte.commons.json.Jsons
import io.airbyte.integrations.destination.dev_null.TestingDestinations.TestDestinationType
import io.airbyte.protocol.models.v0.*
import java.util.*
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Disabled
import org.junit.jupiter.api.Test
open class TestingSilentDestinationAcceptanceTest : DestinationAcceptanceTest() {
override val imageName = "airbyte/destination-dev-null:dev"
override val isCloudTest = false
override fun getConfig(): JsonNode {
return Jsons.jsonNode(
Collections.singletonMap(
"test_destination",
Collections.singletonMap("test_destination_type", TestDestinationType.SILENT.name)
)
)
}
override fun getFailCheckConfig(): JsonNode {
return Jsons.jsonNode(
Collections.singletonMap(
"test_destination",
Collections.singletonMap("test_destination_type", "invalid")
)
)
}
override fun retrieveRecords(
testEnv: TestDestinationEnv?,
streamName: String,
namespace: String,
streamSchema: JsonNode
): List<JsonNode> {
return emptyList()
}
override fun setup(testEnv: TestDestinationEnv, TEST_SCHEMAS: HashSet<String>) {
// do nothing
}
override fun tearDown(testEnv: TestDestinationEnv) {
// do nothing
}
override fun assertSameMessages(
expected: List<AirbyteMessage>,
actual: List<AirbyteRecordMessage>,
pruneAirbyteInternalFields: Boolean
) {
Assertions.assertEquals(0, actual.size)
}
open override fun getDefaultSchema(config: JsonNode): String? {
return super.getDefaultSchema(config) ?: "default_schema"
}
// Skip because `retrieveRecords` returns an empty list at all times.
@Disabled @Test override fun testSyncNotFailsWithNewFields() {}
@Disabled @Test override fun testAirbyteTimeTypes() {}
// This test assumes that dedup support means normalization support.
// Override it to do nothing.
@Disabled
@Test
@Throws(Exception::class)
override fun testIncrementalDedupeSync() {
super.testIncrementalDedupeSync()
}
}

View File

@@ -1,34 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import io.airbyte.cdk.integrations.base.adaptive.AdaptiveSourceRunner
import io.airbyte.commons.features.EnvVariableFeatureFlags
import io.airbyte.commons.features.FeatureFlagsWrapper
import io.airbyte.commons.json.Jsons
import io.airbyte.commons.resources.MoreResources
import io.airbyte.protocol.models.v0.ConnectorSpecification
import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test
internal class CloudDevNullDestinationTest {
@Test
@Throws(Exception::class)
fun testSpec() {
val actual =
TestingDestinations(
FeatureFlagsWrapper.overridingDeploymentMode(
EnvVariableFeatureFlags(),
AdaptiveSourceRunner.CLOUD_MODE
)
)
.spec()
val expected =
Jsons.deserialize(
MoreResources.readResource("expected_spec_cloud.json"),
ConnectorSpecification::class.java
)
Assertions.assertEquals(expected, actual)
}
}

View File

@@ -1,62 +0,0 @@
/*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.dev_null
import com.google.common.collect.ImmutableMap
import io.airbyte.commons.json.Jsons
import io.airbyte.protocol.models.v0.AirbyteMessage
import io.airbyte.protocol.models.v0.AirbyteRecordMessage
import java.time.Instant
import java.util.*
import java.util.function.Consumer
import org.junit.jupiter.api.Test
import org.mockito.Mockito
import org.mockito.Mockito.mock
/**
* This source is designed to be a switch statement for our suite of highly-specific test sources.
*/
class ThrottledDestinationTest {
@Test
@Throws(Exception::class)
fun test() {
val outputRecordCollector: Consumer<AirbyteMessage> = Mockito.mock()
val config =
Jsons.jsonNode(
Collections.singletonMap(
"test_destination",
Collections.singletonMap("millis_per_record", 10)
)
)
val consumer = ThrottledDestination().getConsumer(config, mock(), outputRecordCollector)
consumer.accept(anotherRecord)
consumer.accept(anotherRecord)
consumer.accept(anotherRecord)
consumer.accept(anotherRecord)
consumer.accept(anotherRecord)
}
companion object {
private val anotherRecord: AirbyteMessage
get() =
AirbyteMessage()
.withType(AirbyteMessage.Type.RECORD)
.withRecord(
AirbyteRecordMessage()
.withStream("data")
.withEmittedAt(Instant.now().toEpochMilli())
.withData(
Jsons.jsonNode(
ImmutableMap.of(
"column1",
"contents1 " + Instant.now().toEpochMilli()
)
)
)
)
}
}

View File

@@ -0,0 +1,9 @@
{
"test_destination": {
"test_destination_type": "LOGGING",
"logging_config": {
"logging_type": "FirstN",
"max_entry_count": 100
}
}
}

View File

@@ -130,7 +130,6 @@ class AirbyteBulkConnectorPlugin implements Plugin<Project> {
]
}
// The name integrationTestJava is required by airbyte-ci.
project.sourceSets {
integrationTestJava {
kotlin {

View File

@@ -47,25 +47,26 @@ This mode throws an exception after receiving a configurable number of messages.
The OSS and Cloud variants have the same version number starting from version `0.2.2`.
| Version | Date | Pull Request | Subject |
| :------ | :--------- | :------------------------------------------------------- | :-------------------------------------------------------- |
| 0.6.1 | 2024-09-20 | [45715](https://github.com/airbytehq/airbyte/pull/45715) | add destination to cloud registry |
| Version | Date | Pull Request | Subject |
|:--------|:-----------|:---------------------------------------------------------|:-----------------------------------------------------------|
| 0.7.0 | 2024-09-20 | [45704](https://github.com/airbytehq/airbyte/pull/45704) | |
| 0.6.1 | 2024-09-20 | [45715](https://github.com/airbytehq/airbyte/pull/45715) | add destination to cloud registry |
| 0.6.0 | 2024-09-18 | [45651](https://github.com/airbytehq/airbyte/pull/45651) | merge destination-e2e(OSS) and destination-dev-null(cloud) |
| 0.5.0 | 2024-09-18 | [45650](https://github.com/airbytehq/airbyte/pull/45650) | upgrade cdk |
| 0.4.1 | 2024-09-18 | [45649](https://github.com/airbytehq/airbyte/pull/45649) | convert test code to kotlin |
| 0.4.0 | 2024-09-18 | [45648](https://github.com/airbytehq/airbyte/pull/45648) | convert production code to kotlin |
| 0.3.6 | 2024-05-09 | [38097](https://github.com/airbytehq/airbyte/pull/38097) | Support dedup |
| 0.3.5 | 2024-04-29 | [37366](https://github.com/airbytehq/airbyte/pull/37366) | Support refreshes |
| 0.3.4 | 2024-04-16 | [37366](https://github.com/airbytehq/airbyte/pull/37366) | Fix NPE |
| 0.3.3 | 2024-04-16 | [37366](https://github.com/airbytehq/airbyte/pull/37366) | Fix Log trace messages |
| 0.3.2 | 2024-02-14 | [36812](https://github.com/airbytehq/airbyte/pull/36812) | Log trace messages |
| 0.3.1 | 2024-02-14 | [35278](https://github.com/airbytehq/airbyte/pull/35278) | Adopt CDK 0.20.6 |
| 0.3.0 | 2023-05-08 | [25776](https://github.com/airbytehq/airbyte/pull/25776) | Standardize spec and change property field to non-keyword |
| 0.2.4 | 2022-06-17 | [13864](https://github.com/airbytehq/airbyte/pull/13864) | Updated stacktrace format for any trace message errors |
| 0.2.3 | 2022-02-14 | [10256](https://github.com/airbytehq/airbyte/pull/10256) | Add `-XX:+ExitOnOutOfMemoryError` JVM option |
| 0.2.2 | 2022-01-29 | [\#9745](https://github.com/airbytehq/airbyte/pull/9745) | Integrate with Sentry. |
| 0.2.1 | 2021-12-19 | [\#8824](https://github.com/airbytehq/airbyte/pull/8905) | Fix documentation URL. |
| 0.2.0 | 2021-12-16 | [\#8824](https://github.com/airbytehq/airbyte/pull/8824) | Add multiple logging modes. |
| 0.1.0 | 2021-05-25 | [\#3290](https://github.com/airbytehq/airbyte/pull/3290) | Create initial version. |
| 0.5.0 | 2024-09-18 | [45650](https://github.com/airbytehq/airbyte/pull/45650) | upgrade cdk |
| 0.4.1 | 2024-09-18 | [45649](https://github.com/airbytehq/airbyte/pull/45649) | convert test code to kotlin |
| 0.4.0 | 2024-09-18 | [45648](https://github.com/airbytehq/airbyte/pull/45648) | convert production code to kotlin |
| 0.3.6 | 2024-05-09 | [38097](https://github.com/airbytehq/airbyte/pull/38097) | Support dedup |
| 0.3.5 | 2024-04-29 | [37366](https://github.com/airbytehq/airbyte/pull/37366) | Support refreshes |
| 0.3.4 | 2024-04-16 | [37366](https://github.com/airbytehq/airbyte/pull/37366) | Fix NPE |
| 0.3.3 | 2024-04-16 | [37366](https://github.com/airbytehq/airbyte/pull/37366) | Fix Log trace messages |
| 0.3.2 | 2024-02-14 | [36812](https://github.com/airbytehq/airbyte/pull/36812) | Log trace messages |
| 0.3.1 | 2024-02-14 | [35278](https://github.com/airbytehq/airbyte/pull/35278) | Adopt CDK 0.20.6 |
| 0.3.0 | 2023-05-08 | [25776](https://github.com/airbytehq/airbyte/pull/25776) | Standardize spec and change property field to non-keyword |
| 0.2.4 | 2022-06-17 | [13864](https://github.com/airbytehq/airbyte/pull/13864) | Updated stacktrace format for any trace message errors |
| 0.2.3 | 2022-02-14 | [10256](https://github.com/airbytehq/airbyte/pull/10256) | Add `-XX:+ExitOnOutOfMemoryError` JVM option |
| 0.2.2 | 2022-01-29 | [\#9745](https://github.com/airbytehq/airbyte/pull/9745) | Integrate with Sentry. |
| 0.2.1 | 2021-12-19 | [\#8824](https://github.com/airbytehq/airbyte/pull/8905) | Fix documentation URL. |
| 0.2.0 | 2021-12-16 | [\#8824](https://github.com/airbytehq/airbyte/pull/8824) | Add multiple logging modes. |
| 0.1.0 | 2021-05-25 | [\#3290](https://github.com/airbytehq/airbyte/pull/3290) | Create initial version. |
</details>