Load CDK E2ETest Destination w/o tests (#45147)
This commit is contained in:
@@ -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 }
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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() {}
|
||||
}
|
||||
@@ -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")
|
||||
|
||||
@@ -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())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -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))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
)
|
||||
}
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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 {}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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" }
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -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,
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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."
|
||||
@@ -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 {
|
||||
|
||||
@@ -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(
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
|
||||
@@ -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()
|
||||
)
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
|
||||
@@ -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() }
|
||||
|
||||
@@ -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)
|
||||
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -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}"
|
||||
)
|
||||
}
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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) }
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -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() {}
|
||||
}
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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) },
|
||||
)
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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,
|
||||
)
|
||||
}
|
||||
},
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -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,
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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>,
|
||||
@@ -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
|
||||
//}
|
||||
|
||||
@@ -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" ]
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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() {}
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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?)
|
||||
}
|
||||
@@ -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")
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,3 @@
|
||||
# Default to OSS in the absence of an env variable
|
||||
deployment:
|
||||
mode: oss
|
||||
@@ -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"
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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"
|
||||
}
|
||||
}
|
||||
@@ -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()
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
@@ -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)),
|
||||
)
|
||||
}
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,2 @@
|
||||
deployment:
|
||||
mode: oss
|
||||
@@ -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)
|
||||
}
|
||||
}
|
||||
@@ -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()
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,9 @@
|
||||
{
|
||||
"test_destination": {
|
||||
"test_destination_type": "LOGGING",
|
||||
"logging_config": {
|
||||
"logging_type": "FirstN",
|
||||
"max_entry_count": 100
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -130,7 +130,6 @@ class AirbyteBulkConnectorPlugin implements Plugin<Project> {
|
||||
]
|
||||
}
|
||||
|
||||
// The name integrationTestJava is required by airbyte-ci.
|
||||
project.sourceSets {
|
||||
integrationTestJava {
|
||||
kotlin {
|
||||
|
||||
@@ -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>
|
||||
Reference in New Issue
Block a user