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

chore: dataflow cdk connector building guide (#70217)

This commit is contained in:
Jimmy Ma
2025-11-26 08:21:05 -08:00
committed by GitHub
parent 6fe83b03cb
commit a960a0f2e3
12 changed files with 8277 additions and 0 deletions

View File

@@ -0,0 +1,918 @@
# Dataflow CDK Coding Standards
**Summary:** Best practices for implementing Airbyte destination connectors. Covers Kotlin style, Micronaut DI patterns, async/coroutines, SQL generation, error handling, and common pitfalls. Follow these standards for maintainable, production-ready code.
---
## Code Organization
### Package Structure
```
destination-{db}/src/main/kotlin/io/airbyte/integrations/destination/{db}/
├── {DB}Destination.kt # Entry point (main())
├── {DB}BeanFactory.kt # Micronaut DI setup
├── check/{DB}Checker.kt # Connection validation
├── client/
│ ├── {DB}AirbyteClient.kt # Database operations
│ └── {DB}SqlGenerator.kt # SQL generation
├── config/
│ ├── {DB}NameGenerators.kt # Table/column naming
│ └── {DB}DirectLoadDatabaseInitialStatusGatherer.kt
├── dataflow/
│ ├── {DB}Aggregate.kt # Record accumulation
│ └── {DB}AggregateFactory.kt # Create aggregates
├── spec/
│ ├── {DB}Specification.kt # Config schema
│ └── {DB}Configuration.kt # Runtime config
└── write/
├── {DB}Writer.kt # Orchestration
└── load/{DB}InsertBuffer.kt # Batch writes
```
### File Naming
**Pattern:** `{DatabaseName}{ComponentType}.kt`
**Examples:**
- `SnowflakeAirbyteClient.kt`
- `ClickhouseSqlGenerator.kt`
- `PostgresInsertBuffer.kt`
**Avoid:** Generic names like `Client.kt`, `Generator.kt`
---
## Kotlin Style
### Naming Conventions
```kotlin
// Classes: PascalCase with prefix
class SnowflakeAirbyteClient
class ClickhouseInsertBuffer
// Functions: camelCase, verb-based
fun createTable(...): String
fun toSnowflakeCompatibleName(): String
suspend fun flush()
// Variables: camelCase, descriptive
val snowflakeClient: SnowflakeAirbyteClient
private val columnNameMapping: ColumnNameMapping
// Constants: SCREAMING_SNAKE_CASE
internal const val DATA_SOURCE_CONNECTION_TIMEOUT_MS = 30000L
internal const val CSV_FIELD_SEPARATOR = ','
const val DEFAULT_FLUSH_LIMIT = 1000
// Companion object constants
companion object {
const val DATETIME_WITH_PRECISION = "DateTime64(3)"
const val PROTOCOL = "http"
}
```
### Data Classes vs Regular Classes
```kotlin
// ✅ Data classes: Immutable config/value objects
data class SnowflakeConfiguration(
val host: String,
val database: String,
val schema: String,
) : DestinationConfiguration()
// ✅ Regular classes: Stateful components
@Singleton
class SnowflakeAirbyteClient(
private val dataSource: DataSource,
private val sqlGenerator: SqlGenerator,
) : TableOperationsClient {
// Methods with side effects
}
```
### Extension Functions
```kotlin
// ✅ SQL logging
fun String.andLog(): String {
log.info { this.trim() }
return this
}
// ✅ Name compatibility
fun String.toSnowflakeCompatibleName(): String {
return this.uppercase().replace(Regex("[^A-Z0-9_]"), "_")
}
// ✅ Type utilities
fun String.sqlNullable(): String = "Nullable($this)"
fun ColumnType.typeDecl() = typeDecl(this.type, this.nullable)
// ✅ Quoting
fun String.quote() = "\"$this\"" // Snowflake
```
---
## Dependency Injection (Micronaut)
### Always Use Constructor Injection
```kotlin
// ✅ DO: Constructor injection
@Singleton
class SnowflakeAirbyteClient(
private val dataSource: DataSource,
private val sqlGenerator: SnowflakeDirectLoadSqlGenerator,
private val columnUtils: SnowflakeColumnUtils,
) : TableOperationsClient
// ❌ DON'T: Field injection
@Singleton
class BadExample {
@Inject lateinit var dataSource: DataSource // Avoid!
}
```
### BeanFactory Pattern
```kotlin
@Factory
class SnowflakeBeanFactory {
@Singleton
fun snowflakeConfiguration(
configFactory: SnowflakeConfigurationFactory,
specFactory: SnowflakeMigratingConfigurationSpecificationSupplier,
): SnowflakeConfiguration {
val spec = specFactory.get()
return configFactory.makeWithoutExceptionHandling(spec)
}
@Singleton
@Requires(property = Operation.PROPERTY, notEquals = "spec")
fun snowflakeDataSource(
config: SnowflakeConfiguration,
): HikariDataSource {
return HikariDataSource().apply {
jdbcUrl = buildConnectionString(config)
username = config.username
// ... complex setup
}
}
@Singleton
@Requires(property = Operation.PROPERTY, value = "spec")
fun emptyDataSource(): DataSource {
return object : DataSource {
override fun getConnection(): Connection? = null
}
}
}
```
### Conditional Beans
```kotlin
// ✅ Operation-specific beans
@Singleton
@Requires(property = Operation.PROPERTY, value = "spec")
fun specDataSource(): DataSource {
// Only for --spec operation
}
@Singleton
@Requires(property = Operation.PROPERTY, notEquals = "spec")
fun realDataSource(config: MyConfiguration): HikariDataSource {
// For --check and --write operations
}
```
### lateinit var Usage
```kotlin
// ✅ For lifecycle-initialized state
@Singleton
class SnowflakeWriter(...) : DestinationWriter {
private lateinit var initialStatuses: Map<DestinationStream, DirectLoadInitialStatus>
override suspend fun setup() {
initialStatuses = stateGatherer.gatherInitialStatus(names)
}
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
val status = initialStatuses[stream]!! // Guaranteed after setup()
// ...
}
}
// ✅ For optional state
class InsertBuffer {
private var csvWriter: CsvWriter? = null
private var csvFilePath: Path? = null
}
// ❌ Don't use with nullables or primitives
private lateinit var config: Config? // Compile error!
private lateinit var count: Int // Compile error!
```
---
## Async & Coroutines
### suspend fun Usage
```kotlin
// ✅ Mark I/O operations as suspend
override suspend fun countTable(tableName: TableName): Long? { ... }
override suspend fun createNamespace(namespace: String) { ... }
override suspend fun flush() { ... }
// ✅ Propagate suspend through call chain
override suspend fun flush() {
buffer.flush() // Also suspend
}
// ❌ DON'T block in suspend functions
suspend fun bad() {
Thread.sleep(1000) // Blocks thread!
future.get() // Blocks thread!
}
// ✅ DO use suspend functions
suspend fun good() {
delay(1000) // Suspends without blocking
future.await() // Suspends without blocking
}
```
### runBlocking Usage
```kotlin
// ✅ Use only at non-suspend boundaries
@Singleton
class SnowflakeChecker(...) : DestinationCheckerV2 {
override fun check() { // Can't be suspend (interface constraint)
runBlocking {
try {
client.createNamespace(namespace)
client.createTable(...)
} finally {
client.dropTable(tableName)
}
}
}
}
// ❌ DON'T use inside suspend functions
suspend fun bad() {
runBlocking { // Creates new scope - wrong!
client.execute(query)
}
}
```
### Convert Java Futures
```kotlin
import kotlinx.coroutines.future.await
// ✅ Use .await() for CompletableFuture
internal suspend fun execute(query: String): CommandResponse {
return client.execute(query).await()
}
suspend fun flush() {
val result = clickhouseClient
.insert(tableName, data)
.await() // Not .get()!
}
```
### Error Handling in Async
```kotlin
// ✅ try-finally for cleanup
suspend fun flush() {
csvFilePath?.let { filePath ->
try {
csvWriter?.flush()
csvWriter?.close()
client.putInStage(tableName, filePath.pathString)
client.copyFromStage(tableName, filePath.fileName.toString())
} catch (e: Exception) {
logger.error(e) { "Unable to flush accumulated data." }
throw e
} finally {
// Always cleanup
filePath.deleteIfExists()
csvWriter = null
csvFilePath = null
recordCount = 0
}
}
}
```
---
## SQL Generation Patterns
### Separation of Concerns
```kotlin
// ✅ SqlGenerator: Pure SQL strings, no execution
@Singleton
class SnowflakeDirectLoadSqlGenerator(
private val columnUtils: SnowflakeColumnUtils,
private val config: SnowflakeConfiguration,
) {
fun createTable(...): String { ... }
fun upsertTable(...): String { ... }
fun dropTable(tableName: TableName): String {
return "DROP TABLE IF EXISTS ${fullyQualifiedName(tableName)}"
}
}
// ✅ Client: Execution only, delegates SQL generation
@Singleton
class SnowflakeAirbyteClient(
private val dataSource: DataSource,
private val sqlGenerator: SnowflakeDirectLoadSqlGenerator,
) : TableOperationsClient {
override suspend fun dropTable(tableName: TableName) {
execute(sqlGenerator.dropTable(tableName))
}
private suspend fun execute(sql: String) {
dataSource.connection.use { conn ->
conn.createStatement().executeQuery(sql)
}
}
}
```
### Always Log SQL
```kotlin
private val log = KotlinLogging.logger {}
// ✅ Extension function pattern
fun String.andLog(): String {
log.info { this.trim() }
return this
}
// Usage
fun createTable(...): String {
return """
CREATE TABLE ${fullyQualifiedName(tableName)} (
${columnDeclarations}
)
""".trimIndent().andLog()
}
```
### Quoting and Escaping
```kotlin
// Snowflake: double quotes
internal const val QUOTE = "\""
fun String.quote() = "$QUOTE$this$QUOTE"
"${columnName.quote()}" // "column_name"
// ClickHouse: backticks
"`${tableName.namespace}`.`${tableName.name}`"
// Escape special characters
fun escapeJsonIdentifier(identifier: String): String {
return identifier.replace(QUOTE, "$QUOTE$QUOTE")
}
```
### Multi-line SQL
```kotlin
// ✅ Use trimIndent()
fun createTable(...): String {
return """
CREATE TABLE `${tableName.namespace}`.`${tableName.name}` (
$COLUMN_NAME_AB_RAW_ID String NOT NULL,
$COLUMN_NAME_AB_EXTRACTED_AT DateTime64(3) NOT NULL,
${columnDeclarations}
)
ENGINE = ${engine}
ORDER BY (${orderByColumns})
""".trimIndent().andLog()
}
// ✅ Use StringBuilder for complex statements
fun alterTable(changeset: ColumnChangeset, tableName: TableName): String {
val builder = StringBuilder()
.append("ALTER TABLE `${tableName.namespace}`.`${tableName.name}`")
.appendLine()
changeset.columnsToAdd.forEach { (columnName, columnType) ->
builder.append(" ADD COLUMN `$columnName` ${columnType.typeDecl()},")
}
changeset.columnsToChange.forEach { (columnName, typeChange) ->
builder.append(" MODIFY COLUMN `$columnName` ${typeChange.newType.typeDecl()},")
}
return builder.dropLast(1).toString().andLog() // Remove trailing comma
}
```
---
## Error Handling
### Exception Types
```kotlin
// ✅ ConfigErrorException: User-fixable
throw ConfigErrorException(
"Permission denied: Cannot CREATE tables in schema '$schema'. " +
"Grant CREATE privileges to your role.",
cause
)
throw ConfigErrorException(
"Table '$tableName' exists but lacks Airbyte internal columns. " +
"Delete the table or use a different table prefix."
)
// ✅ TransientErrorException: Retryable
throw TransientErrorException(
"Network timeout connecting to database. Will retry automatically.",
cause
)
// ✅ SystemErrorException: Internal errors
throw SystemErrorException(
"Cannot execute hybrid refresh - unsupported sync mode"
)
```
### Wrap Database Exceptions
```kotlin
private fun handleSnowflakePermissionError(e: SnowflakeSQLException): Nothing {
val errorMessage = e.message?.lowercase() ?: ""
when {
errorMessage.contains("current role has no privileges") -> {
throw ConfigErrorException(
"Permission denied. Grant privileges: " +
"GRANT CREATE, DROP, ALTER ON SCHEMA ${schema} TO ROLE ${role};",
e
)
}
errorMessage.contains("insufficient privileges") -> {
throw ConfigErrorException(
"Insufficient privileges. Contact your database administrator.",
e
)
}
else -> throw e // System error
}
}
// Usage
override suspend fun createNamespace(namespace: String) {
try {
execute(sqlGenerator.createNamespace(namespace))
} catch (e: SnowflakeSQLException) {
handleSnowflakePermissionError(e)
}
}
```
### Return Null for Expected Missing Data
```kotlin
// ✅ Return null when table doesn't exist (expected)
override suspend fun countTable(tableName: TableName): Long? =
try {
dataSource.connection.use { connection ->
val statement = connection.createStatement()
statement.use {
val resultSet = it.executeQuery(sqlGenerator.countTable(tableName))
if (resultSet.next()) resultSet.getLong("count") else 0L
}
}
} catch (e: SQLException) {
log.debug(e) {
"Table ${tableName.toPrettyString()} does not exist. Returning null."
}
null // Expected - not an error
}
```
### Actionable Error Messages
```kotlin
// ✅ DO: Specific and actionable
throw ConfigErrorException(
"Permission denied: Cannot CREATE tables in schema 'public'.\n\n" +
"Required permission:\n" +
" GRANT CREATE ON SCHEMA public TO ROLE your_role;\n\n" +
"Alternatively:\n" +
" 1. Use a different schema where you have permissions\n" +
" 2. Contact your database administrator\n\n" +
"Current user: ${config.username}\n" +
"Current role: ${config.role}",
e
)
// ❌ DON'T: Vague
throw ConfigErrorException("Access denied", e)
throw ConfigErrorException(e.message, e) // Raw database message
```
---
## Logging Patterns
### Log Levels
```kotlin
private val log = KotlinLogging.logger {}
// info: Normal operations
log.info { "Beginning insert into ${tableName.toPrettyString()}..." }
log.info { "Finished insert of $recordCount row(s)..." }
// warn: Unexpected but recoverable
log.warn { "CSV file path is not set: nothing to upload to staging." }
// error: Errors that will fail the operation
log.error(e) { "Unable to flush accumulated data." }
// debug: Detailed diagnostics
log.debug(e) { "Table does not exist. Returning null..." }
```
### Structured Logging
```kotlin
// ✅ Use lambda syntax for lazy evaluation
log.info { "Expensive computation: ${computeExpensiveValue()}" }
// Only called if INFO enabled
// ❌ DON'T: Eager evaluation
log.info("Expensive computation: ${computeExpensiveValue()}")
// Always called, even if INFO disabled
// ✅ Include context
log.info {
"overwriteTable: source=${source.toPrettyString()}, " +
"target=${target.toPrettyString()}, targetExists=$targetExists"
}
log.error(e) {
"Failed to execute query on table ${tableName.toPrettyString()}. " +
"Operation: $operation, SQL State: ${e.sqlState}"
}
```
---
## Resource Management
### JDBC Resources
```kotlin
// ✅ Use .use {} for automatic closing
override suspend fun countTable(tableName: TableName): Long? =
dataSource.connection.use { connection ->
val statement = connection.createStatement()
statement.use {
val resultSet = it.executeQuery(sqlGenerator.countTable(tableName))
if (resultSet.next()) resultSet.getLong("count") else 0L
}
}
// ❌ DON'T: Manual close
val connection = dataSource.connection
try {
// use connection
} finally {
connection.close() // Might not be called if exception in try
}
```
### File Cleanup
```kotlin
// ✅ Use deleteOnExit() for temp files
val tempFile = File.createTempFile("prefix", ".tmp")
tempFile.deleteOnExit()
tempFile.writeText(data)
// ✅ Use deleteIfExists() in finally
suspend fun flush() {
csvFilePath?.let { filePath ->
try {
csvWriter?.flush()
csvWriter?.close()
client.putInStage(tableName, filePath.pathString)
} finally {
filePath.deleteIfExists()
csvWriter = null
csvFilePath = null
}
}
}
```
---
## Common Gotchas
### 1. Thread Safety
```kotlin
// ❌ Mutable state in singleton without synchronization
@Singleton
class Bad {
private var counter = 0 // Race condition!
fun increment() { counter++ }
}
// ✅ Use lateinit only for lifecycle initialization (set once)
@Singleton
class Good {
private lateinit var initialStatuses: Map<...> // Set once in setup()
}
```
### 2. Micronaut Pitfalls
```kotlin
// ❌ Forgetting @Singleton
class MyService { // Won't be managed by DI!
}
// ❌ Operation beans without @Requires
@Singleton
fun specDataSource(): DataSource {
// Created for ALL operations!
}
// ✅ Use @Requires
@Singleton
@Requires(property = Operation.PROPERTY, value = "spec")
fun specDataSource(): DataSource {
// Only for spec operation
}
```
### 3. SQL Injection & Batching
```kotlin
// ❌ Not batching inserts
records.forEach { insertRecord(it) } // One at a time!
// ✅ Use buffering
class InsertBuffer(flushLimit: Int = 1000) {
private val buffer = mutableListOf<Record>()
fun accumulate(record: Record) {
buffer.add(record)
if (buffer.size >= flushLimit) {
runBlocking { flush() }
}
}
suspend fun flush() {
writeBatchToDatabase(buffer)
buffer.clear()
}
}
```
### 4. Resource Leaks
```kotlin
// ❌ Not closing resources
val connection = dataSource.connection
// use connection
// Forgot to close!
// ✅ Use .use {}
dataSource.connection.use { connection ->
// Automatically closed even if exception
}
```
### 5. Async Pitfalls
```kotlin
// ❌ Blocking in suspend functions
suspend fun bad() {
Thread.sleep(1000) // Blocks!
future.get() // Blocks!
}
// ✅ Use suspend functions
suspend fun good() {
delay(1000) // Suspends
future.await() // Suspends
}
// ❌ Not propagating suspend
class Bad {
fun flush() { // Not suspend
runBlocking { client.execute() } // Creates new scope!
}
}
// ✅ Keep suspend throughout
class Good {
suspend fun flush() { // Propagate suspend
client.execute()
}
}
```
### 6. CDK-Specific Gotchas
```kotlin
// ❌ Not using TableCatalog for column mapping
val mappedColumn = columnName.toUpperCase() // Manual!
// ✅ Use TableCatalog
val mappedColumn = columnNameMapping[columnName]!!
// ❌ Creating tables in Writer.setup()
override suspend fun setup() {
client.createTable(...) // StreamLoader does this!
}
// ✅ Only create namespaces in setup()
override suspend fun setup() {
namespaces.forEach { client.createNamespace(it) }
}
// ❌ Finalizing in InsertBuffer.flush()
suspend fun flush() {
writeToDatabase()
client.upsertTable(temp, final) // DON'T - StreamLoader does this!
}
// ✅ Just write records
suspend fun flush() {
writeToDatabase() // That's it!
}
```
### 7. Null Safety
```kotlin
// ❌ Unnecessary !! operator
val value = map[key]!! // Throws if missing
// ✅ Handle null explicitly
val value = map[key] ?: throw IllegalStateException("Key not found: $key")
val value = map[key]?.let { process(it) } // Safe call
// ❌ lateinit with wrong types
private lateinit var config: Config? // Compile error!
private lateinit var count: Int // Compile error!
// ✅ Use nullable var or non-nullable
private var config: Config? = null
private var count: Int = 0
```
---
## Testing Patterns
### Test Structure
```kotlin
internal class SnowflakeDirectLoadSqlGeneratorTest {
private lateinit var sqlGenerator: SnowflakeDirectLoadSqlGenerator
private val uuidGenerator: UUIDGenerator = mockk()
@BeforeEach
fun setUp() {
sqlGenerator = SnowflakeDirectLoadSqlGenerator(
columnUtils,
uuidGenerator,
config,
nameUtils,
)
}
@Test
fun testGenerateCountTableQuery() {
val sql = sqlGenerator.countTable(tableName)
assertTrue(sql.contains("SELECT COUNT(*)"))
assertTrue(sql.contains(tableName.name))
}
}
```
### Test Naming
```kotlin
// ✅ Descriptive test names
@Test
fun testGenerateCountTableQuery() { ... }
@Test
fun testGenerateCreateTableStatement() { ... }
// ✅ Backtick format for readability
@Test
fun `test extractPks with single primary key`() { ... }
@Test
fun `test extractPks with multiple primary keys`() { ... }
```
### Test Cleanup
```kotlin
// ✅ Use try-finally
override fun check() {
runBlocking {
try {
client.createNamespace(namespace)
client.createTable(table, ...)
val count = client.countTable(table)
require(count == 1L)
} finally {
client.dropTable(table)
}
}
}
```
---
## Style Summary
**Naming:**
- Classes: PascalCase with prefix (`SnowflakeAirbyteClient`)
- Functions: camelCase, verbs (`createTable`, `flush`)
- Variables: camelCase, descriptive (`columnNameMapping`)
- Constants: SCREAMING_SNAKE_CASE (`DEFAULT_FLUSH_LIMIT`)
**DI:**
- Always constructor injection
- Use `@Factory` for complex setup
- Use `@Requires` for conditional beans
- Use `lateinit var` only for lifecycle state
**Async:**
- Mark I/O as `suspend`
- Use `.use {}` for resources
- Use `.await()` for futures
- Propagate `suspend` through call chain
**SQL:**
- Separate generation from execution
- Always log SQL (`.andLog()`)
- Use `.trimIndent()` for multi-line
- Quote all identifiers
**Error:**
- `ConfigErrorException` for user errors
- `TransientErrorException` for retryable
- `SystemErrorException` for internal
- Return `null` for expected missing data
**Logging:**
- Use lambda syntax (lazy evaluation)
- Include context (table names, operations)
- Appropriate levels (info/warn/error/debug)
**Resources:**
- `.use {}` for JDBC/closeable
- `deleteOnExit()` and `finally` for files
- Clean up in `finally` blocks
---
## Quick Reference: Most Common Mistakes
1. ❌ Forgetting `@Singleton` on service classes
2. ❌ Not logging generated SQL
3. ❌ Blocking in suspend functions (`Thread.sleep`, `.get()`)
4. ❌ Not using `.use {}` for resource cleanup
5. ❌ Missing `.trimIndent()` on SQL strings
6. ❌ Using `!!` without proving non-null
7. ❌ Not batching database operations
8. ❌ Creating tables in `Writer.setup()` (StreamLoader does this)
9. ❌ Finalizing in `InsertBuffer.flush()` (StreamLoader does this)
10. ❌ Not using `TableCatalog` for column mapping

View File

@@ -0,0 +1,803 @@
# Dataflow CDK Architecture
**Summary:** The Airbyte Dataflow CDK is a framework that orchestrates destination connector write operations. You implement 4 database-specific components (SQL generator, client, insert buffer, column utilities). The CDK handles message parsing, data flow, table lifecycle, state management, and error handling. Result: Write ~4 custom components, get all sync modes (append, dedupe, overwrite) for free.
---
## Architecture Overview
### Entry Point to Database
```
main()
→ AirbyteDestinationRunner.run(*args)
→ Parse CLI (--spec, --check, --write)
→ Create Micronaut context
→ Select Operation (SpecOperation, CheckOperation, WriteOperationV2)
→ Execute operation
```
**Write Operation Flow:**
```
WriteOperationV2.execute()
→ DestinationLifecycle.run()
1. Writer.setup() [Create namespaces]
2. Initialize streams [Create StreamLoaders]
3. runDataPipeline() [Process messages]
4. Finalize streams [MERGE/SWAP/cleanup]
5. Teardown [Close connections]
```
### Data Flow Pipeline
**stdin → Database:**
```
Airbyte Platform Connector Pipeline Database
| | |
|-- RECORD messages ------>| |
|-- STATE messages -------->| |
| | |
| Parse JSON |
| Transform types |
| Map column names |
| Batch records |
| | |
| Aggregate.accept() |
| ↓ |
| InsertBuffer.accumulate() |
| | |
| [Buffering] |
| | |
| Aggregate.flush() |
| ↓ |
| InsertBuffer.flush() --------------->| Write batch
| | |
| [Repeat] |
| | |
| StreamLoader.close() |
| ↓ |
| MERGE/SWAP/nothing -------------->| Finalize
|<----- STATE emitted -----| |
```
**Key Insight:** Your `InsertBuffer` only writes batches. The framework handles message parsing, batching triggers, and finalization strategy (MERGE vs SWAP vs direct).
---
## Core Abstractions
### StreamLoader (CDK-Provided)
**Purpose:** Orchestrates per-stream write lifecycle
**You don't implement** - you instantiate the right one based on sync mode
**4 Variants:**
| StreamLoader | Mode | Strategy | Use Case |
|--------------|------|----------|----------|
| `DirectLoadTableAppendStreamLoader` | Append | Direct write to final table | Logs, append-only data |
| `DirectLoadTableDedupStreamLoader` | Dedupe | Temp table → MERGE with PK dedup | Incremental sync with PK |
| `DirectLoadTableAppendTruncateStreamLoader` | Overwrite | Temp table → SWAP | Full refresh without PK |
| `DirectLoadTableDedupTruncateStreamLoader` | Dedupe + Overwrite | Temp table → dedupe → SWAP | Full refresh with PK |
**Lifecycle:**
```kotlin
StreamLoader.start() {
- Check if final table exists
- Create/evolve final table
- Create temp table if needed (dedupe/truncate)
- Store target table name in state
}
[Records flow through pipeline → your InsertBuffer writes to table]
StreamLoader.close(streamCompleted) {
if (streamCompleted) {
// Dedupe: MERGE temp → final
// Truncate: SWAP temp ↔ final
// Append: nothing (already in final)
}
// Always cleanup temp tables
}
```
**Selection Pattern:**
```kotlin
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
return when (stream.minimumGenerationId) {
0L -> when (stream.importType) {
is Dedupe -> DirectLoadTableDedupStreamLoader(...)
else -> DirectLoadTableAppendStreamLoader(...)
}
stream.generationId -> when (stream.importType) {
is Dedupe -> DirectLoadTableDedupTruncateStreamLoader(...)
else -> DirectLoadTableAppendTruncateStreamLoader(...)
}
else -> throw SystemErrorException("Hybrid refresh not supported")
}
}
```
### Aggregate (Pattern-Based)
**Purpose:** Accumulates records, triggers flushing
**Your Implementation (3 lines):**
```kotlin
class MyAggregate(private val buffer: MyInsertBuffer) : Aggregate {
override fun accept(record: RecordDTO) {
buffer.accumulate(record.fields)
}
override suspend fun flush() {
buffer.flush()
}
}
```
**Created by:** `AggregateFactory` per stream
```kotlin
@Factory
class MyAggregateFactory(
private val client: MyAirbyteClient,
private val streamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>,
) : AggregateFactory {
override fun create(key: StoreKey): Aggregate {
// Get table name set by StreamLoader
val tableName = streamStateStore.get(key)!!.tableName
val buffer = MyInsertBuffer(tableName, client)
return MyAggregate(buffer)
}
}
```
### TableOperationsClient (You Implement)
**Purpose:** Database primitive operations
**Key Methods:**
```kotlin
interface TableOperationsClient {
suspend fun createNamespace(namespace: String)
suspend fun namespaceExists(namespace: String): Boolean
suspend fun createTable(stream, tableName, columnMapping, replace)
suspend fun tableExists(table: TableName): Boolean
suspend fun dropTable(tableName: TableName)
suspend fun countTable(tableName: TableName): Long?
suspend fun getGenerationId(tableName: TableName): Long
suspend fun overwriteTable(source, target) // For truncate mode
suspend fun copyTable(columnMapping, source, target)
suspend fun upsertTable(stream, columnMapping, source, target) // For dedupe
}
```
**Pattern:**
```kotlin
@Singleton
class MyAirbyteClient(
private val dataSource: DataSource,
private val sqlGenerator: MySqlGenerator,
) : TableOperationsClient, TableSchemaEvolutionClient {
override suspend fun createTable(...) {
execute(sqlGenerator.createTable(...))
}
override suspend fun upsertTable(...) {
execute(sqlGenerator.upsertTable(...)) // MERGE statement
}
private suspend fun execute(sql: String) {
dataSource.connection.use { conn ->
conn.createStatement().use { stmt ->
stmt.executeQuery(sql)
}
}
}
}
```
**Separation:** Client executes SQL, SqlGenerator generates SQL
### TableSchemaEvolutionClient (You Implement)
**Purpose:** Automatic schema adaptation
**4-Step Process:**
```kotlin
// 1. Discover current schema from database
suspend fun discoverSchema(tableName): TableSchema
// 2. Compute expected schema from stream
fun computeSchema(stream, columnMapping): TableSchema
// 3. Compare (automatic by CDK)
val changeset = ColumnChangeset(
columnsToAdd = ...,
columnsToDrop = ...,
columnsToChange = ...,
)
// 4. Apply changes
suspend fun applyChangeset(..., changeset)
```
**When Called:** Automatically by `StreamLoader.start()` if table exists
**Operations:**
- **Add column:** `ALTER TABLE ADD COLUMN`
- **Drop column:** `ALTER TABLE DROP COLUMN`
- **Widen type:** `ALTER TABLE ALTER COLUMN TYPE` (safe)
- **Narrow type:** Temp column + cast + rename (complex)
### DestinationWriter (Pattern-Based)
**Purpose:** Orchestration layer
**Your Implementation:**
```kotlin
@Singleton
class MyWriter(
private val names: TableCatalog,
private val stateGatherer: DatabaseInitialStatusGatherer<DirectLoadInitialStatus>,
private val streamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>,
private val client: MyAirbyteClient,
private val tempTableNameGenerator: TempTableNameGenerator,
) : DestinationWriter {
private lateinit var initialStatuses: Map<DestinationStream, DirectLoadInitialStatus>
override suspend fun setup() {
// Create all namespaces
names.values
.map { it.tableNames.finalTableName!!.namespace }
.toSet()
.forEach { client.createNamespace(it) }
// Gather initial state (table exists? gen ID? columns?)
initialStatuses = stateGatherer.gatherInitialStatus(names)
}
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
val initialStatus = initialStatuses[stream]!!
val tableNames = names[stream]!!.tableNames
val columnMapping = names[stream]!!.columnNameMapping
return /* Select appropriate StreamLoader */
}
}
```
**Key Responsibilities:**
- Create namespaces (schema/database)
- Gather initial table state
- Select correct StreamLoader for each stream
- **Does NOT:** Create tables, write data, perform schema evolution (StreamLoader does)
---
## What the CDK Provides
### Automatic Services
| Component | Responsibilities | Your Interaction |
|-----------|-----------------|------------------|
| **DestinationLifecycle** | Overall orchestration | None - runs automatically |
| **Data Pipeline** | Parse messages, transform types, batch records | Configure via ColumnUtils |
| **4 StreamLoaders** | Table lifecycle, finalization strategy | Instantiate the right one |
| **StreamStateStore** | Coordinate InsertBuffer ↔ StreamLoader | Read from in AggregateFactory |
| **TableCatalog** | Column name mapping (logical → physical) | Query for mapped column names |
| **State Management** | Track checkpoints, emit STATE messages | Automatic after successful flush |
| **Error Handling** | Classify errors, emit TRACE messages | Throw ConfigError/SystemError |
### Base Classes with Defaults
| Base Class | Purpose | Customization Needed |
|------------|---------|---------------------|
| `BaseDirectLoadInitialStatusGatherer` | Gather table state before sync | Usually none - just extend |
| `DefaultTempTableNameGenerator` | Generate temp table names | Usually none - use as-is |
---
## What You Implement
### 4 Core Custom Components
| Component | Effort | Purpose | Lines of Code |
|-----------|--------|---------|---------------|
| **SQL Generator** | High | Generate DB-specific SQL | 300-500 |
| **Database Client** | High | Execute SQL, handle errors | 400-600 |
| **Insert Buffer** | Medium | Efficient batch writes | 200-300 |
| **Column Utilities** | Medium | Type mapping, column declarations | 100-200 |
### Pattern-Based Components
| Component | Effort | Purpose | Lines of Code |
|-----------|--------|---------|---------------|
| **Configuration** | Low | Config spec, factory, validation | 100-150 |
| **Name Generators** | Low | Table/column name formatting | 50-100 |
| **Checker** | Low | Connection validation | 50-80 |
| **Writer** | Low | Orchestration (setup, select loaders) | 80-120 |
### Boilerplate Components
| Component | Effort | Purpose | Lines of Code |
|-----------|--------|---------|---------------|
| **Aggregate** | Minimal | Delegate to buffer | 10-15 |
| **AggregateFactory** | Minimal | Create aggregate per stream | 20-30 |
| **WriteOperationV2** | Minimal | Entry point for write operation | 10-15 |
| **BeanFactory** | Low | Micronaut DI setup | 50-100 |
**Total:** ~20 components, ~2000-3000 lines of code
**Critical Path:** SqlGenerator → Client → InsertBuffer → ColumnUtils
---
## Key Concepts
### Temp Tables Strategy
**Why?**
- **Atomic semantics:** All-or-nothing commit
- **Isolation:** Transform without affecting final table
- **Easy rollback:** Just drop temp on failure
- **Performance:** Write without locks/constraints
**When?**
- **Dedupe mode:** Temp table → dedupe via MERGE → final table
- **Truncate mode:** Temp table → SWAP with final table
- **Append mode:** No temp table (direct write)
**Naming:** `_airbyte_tmp_{uuid}_{timestamp}` in internal schema
**Lifecycle:**
```
StreamLoader.start() → createTable(tempTable)
Records written to tempTable via InsertBuffer
StreamLoader.close() → finalize from tempTable → dropTable(tempTable)
```
### Zero Downtime Architecture
**Key Guarantee:** Readers never see empty or partial tables during sync.
**How:** Temp table + atomic swap pattern
- Write new data to temp table (readers see old data)
- Atomic SWAP/EXCHANGE operation (milliseconds)
- Readers instantly see new data
**Why Atomic:**
- SWAP/EXCHANGE are metadata operations (not data copying)
- Database guarantees atomicity
- Old data visible until swap completes
**Traditional ETL comparison:**
- DROP TABLE → CREATE TABLE = downtime (table doesn't exist)
- TRUNCATE → INSERT = downtime (empty table visible)
- Temp → SWAP = zero downtime (always consistent snapshot)
**Use Cases:**
- 24/7 dashboards (cannot tolerate "table not found")
- Production APIs (empty results = outage)
- Long-running syncs (old data available until completion)
### Sync Modes Mapping
| User Setting | minimumGenerationId | importType | StreamLoader |
|--------------|---------------------|------------|--------------|
| Incremental | 0 | Append | `DirectLoadTableAppendStreamLoader` |
| Incremental | 0 | Dedupe | `DirectLoadTableDedupStreamLoader` |
| Full Refresh (Append) | generationId | Append | `DirectLoadTableAppendTruncateStreamLoader` |
| Full Refresh (Overwrite) | generationId | Dedupe | `DirectLoadTableDedupTruncateStreamLoader` |
**Generation IDs:**
- `minimumGenerationId = 0`: Keep all existing data (incremental)
- `minimumGenerationId = generationId`: Replace all data (full refresh)
- `minimumGenerationId != 0 and != generationId`: Hybrid refresh (NOT SUPPORTED)
**Resume Logic (Truncate Mode):**
When StreamLoader.start() is called:
1. If tempTable exists AND tempTable.generationId == stream.generationId → Resume interrupted sync (write to temp)
2. If realTable exists AND realTable.generationId == stream.generationId → Sync already completed (write to real)
3. Otherwise → New sync (drop stale temp if exists, create fresh temp)
**Generation ID Purpose:**
- Uniquely identifies each sync attempt
- Written to `_airbyte_generation_id` column in every record
- Enables detection of interrupted syncs vs completed syncs
- Allows safe resume without duplicate work
### Deduplication Logic
**Dedupe StreamLoader uses window function:**
```sql
WITH deduped AS (
SELECT *, ROW_NUMBER() OVER (
PARTITION BY primary_key
ORDER BY _airbyte_extracted_at DESC
) AS rn
FROM temp_table
)
SELECT * FROM deduped WHERE rn = 1
```
**Then MERGE into final:**
```sql
MERGE INTO final_table AS target
USING deduped AS source
ON target.pk = source.pk
WHEN MATCHED AND source.cursor > target.cursor THEN UPDATE ...
WHEN NOT MATCHED THEN INSERT ...
```
**CDC Handling (if enabled):**
```sql
MERGE INTO final_table AS target
USING deduped AS source
ON target.pk = source.pk
WHEN MATCHED AND source._ab_cdc_deleted_at IS NOT NULL THEN DELETE -- Hard delete
WHEN MATCHED AND source.cursor > target.cursor THEN UPDATE ...
WHEN NOT MATCHED AND source._ab_cdc_deleted_at IS NULL THEN INSERT ...
```
### State Management
**Critical Guarantee:** STATE only emitted after data persisted to database.
**Flow:**
```
RECORD messages → buffer
STATE message → flush buffers → database COMMIT → emit STATE
```
**Timing:**
- STATE boundaries = commit points
- InsertBuffer.flush() must complete before STATE emitted
- StreamLoader.close() finalization completes before final STATE
**Recovery on Failure:**
- Platform retries from last emitted STATE
- Records after last STATE are re-sent
- Destination must be idempotent (generation IDs help)
### Airbyte Metadata Columns
**Always included (managed by framework):**
| Column | Type | Purpose |
|--------|------|---------|
| `_airbyte_raw_id` | UUID/String | Unique record identifier |
| `_airbyte_extracted_at` | Timestamp | Extraction timestamp |
| `_airbyte_meta` | JSON | Errors, warnings, transformations |
| `_airbyte_generation_id` | Integer | Sync generation tracking |
**Filtered out** during schema discovery and computation - never in ColumnChangeset
---
## Data Transformation
### Type Conversion
**AirbyteValue → Database Format:**
```
Pipeline receives: JSON message
↓ Deserialize
AirbyteValue (StringValue, IntegerValue, etc.)
↓ Transform
Database-specific format (via ColumnUtils)
↓ Buffer
InsertBuffer accumulates
↓ Format
CSV, binary, JSON, etc.
↓ Write
Database client writes batch
```
**Example:**
```json
// Source
{"id": 123, "name": "Alice", "created_at": "2024-01-01T12:00:00Z"}
// After transformation (RecordDTO.fields)
{
"id": IntegerValue(123),
"name": StringValue("Alice"),
"created_at": TimestampValue("2024-01-01T12:00:00Z"),
"_airbyte_raw_id": StringValue("uuid..."),
"_airbyte_extracted_at": TimestampValue("2024-01-01T12:00:00Z"),
"_airbyte_meta": ObjectValue(...),
"_airbyte_generation_id": IntegerValue(42)
}
// InsertBuffer formats for database
CSV: "123,Alice,2024-01-01 12:00:00,uuid...,2024-01-01 12:00:00,{},42"
Binary: [0x7B, 0x00, ...] (database-specific format)
```
### Column Name Mapping
**Logical → Physical:**
```
Stream schema: {"field_name": StringType}
ColumnNameGenerator: "field_name" → "FIELD_NAME" (Snowflake)
→ "field_name" (ClickHouse)
TableCatalog stores: {"field_name": "FIELD_NAME"}
Your code queries: columnMapping["field_name"] → "FIELD_NAME"
```
**Use TableCatalog, don't implement manually**
---
## Error Handling
### Exception Types
| Exception | When to Use | Platform Action |
|-----------|-------------|-----------------|
| `ConfigErrorException` | User-fixable (bad creds, permissions, invalid config) | NO RETRY - notify user |
| `TransientErrorException` | Temporary (network timeout, DB unavailable) | RETRY with backoff |
| `SystemErrorException` | Internal errors, bugs | LIMITED RETRY - likely bug |
**Pattern:**
```kotlin
try {
connection.executeQuery(sql)
} catch (e: SQLException) {
when {
e.message?.contains("permission") == true ->
throw ConfigErrorException("Permission denied. Grant privileges.", e)
e.message?.contains("timeout") == true ->
throw TransientErrorException("Network timeout. Will retry.", e)
else ->
throw SystemErrorException("Unexpected SQL error", e)
}
}
```
### Graceful Degradation
**On Failure:**
```kotlin
StreamLoader.close(streamCompleted = false) {
// Skip finalization (no MERGE/SWAP)
// Drop temp tables (cleanup)
// Final table unchanged
}
```
**Result:**
- Final table untouched (atomicity preserved)
- Temp tables cleaned up
- Platform retries from last STATE checkpoint
---
## Integration Points
**Where framework calls your code:**
| Phase | Framework Calls | Your Code Executes |
|-------|----------------|-------------------|
| **Setup** | `Writer.setup()` | Create namespaces, gather initial state |
| **Stream Init** | `Writer.createStreamLoader()` | Select appropriate StreamLoader |
| | `StreamLoader.start()` | `createTable()`, `ensureSchemaMatches()` |
| | `AggregateFactory.create()` | Create InsertBuffer with target table |
| **Data Flow** | `Aggregate.accept()` | `InsertBuffer.accumulate()` |
| | `Aggregate.flush()` | `InsertBuffer.flush()` → write batch |
| **Finalize** | `StreamLoader.close()` | `upsertTable()` or `overwriteTable()` |
| | Always | `dropTable(tempTable)` |
**Key Insight:** Framework orchestrates when to call what. You implement the "what" (database operations), framework handles the "when" and "how".
---
## Common Questions
### Why separate SqlGenerator and Client?
**Separation of concerns:**
- SqlGenerator: Pure functions, testable, no side effects
- Client: I/O operations, error handling, resource management
**Benefits:**
- Test SQL generation without database
- Reuse SQL across multiple execution contexts
- Easier to debug (separate SQL bugs from execution bugs)
### When is schema evolution triggered?
**Automatically during `StreamLoader.start()` if:**
- Final table exists
- Stream schema has changed since last sync
**Never triggered if:**
- Table doesn't exist (fresh create)
- Append mode to temp table
- Schema unchanged
### What if my database doesn't support MERGE?
**Options:**
1. **Use temp table + window function + INSERT:**
```sql
-- Dedupe in temp
CREATE TABLE deduped AS SELECT * FROM (
SELECT *, ROW_NUMBER() OVER (PARTITION BY pk ORDER BY cursor DESC) AS rn
FROM temp
) WHERE rn = 1;
-- DELETE + INSERT
DELETE FROM final WHERE pk IN (SELECT pk FROM deduped);
INSERT INTO final SELECT * FROM deduped;
```
2. **Use UPDATE + INSERT (slower):**
```sql
-- Update existing
UPDATE final SET col1 = temp.col1, ... FROM temp WHERE final.pk = temp.pk;
-- Insert new
INSERT INTO final SELECT * FROM temp WHERE pk NOT IN (SELECT pk FROM final);
```
3. **Use database-specific upsert:**
```sql
-- Postgres: INSERT ... ON CONFLICT
INSERT INTO final SELECT * FROM temp
ON CONFLICT (pk) DO UPDATE SET col1 = EXCLUDED.col1, ...;
-- MySQL: INSERT ... ON DUPLICATE KEY UPDATE
INSERT INTO final SELECT * FROM temp
ON DUPLICATE KEY UPDATE col1 = VALUES(col1), ...;
```
### How do I test my implementation?
**Levels:**
1. **Unit tests:** SqlGenerator (no database needed)
2. **Component tests:** TableOperationsSuite (basic operations)
3. **Integration tests:** BasicFunctionalityIntegrationTest (full sync)
**BasicFunctionalityIntegrationTest provides:**
- Append mode test
- Dedupe mode test
- Overwrite mode test
- Schema evolution test
- CDC test
**Use Testcontainers** for reproducible, isolated tests
---
## Performance Considerations
### Batching
**Framework triggers flush at:**
- Time interval (default: 60s)
- Buffer size threshold (configurable)
- State message boundary
- End of stream
**Your InsertBuffer can add:**
- Record count threshold (e.g., 1000 records)
- Byte size threshold (e.g., 10MB)
- File size threshold (for staging)
### Parallelism
**Framework parallelizes:**
- Multiple streams (configurable: `num-open-stream-workers`)
- Task execution (setup, write, finalize)
**Your code should:**
- Be thread-safe in `@Singleton` components
- Use connection pooling (HikariCP)
- Avoid blocking operations in suspend functions
### Memory Management
**Framework provides:**
- Memory reservation system (backpressure)
- Configurable buffer limits
- Automatic pause/resume based on memory
**Your InsertBuffer should:**
- Write to temp files for large batches (not all in memory)
- Stream data to database (don't load entire batch at once)
- Clean up resources in `finally` blocks
---
## CDK Version Pinning
**All production connectors must pin to a specific CDK version:**
**File:** `destination-{db}/gradle.properties`
```properties
cdkVersion=0.1.76 # Pin to specific version
```
**Never use in production:**
```properties
cdkVersion=local # Only for CDK development
```
**The `airbyte-bulk-connector` plugin:**
- Reads `cdkVersion` from `gradle.properties`
- Resolves Maven artifacts: `io.airbyte.bulk-cdk:bulk-cdk-core-load:0.1.76`
- Or uses local project references if `cdkVersion=local`
**Verify pinning:**
```bash
./gradlew :destination-{db}:dependencies --configuration runtimeClasspath | grep bulk-cdk
```
**Expected:** `io.airbyte.bulk-cdk:bulk-cdk-core-load:0.1.76` (not `project :airbyte-cdk:bulk:...`)
**Upgrade CDK:**
```bash
./gradlew destination-{db}:upgradeCdk --cdkVersion=0.1.76
```
---
## Summary Checklist
**What you must provide:**
- [ ] CDK version pinned in `gradle.properties`
- [ ] SQL Generator with all operations
- [ ] Database Client implementing TableOperationsClient + TableSchemaEvolutionClient
- [ ] InsertBuffer with efficient batch writes
- [ ] Column Utilities for type mapping
- [ ] Configuration (spec, factory)
- [ ] Name Generators (table, column)
- [ ] Checker for connection validation
- [ ] Writer for orchestration
- [ ] Aggregate (3-line delegation)
- [ ] AggregateFactory (create buffer)
- [ ] BeanFactory (Micronaut DI)
**What the CDK provides:**
- [ ] 4 StreamLoader implementations (you just instantiate)
- [ ] Data pipeline (parse, transform, batch)
- [ ] State management (checkpointing)
- [ ] Error handling (classify, emit TRACE)
- [ ] TableCatalog (column name mapping)
- [ ] StreamStateStore (coordinate buffer ↔ loader)
**Result:**
- [ ] All sync modes work (append, dedupe, overwrite)
- [ ] Schema evolution automatic
- [ ] CDC support (if configured)
- [ ] State management automatic
- [ ] Error recovery automatic
**Effort:** ~1 week for experienced developer (4 core components + patterns + testing)

File diff suppressed because it is too large Load Diff

View File

@@ -0,0 +1,581 @@
# Preflight Checklist: Database Knowledge
**Summary:** Essential database knowledge required before implementing a destination connector. Complete this checklist to ensure you have the information needed for the 4 core components (SQL Generator, Database Client, Insert Buffer, Column Utilities).
**Time to complete:** 2-4 hours of research for unfamiliar database
---
## 1. Connection & Client Setup
**For SQL Generator + Database Client**
### Driver & Connection
- [ ] **Driver available?** JDBC, native client library, HTTP API?
- [ ] **Connection string format?** `jdbc:db://host:port/database` or custom URI?
- [ ] **Connection pooling?** HikariCP, built-in, or manual management?
**Quick validation:**
```kotlin
// Can you write this code?
val connection = /* create connection */
connection.execute("SELECT 1")
connection.close()
```
### Authentication
- [ ] **Auth methods supported?** Username/password, API key, OAuth, certificates?
- [ ] **How to configure?** Connection string params? Separate auth object?
- [ ] **Role/privileges required?** RBAC model? Default roles?
**Quick validation:**
```bash
# Can you connect with test credentials?
{database-cli} -h localhost -u testuser -p testpass -d testdb -c "SELECT 1"
```
### Test Environment
- [ ] **Testcontainers available?** Docker image exists? Version to use?
- [ ] **Or local setup?** Installation instructions? Port configuration?
- [ ] **Cleanup strategy?** Can drop/recreate test databases?
---
## 2. Namespace Management
**For SQL Generator: `createNamespace()`, `namespaceExists()`**
### Namespace Concept
- [ ] **Has namespaces?** Schema, database, catalog, or none?
- [ ] **Qualification?** `schema.table` or `database.table` or just `table`?
- [ ] **Implicit creation?** Created on first use or requires explicit CREATE?
### SQL Operations
- [ ] **Create syntax?** `CREATE SCHEMA`, `CREATE DATABASE`, or N/A?
- [ ] **Check existence?** Query `information_schema`, system catalog, or API?
- [ ] **Drop syntax?** `DROP SCHEMA CASCADE`? Restrictions?
**Quick validation:**
```sql
-- Can you write these queries?
CREATE SCHEMA test_schema;
SELECT schema_name FROM information_schema.schemata WHERE schema_name = 'test_schema';
DROP SCHEMA test_schema CASCADE;
```
---
## 3. Table Management
**For SQL Generator: `createTable()`, `dropTable()`, `tableExists()`, `countTable()`**
### Table Operations
- [ ] **Create syntax?** Column definitions, constraints, indexes?
- [ ] **Check existence?** Query system catalog? Try and catch error?
- [ ] **Drop syntax?** `DROP TABLE IF EXISTS`?
- [ ] **Count rows?** `SELECT COUNT(*)`? Performance considerations?
- [ ] **Introspect schema?** `DESCRIBE`, `information_schema.columns`, client API?
**Quick validation:**
```sql
-- Can you write these queries?
CREATE TABLE test_table (id BIGINT, name VARCHAR);
SELECT table_name FROM information_schema.tables WHERE table_name = 'test_table';
SELECT COUNT(*) FROM test_table;
SELECT column_name, data_type FROM information_schema.columns WHERE table_name = 'test_table';
DROP TABLE IF EXISTS test_table;
```
### Atomic Operations
- [ ] **Rename/swap tables?** `ALTER TABLE RENAME`? `SWAP WITH`? `EXCHANGE TABLES`?
- [ ] **Atomic swap available?** Or need DROP + RENAME pattern?
**Quick validation:**
```sql
-- Can you atomically swap tables?
CREATE TABLE old_table (id INT);
CREATE TABLE new_table (id INT);
ALTER TABLE old_table SWAP WITH new_table; -- Or your DB's syntax
```
---
## 4. Type System
**For Column Utilities: `toDialectType()`**
### Type Mapping Required
- [ ] **String type?** VARCHAR, TEXT, STRING, CLOB? Max length?
- [ ] **Integer type?** INT, BIGINT, INTEGER, NUMBER? Range?
- [ ] **Decimal type?** DECIMAL(p,s), NUMERIC, FLOAT, DOUBLE? Precision?
- [ ] **Boolean type?** BOOLEAN, TINYINT(1), BIT?
- [ ] **Date type?** DATE format? Storage (days, string, etc.)?
- [ ] **Time type?** TIME, supports timezone? Format?
- [ ] **Timestamp type?** TIMESTAMP, TIMESTAMPTZ, DATETIME? Timezone handling?
- [ ] **JSON type?** JSON, JSONB, VARIANT, or TEXT fallback?
- [ ] **Array type?** Native arrays or JSON encoding?
- [ ] **Binary type?** BLOB, BYTEA, BINARY, or Base64?
**Quick validation:**
```sql
-- Can you create table with all these types?
CREATE TABLE type_test (
str_col VARCHAR,
int_col BIGINT,
dec_col DECIMAL(38,9),
bool_col BOOLEAN,
date_col DATE,
time_col TIME WITH TIME ZONE,
ts_col TIMESTAMP WITH TIME ZONE,
json_col JSONB,
arr_col JSONB, -- or native array type
bin_col BYTEA
);
```
### Nullable Handling
- [ ] **Default nullable?** Or default NOT NULL?
- [ ] **Syntax?** `NULL`/`NOT NULL` suffix? `Nullable()` wrapper?
**Quick validation:**
```sql
CREATE TABLE nullable_test (
nullable_col VARCHAR,
not_null_col VARCHAR NOT NULL
);
```
### Airbyte Metadata Columns
- [ ] **UUID storage?** Native UUID type or VARCHAR(36)?
- [ ] **Timestamp storage?** Millisecond precision? Timezone?
- [ ] **JSON storage?** JSONB, JSON, VARIANT, or TEXT?
- [ ] **Integer storage?** For generation_id?
**Required types:**
```sql
CREATE TABLE airbyte_test (
_airbyte_raw_id VARCHAR NOT NULL, -- UUID as string
_airbyte_extracted_at TIMESTAMP NOT NULL, -- Extraction time
_airbyte_meta JSONB NOT NULL, -- Metadata
_airbyte_generation_id BIGINT -- Generation tracking
);
```
---
## 5. Batch Insert Strategy
**For Insert Buffer: `accumulate()`, `flush()`**
### Bulk Insert Methods
- [ ] **Multi-row INSERT?** `INSERT INTO t VALUES (...), (...)`? Row limit?
- [ ] **COPY/LOAD command?** `COPY FROM`, `LOAD DATA`, bulk API?
- [ ] **Staging support?** File upload, external stages, cloud storage?
- [ ] **Binary format?** Native binary protocol for faster inserts?
### Performance Considerations
- [ ] **Optimal batch size?** 100 rows? 1000? 10000?
- [ ] **Compression support?** GZIP, LZ4, Snappy?
- [ ] **Transaction size limits?** Max statement size? Max transaction duration?
**Quick validation:**
```sql
-- Can you insert multiple rows at once?
INSERT INTO test_table (id, name) VALUES
(1, 'Alice'),
(2, 'Bob'),
(3, 'Charlie');
-- Or does your DB prefer COPY/bulk API?
COPY test_table FROM '/path/to/data.csv' WITH (FORMAT CSV);
```
**Decision needed:**
- Simple multi-row INSERT (easy, decent performance)
- CSV staging + COPY (faster for large datasets)
- Binary protocol (fastest, more complex)
- Native bulk API (database-specific)
---
## 6. Deduplication & Upsert
**For SQL Generator: `upsertTable()`**
### Upsert Mechanism
- [ ] **MERGE statement?** Full support with DELETE, UPDATE, INSERT clauses?
- [ ] **INSERT ON CONFLICT?** PostgreSQL-style upsert?
- [ ] **REPLACE INTO?** MySQL-style replace?
- [ ] **None?** Need temp table + window function + DELETE + INSERT approach?
**Quick validation:**
```sql
-- Test 1: Can you upsert with MERGE?
MERGE INTO target USING source ON target.pk = source.pk
WHEN MATCHED THEN UPDATE SET ...
WHEN NOT MATCHED THEN INSERT ...;
-- Test 2: Or INSERT ON CONFLICT?
INSERT INTO target VALUES (...)
ON CONFLICT (pk) DO UPDATE SET ...;
-- Test 3: Or manual approach?
DELETE FROM target WHERE pk IN (SELECT pk FROM source);
INSERT INTO target SELECT * FROM source;
```
### Window Functions
- [ ] **ROW_NUMBER() OVER?** For deduplication?
- [ ] **PARTITION BY?** For grouping by primary key?
- [ ] **ORDER BY in window?** For selecting latest record?
**Quick validation:**
```sql
-- Can you deduplicate with window function?
SELECT * FROM (
SELECT *, ROW_NUMBER() OVER (
PARTITION BY pk
ORDER BY updated_at DESC
) AS rn
FROM table
) WHERE rn = 1;
```
---
## 7. Schema Evolution
**For SQL Generator: `alterTable()`, Client: `discoverSchema()`, `computeSchema()`**
### ALTER TABLE Support
- [ ] **ADD COLUMN?** Syntax? Can add multiple at once?
- [ ] **DROP COLUMN?** Syntax? Restrictions?
- [ ] **MODIFY/ALTER COLUMN TYPE?** Direct type change? Requires USING clause?
- [ ] **RENAME COLUMN?** Supported?
- [ ] **ALTER CONSTRAINT?** ADD/DROP NOT NULL?
**Quick validation:**
```sql
CREATE TABLE schema_test (id INT);
-- Add column
ALTER TABLE schema_test ADD COLUMN name VARCHAR;
-- Drop column
ALTER TABLE schema_test DROP COLUMN name;
-- Change type
ALTER TABLE schema_test ALTER COLUMN id TYPE BIGINT;
-- Change nullable
ALTER TABLE schema_test ALTER COLUMN id DROP NOT NULL;
```
### Schema Introspection
- [ ] **System catalog?** `information_schema.columns`? `DESCRIBE TABLE`? Client API?
- [ ] **Returns?** Column name, type, nullable, default, precision?
**Quick validation:**
```sql
-- Can you introspect schema?
SELECT column_name, data_type, is_nullable
FROM information_schema.columns
WHERE table_name = 'schema_test';
```
### Type Changes
- [ ] **Safe widening?** INT → BIGINT? VARCHAR(50) → VARCHAR(100)?
- [ ] **Unsafe narrowing?** Requires temp column approach? Or table recreation?
- [ ] **Incompatible changes?** VARCHAR → INT? How to handle?
---
## 8. Identifier Rules
**For SQL Generator: All methods**
### Case Sensitivity
- [ ] **Default case?** Lowercase, uppercase, or preserve?
- [ ] **Quoted behavior?** Preserves case? Case-sensitive comparison?
- [ ] **Quoting syntax?** Double quotes, backticks, square brackets?
**Quick validation:**
```sql
-- What happens here?
CREATE TABLE MyTable (MyColumn INT);
SELECT * FROM mytable; -- Does this work?
SELECT * FROM "MyTable"; -- Does this preserve case?
```
### Reserved Keywords
- [ ] **How to use reserved words?** Must quote? Error if unquoted?
- [ ] **Common reserved keywords?** SELECT, ORDER, USER, TABLE, etc.?
**Quick validation:**
```sql
-- Can you use reserved keywords with quoting?
CREATE TABLE "order" ("user" VARCHAR);
```
### Special Characters
- [ ] **Allowed in identifiers?** Underscores? Hyphens? Unicode?
- [ ] **Max identifier length?** 63 chars (Postgres)? 255 (Snowflake)?
---
## 9. Error Handling
**For Database Client: Error classification**
### Error Codes
- [ ] **SQL State codes?** (e.g., 23505 = unique violation)
- [ ] **Error code system?** Numeric codes? String codes?
- [ ] **Exception hierarchy?** SQLException? Database-specific exceptions?
**Quick validation:**
```kotlin
// Can you classify errors?
try {
connection.execute("INSERT INTO t VALUES (1, 1)") // Duplicate PK
connection.execute("INSERT INTO t VALUES (1, 2)") // Duplicate PK
} catch (e: SQLException) {
println("SQL State: ${e.sqlState}") // 23505?
println("Error code: ${e.errorCode}")
println("Message: ${e.message}")
}
```
### Permission Errors
- [ ] **How to detect?** Error message keywords? SQL State?
- [ ] **Common patterns?** "permission denied", "insufficient privileges"?
---
## 10. Testing Setup
**For all phases**
### Test Environment
- [ ] **Testcontainers image?** `{database}:latest`? Specific version?
- [ ] **Or local install?** Installation steps? Default config?
- [ ] **Port?** Default port number? Configurable?
### Test Operations
- [ ] **Insert test data?** Prepared statements? Bulk API?
- [ ] **Read data back?** ResultSet iteration? Native API?
- [ ] **Type conversion?** AirbyteValue → DB types in tests?
---
## Quick Database Survey
**Answer these to assess readiness:**
| Question | Your Answer | Needed For |
|----------|-------------|------------|
| JDBC driver or native client? | | Phase 1 |
| Connection string format? | | Phase 1 |
| CREATE SCHEMA or CREATE DATABASE? | | Phase 2 |
| CREATE TABLE syntax? | | Phase 3 |
| String type (VARCHAR/TEXT/STRING)? | | Phase 3 |
| Integer type (INT/BIGINT/NUMBER)? | | Phase 3 |
| JSON type (JSONB/JSON/VARIANT)? | | Phase 3 |
| Timestamp type with timezone? | | Phase 3 |
| Has MERGE or INSERT ON CONFLICT? | | Phase 9 |
| Has window functions (ROW_NUMBER)? | | Phase 9 |
| Has ALTER TABLE ADD/DROP COLUMN? | | Phase 8 |
| System catalog for introspection? | | Phase 8 |
| Atomic table swap/exchange? | | Phase 6 |
| Optimal batch size for inserts? | | Phase 5 |
---
## Research Template
**Use this to document your findings:**
```markdown
# {Database} Connector Preflight Research
## 1. Connection & Client
- **Driver:** [JDBC / Native / HTTP API]
- **Maven coordinates:** `group:artifact:version`
- **Connection string:** `protocol://host:port/database?options`
- **Auth method:** [username/password / API key / other]
- **Connection pooling:** [HikariCP / built-in / manual]
## 2. Namespaces
- **Concept:** [Schema / Database / None]
- **Create:** `CREATE SCHEMA name` or `CREATE DATABASE name`
- **Check exists:** `SELECT ... FROM information_schema.schemata`
- **Drop:** `DROP SCHEMA name CASCADE`
## 3. Tables
- **Create:** `CREATE TABLE schema.table (cols...)`
- **Check exists:** `SELECT ... FROM information_schema.tables`
- **Drop:** `DROP TABLE IF EXISTS schema.table`
- **Describe:** `DESCRIBE TABLE` or `SELECT ... FROM information_schema.columns`
- **Swap:** `ALTER TABLE ... SWAP WITH ...` or recreation needed
## 4. Type Mapping
| Airbyte Type | Database Type | Notes |
|--------------|---------------|-------|
| String | VARCHAR / TEXT | Length limit? |
| Integer | BIGINT | Range? |
| Number | DECIMAL(38,9) | Precision? |
| Boolean | BOOLEAN | Or TINYINT? |
| Date | DATE | Format? |
| Timestamp+TZ | TIMESTAMPTZ | Precision? |
| JSON | JSONB | Or JSON / TEXT? |
| Array | JSONB | Or native array? |
## 5. Bulk Insert
- **Best method:** [Multi-row INSERT / COPY / Staging / Bulk API]
- **Batch size:** [1000 / 10000 / custom]
- **Compression:** [GZIP / LZ4 / None]
- **Example:** `COPY table FROM file WITH (...)`
## 6. Upsert
- **Method:** [MERGE / INSERT ON CONFLICT / REPLACE / Manual DELETE+INSERT]
- **Syntax:** `MERGE INTO ... USING ... ON ... WHEN MATCHED ...`
- **Window functions:** [ROW_NUMBER OVER supported? YES/NO]
## 7. Schema Evolution
- **ADD COLUMN:** `ALTER TABLE t ADD COLUMN c type`
- **DROP COLUMN:** `ALTER TABLE t DROP COLUMN c`
- **CHANGE TYPE:** `ALTER TABLE t ALTER COLUMN c TYPE newtype` or temp column approach
- **Discover schema:** `SELECT ... FROM information_schema.columns`
## 8. Identifiers
- **Case:** [Lowercase / Uppercase / Preserve]
- **Quoting:** [" / ` / [] ]
- **Reserved keywords:** [List common ones]
- **Max length:** [63 / 255 / other]
## 9. Errors
- **SQL State codes:** [23505 = unique violation, etc.]
- **Exception type:** [SQLException / DatabaseException / custom]
- **Permission errors:** [Pattern in error message]
## 10. Testing
- **Testcontainers:** `{database}Container("{db}:latest")`
- **Or local:** [Installation command]
- **Default port:** [5432 / 3306 / etc.]
```
---
## Validation Questions
**Before starting Phase 1, can you answer YES to all?**
### Critical (Must Have)
- [ ] I can establish a connection programmatically
- [ ] I know how to execute a simple query
- [ ] I know the CREATE TABLE syntax
- [ ] I know how to map basic types (String, Integer, Boolean, Timestamp)
- [ ] I can insert records programmatically
- [ ] I can read records back for verification
- [ ] I have a test environment (Testcontainers or local)
### Important (Needed for Full Features)
- [ ] I know how to create/drop namespaces (schemas/databases)
- [ ] I know how to check if table/namespace exists
- [ ] I know the system catalog for schema introspection
- [ ] I know how to handle JSON/JSONB data
- [ ] I know at least one bulk insert method
- [ ] I know how to swap/rename tables atomically (or workaround)
### Nice to Have (Can Research During Implementation)
- [ ] I know the optimal batch size for bulk inserts
- [ ] I know how to handle all edge case types (arrays, binary, etc.)
- [ ] I know database-specific optimization tricks
- [ ] I know all error codes for better error messages
---
## Red Flags (May Need Alternative Approach)
**If you answer YES to any, plan workarounds:**
- [ ] **No MERGE or INSERT ON CONFLICT?** → Use temp table + window function + DELETE + INSERT
- [ ] **No ALTER TABLE?** → Use table recreation for schema changes
- [ ] **No window functions?** → Dedupe via application logic (slower)
- [ ] **No atomic swap?** → Use temp table + DROP + RENAME (brief inconsistency window)
- [ ] **No JSON type?** → Use TEXT with JSON string encoding
- [ ] **No schemas/namespaces?** → Use table name prefixes
---
## Time Estimates by Familiarity
| Database Familiarity | Research Time | Total Implementation |
|---------------------|---------------|---------------------|
| **Expert** (daily use) | 30 min | 3-4 days |
| **Familiar** (used before) | 2-3 hours | 4-5 days |
| **New** (never used) | 4-8 hours | 5-7 days |
| **Exotic** (limited docs) | 8-16 hours | 7-10 days |
**Recommendation:** Spend the research time upfront. It pays off during implementation.
---
## Helpful Resources
### Documentation to Review
- [ ] Database SQL reference (CREATE, ALTER, DROP, INSERT)
- [ ] JDBC driver documentation (if applicable)
- [ ] Native client SDK documentation
- [ ] Type system reference
- [ ] System catalog documentation (information_schema or equivalent)
- [ ] Transaction and isolation level documentation
### Code References
- [ ] Existing connector for similar database (Postgres → MySQL, Snowflake → Databricks)
- [ ] Database driver examples and sample code
- [ ] Testcontainers examples
### Testing
- [ ] Set up local database instance or Testcontainers
- [ ] Connect via CLI and test all operations manually
- [ ] Write a simple JDBC/client test program
---
## Next Steps After Completing Checklist
**If you can answer all Critical questions:**
**Proceed to Phase 0 (Scaffolding)** in step-by-step-guide.md
**If you're missing Important knowledge:**
**Research those areas first** - they're needed by Phase 6-9
**If you're missing Nice to Have knowledge:**
**Start implementation anyway** - research these as needed during development
**If you hit Red Flags:**
**Review implementation-reference.md** for alternative approaches
**Consider asking in Airbyte community** if workarounds exist
---
## Checklist Summary
**Complete these research tasks (2-4 hours):**
1. ✅ Set up test environment (Testcontainers or local)
2. ✅ Test connection via CLI and code
3. ✅ Write CREATE/DROP operations for namespace and table
4. ✅ Map all Airbyte types to database types
5. ✅ Test bulk insert (multi-row or COPY or API)
6. ✅ Test upsert mechanism (MERGE or INSERT ON CONFLICT or workaround)
7. ✅ Test schema introspection (system catalog queries)
8. ✅ Test ALTER TABLE operations
9. ✅ Document identifier rules (case, quoting, keywords)
10. ✅ Document error codes and exception types
**Output:** Research document with all findings
**Then:** Proceed to step-by-step-guide.md Phase 0
**Estimated effort to implement:** 3-7 days depending on database familiarity

View File

@@ -0,0 +1,252 @@
# Step-by-Step Guide: Building a Destination Connector
**Summary:** Complete guide to building a Dataflow CDK destination connector from scratch. Follow the numbered guides in sequence to build incrementally with clear milestones and quick feedback loops.
---
## Prerequisites
- Familiarity with Kotlin and your target database
- Understanding of [dataflow-cdk.md](../dataflow-cdk.md) (architecture overview)
- Understanding of [implementation-reference.md](../implementation-reference.md) (component reference)
- Database credentials or Testcontainers setup
---
## Development Flow
### Path 1: Fast Path (Working Connector)
**Goal:** Get a working connector with basic sync modes as quickly as possible
**Timeline:** 2-3 days
**Steps:**
1. **1-getting-started.md** (Phases 0-1, ~4 hours)
- Project scaffolding and build setup
- Spec operation implementation
- **Milestone:** `./destination-{db} --spec` works
2. **2-database-setup.md** (Phases 2-5, ~6 hours)
- Database connectivity and basic operations
- Namespace and table operations
- Check operation implementation
- **Milestone:** `./destination-{db} --check --config config.json` works
3. **3-write-infrastructure.md** (Phases 6-7, ~4 hours)
- Name generators and DI setup
- Write operation infrastructure
- Understanding test contexts
- **Milestone:** DI configured, ready for business logic
4. **4-write-operations.md** (Phases 8-11, ~8 hours)
- InsertBuffer, Aggregate, Writer implementation
- Append mode (direct writes)
- Generation ID support
- Overwrite mode (atomic swap)
- **Milestone:** `./destination-{db} --write` works with append + overwrite modes
**Result:** Working connector suitable for PoC and simple use cases
---
### Path 2: Production Path (Full-Featured Connector)
**Goal:** Production-ready connector with all enterprise features
**Timeline:** 5-7 days
**Steps:**
1-4. Complete Fast Path (above)
5. **5-advanced-features.md** (Phases 12-15, ~12 hours)
- Schema evolution (automatic column add/drop/modify)
- Dedupe mode (MERGE with primary key)
- CDC support (hard/soft deletes)
- Optimization and polish
- **Milestone:** Full-featured, production-ready connector
6. **6-testing.md** (~2 hours)
- Run BasicFunctionalityIntegrationTest
- Validate all sync modes
- Test schema evolution and CDC
- **Milestone:** All tests passing
**Result:** Production-ready connector with all features
---
### Path 3: Debug Path (Troubleshooting)
**Goal:** Fix issues quickly
**When:** Encountering errors during development
**Steps:**
1. Check **7-troubleshooting.md** for common errors
- Test context confusion
- Dependency injection errors
- Quick fixes and patterns
2. Return to phase guide with solution
**Result:** Unblocked and back to development
---
## Milestone Summary
| Guide | Phases | What Works | Lines | Time | Prerequisites |
|-------|--------|------------|-------|------|---------------|
| **1-getting-started.md** | 0-1 | --spec | ~626 | 4h | None |
| **2-database-setup.md** | 2-5 | --check | ~902 | 6h | Guide 1 |
| **3-write-infrastructure.md** | 6-7 | DI ready | ~855 | 4h | Guide 2 |
| **4-write-operations.md** | 8-11 | --write (append, overwrite) | ~869 | 8h | Guide 3 |
| **5-advanced-features.md** | 12-15 | All features | ~1020 | 12h | Guide 4 |
| **6-testing.md** | Tests | All tests pass | ~878 | 2h | Guide 5 |
| **7-troubleshooting.md** | Reference | Debug help | ~280 | As needed | Any |
---
## What You'll Build
### After Guide 1 (Getting Started)
- ✅ Project compiles and builds
- ✅ Docker image builds
-`--spec` operation returns connector capabilities
### After Guide 2 (Database Setup)
- ✅ Database connection established
- ✅ Namespace (schema/database) creation
- ✅ Table creation, drop, count operations
-`--check` operation validates configuration
### After Guide 3 (Write Infrastructure)
- ✅ Name generators (table, column, temp table)
- ✅ TableCatalog DI setup
- ✅ Write operation entry point
- ✅ Understanding of test contexts (critical!)
### After Guide 4 (Write Operations)
- ✅ InsertBuffer with efficient batch writes
- ✅ Aggregate and AggregateFactory
- ✅ Writer orchestration
- ✅ Append mode (direct insert)
- ✅ Overwrite mode (temp table + atomic swap)
- ✅ Generation ID tracking
-`--write` operation works for basic syncs
### After Guide 5 (Advanced Features)
- ✅ Schema evolution (automatic schema changes)
- ✅ Dedupe mode (MERGE with PK)
- ✅ CDC support (hard/soft deletes)
- ✅ Performance optimization
- ✅ Production-ready connector
### After Guide 6 (Testing)
- ✅ All integration tests passing
- ✅ All sync modes validated
- ✅ Schema evolution tested
- ✅ Ready for deployment
---
## Key Concepts Per Guide
### Guide 1: Getting Started
- CDK version pinning
- Micronaut DI basics
- Specification and configuration classes
- JSON schema generation
### Guide 2: Database Setup
- SqlGenerator pattern (SQL generation separate from execution)
- TableOperationsClient interface
- Testcontainers for local testing
- Component vs integration tests
### Guide 3: Write Infrastructure
- Name generators and column mapping
- StreamStateStore pattern
- Test contexts (component vs integration vs basic functionality)
- Common DI errors and fixes
### Guide 4: Write Operations
- InsertBuffer pattern (database-specific)
- StreamLoader variants (4 types)
- Writer.createStreamLoader() decision logic
- Temp table + atomic swap strategy
### Guide 5: Advanced Features
- Schema evolution (discover → compute → compare → apply)
- MERGE/UPSERT implementation
- Window functions for deduplication
- CDC handling (hard vs soft delete)
### Guide 6: Testing
- BasicFunctionalityIntegrationTest structure
- Testing all sync modes
- Validating schema evolution
- End-to-end validation
### Guide 7: Troubleshooting
- Test context confusion
- Micronaut DI errors
- Quick reference fixes
---
## Architecture Recap
Before starting, understand these key patterns:
**Component Roles:**
- **SqlGenerator:** Generates SQL (pure functions, testable)
- **Client:** Executes SQL (I/O, error handling)
- **InsertBuffer:** Efficient batch writes (database-specific)
- **StreamLoader:** Orchestrates table lifecycle (CDK-provided, you select)
- **Writer:** High-level orchestration (you implement, minimal logic)
**Data Flow:**
```
Platform → stdin → Lifecycle → Writer.setup()
→ createStreamLoader()
→ AggregateFactory.create()
→ InsertBuffer
→ Database
→ StreamLoader.close()
→ STATE → stdout → Platform
```
**Testing Strategy:**
- **Component tests:** Test individual operations (create table, insert, etc.)
- **Integration tests:** Test write initialization and lifecycle
- **Basic functionality tests:** End-to-end validation of all features
---
## Getting Help
**During development:**
- Check 7-troubleshooting.md first
- Review dataflow-cdk.md for architecture questions
- Review implementation-reference.md for API details
- Look at destination-snowflake or destination-clickhouse for examples
**Common pitfalls:**
- Not reading test contexts section (causes confusion in Phase 7)
- Missing DI registration (causes "No bean found" errors)
- Skipping CDK version pinning (causes build issues)
- Not understanding StreamLoader variants (causes wrong finalization)
---
## Next Steps
**Start with:** [1-getting-started.md](./1-getting-started.md)
**References:**
- [Architecture Overview](../dataflow-cdk.md)
- [Implementation Reference](../implementation-reference.md)
- [Coding Standards](../coding-standards.md)
- [Preflight Checklist](../preflight-checklist.md)

View File

@@ -0,0 +1,614 @@
# Getting Started: Project Setup and Spec Operation
**Prerequisites:** None (start here)
**Summary:** Paint-by-numbers guide to implementing a destination connector. 14 phases (0-13) with clear tasks, code patterns, and test validation. Build incrementally with quick feedback loops. After Phase 1, --spec works. After Phase 5, --check works. After Phase 7, you have a working append-only connector. Full feature set by Phase 11.
**Prerequisites:**
- Familiarity with Kotlin and your target database
- Understanding of dataflow-cdk.md (architecture overview)
- Database credentials or Testcontainers setup
---
## Phase 0: Scaffolding
**Goal:** Empty project structure that builds
**Checkpoint:** Project compiles
### Step 0.1: Create Directory Structure
```bash
cd airbyte-integrations/connectors
mkdir -p destination-{db}/src/main/kotlin/io/airbyte/integrations/destination/{db}
mkdir -p destination-{db}/src/test/kotlin/io/airbyte/integrations/destination/{db}
mkdir -p destination-{db}/src/test-integration/kotlin/io/airbyte/integrations/destination/{db}
```
**Create subdirectories:**
```bash
cd destination-{db}/src/main/kotlin/io/airbyte/integrations/destination/{db}
mkdir check client config dataflow spec write
mkdir write/load write/transform
```
### Step 0.2: Create gradle.properties with CDK Version Pin
**File:** `destination-{db}/gradle.properties`
```properties
# Pin to latest stable Bulk CDK version
# Check airbyte-cdk/bulk/version.properties for latest
cdkVersion=0.1.76
```
**IMPORTANT:** Always use a pinned version for production connectors.
**When to use `cdkVersion=local`:**
- Only when actively developing CDK features
- For faster iteration when modifying CDK code
- Switch back to pinned version before merging
**To upgrade CDK version later:**
```bash
./gradlew destination-{db}:upgradeCdk --cdkVersion=0.1.76
```
### Step 0.3: Create build.gradle.kts
**File:** `destination-{db}/build.gradle.kts`
**Reference:** `destination-snowflake/build.gradle.kts` or `destination-clickhouse/build.gradle.kts`
```kotlin
plugins {
id("airbyte-bulk-connector")
}
airbyteBulkConnector {
core = "load" // For destinations
toolkits = listOf("load-db") // Database toolkit
}
dependencies {
// Database driver
implementation("your.database:driver:version")
// Add other specific dependencies as needed
}
```
**How it works:**
- The `airbyte-bulk-connector` plugin reads `cdkVersion` from `gradle.properties`
- If `cdkVersion=0.1.76`: Resolves Maven artifacts `io.airbyte.bulk-cdk:bulk-cdk-core-load:0.1.76`
- If `cdkVersion=local`: Uses project references `:airbyte-cdk:bulk:core:load`
- Automatically adds CDK dependencies, Micronaut, test fixtures
**No need to manually declare CDK dependencies** - the plugin handles it
### Step 0.4: Create metadata.yaml
**File:** `destination-{db}/metadata.yaml`
```yaml
data:
connectorType: destination
connectorSubtype: database
dockerImageTag: 0.1.0
dockerRepository: airbyte/destination-{db}
documentationUrl: https://docs.airbyte.com/integrations/destinations/{db}
githubIssueLabel: destination-{db}
icon: {db}.svg # Add icon file to src/main/resources
license: ELv2
name: {Database Name}
connectorBuildOptions:
# Use latest Java connector base image
# Find latest at: https://hub.docker.com/r/airbyte/java-connector-base/tags
baseImage: docker.io/airbyte/java-connector-base:2.0.3@sha256:119b8506bca069bbc8357a275936c7e2b0994e6947b81f1bf8d6ce9e16db7d47
connectorIPCOptions:
dataChannel:
version: "0.0.2"
supportedSerialization: ["JSONL", "PROTOBUF"]
supportedTransport: ["SOCKET", "STDIO"]
registryOverrides:
oss:
enabled: true
cloud:
enabled: false # Set true when ready for Airbyte Cloud
releaseStage: alpha # alpha → beta → generally_available
supportLevel: community
tags:
- language:java
connectorTestSuitesOptions:
- suite: unitTests
- suite: integrationTests
metadataSpecVersion: "1.0"
```
**Key fields:**
- `dockerRepository`: Full image name (e.g., `airbyte/destination-{db}`)
- `dockerImageTag`: Version (start with `0.1.0`)
- `baseImage`: Java connector base image (with digest for reproducibility)
- `releaseStage`: Start with `alpha`, promote to `beta``generally_available`
**To find latest base image:**
```bash
# Check what other connectors use
grep "baseImage:" airbyte-integrations/connectors/destination-*/metadata.yaml | sort | uniq -c | sort -rn | head -3
```
### Step 0.5: Configure Docker Build in build.gradle.kts
**File:** Update `destination-{db}/build.gradle.kts`
```kotlin
plugins {
id("application")
id("airbyte-bulk-connector")
id("io.airbyte.gradle.docker") // Docker build support
id("airbyte-connector-docker-convention") // Reads metadata.yaml
}
airbyteBulkConnector {
core = "load"
toolkits = listOf("load-db")
}
application {
mainClass = "io.airbyte.integrations.destination.{db}.{DB}DestinationKt"
applicationDefaultJvmArgs = listOf(
"-XX:+ExitOnOutOfMemoryError",
"-XX:MaxRAMPercentage=75.0"
)
}
dependencies {
// Database driver
implementation("your.database:driver:version")
}
```
**What the plugins do:**
- `io.airbyte.gradle.docker`: Provides Docker build tasks
- `airbyte-connector-docker-convention`: Reads metadata.yaml, generates build args
### Step 0.6: Create Main Entry Point
**File:** `destination-{db}/src/main/kotlin/.../{DB}Destination.kt`
```kotlin
package io.airbyte.integrations.destination.{db}
import io.airbyte.cdk.AirbyteDestinationRunner
fun main(args: Array<String>) {
AirbyteDestinationRunner.run(*args)
}
```
**That's it!** The framework handles everything else.
### Step 0.7: Verify Build
```bash
$ ./gradlew :destination-{db}:build
```
**Expected:** Build succeeds
**Troubleshooting:**
- Missing dependencies? Check `build.gradle.kts`
- Package name mismatches? Verify all files use consistent package
- Micronaut scanning issues? Ensure `@Singleton` annotations present
- metadata.yaml syntax errors? Validate YAML format
### Step 0.8: Create application-connector.yml
**File:** `src/main/resources/application-connector.yml`
```yaml
# This file is loaded by the connector at runtime (in Docker)
# The platform may override these via environment variables
airbyte:
destination:
core:
# Default type handling
types:
unions: DEFAULT
# Data channel configuration (required)
data-channel:
medium: STDIO # STDIO or SOCKET (platform sets this)
format: JSONL # JSONL or PROTOBUF
# Namespace mapping (required)
mappers:
namespace-mapping-config-path: "" # Empty = no custom mapping (identity)
# File transfer (required)
file-transfer:
enabled: false # true for cloud storage destinations, false for databases
# Reduce noise in logs
logger:
levels:
com.zaxxer.hikari: ERROR
com.zaxxer.hikari.pool: ERROR
```
**Critical:** Without this file, the connector will crash with DI errors:
```
Failed to inject value for parameter [dataChannelMedium]
Failed to inject value for parameter [namespaceMappingConfigPath]
Failed to inject value for parameter [fileTransferEnabled]
```
**All required properties:**
-`types.unions`: How to handle union types
-`data-channel.medium`: STDIO or SOCKET
-`data-channel.format`: JSONL or PROTOBUF
-`mappers.namespace-mapping-config-path`: Namespace mapping file path (empty for identity)
-`file-transfer.enabled`: Whether connector transfers files (false for databases)
### Step 0.9: Build Docker Image
```bash
$ ./gradlew :destination-{db}:assemble
```
**What this does:**
1. Compiles code
2. Runs unit tests
3. Creates distribution TAR
4. Builds Docker image (includes application-connector.yml)
**Expected output:**
```
BUILD SUCCESSFUL
...
> Task :airbyte-integrations:connectors:destination-{db}:dockerBuildx
Building image: airbyte/destination-{db}:0.1.0
```
**Verify image was created:**
```bash
$ docker images | grep destination-{db}
```
**Expected:**
```
airbyte/destination-{db} 0.1.0 abc123def456 2 minutes ago 500MB
```
**Checkpoint:** Project compiles and Docker image builds successfully
---
## Phase 1: Spec Operation
**Goal:** Implement --spec operation (returns connector configuration schema)
**Checkpoint:** Spec test passes
### Step 1.1: Understand Configuration Classes
**Two classes work together for configuration:**
| Class | Purpose | Used By |
|-------|---------|---------|
| `{DB}Specification` | Defines UI form schema (what users see) | Spec operation (generates JSON schema) |
| `{DB}Configuration` | Runtime config object (what your code uses) | Check and Write operations |
**Flow:**
```
User fills UI form
Platform sends JSON matching Specification schema
ConfigurationFactory parses JSON → Configuration object
Your code uses Configuration object
```
### Step 1.2: Create Specification Class
**Purpose:** Defines the configuration form users fill in Airbyte UI
**File:** `spec/{DB}Specification.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.spec
import com.fasterxml.jackson.annotation.JsonProperty
import com.fasterxml.jackson.annotation.JsonPropertyDescription
import io.airbyte.cdk.command.ConfigurationSpecification
import io.micronaut.context.annotation.Singleton
@Singleton
open class {DB}Specification : ConfigurationSpecification() {
@get:JsonProperty("hostname")
@get:JsonPropertyDescription("Hostname of the database server")
val hostname: String = ""
@get:JsonProperty("port")
@get:JsonPropertyDescription("Port of the database server")
val port: Int = 5432 // Your DB's default port
@get:JsonProperty("database")
@get:JsonPropertyDescription("Name of the database")
val database: String = ""
@get:JsonProperty("username")
@get:JsonPropertyDescription("Username for authentication")
val username: String = ""
@get:JsonProperty("password")
@get:JsonPropertyDescription("Password for authentication")
val password: String = ""
}
```
**Key annotations:**
- `@JsonProperty("field_name")` - Field name in JSON
- `@JsonPropertyDescription("...")` - Help text in UI
- `@JsonSchemaTitle("Title")` - Label in UI (optional, defaults to property name)
- `@JsonSchemaInject(json = """{"airbyte_secret": true}""")` - Mark as secret (passwords, API keys)
### Step 1.3: Create Configuration and Factory
**Purpose:** Runtime configuration object your code actually uses
**File:** `spec/{DB}Configuration.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.spec
import io.airbyte.cdk.load.command.DestinationConfiguration
import io.airbyte.cdk.load.command.DestinationConfigurationFactory
import io.micronaut.context.annotation.Singleton
// Runtime configuration (used by your code)
data class {DB}Configuration(
val hostname: String,
val port: Int,
val database: String,
val username: String,
val password: String,
) : DestinationConfiguration()
// Factory: Converts Specification → Configuration
@Singleton
class {DB}ConfigurationFactory :
DestinationConfigurationFactory<{DB}Specification, {DB}Configuration> {
override fun makeWithoutExceptionHandling(
pojo: {DB}Specification
): {DB}Configuration {
return {DB}Configuration(
hostname = pojo.hostname,
port = pojo.port,
database = pojo.database,
username = pojo.username,
password = pojo.password,
)
}
}
```
**Why two classes?**
- **Specification:** JSON schema annotations, defaults, UI metadata
- **Configuration:** Clean runtime object, validated values, no Jackson overhead
- **Factory:** Validation and transformation layer between them
**Simple rule:**
- Specification = What users configure
- Configuration = What your code uses
### Step 1.4: Create Specification Extension
**Purpose:** Declares what sync modes your connector supports
**File:** `spec/{DB}SpecificationExtension.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.spec
import io.airbyte.cdk.load.spec.DestinationSpecificationExtension
import io.airbyte.protocol.models.v0.DestinationSyncMode
import io.micronaut.context.annotation.Singleton
@Singleton
class {DB}SpecificationExtension : DestinationSpecificationExtension {
override val supportedSyncModes =
listOf(
DestinationSyncMode.OVERWRITE,
DestinationSyncMode.APPEND,
DestinationSyncMode.APPEND_DEDUP,
)
override val supportsIncremental = true
// Optional: Group configuration fields in UI
override val groups =
listOf(
DestinationSpecificationExtension.Group("connection", "Connection"),
DestinationSpecificationExtension.Group("advanced", "Advanced"),
)
}
```
### Step 1.5: Configure Documentation URL
**File:** `src/main/resources/application.yml`
```yaml
airbyte:
connector:
metadata:
documentation-url: 'https://docs.airbyte.com/integrations/destinations/{db}'
destination:
core:
data-channel:
medium: STDIO # Default for local testing (platform sets this at runtime)
```
**Or in build.gradle.kts (alternative):**
```kotlin
airbyteBulkConnector {
core = "load"
toolkits = listOf("load-db")
// Optional: override documentation URL
// documentationUrl = "https://docs.airbyte.com/integrations/destinations/{db}"
}
```
**Default:** If not specified, uses placeholder URL
### Step 1.6: Create Expected Spec Test File
**File:** `src/test-integration/resources/expected-spec-oss.json`
```json
{
"documentationUrl": "https://docs.airbyte.com/integrations/destinations/{db}",
"connectionSpecification": {
"$schema": "http://json-schema.org/draft-07/schema#",
"title": "{DB} Destination Spec",
"type": "object",
"required": [
"hostname",
"port",
"database",
"username",
"password"
],
"properties": {
"hostname": {
"type": "string",
"title": "Hostname",
"description": "Hostname of the database server"
},
"port": {
"type": "integer",
"title": "Port",
"description": "Port of the database server"
},
"database": {
"type": "string",
"title": "Database",
"description": "Name of the database"
},
"username": {
"type": "string",
"title": "Username",
"description": "Username for authentication"
},
"password": {
"type": "string",
"title": "Password",
"description": "Password for authentication",
"airbyte_secret": true
}
},
"groups": [
{"id": "connection", "title": "Connection"},
{"id": "advanced", "title": "Advanced"}
]
},
"supportsIncremental": true,
"supportsNormalization": false,
"supportsDBT": false,
"supported_destination_sync_modes": [
"overwrite",
"append",
"append_dedup"
]
}
```
**Note:** This file is a snapshot of expected output. Generate it by:
1. Running spec operation manually
2. Copying output to this file
3. Using it for regression testing
### Step 1.7: Create Spec Test
**File:** `src/test-integration/kotlin/.../spec/{DB}SpecTest.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.spec
import io.airbyte.cdk.load.spec.SpecTest
class {DB}SpecTest : SpecTest()
```
**What this tests:**
- Spec operation executes without errors
- Returns valid JSON schema
- Matches expected-spec-oss.json (snapshot test)
- If Cloud-specific: Matches expected-spec-cloud.json
### Step 1.8: Generate and Validate Spec
**Run spec operation to generate the JSON schema:**
```bash
$ ./gradlew :destination-{db}:run --args='--spec'
```
**Expected output (stdout):**
```json
{
"type": "SPEC",
"spec": {
"documentationUrl": "https://docs.airbyte.com/integrations/destinations/{db}",
"connectionSpecification": { ... },
"supportsIncremental": true,
"supported_destination_sync_modes": ["overwrite", "append", "append_dedup"]
}
}
```
**Copy the `spec` object** (not the outer wrapper) to:
```bash
# Create resources directory
mkdir -p src/test-integration/resources
# Manually copy the "spec" portion to this file:
# src/test-integration/resources/expected-spec-oss.json
```
**Tip:** Use `jq` to format: `./gradlew :destination-{db}:run --args='--spec' | jq .spec > expected-spec-oss.json`
### Step 1.9: Run Spec Test
```bash
$ ./gradlew :destination-{db}:integrationTestSpecOss
```
**Expected:**
```
✓ testSpecOss
```
**Troubleshooting:**
- **Spec operation fails:** Check `application.yml` has documentation-url, verify Specification class has Jackson annotations
- **Spec test fails:** Actual spec doesn't match expected-spec-oss.json - update expected file with correct output
**Checkpoint:** `integrationTestSpecOss` passes, --spec operation returns valid JSON schema
---
## Next Steps
**Next:** Continue to [2-database-setup.md](./2-database-setup.md) to implement database connectivity and the check operation.

View File

@@ -0,0 +1,834 @@
# Database Setup: Connectivity, Operations, and Check
**Prerequisites:** Complete [1-getting-started.md](./1-getting-started.md) - Your connector's `--spec` operation must be working.
## What You'll Build
After completing this guide, your connector will have:
- Database connectivity
- Namespace operations
- Basic table operations (create, drop, count)
- `--check` operation working
---
## Phase 2: Database Connectivity
**Goal:** Establish database connection
**Checkpoint:** Can connect to database and execute simple query
### Step 2.1: Create BeanFactory with DataSource
**File:** `{DB}BeanFactory.kt`
```kotlin
package io.airbyte.integrations.destination.{db}
import io.airbyte.cdk.Operation
import io.airbyte.integrations.destination.{db}.spec.*
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Requires
import io.micronaut.context.annotation.Singleton
import javax.sql.DataSource
@Factory
class {DB}BeanFactory {
@Singleton
fun configuration(
configFactory: {DB}ConfigurationFactory,
specFactory: MigratingConfigurationSpecificationSupplier<{DB}Specification>,
): {DB}Configuration {
val spec = specFactory.get()
return configFactory.makeWithoutExceptionHandling(spec)
}
@Singleton
@Requires(property = Operation.PROPERTY, notEquals = "spec")
fun dataSource(config: {DB}Configuration): DataSource {
// For JDBC databases:
return HikariDataSource().apply {
jdbcUrl = "jdbc:{db}://${config.hostname}:${config.port}/${config.database}"
username = config.username
password = config.password
maximumPoolSize = 10
connectionTimeout = 30000
}
// For non-JDBC: Create your native client here
}
@Singleton
@Requires(property = Operation.PROPERTY, value = "spec")
fun emptyDataSource(): DataSource {
return object : DataSource {
override fun getConnection() = null
override fun getConnection(username: String?, password: String?) = null
override fun unwrap(iface: Class<*>?) = throw UnsupportedOperationException()
override fun isWrapperFor(iface: Class<*>?) = false
override fun getLogWriter() = null
override fun setLogWriter(out: java.io.PrintWriter?) {}
override fun setLoginTimeout(seconds: Int) {}
override fun getLoginTimeout() = 0
override fun getParentLogger() = throw UnsupportedOperationException()
}
}
}
```
### Step 2.2: Add Testcontainers Dependency
**File:** Update `build.gradle.kts`
```kotlin
dependencies {
// Existing dependencies...
// Testcontainers for automated testing (recommended)
testImplementation("org.testcontainers:testcontainers:1.19.0")
testImplementation("org.testcontainers:{db}:1.19.0") // e.g., postgresql, mysql, etc.
// Or for databases without specific module:
// testImplementation("org.testcontainers:jdbc:1.19.0")
}
```
**Check available modules:** https://www.testcontainers.org/modules/databases/
### Step 2.3: Create Test Configuration with Testcontainers
**File:** `src/test-integration/kotlin/.../component/{DB}TestConfigFactory.kt`
**Primary approach (Testcontainers - recommended):**
```kotlin
package io.airbyte.integrations.destination.{db}.component
import io.airbyte.cdk.command.MigratingConfigurationSpecificationSupplier
import io.airbyte.integrations.destination.{db}.spec.*
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Primary
import io.micronaut.context.annotation.Requires
import io.micronaut.context.annotation.Singleton
import org.testcontainers.containers.{DB}Container // e.g., PostgreSQLContainer
@Factory
@Requires(env = ["component"])
class {DB}TestConfigFactory {
@Singleton
@Primary
fun testContainer(): {DB}Container<*> {
// Example for PostgreSQL:
val container = PostgreSQLContainer("postgres:15-alpine")
.withDatabaseName("test")
.withUsername("test")
.withPassword("test")
// Example for MySQL:
// val container = MySQLContainer("mysql:8.0")
// .withDatabaseName("test")
// Example for generic JDBC:
// val container = JdbcDatabaseContainer("{db}:latest")
container.start()
return container
}
@Singleton
@Primary
fun testConfig(container: {DB}Container<*>): {DB}Configuration {
return {DB}Configuration(
hostname = container.host,
port = container.firstMappedPort,
database = container.databaseName,
username = container.username,
password = container.password,
)
}
@Singleton
@Primary
fun testSpecSupplier(
config: {DB}Configuration
): MigratingConfigurationSpecificationSupplier<{DB}Specification> {
return object : MigratingConfigurationSpecificationSupplier<{DB}Specification> {
override fun get() = {DB}Specification()
}
}
}
```
**Alternative: Environment variables (for local development)**
```kotlin
@Singleton
@Primary
fun testConfig(): {DB}Configuration {
return {DB}Configuration(
hostname = System.getenv("DB_HOSTNAME") ?: "localhost",
port = System.getenv("DB_PORT")?.toInt() ?: 5432,
database = System.getenv("DB_DATABASE") ?: "test",
username = System.getenv("DB_USERNAME") ?: "test",
password = System.getenv("DB_PASSWORD") ?: "test",
)
}
```
**Why Testcontainers (recommended)?**
- ✅ Isolated test environment (no conflicts with other tests)
- ✅ Works in CI without setup
- ✅ Reproducible across machines
- ✅ Automatic cleanup
- ✅ No manual database installation needed
**When to use environment variables?**
- Local development with existing database
- Database not supported by Testcontainers
- Debugging against specific database version
### Step 2.4: Create Minimal Test Client
**File:** `src/test-integration/kotlin/.../component/{DB}TestTableOperationsClient.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.component
import io.airbyte.cdk.load.component.TestTableOperationsClient
import io.airbyte.cdk.load.data.AirbyteValue
import io.airbyte.cdk.load.table.TableName
import io.micronaut.context.annotation.Requires
import io.micronaut.context.annotation.Singleton
import javax.sql.DataSource
@Requires(env = ["component"])
@Singleton
class {DB}TestTableOperationsClient(
private val dataSource: DataSource,
) : TestTableOperationsClient {
override suspend fun ping() {
dataSource.connection.use { connection ->
connection.createStatement().use { statement ->
statement.executeQuery("SELECT 1")
}
}
}
override suspend fun dropNamespace(namespace: String) {
// Implement in Phase 2
TODO("Implement in Phase 2")
}
override suspend fun insertRecords(
table: TableName,
records: List<Map<String, AirbyteValue>>
) {
// Implement in Phase 3
TODO("Implement in Phase 3")
}
override suspend fun readTable(table: TableName): List<Map<String, Any>> {
// Implement in Phase 3
TODO("Implement in Phase 3")
}
}
```
### Step 2.5: Create TableOperationsTest (Minimal)
**File:** `src/test-integration/kotlin/.../component/{DB}TableOperationsTest.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.component
import io.airbyte.cdk.load.component.TableOperationsClient
import io.airbyte.cdk.load.component.TableOperationsSuite
import io.airbyte.cdk.load.component.TestTableOperationsClient
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
import org.junit.jupiter.api.Test
@MicronautTest(environments = ["component"])
class {DB}TableOperationsTest(
override val client: TableOperationsClient,
override val testClient: TestTableOperationsClient,
) : TableOperationsSuite {
@Test
override fun `connect to database`() {
super.`connect to database`()
}
// Other tests commented out for now - implement in later phases
// @Test
// override fun `create and drop namespaces`() { super.`create and drop namespaces`() }
}
```
### Step 2.6: Validate Connection
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentConnectToDatabase # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # Regression check
```
**Troubleshooting:**
- Connection refused? Check hostname/port in test config
- Authentication failed? Verify username/password
- Database doesn't exist? Create test database first or use Testcontainers
- Timeout? Check firewall/network connectivity
**Checkpoint:** Can connect to database + all previous phases still work
---
---
## Phase 3: Namespace Operations
**Goal:** Create and drop schemas/databases
**Checkpoint:** Can manage namespaces
### Step 3.1: Create SQL Generator (Namespace Methods)
**File:** `client/{DB}SqlGenerator.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.client
import io.airbyte.cdk.load.table.TableName
import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Singleton
private val log = KotlinLogging.logger {}
fun String.andLog(): String {
log.info { this.trim() }
return this
}
@Singleton
class {DB}SqlGenerator {
fun createNamespace(namespace: String): String {
// Postgres/MySQL: CREATE SCHEMA
return "CREATE SCHEMA IF NOT EXISTS ${namespace.quote()}".andLog()
// Or for databases without schemas:
return "CREATE DATABASE IF NOT EXISTS ${namespace.quote()}".andLog()
}
fun namespaceExists(namespace: String): String {
// Postgres:
return """
SELECT schema_name
FROM information_schema.schemata
WHERE schema_name = '${namespace}'
""".trimIndent().andLog()
// Or query your DB's system catalog
}
private fun String.quote() = "\"$this\"" // Or backticks, brackets, etc.
private fun fullyQualifiedName(tableName: TableName) =
"${tableName.namespace.quote()}.${tableName.name.quote()}"
}
```
### Step 2.2: Create Database Client (Namespace Methods)
**File:** `client/{DB}AirbyteClient.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.client
import io.airbyte.cdk.load.component.TableOperationsClient
import io.airbyte.cdk.load.component.TableSchemaEvolutionClient
import io.airbyte.integrations.destination.{db}.spec.{DB}Configuration
import io.micronaut.context.annotation.Singleton
import javax.sql.DataSource
@Singleton
class {DB}AirbyteClient(
private val dataSource: DataSource,
private val sqlGenerator: {DB}SqlGenerator,
private val config: {DB}Configuration,
) : TableOperationsClient, TableSchemaEvolutionClient {
override suspend fun createNamespace(namespace: String) {
execute(sqlGenerator.createNamespace(namespace))
}
override suspend fun namespaceExists(namespace: String): Boolean {
return dataSource.connection.use { connection ->
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sqlGenerator.namespaceExists(namespace))
rs.next() // Returns true if namespace exists
}
}
}
private fun execute(sql: String) {
dataSource.connection.use { connection ->
connection.createStatement().use { statement ->
statement.execute(sql)
}
}
}
// Stub other methods for now
override suspend fun createTable(...) = TODO("Phase 3")
override suspend fun dropTable(...) = TODO("Phase 3")
override suspend fun tableExists(...) = TODO("Phase 3")
override suspend fun countTable(...) = TODO("Phase 3")
override suspend fun getGenerationId(...) = TODO("Phase 3")
override suspend fun copyTable(...) = TODO("Phase 7")
override suspend fun overwriteTable(...) = TODO("Phase 6")
override suspend fun upsertTable(...) = TODO("Phase 9")
override suspend fun discoverSchema(...) = TODO("Phase 8")
override fun computeSchema(...) = TODO("Phase 8")
override suspend fun ensureSchemaMatches(...) = TODO("Phase 8")
override suspend fun applyChangeset(...) = TODO("Phase 8")
}
```
### Step 2.3: Update Test Client
**File:** Update `{DB}TestTableOperationsClient.kt`
```kotlin
override suspend fun dropNamespace(namespace: String) {
dataSource.connection.use { connection ->
connection.createStatement().use { statement ->
// Postgres/MySQL:
statement.execute("DROP SCHEMA IF EXISTS \"${namespace}\" CASCADE")
// Or for databases without schemas:
statement.execute("DROP DATABASE IF EXISTS `${namespace}`")
}
}
}
```
### Step 2.4: Register Client in BeanFactory
**File:** Update `{DB}BeanFactory.kt`
```kotlin
@Singleton
fun client(
dataSource: DataSource,
sqlGenerator: {DB}SqlGenerator,
config: {DB}Configuration,
): {DB}AirbyteClient {
return {DB}AirbyteClient(dataSource, sqlGenerator, config)
}
```
### Step 2.5: Enable Test in TableOperationsTest
**File:** Update `{DB}TableOperationsTest.kt`
```kotlin
@Test
override fun `create and drop namespaces`() {
super.`create and drop namespaces`()
}
```
### Step 2.6: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentCreateAndDropNamespaces # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 2 tests should pass
```
**Checkpoint:** Can manage namespaces + all previous phases still work
---
---
## Phase 4: Basic Table Operations
**Goal:** Create tables, insert data, count rows
**Checkpoint:** Can perform basic table CRUD
### Step 4.1: Create Column Utilities
**File:** `client/{DB}ColumnUtils.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.client
import io.airbyte.cdk.load.data.*
import io.micronaut.context.annotation.Singleton
@Singleton
class {DB}ColumnUtils {
fun toDialectType(type: AirbyteType): String = when (type) {
BooleanType -> "BOOLEAN"
IntegerType -> "BIGINT"
NumberType -> "DECIMAL(38, 9)"
StringType -> "VARCHAR" // or TEXT
DateType -> "DATE"
TimeTypeWithTimezone -> "TIME WITH TIME ZONE"
TimeTypeWithoutTimezone -> "TIME"
TimestampTypeWithTimezone -> "TIMESTAMP WITH TIME ZONE"
TimestampTypeWithoutTimezone -> "TIMESTAMP"
is ArrayType, ArrayTypeWithoutSchema -> "JSONB" // or TEXT
is ObjectType, ObjectTypeWithEmptySchema, ObjectTypeWithoutSchema -> "JSONB"
is UnionType, is UnknownType -> "JSONB" // or VARCHAR as fallback
else -> "VARCHAR"
}
fun formatColumn(name: String, type: AirbyteType, nullable: Boolean): String {
val typeDecl = toDialectType(type)
val nullableDecl = if (nullable) "" else " NOT NULL"
return "\"$name\" $typeDecl$nullableDecl"
}
}
```
**Database-specific adjustments:**
- **Snowflake:** `VARCHAR` (no length), `VARIANT` for JSON
- **ClickHouse:** `String`, `Nullable()` wrapper, `DateTime64(3)`
- **MySQL:** `VARCHAR(65535)`, `JSON` type
- **BigQuery:** `STRING`, `JSON`, specific types
### Step 3.2: Add Table Methods to SQL Generator
**File:** Update `{DB}SqlGenerator.kt`
```kotlin
fun createTable(
stream: DestinationStream,
tableName: TableName,
columnMapping: ColumnNameMapping,
replace: Boolean
): String {
val replaceClause = if (replace) "OR REPLACE " else ""
val columnDeclarations = stream.schema.asColumns()
.filter { (name, _) -> name !in AIRBYTE_META_COLUMNS }
.map { (name, type) ->
val mappedName = columnMapping[name]!!
columnUtils.formatColumn(mappedName, type.type, type.nullable)
}
.joinToString(",\n ")
return """
CREATE ${replaceClause}TABLE ${fullyQualifiedName(tableName)} (
"_airbyte_raw_id" VARCHAR NOT NULL,
"_airbyte_extracted_at" TIMESTAMP NOT NULL,
"_airbyte_meta" JSONB NOT NULL,
"_airbyte_generation_id" BIGINT,
$columnDeclarations
)
""".trimIndent().andLog()
}
fun dropTable(tableName: TableName): String {
return "DROP TABLE IF EXISTS ${fullyQualifiedName(tableName)}".andLog()
}
fun countTable(tableName: TableName): String {
return """
SELECT COUNT(*) AS count
FROM ${fullyQualifiedName(tableName)}
""".trimIndent().andLog()
}
fun getGenerationId(tableName: TableName): String {
return """
SELECT "_airbyte_generation_id" AS generation_id
FROM ${fullyQualifiedName(tableName)}
LIMIT 1
""".trimIndent().andLog()
}
private val AIRBYTE_META_COLUMNS = setOf(
"_airbyte_raw_id",
"_airbyte_extracted_at",
"_airbyte_meta",
"_airbyte_generation_id"
)
```
### Step 3.3: Implement Table Operations in Client
**File:** Update `{DB}AirbyteClient.kt`
```kotlin
override suspend fun createTable(
stream: DestinationStream,
tableName: TableName,
columnNameMapping: ColumnNameMapping,
replace: Boolean
) {
execute(sqlGenerator.createTable(stream, tableName, columnNameMapping, replace))
}
override suspend fun dropTable(tableName: TableName) {
execute(sqlGenerator.dropTable(tableName))
}
override suspend fun tableExists(table: TableName): Boolean {
return countTable(table) != null
}
override suspend fun countTable(tableName: TableName): Long? =
try {
dataSource.connection.use { connection ->
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sqlGenerator.countTable(tableName))
if (rs.next()) rs.getLong("count") else 0L
}
}
} catch (e: SQLException) {
log.debug(e) { "Table ${tableName} does not exist. Returning null." }
null // Expected - table doesn't exist
}
override suspend fun getGenerationId(tableName: TableName): Long {
try {
dataSource.connection.use { connection ->
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sqlGenerator.getGenerationId(tableName))
if (rs.next()) {
rs.getLong("generation_id") ?: 0L
} else {
0L
}
}
}
} catch (e: SQLException) {
log.debug(e) { "Failed to retrieve generation ID, returning 0" }
return 0L
}
}
```
### Step 3.4: Implement Test Client Insert/Read
**File:** Update `{DB}TestTableOperationsClient.kt`
```kotlin
override suspend fun insertRecords(
table: TableName,
records: List<Map<String, AirbyteValue>>
) {
if (records.isEmpty()) return
dataSource.connection.use { connection ->
records.forEach { record ->
val columns = record.keys.joinToString(", ") { "\"$it\"" }
val placeholders = record.keys.joinToString(", ") { "?" }
val sql = """
INSERT INTO "${table.namespace}"."${table.name}" ($columns)
VALUES ($placeholders)
"""
connection.prepareStatement(sql).use { statement ->
record.values.forEachIndexed { index, value ->
setParameter(statement, index + 1, value)
}
statement.executeUpdate()
}
}
}
}
override suspend fun readTable(table: TableName): List<Map<String, Any>> {
val results = mutableListOf<Map<String, Any>>()
dataSource.connection.use { connection ->
val sql = "SELECT * FROM \"${table.namespace}\".\"${table.name}\""
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sql)
val metadata = rs.metaData
while (rs.next()) {
val row = mutableMapOf<String, Any>()
for (i in 1..metadata.columnCount) {
val columnName = metadata.getColumnName(i)
val value = rs.getObject(i)
if (value != null) {
row[columnName] = value
}
}
results.add(row)
}
}
}
return results
}
private fun setParameter(statement: PreparedStatement, index: Int, value: AirbyteValue) {
when (value) {
is StringValue -> statement.setString(index, value.value)
is IntegerValue -> statement.setLong(index, value.value)
is NumberValue -> statement.setBigDecimal(index, value.value)
is BooleanValue -> statement.setBoolean(index, value.value)
is TimestampValue -> statement.setTimestamp(index, Timestamp.from(value.value))
is DateValue -> statement.setDate(index, Date.valueOf(value.value))
is ObjectValue -> statement.setString(index, value.toJson()) // JSON as string
is ArrayValue -> statement.setString(index, value.toJson()) // JSON as string
is NullValue -> statement.setNull(index, Types.VARCHAR)
else -> statement.setString(index, value.toString())
}
}
```
**Note:** For non-JDBC databases, use native client APIs for insert/read
### Step 3.5: Enable Table Tests
**File:** Update `{DB}TableOperationsTest.kt`
```kotlin
@Test
override fun `create and drop tables`() {
super.`create and drop tables`()
}
@Test
override fun `insert records`() {
super.`insert records`()
}
@Test
override fun `count table rows`() {
super.`count table rows`()
}
```
### Step 3.6: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentCreateAndDropTables \
:destination-{db}:testComponentInsertRecords \
:destination-{db}:testComponentCountTableRows # 3 tests should pass
$ ./gradlew :destination-{db}:componentTest # 5 tests should pass
```
**Checkpoint:** Can manage tables and data + all previous phases still work
---
---
## Phase 5: Check Operation
**Goal:** Implement --check operation (validates database connection)
**Checkpoint:** Check operation works
### Step 5.1: Create Checker
**File:** `check/{DB}Checker.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.check
import io.airbyte.cdk.load.check.DestinationCheckerV2
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.data.*
import io.airbyte.cdk.load.table.ColumnNameMapping
import io.airbyte.cdk.load.table.TableName
import io.airbyte.integrations.destination.{db}.client.{DB}AirbyteClient
import io.airbyte.integrations.destination.{db}.spec.{DB}Configuration
import io.micronaut.context.annotation.Singleton
import kotlinx.coroutines.runBlocking
import java.util.UUID
@Singleton
class {DB}Checker(
private val client: {DB}AirbyteClient,
private val config: {DB}Configuration,
) : DestinationCheckerV2 {
override fun check() {
val testNamespace = config.database
val testTableName = "_airbyte_connection_test_${UUID.randomUUID()}"
val tableName = TableName(testNamespace, testTableName)
runBlocking {
try {
client.createNamespace(testNamespace)
val testStream = createTestStream()
val columnMapping = ColumnNameMapping(mapOf("test_col" to "test_col"))
client.createTable(testStream, tableName, columnMapping, replace = false)
val count = client.countTable(tableName)
require(count == 0L) { "Expected empty table, got $count rows" }
} finally {
client.dropTable(tableName)
}
}
}
private fun createTestStream(): DestinationStream {
return DestinationStream(
descriptor = DestinationStream.Descriptor(namespace = config.database, name = "test"),
importType = DestinationStream.ImportType.APPEND,
schema = ObjectType(linkedMapOf("test_col" to FieldType(StringType, true))),
generationId = 0,
minimumGenerationId = 0,
syncId = 0,
)
}
}
```
### Step 5.2: Create Check Integration Test
**File:** `src/test-integration/kotlin/.../check/{DB}CheckTest.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.check
import io.airbyte.cdk.load.check.CheckIntegrationTest
import io.airbyte.cdk.load.check.CheckTestConfig
import io.airbyte.integrations.destination.{db}.spec.{DB}Specification
import java.nio.file.Path
class {DB}CheckTest :
CheckIntegrationTest<{DB}Specification>(
successConfigFilenames = listOf(
CheckTestConfig(configPath = Path.of("secrets/config.json")),
),
failConfigFilenamesAndFailureReasons = emptyMap(),
)
```
### Step 5.3: Validate Check Operation
**Validate:**
```bash
$ ./gradlew :destination-{db}:integrationTestCheckSuccessConfigs # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 5 tests should pass
$ ./gradlew :destination-{db}:integrationTest # testSpecOss, testSuccessConfigs should pass
```
**Checkpoint:** --check operation works + all previous phases still work
---
---
## Next Steps
**Next:** Continue to [3-write-infrastructure.md](./3-write-infrastructure.md) to set up the write operation infrastructure.

View File

@@ -0,0 +1,596 @@
# Write Infrastructure: DI Setup and Test Contexts
**Prerequisites:** Complete [2-database-setup.md](./2-database-setup.md) - Your connector's `--check` operation must be working.
## What You'll Build
After completing this guide, you'll have:
- Name generators (table, column, temp)
- TableCatalog DI setup
- Write operation entry point
- Understanding of test contexts (CRITICAL!)
---
## Phase 6: Name Generators & TableCatalog DI
**Goal:** Create name generator beans required for TableCatalog instantiation
**Checkpoint:** Compilation succeeds without DI errors
**📋 Dependency Context:** TableCatalog (auto-instantiated by CDK) requires these three @Singleton beans:
- RawTableNameGenerator
- FinalTableNameGenerator
- ColumnNameGenerator
Without these beans, you'll get **"Error instantiating TableCatalog"** or **"No bean of type [FinalTableNameGenerator]"** errors in Phase 7 write tests.
### Step 6.1: Create RawTableNameGenerator
**File:** `config/{DB}NameGenerators.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.config
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.orchestration.db.RawTableNameGenerator
import io.airbyte.cdk.load.table.TableName
import io.airbyte.integrations.destination.{db}.spec.{DB}Configuration
import io.micronaut.context.annotation.Singleton
@Singleton
class {DB}RawTableNameGenerator(
private val config: {DB}Configuration,
) : RawTableNameGenerator {
override fun getTableName(descriptor: DestinationStream.Descriptor): TableName {
// Raw tables typically go to internal schema
// Modern CDK uses final tables directly, so raw tables are rarely used
val namespace = config.database // Or config.internalSchema if you have one
val name = "_airbyte_raw_${descriptor.namespace}_${descriptor.name}".toDbCompatible()
return TableName(namespace, name)
}
}
```
**Notes:**
- `@Singleton` annotation is **REQUIRED** - without it, Micronaut cannot inject this bean
- RawTableNameGenerator is legacy from two-stage sync (raw → final tables)
- Modern connectors typically use final tables only, but interface must be implemented
- Keep implementation simple (identity mapping is fine)
### Step 6.2: Create FinalTableNameGenerator
**Add to same file:** `config/{DB}NameGenerators.kt`
```kotlin
@Singleton
class {DB}FinalTableNameGenerator(
private val config: {DB}Configuration,
) : FinalTableNameGenerator {
override fun getTableName(descriptor: DestinationStream.Descriptor): TableName {
val namespace = descriptor.namespace?.toDbCompatible()
?: config.database
val name = descriptor.name.toDbCompatible()
return TableName(namespace, name)
}
}
```
**What this does:**
- Maps Airbyte stream descriptor → database table name
- Handles namespace mapping (if source has schemas/databases)
- Applies database-specific name transformation rules
**Example transforms:**
```kotlin
// Input: descriptor(namespace="public", name="users")
// Output: TableName("public", "users")
// Input: descriptor(namespace=null, name="customers")
// Output: TableName("my_database", "customers") // Uses config.database as fallback
```
### Step 6.3: Create ColumnNameGenerator
**Add to same file:** `config/{DB}NameGenerators.kt`
```kotlin
@Singleton
class {DB}ColumnNameGenerator : ColumnNameGenerator {
override fun getColumnName(column: String): ColumnNameGenerator.ColumnName {
val dbName = column.toDbCompatible()
return ColumnNameGenerator.ColumnName(
canonicalName = dbName,
displayName = dbName,
)
}
}
```
**What this does:**
- Maps Airbyte column names → database column names
- Applies database-specific transformations (case, special chars)
**Example transforms:**
```kotlin
// Snowflake: uppercase
"userId" "USERID"
// Postgres/ClickHouse: lowercase
"userId" "userid"
// MySQL: preserve case
"userId" "userId"
```
### Step 6.4: Add Name Transformation Helper
**Add to same file:** `config/{DB}NameGenerators.kt`
```kotlin
// Helper function for database-specific name transformations
private fun String.toDbCompatible(): String {
// Snowflake: uppercase
return this.uppercase()
// ClickHouse/Postgres: lowercase
return this.lowercase()
// MySQL: preserve case, but sanitize special chars
return this.replace(Regex("[^a-zA-Z0-9_]"), "_")
// Custom rules: Apply your database's naming conventions
// - Max length limits
// - Reserved word handling
// - Character restrictions
}
```
**Database-specific examples:**
**Snowflake:**
```kotlin
private fun String.toDbCompatible() = this.uppercase()
```
**ClickHouse:**
```kotlin
private fun String.toDbCompatible() = this.lowercase()
```
**Postgres (strict):**
```kotlin
private fun String.toDbCompatible(): String {
val sanitized = this
.lowercase()
.replace(Regex("[^a-z0-9_]"), "_")
.take(63) // Postgres identifier limit
// Handle reserved words
return if (sanitized in POSTGRES_RESERVED_WORDS) {
"_$sanitized"
} else {
sanitized
}
}
private val POSTGRES_RESERVED_WORDS = setOf("user", "table", "select", ...)
```
### Step 6.5: Register TempTableNameGenerator in BeanFactory
**File:** Update `{DB}BeanFactory.kt`
```kotlin
@Singleton
fun tempTableNameGenerator(config: {DB}Configuration): TempTableNameGenerator {
return DefaultTempTableNameGenerator(
internalNamespace = config.database // Or config.internalSchema if you have one
)
}
```
**What this does:**
- Temp tables are used during overwrite/dedupe operations
- CDK provides `DefaultTempTableNameGenerator` implementation
- Just needs to know which namespace to use for temp tables
**Why register as bean?**
- TempTableNameGenerator is an interface, not a class
- CDK provides implementation, but YOU must register it
- Used by Writer to create staging tables
### Step 6.6: Verify Compilation
**Validate:**
```bash
$ ./gradlew :destination-{db}:compileKotlin # BUILD SUCCESSFUL
$ ./gradlew :destination-{db}:componentTest # 5 tests should pass
$ ./gradlew :destination-{db}:integrationTest # testSpecOss, testSuccessConfigs should pass
```
**If you see DI errors:**
- Check all three classes have `@Singleton` annotation
- Verify package name matches your connector structure
- Ensure classes implement correct interfaces:
- `RawTableNameGenerator` (from `io.airbyte.cdk.load.orchestration.db`)
- `FinalTableNameGenerator` (from `io.airbyte.cdk.load.orchestration.db`)
- `ColumnNameGenerator` (from `io.airbyte.cdk.load.orchestration.db`)
**Checkpoint:** Name generators registered + all previous phases still work
---
---
⚠️ **IMPORTANT: Before starting Phase 7, read [Understanding Test Contexts](./7-troubleshooting.md#understanding-test-contexts) in the troubleshooting guide. Phase 7 introduces integration tests which behave differently than the component tests you've been using.**
---
## Phase 7: Write Operation Infrastructure
**Goal:** Create write operation infrastructure beans (no business logic yet)
**Checkpoint:** Write operation initializes with real catalog (no DI errors)
**📋 Dependency Context:** This phase creates PURE INFRASTRUCTURE:
- WriteOperationV2 (enables --write command)
- DatabaseInitialStatusGatherer (checks existing tables before write)
- ColumnNameMapper (maps column names)
**NO business logic** (Writer/Aggregate/Buffer come in Phase 8)
**Key insight:** Separate infrastructure DI from business logic DI to catch errors incrementally.
### Step 7.1: Create WriteOperationV2
**File:** `cdk/WriteOperationV2.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.cdk
import io.airbyte.cdk.Operation
import io.airbyte.cdk.load.dataflow.DestinationLifecycle
import io.micronaut.context.annotation.Primary
import io.micronaut.context.annotation.Requires
import io.micronaut.context.annotation.Singleton
@Primary
@Singleton
@Requires(property = Operation.PROPERTY, value = "write")
class WriteOperationV2(
private val d: DestinationLifecycle,
) : Operation {
override fun execute() {
d.run()
}
}
```
⚠️ **REQUIRED:** This bean enables --write command
**What this does:**
- Declares a @Primary implementation of Operation for write mode
- Delegates to DestinationLifecycle (provided by CDK)
- DestinationLifecycle orchestrates the full write pipeline:
- Setup (create namespaces, gather initial state)
- Open (start data flow)
- Process (accept records)
- Close (flush buffers, finalize)
**Without this bean:**
```
IllegalStateException: A legal sync requires a declared @Singleton of a type that implements LoadStrategy
```
**Why @Primary?**
- CDK provides default implementations for spec/check operations
- WriteOperationV2 overrides default for write operation only
- @Primary tells Micronaut to prefer this bean over CDK defaults
**Why in `cdk/` package?**
- Pure framework integration (no database-specific code)
- Many connectors keep this file identical across databases
- Signals "this is infrastructure, not business logic"
### Step 7.2: Create DatabaseInitialStatusGatherer
**File:** `config/{DB}DirectLoadDatabaseInitialStatusGatherer.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.config
import io.airbyte.cdk.load.component.TableOperationsClient
import io.airbyte.cdk.load.orchestration.db.*
import io.micronaut.context.annotation.Singleton
@Singleton
class {DB}DirectLoadDatabaseInitialStatusGatherer(
tableOperationsClient: TableOperationsClient,
tempTableNameGenerator: TempTableNameGenerator,
) : BaseDirectLoadInitialStatusGatherer(
tableOperationsClient,
tempTableNameGenerator,
)
```
**What this does:**
- Extends CDK base class that checks which tables already exist
- Used by Writer.setup() to determine:
- Does target table exist?
- Does temp table exist (from previous failed sync)?
- What generation ID is in the table?
**Base class provides:**
```kotlin
suspend fun gatherInitialStatus(catalog: TableCatalog): Map<DestinationStream, DirectLoadInitialStatus>
```
Returns status like:
```kotlin
DirectLoadInitialStatus(
finalTableExists = true,
tempTableExists = false,
finalTableGenerationId = 42L,
)
```
⚠️ **MISSING IN V1 GUIDE:** This step existed as code but bean registration was missing!
### Step 7.3: Register DatabaseInitialStatusGatherer in BeanFactory
**File:** Update `{DB}BeanFactory.kt`
```kotlin
@Singleton
fun initialStatusGatherer(
client: TableOperationsClient,
tempTableNameGenerator: TempTableNameGenerator,
): DatabaseInitialStatusGatherer<DirectLoadInitialStatus> {
return {DB}DirectLoadDatabaseInitialStatusGatherer(client, tempTableNameGenerator)
}
```
⚠️ **CRITICAL:** This bean registration was MISSING in V1 guide!
**Why this is needed:**
- Writer requires `DatabaseInitialStatusGatherer<DirectLoadInitialStatus>` injection
- Without this bean: `No bean of type [DatabaseInitialStatusGatherer] exists`
- Class exists but bean registration forgotten → DI error
**Why use factory method instead of class @Singleton?**
- DatabaseInitialStatusGatherer is generic: `DatabaseInitialStatusGatherer<DirectLoadInitialStatus>`
- Micronaut needs explicit return type for generic beans
- Factory method provides type safety
### Step 7.4: Create ColumnNameMapper
**File:** `write/transform/{DB}ColumnNameMapper.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.write.transform
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.dataflow.transform.ColumnNameMapper
import io.airbyte.cdk.load.table.TableCatalog
import io.micronaut.context.annotation.Singleton
@Singleton
class {DB}ColumnNameMapper(
private val names: TableCatalog,
) : ColumnNameMapper {
override fun getMappedColumnName(
stream: DestinationStream,
columnName: String
): String? {
return names[stream]?.columnNameMapping?.get(columnName)
}
}
```
**What this does:**
- Used by record transformer to map Airbyte column names → database column names
- During transformation pipeline:
```kotlin
// Input record: {"userId": 123, "email": "test@example.com"}
// Snowflake output: {"USERID": 123, "EMAIL": "test@example.com"}
// Postgres output: {"userid": 123, "email": "test@example.com"}
```
**How it works:**
- TableCatalog (provided by CDK) contains column name mappings
- Column names already generated by ColumnNameGenerator (Phase 6)
- ColumnNameMapper just looks up the mapping
**Why separate from ColumnNameGenerator?**
- ColumnNameGenerator: Creates mappings (Phase 6)
- ColumnNameMapper: Uses mappings during transform (Phase 7)
- Separation of concerns: generation vs. application
### Step 7.5: Register AggregatePublishingConfig in BeanFactory
**File:** Update `{DB}BeanFactory.kt`
```kotlin
@Singleton
fun aggregatePublishingConfig(dataChannelMedium: DataChannelMedium): AggregatePublishingConfig {
// Different settings for STDIO vs SOCKET mode
return if (dataChannelMedium == DataChannelMedium.STDIO) {
AggregatePublishingConfig(
maxRecordsPerAgg = 10_000_000_000_000L,
maxEstBytesPerAgg = 350_000_000L,
maxEstBytesAllAggregates = 350_000_000L * 5,
)
} else {
// SOCKET mode (faster IPC)
AggregatePublishingConfig(
maxRecordsPerAgg = 10_000_000_000_000L,
maxEstBytesPerAgg = 350_000_000L,
maxEstBytesAllAggregates = 350_000_000L * 5,
maxBufferedAggregates = 6,
)
}
}
```
**What this configures:**
- `maxRecordsPerAgg`: Flush aggregate after this many records
- `maxEstBytesPerAgg`: Flush aggregate after this many bytes
- `maxEstBytesAllAggregates`: Total memory limit across all streams
- `maxBufferedAggregates`: Backpressure threshold (SOCKET mode only)
**Why this is required:**
- Controls memory usage and batching behavior
- CDK's data pipeline needs this configuration
- Without it: `No bean of type [AggregatePublishingConfig] exists`
**Default values:**
- Use Snowflake/ClickHouse values as template (shown above)
- Tune later based on performance requirements
- Start with defaults - they work for most databases
### Step 7.6: Create WriteInitializationTest
**File:** `src/test-integration/kotlin/.../write/{DB}WriteInitTest.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.write
import io.airbyte.cdk.load.write.WriteInitializationTest
import io.airbyte.integrations.destination.{db}.spec.{DB}Specification
import java.nio.file.Path
/**
* Validates write operation can initialize with real catalog.
* Catches missing beans that ConnectorWiringSuite (with mock catalog) doesn't test.
*
* This test spawns a real write process (same as Docker) and validates:
* - TableCatalog can be instantiated (requires name generators from Phase 6)
* - Write operation can be created (requires WriteOperationV2 from Phase 7.1)
* - All write infrastructure beans exist (DatabaseInitialStatusGatherer, ColumnNameMapper, etc.)
*
* Does NOT validate data writing - that's Phase 8 (ConnectorWiringSuite)
*/
class {DB}WriteInitTest : WriteInitializationTest<{DB}Specification>(
configContents = Path.of("secrets/config.json").toFile().readText(),
configSpecClass = {DB}Specification::class.java,
)
```
**That's it!** Just 10 lines - extend `WriteInitializationTest` and provide your config.
**What WriteInitializationTest provides (from CDK):**
- ✅ Test method: `writer can be instantiated with real catalog`
- ✅ Minimal catalog (one stream) - hardcoded in base class
- ✅ Spawns real write process (same as Docker)
- ✅ Validates DI initialization only (no data writing)
- ✅ Clear error messages pointing to missing beans
**Why this test matters:**
- **ConnectorWiringSuite** (Phase 8) uses MockDestinationCatalog → doesn't test TableCatalog DI
- **WriteInitializationTest** uses REAL catalog parsing → catches TableCatalog DI errors
- **Catches:** Missing name generators, missing WriteOperationV2, missing beans
**Test progression:**
```
Phase 6: Name generators exist
Phase 7: WriteInitTest validates they work with real catalog
Phase 8: ConnectorWiringSuite validates full write path with mock catalog
```
### Step 7.6: Create Test Config File
**File:** `secrets/config.json`
```bash
$ mkdir -p destination-{db}/secrets
```
**File:** `destination-{db}/secrets/config.json`
```json
{
"hostname": "localhost",
"port": 5432,
"database": "test",
"username": "test",
"password": "test"
}
```
**For CI/local with Testcontainers:**
- WriteInitTest doesn't use Testcontainers (integration test, not component test)
- Provide real database credentials or use local database
- Alternatively: Update test to read from environment variables
**Note:** Add `secrets/` to `.gitignore` to avoid committing credentials
### Step 7.7: Validate WriteInitializationTest
**Validate:**
```bash
$ ./gradlew :destination-{db}:integrationTestWriterCanBeInstantiatedWithRealCatalog # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 5 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass
```
**If this FAILS with DI errors:**
**"Error instantiating TableCatalog"** or **"No bean of type [FinalTableNameGenerator]"**
→ Missing name generator from Phase 6
→ Check @Singleton annotation on all three generators
**"No bean of type [DatabaseInitialStatusGatherer]"**
→ Missing bean registration in BeanFactory (Step 7.3)
→ Add `initialStatusGatherer()` factory method
**"IllegalStateException: No LoadStrategy"** or **"A legal sync requires a declared @Singleton"**
→ Missing WriteOperationV2 (Step 7.1)
→ Create `cdk/WriteOperationV2.kt` with @Primary @Singleton
**"Failed to inject ColumnNameMapper"**
→ Missing @Singleton annotation on ColumnNameMapper
→ Check `write/transform/{DB}ColumnNameMapper.kt`
**Checkpoint:** Write operation infrastructure exists + all previous phases still work
---
---
## Troubleshooting Reference
**If you encounter errors during Phases 6-7, refer to:**
### Understanding Test Contexts
Phase 7 introduces **integration tests** which behave differently than component tests:
- **Component tests** (Phases 2-5): Use MockDestinationCatalog, bypass name generators
- **Integration tests** (Phase 7+): Use real catalog parsing, require name generators
**Why this matters:**
- Component tests passing ≠ integration tests passing
- Integration tests catch missing name generators, bean registrations, and DI issues
- Docker runtime uses same context as integration tests
📖 **Full explanation:** [Understanding Test Contexts](./7-troubleshooting.md#understanding-test-contexts)
---
### Common DI Errors in Phase 7
**Quick fixes for the most common errors:**
| Error | Likely Cause | Fix Guide |
|-------|--------------|-----------|
| "No bean of type [FinalTableNameGenerator]" | Missing name generator classes | Phase 6, Step 6.1-6.3 |
| "No bean of type [DatabaseInitialStatusGatherer]" | Missing bean registration | Phase 7, Step 7.3 |
| "A legal sync requires a declared @Singleton" | Missing WriteOperationV2 | Phase 7, Step 7.1 |
| "Failed to inject value for parameter [dataChannelMedium]" | Missing application-connector.yml | Phase 0, Step 0.8 |
📖 **Detailed troubleshooting:** [Common DI Errors & Fixes](./7-troubleshooting.md#common-di-errors--fixes)
**Next:** Continue to [4-write-operations.md](./4-write-operations.md) to implement the core write business logic.

View File

@@ -0,0 +1,769 @@
# Write Operations: Core Sync Functionality
**Prerequisites:** Complete [3-write-infrastructure.md](./3-write-infrastructure.md) - Your DI setup must be complete and you must understand test contexts.
**What You'll Build:** After completing this guide, you'll have a working connector with:
- InsertBuffer with efficient batch writes
- Aggregate and AggregateFactory
- Writer orchestration
- Append mode (direct insert)
- Overwrite mode (temp table + atomic swap)
- Generation ID tracking
- `--write` operation working for basic syncs
---
## Phase 8: Writer & Append Mode (Business Logic)
**Goal:** Implement actual data writing (Writer, Aggregate, InsertBuffer)
**Checkpoint:** Can write one record end-to-end
**📋 Dependency Context:** Now that infrastructure exists (Phases 6-7), add business logic:
- InsertBuffer (accumulates and flushes records to database)
- Aggregate (processes transformed records)
- AggregateFactory (creates Aggregate instances)
- Writer (orchestrates setup and creates StreamLoaders)
**Key insight:** Infrastructure DI (Phase 7) is separate from business logic DI (Phase 8).
Phase 7 validates "can we start?" Phase 8 validates "can we write data?"
### Step 8.1: Create InsertBuffer
**File:** `write/load/{DB}InsertBuffer.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.write.load
import io.airbyte.cdk.load.data.AirbyteValue
import io.airbyte.cdk.load.table.TableName
import io.airbyte.integrations.destination.{db}.client.{DB}AirbyteClient
import io.github.oshai.kotlinlogging.KotlinLogging
private val log = KotlinLogging.logger {}
/**
* Accumulates records and flushes to database in batches.
*
* NOT a @Singleton - created per-stream by AggregateFactory
*/
class {DB}InsertBuffer(
private val tableName: TableName,
private val client: {DB}AirbyteClient,
private val flushLimit: Int = 1000,
) {
private val buffer = mutableListOf<Map<String, AirbyteValue>>()
private var recordCount = 0
fun accumulate(recordFields: Map<String, AirbyteValue>) {
buffer.add(recordFields)
recordCount++
if (recordCount >= flushLimit) {
kotlinx.coroutines.runBlocking { flush() }
}
}
suspend fun flush() {
if (buffer.isEmpty()) return
try {
log.info { "Flushing $recordCount records to ${tableName}..." }
// Simple multi-row INSERT for now
// (Optimize in Phase 15: CSV staging, COPY, bulk APIs)
buffer.forEach { record ->
insertRecord(tableName, record)
}
log.info { "Finished flushing $recordCount records" }
} finally {
buffer.clear()
recordCount = 0
}
}
private suspend fun insertRecord(
tableName: TableName,
record: Map<String, AirbyteValue>
) {
val columns = record.keys.joinToString(", ") { "\"$it\"" }
val placeholders = record.keys.joinToString(", ") { "?" }
val sql = """
INSERT INTO "${tableName.namespace}"."${tableName.name}" ($columns)
VALUES ($placeholders)
"""
client.executeInsert(sql, record.values.toList())
}
}
```
**Key points:**
- **NOT @Singleton** - one buffer per stream
- Simple implementation: single-row inserts
- Phase 15 (Optimization) replaces with bulk loading
**Why not @Singleton?**
- Each stream needs its own buffer
- Buffers hold stream-specific state (table name, accumulated records)
- AggregateFactory creates one buffer per stream
### Step 8.2: Add executeInsert() to Client
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
// Add this method to {DB}AirbyteClient
fun executeInsert(sql: String, values: List<AirbyteValue>) {
dataSource.connection.use { connection ->
connection.prepareStatement(sql).use { statement ->
values.forEachIndexed { index, value ->
setParameter(statement, index + 1, value)
}
statement.executeUpdate()
}
}
}
private fun setParameter(statement: PreparedStatement, index: Int, value: AirbyteValue) {
when (value) {
is StringValue -> statement.setString(index, value.value)
is IntegerValue -> statement.setLong(index, value.value)
is NumberValue -> statement.setBigDecimal(index, value.value)
is BooleanValue -> statement.setBoolean(index, value.value)
is TimestampValue -> statement.setTimestamp(index, Timestamp.from(value.value))
is DateValue -> statement.setDate(index, Date.valueOf(value.value))
is TimeValue -> statement.setTime(index, Time.valueOf(value.value.toLocalTime()))
is ObjectValue -> statement.setString(index, value.toJson()) // JSON as string
is ArrayValue -> statement.setString(index, value.toJson()) // JSON as string
is NullValue -> statement.setNull(index, Types.VARCHAR)
else -> statement.setString(index, value.toString())
}
}
```
**Note:** For non-JDBC databases, use native client APIs (e.g., MongoDB insertOne, ClickHouse native client)
### Step 8.3: Create Aggregate
**File:** `dataflow/{DB}Aggregate.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.dataflow
import io.airbyte.cdk.load.dataflow.aggregate.Aggregate
import io.airbyte.cdk.load.dataflow.transform.RecordDTO
import io.airbyte.integrations.destination.{db}.write.load.{DB}InsertBuffer
/**
* Processes transformed records for a single stream.
*
* Dataflow pipeline: Raw record → Transform → RecordDTO → Aggregate.accept() → InsertBuffer
*
* NOT a @Singleton - created per-stream by AggregateFactory
*/
class {DB}Aggregate(
private val buffer: {DB}InsertBuffer,
) : Aggregate {
override fun accept(record: RecordDTO) {
buffer.accumulate(record.fields)
}
override suspend fun flush() {
buffer.flush()
}
}
```
**What this does:**
- Receives transformed records from CDK dataflow pipeline
- Delegates to InsertBuffer for accumulation
- Implements flush() for end-of-stream flushing
**Dataflow pipeline:**
```
Platform → JSONL records
AirbyteMessageDeserializer (CDK)
RecordTransformer (CDK, uses ColumnNameMapper from Phase 7)
RecordDTO (transformed record with mapped column names)
Aggregate.accept() ← YOUR CODE STARTS HERE
InsertBuffer.accumulate()
Database
```
### Step 8.4: Create AggregateFactory
**File:** `dataflow/{DB}AggregateFactory.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.dataflow
import io.airbyte.cdk.load.dataflow.aggregate.Aggregate
import io.airbyte.cdk.load.dataflow.aggregate.AggregateFactory
import io.airbyte.cdk.load.orchestration.db.DirectLoadTableExecutionConfig
import io.airbyte.cdk.load.state.StoreKey
import io.airbyte.cdk.load.state.StreamStateStore
import io.airbyte.integrations.destination.{db}.client.{DB}AirbyteClient
import io.airbyte.integrations.destination.{db}.write.load.{DB}InsertBuffer
import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Singleton
@Factory
class {DB}AggregateFactory(
private val client: {DB}AirbyteClient,
private val streamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>,
) : AggregateFactory {
@Singleton
override fun create(key: StoreKey): Aggregate {
// StreamStateStore contains execution config for each stream
// Config includes table name, column mapping, etc.
val tableName = streamStateStore.get(key)!!.tableName
val buffer = {DB}InsertBuffer(
tableName = tableName,
client = client,
)
return {DB}Aggregate(buffer)
}
}
```
**What this does:**
- @Factory class provides factory method for creating Aggregates
- create() called once per stream at start of sync
- StreamStateStore provides table name for the stream
- Creates InsertBuffer → Aggregate chain
**Why factory pattern?**
- Aggregate needs per-stream state (table name)
- Can't use constructor injection (dynamic stream list)
- Factory receives StoreKey, looks up stream config, creates Aggregate
### Step 8.5: Create Writer
**File:** `write/{DB}Writer.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.write
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.orchestration.db.*
import io.airbyte.cdk.load.state.StreamStateStore
import io.airbyte.cdk.load.table.TableCatalog
import io.airbyte.cdk.load.write.DestinationWriter
import io.airbyte.cdk.load.write.StreamLoader
import io.airbyte.integrations.destination.{db}.client.{DB}AirbyteClient
import io.micronaut.context.annotation.Singleton
@Singleton
class {DB}Writer(
private val names: TableCatalog,
private val stateGatherer: DatabaseInitialStatusGatherer<DirectLoadInitialStatus>,
private val streamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>,
private val client: {DB}AirbyteClient,
private val tempTableNameGenerator: TempTableNameGenerator,
) : DestinationWriter {
private lateinit var initialStatuses: Map<DestinationStream, DirectLoadInitialStatus>
override suspend fun setup() {
// Create all namespaces
names.values
.map { it.tableNames.finalTableName!!.namespace }
.toSet()
.forEach { client.createNamespace(it) }
// Gather initial state (which tables exist, generation IDs, etc.)
initialStatuses = stateGatherer.gatherInitialStatus(names)
}
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
// Defensive: Handle streams not in catalog (for test compatibility)
val initialStatus = if (::initialStatuses.isInitialized) {
initialStatuses[stream] ?: DirectLoadInitialStatus(null, null)
} else {
DirectLoadInitialStatus(null, null)
}
val tableNameInfo = names[stream]
val (realTableName, tempTableName, columnNameMapping) = if (tableNameInfo != null) {
// Stream in catalog - use configured names
Triple(
tableNameInfo.tableNames.finalTableName!!,
tempTableNameGenerator.generate(tableNameInfo.tableNames.finalTableName!!),
tableNameInfo.columnNameMapping
)
} else {
// Dynamic stream (test-generated) - use descriptor names directly
val tableName = TableName(
namespace = stream.mappedDescriptor.namespace ?: "test",
name = stream.mappedDescriptor.name
)
Triple(tableName, tempTableNameGenerator.generate(tableName), ColumnNameMapping(emptyMap()))
}
// Phase 8: Append mode only
// Phase 10: Add truncate mode (minimumGenerationId = generationId)
// Phase 13: Add dedupe mode (importType is Dedupe)
return DirectLoadTableAppendStreamLoader(
stream,
initialStatus,
realTableName,
tempTableName,
columnNameMapping,
client, // TableOperationsClient
client, // TableSchemaEvolutionClient
streamStateStore,
)
}
}
```
**What this does:**
- **setup()**: Creates namespaces, gathers initial table state
- **createStreamLoader()**: Creates StreamLoader for each stream
- AppendStreamLoader: Just insert records (this phase)
- TruncateStreamLoader: Overwrite table (Phase 10)
- DedupStreamLoader: Upsert with primary key (Phase 13)
**Defensive pattern (lines 27-52):**
- Handles ConnectorWiringSuite creating dynamic test streams
- Test streams not in TableCatalog → use descriptor names directly
- Prevents NullPointerException in tests
**StreamLoader responsibilities:**
- start(): Create/prepare table
- accept(): Add record to buffer
- complete(): Flush and finalize
**CDK provides implementations:**
- DirectLoadTableAppendStreamLoader
- DirectLoadTableAppendTruncateStreamLoader
- DirectLoadTableDedupStreamLoader
- DirectLoadTableDedupTruncateStreamLoader
### Step 8.6: Create ConnectorWiringSuite Test
**File:** `src/test-integration/kotlin/.../component/{DB}WiringTest.kt`
```kotlin
package io.airbyte.integrations.destination.{db}.component
import io.airbyte.cdk.load.component.ConnectorWiringSuite
import io.airbyte.cdk.load.component.TableOperationsClient
import io.airbyte.cdk.load.dataflow.aggregate.AggregateFactory
import io.airbyte.cdk.load.write.DestinationWriter
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
import org.junit.jupiter.api.Test
@MicronautTest(environments = ["component"])
class {DB}WiringTest(
override val writer: DestinationWriter,
override val client: TableOperationsClient,
override val aggregateFactory: AggregateFactory,
) : ConnectorWiringSuite {
// Optional: Override test namespace if different from "test"
// override val testNamespace = "my_database"
@Test
override fun `all beans are injectable`() {
super.`all beans are injectable`()
}
@Test
override fun `writer setup completes`() {
super.`writer setup completes`()
}
@Test
override fun `can create append stream loader`() {
super.`can create append stream loader`()
}
@Test
override fun `can write one record`() {
super.`can write one record`()
}
}
```
**What ConnectorWiringSuite does:**
**Test 1: `all beans are injectable`**
- Validates all DI beans exist
- Catches missing @Singleton annotations
- Catches circular dependencies
**Test 2: `writer setup completes`**
- Calls Writer.setup()
- Validates namespace creation works
- Catches database connection errors
**Test 3: `can create append stream loader`**
- Calls Writer.createStreamLoader()
- Validates StreamLoader instantiation
- Catches missing StreamLoader dependencies
**Test 4: `can write one record`** ← MOST IMPORTANT
- Creates test stream
- Calls StreamLoader.start() → creates table
- Calls Aggregate.accept() → buffers record
- Calls Aggregate.flush() → writes to database
- Validates record appears in database
- **END-TO-END validation of full write path!**
**Test context:**
- Uses MockDestinationCatalog (fast, no real catalog parsing)
- Uses Testcontainers database
- Component test (not integration test)
**Why MockDestinationCatalog?**
- Fast iteration (no catalog JSON parsing)
- Creates dynamic test streams
- Focuses on write logic, not catalog parsing
### Step 8.7: Validate ConnectorWiringSuite
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentAllBeansAreInjectable \
:destination-{db}:testComponentWriterSetupCompletes \
:destination-{db}:testComponentCanCreateAppendStreamLoader \
:destination-{db}:testComponentCanWriteOneRecord # 4 tests should pass
$ ./gradlew :destination-{db}:componentTest # 9 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass
```
**If `can write one record` FAILS:**
**DI errors:**
→ Check Phase 7 infrastructure (WriteOperationV2, DatabaseInitialStatusGatherer, ColumnNameMapper)
→ Check Phase 6 name generators all have @Singleton
**Table creation errors:**
→ Check TableOperationsClient.createTable() implementation (Phase 4)
→ Check SqlGenerator.createTable() SQL syntax
**Insert errors:**
→ Check InsertBuffer.insertRecord() implementation
→ Check client.executeInsert() and setParameter() logic
→ Check column name mapping
**Record not found in database:**
→ Check buffer.flush() is called
→ Check SQL INSERT statement is correct
→ Query database directly to debug
**Checkpoint:** First working sync + all previous phases still work
---
---
## Phase 9: Generation ID Support
**Goal:** Track sync generations for refresh handling
**Checkpoint:** Can retrieve generation IDs
**📋 What's a Generation ID?**
- Unique identifier for each sync run
- Used to distinguish "old data" from "new data" during refreshes
- Stored in `_airbyte_generation_id` column
**When used:**
- Full refresh: minimumGenerationId = generationId (replace all data)
- Incremental: minimumGenerationId = 0 (keep all data)
### Step 9.1: Enable Generation ID Test
**File:** Update `src/test-integration/kotlin/.../component/{DB}TableOperationsTest.kt`
```kotlin
@Test
override fun `get generation id`() {
super.`get generation id`()
}
```
**What this tests:**
- TableOperationsClient.getGenerationId() returns correct value
- Returns 0L for tables without generation ID
- Returns actual generation ID from `_airbyte_generation_id` column
### Step 9.2: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentGetGenerationId # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 10 tests should pass
```
**Checkpoint:** Generation ID tracking works + all previous phases still work
---
---
## Phase 10: Overwrite Mode
**Goal:** Support full refresh (replace all data)
**Checkpoint:** Can replace table contents atomically
**📋 How Overwrite Works:**
1. Write new data to temp table
2. Atomically swap temp table with final table
3. Drop old table
**Sync modes:**
- **Append** (Phase 8): INSERT into existing table
- **Overwrite** (Phase 10): SWAP temp table with final table
### Step 10.1: Implement overwriteTable() in SQL Generator
**File:** Update `client/{DB}SqlGenerator.kt`
```kotlin
fun overwriteTable(source: TableName, target: TableName): List<String> {
// Option 1: SWAP (Snowflake)
return listOf(
"ALTER TABLE ${fullyQualifiedName(target)} SWAP WITH ${fullyQualifiedName(source)}".andLog(),
"DROP TABLE IF EXISTS ${fullyQualifiedName(source)}".andLog(),
)
// Option 2: EXCHANGE (ClickHouse)
return listOf(
"EXCHANGE TABLES ${fullyQualifiedName(target)} AND ${fullyQualifiedName(source)}".andLog(),
"DROP TABLE IF EXISTS ${fullyQualifiedName(source)}".andLog(),
)
// Option 3: DROP + RENAME (fallback for most databases)
return listOf(
"DROP TABLE IF EXISTS ${fullyQualifiedName(target)}".andLog(),
"ALTER TABLE ${fullyQualifiedName(source)} RENAME TO ${target.name.quote()}".andLog(),
)
// Option 4: BEGIN TRANSACTION + DROP + RENAME + COMMIT (for ACID guarantees)
return listOf(
"BEGIN TRANSACTION".andLog(),
"DROP TABLE IF EXISTS ${fullyQualifiedName(target)}".andLog(),
"ALTER TABLE ${fullyQualifiedName(source)} RENAME TO ${target.name.quote()}".andLog(),
"COMMIT".andLog(),
)
}
```
**Database-specific notes:**
- **Snowflake**: SWAP is atomic and instant (metadata operation)
- **ClickHouse**: EXCHANGE is atomic
- **Postgres/MySQL**: DROP + RENAME requires transaction for atomicity
- **BigQuery**: CREATE OR REPLACE TABLE (different pattern)
### Step 10.2: Implement overwriteTable() in Client
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override suspend fun overwriteTable(
sourceTableName: TableName,
targetTableName: TableName
) {
val statements = sqlGenerator.overwriteTable(sourceTableName, targetTableName)
statements.forEach { execute(it) }
}
```
### Step 10.3: Update Writer for Truncate Mode
**File:** Update `write/{DB}Writer.kt`
```kotlin
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
// Defensive: Handle streams not in catalog (for test compatibility)
val initialStatus = if (::initialStatuses.isInitialized) {
initialStatuses[stream] ?: DirectLoadInitialStatus(null, null)
} else {
DirectLoadInitialStatus(null, null)
}
val tableNameInfo = names[stream]
val (realTableName, tempTableName, columnNameMapping) = if (tableNameInfo != null) {
Triple(
tableNameInfo.tableNames.finalTableName!!,
tempTableNameGenerator.generate(tableNameInfo.tableNames.finalTableName!!),
tableNameInfo.columnNameMapping
)
} else {
val tableName = TableName(
namespace = stream.mappedDescriptor.namespace ?: "test",
name = stream.mappedDescriptor.name
)
Triple(tableName, tempTableNameGenerator.generate(tableName), ColumnNameMapping(emptyMap()))
}
// Choose StreamLoader based on sync mode
return when (stream.minimumGenerationId) {
0L -> DirectLoadTableAppendStreamLoader(
stream, initialStatus, realTableName, tempTableName,
columnNameMapping, client, client, streamStateStore
)
stream.generationId -> DirectLoadTableAppendTruncateStreamLoader(
stream, initialStatus, realTableName, tempTableName,
columnNameMapping, client, client, streamStateStore
)
else -> throw SystemErrorException("Hybrid refresh not supported")
}
}
```
**What changed:**
- Added `when` statement to choose StreamLoader based on `minimumGenerationId`
- `minimumGenerationId = 0`: Append mode (keep old data)
- `minimumGenerationId = generationId`: Truncate mode (replace old data)
**StreamLoader behavior:**
- **AppendStreamLoader**: Writes directly to final table
- **AppendTruncateStreamLoader**: Writes to temp table, then swaps
### Step 10.4: Enable Tests
**File:** Update `src/test-integration/kotlin/.../component/{DB}TableOperationsTest.kt`
```kotlin
@Test
override fun `overwrite tables`() {
super.`overwrite tables`()
}
```
### Step 10.5: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentOverwriteTables # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 11 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass
```
**Checkpoint:** Full refresh mode works + all previous phases still work
---
---
## Phase 11: Copy Operation
**Goal:** Support table copying (used internally by some modes)
**Checkpoint:** Can copy data between tables
**📋 When Copy is Used:**
- Dedupe mode: Copy deduplicated data from temp to final
- Some overwrite implementations: Copy instead of swap
- Schema evolution: Copy to new schema
### Step 11.1: Implement copyTable() in SQL Generator
**File:** Update `client/{DB}SqlGenerator.kt`
```kotlin
fun copyTable(
columnMapping: ColumnNameMapping,
source: TableName,
target: TableName
): String {
val columnList = columnMapping.values.joinToString(", ") { "\"$it\"" }
return """
INSERT INTO ${fullyQualifiedName(target)} ($columnList)
SELECT $columnList
FROM ${fullyQualifiedName(source)}
""".trimIndent().andLog()
}
```
**What this does:**
- Copies all rows from source to target
- Only copies mapped columns (not all columns)
- Preserves data types (SELECT → INSERT)
**Alternative: Include Airbyte metadata columns explicitly:**
```kotlin
fun copyTable(
columnMapping: ColumnNameMapping,
source: TableName,
target: TableName
): String {
// Include Airbyte metadata + user columns
val allColumns = listOf(
"_airbyte_raw_id",
"_airbyte_extracted_at",
"_airbyte_meta",
"_airbyte_generation_id"
) + columnMapping.values
val columnList = allColumns.joinToString(", ") { "\"$it\"" }
return """
INSERT INTO ${fullyQualifiedName(target)} ($columnList)
SELECT $columnList
FROM ${fullyQualifiedName(source)}
""".trimIndent().andLog()
}
```
### Step 11.2: Implement copyTable() in Client
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override suspend fun copyTable(
columnNameMapping: ColumnNameMapping,
sourceTableName: TableName,
targetTableName: TableName
) {
execute(sqlGenerator.copyTable(columnNameMapping, sourceTableName, targetTableName))
}
```
### Step 11.3: Enable Test
**File:** Update `src/test-integration/kotlin/.../component/{DB}TableOperationsTest.kt`
```kotlin
@Test
override fun `copy tables`() {
super.`copy tables`()
}
```
### Step 11.4: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentCopyTables # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 12 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass
```
**Checkpoint:** Copy operation works + all previous phases still work
---
---
## Next Steps
**Next:** Your connector now works for basic use cases! Continue to [5-advanced-features.md](./5-advanced-features.md) for production-ready features, or jump to [6-testing.md](./6-testing.md) to run the full test suite.

View File

@@ -0,0 +1,756 @@
# Advanced Features: Production-Ready Connector
**Summary:** Add enterprise features to your connector: schema evolution, dedupe mode with primary keys, CDC support for deletions, and performance optimization. After completing this guide, your connector will be production-ready.
---
## Prerequisites
**Complete:** [4-write-operations.md](./4-write-operations.md) - Your connector must have working append and overwrite modes.
---
## What You'll Build
After completing this guide, you'll have a production-ready connector with:
- **Schema evolution:** Automatic column add/drop/modify when source schema changes
- **Dedupe mode:** MERGE with primary key for incremental syncs
- **CDC support:** Handle hard/soft deletes from change data capture sources
- **Performance optimization:** Efficient batch writes for large datasets
- **Production readiness:** Suitable for enterprise use
---
**Goal:** Automatically adapt to schema changes
**Checkpoint:** Can add, drop, and modify columns
**📋 Schema Evolution Scenarios:**
1. **Add column**: Source adds new field → add column to destination
2. **Drop column**: Source removes field → drop column from destination (optional)
3. **Change type**: Source changes field type → alter column (with casting)
4. **Change nullability**: Source changes nullable → alter column constraints
### Step 12.1: Implement discoverSchema()
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override suspend fun discoverSchema(tableName: TableName): TableSchema {
val columns = getColumnsFromDb(tableName)
return TableSchema(columns)
}
private fun getColumnsFromDb(tableName: TableName): Map<String, ColumnType> {
val columns = mutableMapOf<String, ColumnType>()
dataSource.connection.use { connection ->
// Postgres/MySQL: Query information_schema.columns
val sql = """
SELECT column_name, data_type, is_nullable
FROM information_schema.columns
WHERE table_schema = '${tableName.namespace}'
AND table_name = '${tableName.name}'
"""
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sql)
while (rs.next()) {
val columnName = rs.getString("column_name")
// Skip Airbyte metadata columns
if (columnName in AIRBYTE_META_COLUMNS) continue
val dataType = rs.getString("data_type")
.takeWhile { it != '(' } // Strip precision (e.g., VARCHAR(255) → VARCHAR)
val nullable = rs.getString("is_nullable") == "YES"
columns[columnName] = ColumnType(dataType, nullable)
}
}
}
return columns
}
private val AIRBYTE_META_COLUMNS = setOf(
"_airbyte_raw_id",
"_airbyte_extracted_at",
"_airbyte_meta",
"_airbyte_generation_id"
)
```
**Database-specific approaches:**
- **Snowflake**: `DESCRIBE TABLE` or `SHOW COLUMNS`
- **Postgres/MySQL**: `information_schema.columns`
- **ClickHouse**: `system.columns` or client API `getTableSchema()`
- **BigQuery**: `INFORMATION_SCHEMA.COLUMNS`
### Step 12.2: Implement computeSchema()
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override fun computeSchema(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping
): TableSchema {
val columns = stream.schema.asColumns()
.filter { (name, _) -> name !in AIRBYTE_META_COLUMNS }
.mapKeys { (name, _) -> columnNameMapping[name]!! }
.mapValues { (_, field) ->
val dbType = columnUtils.toDialectType(field.type)
.takeWhile { it != '(' } // Strip precision
ColumnType(dbType, field.nullable)
}
return TableSchema(columns)
}
```
**What this does:**
- Converts Airbyte schema → database schema
- Applies column name mapping (Phase 6 generators)
- Uses ColumnUtils.toDialectType() from Phase 4
### Step 12.3: Implement alterTable() - ADD COLUMN
**File:** Update `client/{DB}SqlGenerator.kt`
```kotlin
fun alterTable(
tableName: TableName,
columnsToAdd: Map<String, ColumnType>,
columnsToDrop: Map<String, ColumnType>,
columnsToChange: Map<String, ColumnTypeChange>,
): Set<String> {
val statements = mutableSetOf<String>()
// ADD COLUMN (simplest - implement first)
columnsToAdd.forEach { (name, type) ->
val nullableClause = if (type.nullable) "" else " NOT NULL"
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} ADD COLUMN \"$name\" ${type.type}$nullableClause".andLog()
)
}
// DROP COLUMN (implement second)
columnsToDrop.forEach { (name, _) ->
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} DROP COLUMN \"$name\"".andLog()
)
}
// MODIFY COLUMN (implement last - most complex)
columnsToChange.forEach { (name, typeChange) ->
// See Step 12.4 for implementation
}
return statements
}
```
### Step 12.4: Implement alterTable() - MODIFY COLUMN
**Add to alterTable():**
```kotlin
columnsToChange.forEach { (name, typeChange) ->
when {
// Safe: NOT NULL → NULL (widen constraint)
!typeChange.originalType.nullable && typeChange.newType.nullable -> {
// Postgres/MySQL
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} ALTER COLUMN \"$name\" DROP NOT NULL".andLog()
)
// Or Snowflake
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} MODIFY COLUMN \"$name\" DROP NOT NULL".andLog()
)
}
// Unsafe: Type change - use temp column approach
typeChange.originalType.type != typeChange.newType.type -> {
val tempColumn = "${name}_${UUID.randomUUID().toString().replace("-", "").take(8)}"
val backupColumn = "${name}_backup"
// 1. Add temp column with new type
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} ADD COLUMN \"$tempColumn\" ${typeChange.newType.type}".andLog()
)
// 2. Cast and copy data
statements.add(
"UPDATE ${fullyQualifiedName(tableName)} SET \"$tempColumn\" = CAST(\"$name\" AS ${typeChange.newType.type})".andLog()
)
// 3. Rename original to backup
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} RENAME COLUMN \"$name\" TO \"$backupColumn\"".andLog()
)
// 4. Rename temp to original
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} RENAME COLUMN \"$tempColumn\" TO \"$name\"".andLog()
)
// 5. Drop backup
statements.add(
"ALTER TABLE ${fullyQualifiedName(tableName)} DROP COLUMN \"$backupColumn\"".andLog()
)
}
// Unsafe: NULL → NOT NULL (skip - may fail with existing NULLs)
typeChange.originalType.nullable && !typeChange.newType.nullable -> {
log.info { "Skipping change from nullable to non-nullable for column $name (may have NULL values)" }
}
// No change needed
else -> {
log.debug { "No schema change needed for column $name" }
}
}
}
```
**Alternative: Table Recreation (ClickHouse pattern):**
For databases where ALTER is expensive or impossible (e.g., changing primary key):
```kotlin
fun recreateTable(
stream: DestinationStream,
oldTableName: TableName,
newTableName: TableName,
columnMapping: ColumnNameMapping
): List<String> {
return listOf(
// 1. Create new table with new schema
createTable(stream, newTableName, columnMapping, replace = false),
// 2. Copy data (with type casting)
copyTable(columnMapping, oldTableName, newTableName),
// 3. Drop old table
dropTable(oldTableName),
// 4. Rename new table to old name
"ALTER TABLE ${fullyQualifiedName(newTableName)} RENAME TO ${oldTableName.name.quote()}".andLog()
)
}
```
### Step 12.5: Implement applyChangeset()
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override suspend fun applyChangeset(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
tableName: TableName,
expectedColumns: TableColumns,
columnChangeset: ColumnChangeset,
) {
if (columnChangeset.isNoop()) return
log.info { "Summary of table alterations for ${tableName}:" }
log.info { " Added columns: ${columnChangeset.columnsToAdd.keys}" }
log.info { " Dropped columns: ${columnChangeset.columnsToDrop.keys}" }
log.info { " Modified columns: ${columnChangeset.columnsToChange.keys}" }
val statements = sqlGenerator.alterTable(
tableName,
columnChangeset.columnsToAdd,
columnChangeset.columnsToDrop,
columnChangeset.columnsToChange,
)
statements.forEach { execute(it) }
}
```
### Step 12.6: Implement ensureSchemaMatches()
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override suspend fun ensureSchemaMatches(
stream: DestinationStream,
tableName: TableName,
columnNameMapping: ColumnNameMapping
) {
val actualSchema = discoverSchema(tableName)
val expectedSchema = computeSchema(stream, columnNameMapping)
val changeset = actualSchema.diff(expectedSchema)
if (!changeset.isNoop()) {
log.info { "Schema mismatch detected for ${tableName}, applying changes" }
applyChangeset(stream, columnNameMapping, tableName, expectedSchema.columns, changeset)
} else {
log.debug { "Schema matches for ${tableName}, no changes needed" }
}
}
```
**When is this called?**
- StreamLoader.start() calls ensureSchemaMatches() before writing
- If source schema changed since last sync, applies schema changes
- Automatic - no user intervention needed
### Step 12.7: Validate Schema Evolution
**Validate:**
```bash
$ ./gradlew :destination-{db}:componentTest # 12 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass
```
**Checkpoint:** Schema evolution works + all previous phases still work
---
---
## Phase 13: Dedupe Mode
**Goal:** Support primary key deduplication
**Checkpoint:** Can deduplicate by primary key with "last write wins"
**📋 Dedupe Strategy:**
1. Write all records to temp table
2. Deduplicate in temp table (ROW_NUMBER() by primary key, ordered by cursor DESC)
3. Upsert from temp to final:
- Match on primary key
- Update if cursor is newer
- Insert if no match
**Sync modes:**
- **Append** (Phase 8): Just insert
- **Overwrite** (Phase 10): Swap tables
- **Dedupe** (Phase 13): Upsert with primary key
### Step 13.1: Implement upsertTable() in SQL Generator
**Option A: MERGE Statement (Snowflake, SQL Server, BigQuery)**
**File:** Update `client/{DB}SqlGenerator.kt`
```kotlin
fun upsertTable(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
source: TableName,
target: TableName
): String {
val importType = stream.importType as Dedupe
val pkColumns = importType.primaryKey.map { columnNameMapping[it]!! }
val allColumns = columnNameMapping.values.toList()
// Cursor column (for "last write wins" logic)
val cursorColumn = if (importType.cursor.isNotEmpty()) {
columnNameMapping[importType.cursor.first()]!!
} else {
"_airbyte_extracted_at" // Fallback to extraction timestamp
}
// Deduplication CTE (keep latest record per primary key)
val dedupCte = if (pkColumns.isNotEmpty()) {
"""
WITH deduped AS (
SELECT *, ROW_NUMBER() OVER (
PARTITION BY ${pkColumns.joinToString(", ") { "\"$it\"" }}
ORDER BY "$cursorColumn" DESC, "_airbyte_extracted_at" DESC
) AS rn
FROM ${fullyQualifiedName(source)}
)
SELECT * FROM deduped WHERE rn = 1
"""
} else {
// No primary key - just dedupe by all columns (inefficient but safe)
"SELECT * FROM ${fullyQualifiedName(source)}"
}
// Primary key match condition (handles NULLs)
val pkMatch = pkColumns.joinToString(" AND ") { col ->
"""(target."$col" = source."$col" OR (target."$col" IS NULL AND source."$col" IS NULL))"""
}
// Cursor comparison (for UPDATE condition)
val cursorComparison = """
(
target."$cursorColumn" < source."$cursorColumn"
OR (target."$cursorColumn" = source."$cursorColumn" AND target."_airbyte_extracted_at" < source."_airbyte_extracted_at")
OR (target."$cursorColumn" IS NULL AND source."$cursorColumn" IS NOT NULL)
)
""".trimIndent()
// Column assignments for UPDATE
val columnAssignments = allColumns.joinToString(",\n ") { col ->
"\"$col\" = source.\"$col\""
}
// Column list for INSERT
val columnList = allColumns.joinToString(", ") { "\"$it\"" }
val sourceColumnList = allColumns.joinToString(", ") { "source.\"$it\"" }
return """
MERGE INTO ${fullyQualifiedName(target)} AS target
USING (
$dedupCte
) AS source
ON $pkMatch
WHEN MATCHED AND $cursorComparison THEN UPDATE SET
$columnAssignments
WHEN NOT MATCHED THEN INSERT (
$columnList
) VALUES (
$sourceColumnList
)
""".trimIndent().andLog()
}
```
**Option B: INSERT ... ON CONFLICT (Postgres, SQLite)**
```kotlin
fun upsertTable(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
source: TableName,
target: TableName
): List<String> {
val importType = stream.importType as Dedupe
val pkColumns = importType.primaryKey.map { columnNameMapping[it]!! }
val allColumns = columnNameMapping.values.toList()
val cursorColumn = if (importType.cursor.isNotEmpty()) {
columnNameMapping[importType.cursor.first()]!!
} else {
"_airbyte_extracted_at"
}
val columnList = allColumns.joinToString(", ") { "\"$it\"" }
val updateAssignments = allColumns
.filter { it !in pkColumns } // Don't update PK columns
.joinToString(", ") { "\"$it\" = EXCLUDED.\"$it\"" }
// 1. Dedupe in temp table first
val dedupSql = """
CREATE TEMP TABLE deduped AS
SELECT * FROM (
SELECT *, ROW_NUMBER() OVER (
PARTITION BY ${pkColumns.joinToString(", ") { "\"$it\"" }}
ORDER BY "$cursorColumn" DESC, "_airbyte_extracted_at" DESC
) AS rn
FROM ${fullyQualifiedName(source)}
) WHERE rn = 1
""".trimIndent().andLog()
// 2. Upsert with cursor check
val upsertSql = """
INSERT INTO ${fullyQualifiedName(target)} ($columnList)
SELECT $columnList FROM deduped
ON CONFLICT (${pkColumns.joinToString(", ") { "\"$it\"" }})
DO UPDATE SET $updateAssignments
WHERE ${fullyQualifiedName(target)}."$cursorColumn" < EXCLUDED."$cursorColumn"
OR (${fullyQualifiedName(target)}."$cursorColumn" = EXCLUDED."$cursorColumn"
AND ${fullyQualifiedName(target)}."_airbyte_extracted_at" < EXCLUDED."_airbyte_extracted_at")
""".trimIndent().andLog()
// 3. Cleanup
val cleanupSql = "DROP TABLE deduped".andLog()
return listOf(dedupSql, upsertSql, cleanupSql)
}
```
**Option C: Separate DELETE + INSERT (fallback)**
```kotlin
fun upsertTable(...): List<String> {
val pkColumns = (stream.importType as Dedupe).primaryKey.map { columnNameMapping[it]!! }
val allColumns = columnNameMapping.values.toList()
// 1. Dedupe (same as above)
val dedupSql = "..."
// 2. Delete existing records with matching PKs
val deleteSql = """
DELETE FROM ${fullyQualifiedName(target)}
WHERE (${pkColumns.joinToString(", ") { "\"$it\"" }})
IN (SELECT ${pkColumns.joinToString(", ") { "\"$it\"" }} FROM deduped)
""".trimIndent().andLog()
// 3. Insert all from deduped
val insertSql = """
INSERT INTO ${fullyQualifiedName(target)}
SELECT ${allColumns.joinToString(", ") { "\"$it\"" }} FROM deduped
""".trimIndent().andLog()
// 4. Cleanup
val cleanupSql = "DROP TABLE deduped".andLog()
return listOf(dedupSql, deleteSql, insertSql, cleanupSql)
}
```
### Step 13.2: Implement upsertTable() in Client
**File:** Update `client/{DB}AirbyteClient.kt`
```kotlin
override suspend fun upsertTable(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
sourceTableName: TableName,
targetTableName: TableName
) {
val sql = sqlGenerator.upsertTable(stream, columnNameMapping, sourceTableName, targetTableName)
// Single statement (MERGE)
if (sql is String) {
execute(sql)
} else {
// Multiple statements (INSERT ON CONFLICT, DELETE+INSERT)
sql.forEach { execute(it) }
}
}
```
### Step 13.3: Update Writer for Dedupe Mode
**File:** Update `write/{DB}Writer.kt`
```kotlin
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
val initialStatus = if (::initialStatuses.isInitialized) {
initialStatuses[stream] ?: DirectLoadInitialStatus(null, null)
} else {
DirectLoadInitialStatus(null, null)
}
val tableNameInfo = names[stream]
val (realTableName, tempTableName, columnNameMapping) = if (tableNameInfo != null) {
Triple(
tableNameInfo.tableNames.finalTableName!!,
tempTableNameGenerator.generate(tableNameInfo.tableNames.finalTableName!!),
tableNameInfo.columnNameMapping
)
} else {
val tableName = TableName(
namespace = stream.mappedDescriptor.namespace ?: "test",
name = stream.mappedDescriptor.name
)
Triple(tableName, tempTableNameGenerator.generate(tableName), ColumnNameMapping(emptyMap()))
}
// Choose StreamLoader based on sync mode and import type
return when (stream.minimumGenerationId) {
0L -> when (stream.importType) {
is Dedupe -> DirectLoadTableDedupStreamLoader(
stream, initialStatus, realTableName, tempTableName,
columnNameMapping, client, client, streamStateStore
)
else -> DirectLoadTableAppendStreamLoader(
stream, initialStatus, realTableName, tempTableName,
columnNameMapping, client, client, streamStateStore
)
}
stream.generationId -> when (stream.importType) {
is Dedupe -> DirectLoadTableDedupTruncateStreamLoader(
stream, initialStatus, realTableName, tempTableName,
columnNameMapping, client, client, streamStateStore, tempTableNameGenerator
)
else -> DirectLoadTableAppendTruncateStreamLoader(
stream, initialStatus, realTableName, tempTableName,
columnNameMapping, client, client, streamStateStore
)
}
else -> throw SystemErrorException("Hybrid refresh not supported")
}
}
```
**What changed:**
- Added `when (stream.importType)` check inside generation ID check
- Four StreamLoader types now supported:
- DirectLoadTableAppendStreamLoader (incremental append)
- DirectLoadTableAppendTruncateStreamLoader (full refresh overwrite)
- DirectLoadTableDedupStreamLoader (incremental dedupe)
- DirectLoadTableDedupTruncateStreamLoader (full refresh dedupe)
### Step 13.4: Enable Tests
**File:** Update `src/test-integration/kotlin/.../component/{DB}TableOperationsTest.kt`
```kotlin
@Test
override fun `upsert tables`() {
super.`upsert tables`()
}
```
### Step 13.5: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:testComponentUpsertTables # 1 test should pass
$ ./gradlew :destination-{db}:componentTest # 13 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass
```
**Checkpoint:** Dedupe mode works + all previous phases still work
---
---
## Phase 14: CDC Support (Optional)
**Goal:** Handle source deletions
**Checkpoint:** Can process CDC deletion events
**📋 CDC (Change Data Capture):**
- Tracks INSERT, UPDATE, DELETE operations from source
- Deletion marked with `_ab_cdc_deleted_at` timestamp
- Two modes:
- **Hard delete**: Remove record from destination
- **Soft delete**: Keep record with deletion timestamp
### Step 14.1: Add CDC Configuration
**File:** Update `spec/{DB}Specification.kt`
```kotlin
@get:JsonProperty("cdc_deletion_mode")
@get:JsonPropertyDescription(
"""Whether to execute CDC deletions as hard deletes (propagate source deletions)
or soft deletes (leave tombstone record). Defaults to hard deletes."""
)
val cdcDeletionMode: CdcDeletionMode? = null
enum class CdcDeletionMode(@get:JsonValue val value: String) {
@JsonProperty("hard_delete") HARD_DELETE("Hard delete"),
@JsonProperty("soft_delete") SOFT_DELETE("Soft delete"),
}
```
**File:** Update `spec/{DB}Configuration.kt`
```kotlin
data class {DB}Configuration(
// ... existing fields
val cdcDeletionMode: CdcDeletionMode,
)
// In ConfigurationFactory
override fun makeWithoutExceptionHandling(pojo: {DB}Specification): {DB}Configuration {
return {DB}Configuration(
// ... existing fields
cdcDeletionMode = pojo.cdcDeletionMode ?: CdcDeletionMode.HARD_DELETE,
)
}
```
### Step 14.2: Add CDC Logic to upsertTable()
**File:** Update `client/{DB}SqlGenerator.kt`
```kotlin
fun upsertTable(...): String {
val importType = stream.importType as Dedupe
val pkColumns = importType.primaryKey.map { columnNameMapping[it]!! }
val allColumns = columnNameMapping.values.toList()
val hasCdc = stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN)
val isHardDelete = config.cdcDeletionMode == CdcDeletionMode.HARD_DELETE
val cursorColumn = if (importType.cursor.isNotEmpty()) {
columnNameMapping[importType.cursor.first()]!!
} else {
"_airbyte_extracted_at"
}
// Deduplication CTE (same as before)
val dedupCte = """..."""
val pkMatch = """..."""
val cursorComparison = """..."""
// CDC DELETE clause (must come BEFORE UPDATE)
val cdcDeleteClause = if (hasCdc && isHardDelete) {
"""
WHEN MATCHED AND source."_ab_cdc_deleted_at" IS NOT NULL
AND $cursorComparison THEN DELETE
"""
} else {
""
}
// Skip INSERT for deleted records (hard delete mode)
val cdcSkipInsertClause = if (hasCdc && isHardDelete) {
"AND source.\"_ab_cdc_deleted_at\" IS NULL"
} else {
""
}
val columnAssignments = allColumns.joinToString(",\n ") { "\"$it\" = source.\"$it\"" }
val columnList = allColumns.joinToString(", ") { "\"$it\"" }
val sourceColumnList = allColumns.joinToString(", ") { "source.\"$it\"" }
return """
MERGE INTO ${fullyQualifiedName(target)} AS target
USING (
$dedupCte
) AS source
ON $pkMatch
$cdcDeleteClause
WHEN MATCHED AND $cursorComparison THEN UPDATE SET $columnAssignments
WHEN NOT MATCHED $cdcSkipInsertClause THEN INSERT ($columnList) VALUES ($sourceColumnList)
""".trimIndent().andLog()
}
private val CDC_DELETED_AT_COLUMN = "_ab_cdc_deleted_at"
```
**Key Points:**
- DELETE clause must come **before** UPDATE (SQL execution order)
- Must check cursor (only delete if deletion is newer than existing record)
- Skip INSERT for deleted records (don't re-insert deleted rows)
- Soft delete: No special clauses (just upsert the deletion record with timestamp)
### Step 14.3: Test CDC
CDC tests are typically included in integration tests automatically if you have CDC streams configured. No separate test enablement needed - the framework tests CDC if the stream has `_ab_cdc_deleted_at` column.
### Step 14.4: Validate
**Validate:**
```bash
$ ./gradlew :destination-{db}:componentTest # 13 tests should pass
$ ./gradlew :destination-{db}:integrationTest # 3 tests should pass (CDC tested automatically if applicable)
```
**Checkpoint:** Full CDC support + all previous phases still work
---
---
## Phase 15: Optimization & Polish
**Goal:** Production-ready performance
---
## Next Steps
**Next:** Your connector is now production-ready! Continue to [6-testing.md](./6-testing.md) to run BasicFunctionalityIntegrationTest and validate all features.

View File

@@ -0,0 +1,750 @@
# BasicFunctionalityIntegrationTest Implementation Guide
**Summary:** Comprehensive guide for implementing the full CDK integration test suite. This test validates edge cases, type handling, schema evolution, and CDC support. Required for production certification.
**When to use this:** After Phase 8 (working connector with ConnectorWiringSuite passing)
**Time estimate:** 4-8 hours for complete implementation
---
## What BasicFunctionalityIntegrationTest Validates
**Comprehensive test coverage (50+ scenarios):**
### Data Type Handling
- All Airbyte types (string, integer, number, boolean, date, time, timestamp)
- Nested objects and arrays
- Union types (multiple possible types for one field)
- Unknown types (unrecognized JSON schema types)
- Null values vs unset fields
- Large integers/decimals (precision handling)
### Sync Modes
- `testAppend()` - Incremental append without deduplication
- `testDedupe()` - Incremental append with primary key deduplication
- `testTruncate()` - Full refresh (replace all data)
- `testAppendSchemaEvolution()` - Schema changes during append
### Schema Evolution
- Add column
- Drop column
- Change column type (widening)
- Nullable to non-nullable changes
### CDC Support (if enabled)
- Hard delete (actually remove records)
- Soft delete (tombstone records)
- Delete non-existent records
- Insert + delete in same sync
### Edge Cases
- Empty syncs
- Very large datasets
- Concurrent streams
- State checkpointing
- Error recovery
---
## Prerequisites
Before starting, you must have:
- ✅ Phase 8 complete (ConnectorWiringSuite passing)
- ✅ Phase 13 complete (if testing dedupe mode)
- ✅ Working database connection (Testcontainers or real DB)
- ✅ All sync modes implemented
---
## Step 1: Implement Test Helper Classes
### Step 1.1: Create DestinationDataDumper
**Purpose:** Read data from database for test verification
**File:** `src/test-integration/kotlin/.../{DB}DataDumper.kt`
```kotlin
package io.airbyte.integrations.destination.{db}
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.data.*
import io.airbyte.cdk.load.test.util.OutputRecord
import io.airbyte.cdk.load.test.util.destination.DestinationDataDumper
import javax.sql.DataSource
class {DB}DataDumper(
private val dataSource: DataSource,
) : DestinationDataDumper {
override fun dumpRecords(stream: DestinationStream): List<OutputRecord> {
val tableName = stream.descriptor.name // Or use name generator
val namespace = stream.descriptor.namespace ?: "test"
val records = mutableListOf<OutputRecord>()
dataSource.connection.use { connection ->
val sql = "SELECT * FROM \"$namespace\".\"$tableName\""
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sql)
val metadata = rs.metaData
while (rs.next()) {
val data = mutableMapOf<String, AirbyteValue>()
for (i in 1..metadata.columnCount) {
val columnName = metadata.getColumnName(i)
val value = rs.getObject(i)
// Convert database value to AirbyteValue
data[columnName] = when {
value == null -> NullValue
value is String -> StringValue(value)
value is Int -> IntegerValue(value.toLong())
value is Long -> IntegerValue(value)
value is Boolean -> BooleanValue(value)
value is java.math.BigDecimal -> NumberValue(value)
value is java.sql.Timestamp -> TimestampWithTimezoneValue(value.toInstant().toString())
value is java.sql.Date -> DateValue(value.toLocalDate().toString())
// Add more type conversions as needed
else -> StringValue(value.toString())
}
}
// Extract Airbyte metadata columns
val extractedAt = (data["_airbyte_extracted_at"] as? TimestampWithTimezoneValue)?.value?.toLong() ?: 0L
val generationId = (data["_airbyte_generation_id"] as? IntegerValue)?.value?.toLong() ?: 0L
val meta = data["_airbyte_meta"] // ObjectValue with errors/changes
records.add(
OutputRecord(
extractedAt = extractedAt,
generationId = generationId,
data = data.filterKeys { !it.startsWith("_airbyte") },
airbyteMeta = parseAirbyteMeta(meta)
)
)
}
}
}
return records
}
private fun parseAirbyteMeta(meta: AirbyteValue?): OutputRecord.Meta {
// Parse _airbyte_meta JSON to OutputRecord.Meta
// For now, simple implementation:
return OutputRecord.Meta(syncId = 0)
}
}
```
**What this does:**
- Queries database table for a stream
- Converts database types back to AirbyteValue
- Extracts Airbyte metadata columns
- Returns OutputRecord list for test assertions
### Step 1.2: Create DestinationCleaner
**Purpose:** Clean up test data between test runs
**File:** `src/test-integration/kotlin/.../{DB}Cleaner.kt`
```kotlin
package io.airbyte.integrations.destination.{db}
import io.airbyte.cdk.load.test.util.destination.DestinationCleaner
import javax.sql.DataSource
class {DB}Cleaner(
private val dataSource: DataSource,
private val testNamespace: String = "test",
) : DestinationCleaner {
override fun cleanup() {
dataSource.connection.use { connection ->
// Drop all test tables
val sql = """
SELECT table_name
FROM information_schema.tables
WHERE table_schema = '$testNamespace'
"""
connection.createStatement().use { statement ->
val rs = statement.executeQuery(sql)
val tablesToDrop = mutableListOf<String>()
while (rs.next()) {
tablesToDrop.add(rs.getString("table_name"))
}
// Drop each table
tablesToDrop.forEach { tableName ->
try {
statement.execute("DROP TABLE IF EXISTS \"$testNamespace\".\"$tableName\" CASCADE")
} catch (e: Exception) {
// Ignore errors during cleanup
}
}
}
// Optionally drop test namespace
try {
connection.createStatement().use {
it.execute("DROP SCHEMA IF EXISTS \"$testNamespace\" CASCADE")
}
} catch (e: Exception) {
// Ignore
}
}
}
}
```
**What this does:**
- Finds all tables in test namespace
- Drops them to clean up between tests
- Runs once per test suite (not per test)
---
## Step 2: Create BasicFunctionalityIntegrationTest Class
### Step 2.1: Understand Required Parameters
BasicFunctionalityIntegrationTest has **14 required constructor parameters** (15 for dataflow CDK):
| Parameter | Type | Purpose | Common Value |
|-----------|------|---------|--------------|
| `configContents` | String | Database config JSON | Load from secrets/config.json |
| `configSpecClass` | Class<T> | Specification class | `{DB}Specification::class.java` |
| `dataDumper` | DestinationDataDumper | Read data for verification | `{DB}DataDumper(dataSource)` |
| `destinationCleaner` | DestinationCleaner | Clean between tests | `{DB}Cleaner(dataSource)` |
| `isStreamSchemaRetroactive` | Boolean | Schema changes apply retroactively | `true` (usually) |
| `dedupBehavior` | DedupBehavior? | CDC deletion mode | `DedupBehavior(CdcDeletionMode.HARD_DELETE)` |
| `stringifySchemalessObjects` | Boolean | Convert objects without schema to strings | `false` |
| `schematizedObjectBehavior` | SchematizedNestedValueBehavior | How to handle nested objects | `PASS_THROUGH` or `STRINGIFY` |
| `schematizedArrayBehavior` | SchematizedNestedValueBehavior | How to handle nested arrays | `STRINGIFY` (usually) |
| `unionBehavior` | UnionBehavior | How to handle union types | `STRINGIFY` or `PROMOTE_TO_OBJECT` |
| `supportFileTransfer` | Boolean | Supports file uploads | `false` (for databases) |
| `commitDataIncrementally` | Boolean | Commit during sync vs at end | `true` |
| `allTypesBehavior` | AllTypesBehavior | Type handling configuration | `StronglyTyped(...)` |
| `unknownTypesBehavior` | UnknownTypesBehavior | Unknown type handling | `PASS_THROUGH` |
| `nullEqualsUnset` | Boolean | Null same as missing field | `true` |
| **`useDataFlowPipeline`** | **Boolean** | **Use dataflow CDK architecture** | **`true`** ⭐ **REQUIRED for dataflow CDK** |
### Step 2.2: Create Test Class
**File:** `src/test-integration/kotlin/.../{DB}BasicFunctionalityTest.kt`
```kotlin
package io.airbyte.integrations.destination.{db}
import io.airbyte.cdk.load.test.util.destination.DestinationCleaner
import io.airbyte.cdk.load.test.util.destination.DestinationDataDumper
import io.airbyte.cdk.load.write.AllTypesBehavior
import io.airbyte.cdk.load.write.BasicFunctionalityIntegrationTest
import io.airbyte.cdk.load.write.DedupBehavior
import io.airbyte.cdk.load.write.SchematizedNestedValueBehavior
import io.airbyte.cdk.load.write.UnionBehavior
import io.airbyte.cdk.load.write.UnknownTypesBehavior
import io.airbyte.integrations.destination.{db}.spec.{DB}Specification
import java.nio.file.Path
import javax.sql.DataSource
import org.junit.jupiter.api.BeforeAll
class {DB}BasicFunctionalityTest : BasicFunctionalityIntegrationTest(
configContents = Path.of("secrets/config.json").toFile().readText(),
configSpecClass = {DB}Specification::class.java,
dataDumper = createDataDumper(),
destinationCleaner = createCleaner(),
// Schema behavior
isStreamSchemaRetroactive = true,
// CDC deletion mode
dedupBehavior = DedupBehavior(DedupBehavior.CdcDeletionMode.HARD_DELETE),
// Type handling
stringifySchemalessObjects = false,
schematizedObjectBehavior = SchematizedNestedValueBehavior.PASS_THROUGH,
schematizedArrayBehavior = SchematizedNestedValueBehavior.STRINGIFY,
unionBehavior = UnionBehavior.STRINGIFY,
// Feature support
supportFileTransfer = false, // Database destinations don't transfer files
commitDataIncrementally = true,
// Type system behavior
allTypesBehavior = AllTypesBehavior.StronglyTyped(
integerCanBeLarge = false, // true if your DB has unlimited integers
numberCanBeLarge = false, // true if your DB has unlimited precision
nestedFloatLosesPrecision = false,
),
unknownTypesBehavior = UnknownTypesBehavior.PASS_THROUGH,
nullEqualsUnset = true,
// Dataflow CDK architecture (REQUIRED for new CDK)
useDataFlowPipeline = true, // ⚠️ Must be true for dataflow CDK connectors
) {
companion object {
private lateinit var testDataSource: DataSource
@JvmStatic
@BeforeAll
fun beforeAll() {
// Set up test database (Testcontainers or real DB)
testDataSource = createTestDataSource()
}
private fun createDataDumper(): DestinationDataDumper {
return {DB}DataDumper(testDataSource)
}
private fun createCleaner(): DestinationCleaner {
return {DB}Cleaner(testDataSource)
}
private fun createTestDataSource(): DataSource {
// Initialize Testcontainers or connection pool
val container = {DB}Container("{db}:latest")
container.start()
return HikariDataSource().apply {
jdbcUrl = container.jdbcUrl
username = container.username
password = container.password
}
}
}
// Test methods - uncomment as you implement features
@Test
override fun testAppend() {
super.testAppend()
}
@Test
override fun testTruncate() {
super.testTruncate()
}
@Test
override fun testAppendSchemaEvolution() {
super.testAppendSchemaEvolution()
}
@Test
override fun testDedupe() {
super.testDedupe()
}
}
```
---
## Step 3: Configure Test Parameters
### Quick Reference Table
| Parameter | Typical Value | Purpose |
|-----------|---------------|---------|
| configContents | `Path.of("secrets/config.json").toFile().readText()` | DB connection config |
| configSpecClass | `{DB}Specification::class.java` | Your spec class |
| dataDumper | `{DB}DataDumper(testDataSource)` | Read test data (from Step 1) |
| destinationCleaner | `{DB}Cleaner(testDataSource)` | Cleanup test data (from Step 1) |
| isStreamSchemaRetroactive | `true` | Schema changes apply to existing data |
| supportFileTransfer | `false` | Database destinations don't support files |
| commitDataIncrementally | `true` | Commit batches as written |
| nullEqualsUnset | `true` | Treat `{"x": null}` same as `{}` |
| stringifySchemalessObjects | `false` | Use native JSON if available |
| unknownTypesBehavior | `PASS_THROUGH` | Store unrecognized types as-is |
| unionBehavior | `STRINGIFY` | Convert union types to JSON string |
| schematizedObjectBehavior | `PASS_THROUGH` or `STRINGIFY` | See below |
| schematizedArrayBehavior | `STRINGIFY` | See below |
### Complex Parameters (Database-Specific)
#### dedupBehavior
**Purpose:** How to handle CDC deletions
**Options:**
```kotlin
// Hard delete - remove CDC-deleted records
DedupBehavior(DedupBehavior.CdcDeletionMode.HARD_DELETE)
// Soft delete - keep tombstone records
DedupBehavior(DedupBehavior.CdcDeletionMode.SOFT_DELETE)
// No CDC support yet
null
```
#### allTypesBehavior
**Purpose:** Configure type precision limits
```kotlin
// Snowflake/BigQuery: Unlimited precision
AllTypesBehavior.StronglyTyped(
integerCanBeLarge = true,
numberCanBeLarge = true,
nestedFloatLosesPrecision = false,
)
// MySQL/Postgres: Limited precision
AllTypesBehavior.StronglyTyped(
integerCanBeLarge = false, // BIGINT limits
numberCanBeLarge = false, // DECIMAL limits
nestedFloatLosesPrecision = false,
)
```
#### schematizedObjectBehavior / schematizedArrayBehavior
**Purpose:** How to store nested objects and arrays
**Options:**
- `PASS_THROUGH`: Use native JSON/array types (Postgres JSONB, Snowflake VARIANT)
- `STRINGIFY`: Convert to JSON strings (fallback for databases without native types)
**Recommendations:**
- **Objects:** `PASS_THROUGH` if DB has native JSON, else `STRINGIFY`
- **Arrays:** `STRINGIFY` (most DBs don't have typed arrays, except Postgres)
#### useDataFlowPipeline ⚠️
**Value:** `true` - **REQUIRED for dataflow CDK connectors**
**Why critical:** Setting to `false` uses old CDK code paths that don't work with Aggregate/InsertBuffer pattern. Always use `true`.
---
## ⚠️ CRITICAL: All Tests Must Pass - No Exceptions
**NEVER rationalize test failures as:**
- ❌ "Cosmetic, not functional"
- ❌ "The connector IS working, tests just need adjustment"
- ❌ "Just test framework expectations vs database behavior"
- ❌ "State message comparison issues, not real problems"
- ❌ "Need database-specific adaptations (but haven't made them)"
**Test failures mean ONE of two things:**
### 1. Your Implementation is Wrong (90% of cases)
- State message format doesn't match expected
- Schema evolution doesn't work correctly
- Deduplication logic has bugs
- Type handling is incorrect
**Fix:** Debug and fix your implementation
### 2. Test Expectations Need Tuning (10% of cases)
- Database truly handles something differently (e.g., ClickHouse soft delete only)
- Type precision genuinely differs
- **BUT:** You must document WHY and get agreement this is acceptable
**Fix:** Update test parameters with clear rationale
**Key principle:** If tests fail, the connector is NOT working correctly for production use.
**Example rationalizations to REJECT:**
❌ "Many tests failing due to state message comparison - cosmetic"
→ State messages are HOW Airbyte tracks progress. Wrong state = broken checkpointing!
❌ "Schema evolution needs MongoDB-specific expectations"
→ Implement schema evolution correctly for MongoDB, then tests pass!
❌ "Dedupe tests need configuration"
→ Add the configuration! Don't skip tests!
❌ "Some tests need adaptations"
→ Make the adaptations! Document what's different and why!
**ALL tests must pass or be explicitly skipped with documented rationale approved by maintainers.**
### Common Rationalizations That Are WRONG
**Agent says:** "The 7 failures are specific edge cases - advanced scenarios, not core functionality"
**Reality:**
- Truncate/overwrite mode = **CORE SYNC MODE** used by thousands of syncs
- Generation ID tracking = **REQUIRED for refresh** to work correctly
- "Edge cases" = real user scenarios that WILL happen in production
- "Advanced scenarios" = standard Airbyte features your connector claims to support
**If you don't support a mode:**
- Don't claim to support it (remove from SpecificationExtension)
- Explicitly skip those tests with @Disabled annotation
- Document the limitation clearly
**If you claim to support it (in SpecificationExtension):**
- Tests MUST pass
- No "works for normal cases" excuses
- Users will try to use it and it will break
**Agent says:** "The connector works for normal use cases"
**Reality:**
- Tests define "working"
- "Normal use cases" is undefined - what's normal?
- Users will hit "edge cases" in production
- Failed tests = broken functionality that will cause support tickets
**The rule:** If supportedSyncModes includes OVERWRITE, then testTruncate() must pass.
---
### Specific Scenarios That Are NOT Optional
**Truncate/Overwrite Mode:**
- Used by: Full refresh syncs (very common!)
- Tests: testTruncate()
- **NOT optional** if you declared `DestinationSyncMode.OVERWRITE` in SpecificationExtension
**Generation ID Tracking:**
- Used by: All refresh operations
- Tests: Generation ID assertions in all tests
- **NOT optional** - required for sync modes to work correctly
**State Messages:**
- Used by: Checkpointing and resume
- Tests: State message format validation
- **NOT optional** - wrong state = broken incremental syncs
**Schema Evolution:**
- Used by: When source schema changes
- Tests: testAppendSchemaEvolution()
- **NOT optional** - users will add/remove columns
**Deduplication:**
- Used by: APPEND_DEDUP mode
- Tests: testDedupe()
- **NOT optional** if you declared `DestinationSyncMode.APPEND_DEDUP`
**None of these are "edge cases" - they're core Airbyte features!**
---
## Step 4: Run Tests
### Test Individually
```bash
# Test append mode
$ ./gradlew :destination-{db}:integrationTest --tests "*BasicFunctionalityTest.testAppend"
# Test dedupe mode
$ ./gradlew :destination-{db}:integrationTest --tests "*BasicFunctionalityTest.testDedupe"
# Test schema evolution
$ ./gradlew :destination-{db}:integrationTest --tests "*BasicFunctionalityTest.testAppendSchemaEvolution"
```
### Run Full Suite
```bash
$ ./gradlew :destination-{db}:integrationTest --tests "*BasicFunctionalityTest"
```
**Expected:** All enabled tests pass
**Time:** 5-15 minutes (depending on database and data volume)
---
## Step 5: Debug Common Failures
### Test: testAppend fails with "Record mismatch"
**Cause:** DataDumper not converting types correctly
**Fix:** Check type conversion in DataDumper:
- Timestamps: Ensure timezone handling matches
- Numbers: Check BigDecimal vs Double conversion
- Booleans: Check 1/0 vs true/false
### Test: testDedupe fails with "Expected 1 record, got 2"
**Cause:** Deduplication not working
**Fix:** Check upsertTable() implementation:
- MERGE statement correct?
- Primary key comparison working?
- Cursor field comparison correct?
### Test: testAppendSchemaEvolution fails with "Column not found"
**Cause:** Schema evolution (ALTER TABLE) not working
**Fix:** Check applyChangeset() implementation:
- ADD COLUMN syntax correct?
- DROP COLUMN supported?
- Type changes handled?
### Test: Data type tests fail
**Cause:** Type mapping issues
**Fix:** Check ColumnUtils.toDialectType():
- All Airbyte types mapped?
- Nullable handling correct?
- Precision/scale for decimals?
---
## Step 6: Optional Test Customization
### Skip Tests Not Applicable
```kotlin
// If your DB doesn't support certain features:
// @Test
// override fun testDedupe() {
// // Skip if no MERGE/UPSERT support yet
// }
```
### Add Database-Specific Tests
```kotlin
@Test
fun testDatabaseSpecificFeature() {
// Your custom test
}
```
---
## Reference Implementations
### Snowflake
**File:** `destination-snowflake/src/test-integration/.../SnowflakeBasicFunctionalityTest.kt`
**Parameters:**
- `unionBehavior = UnionBehavior.PROMOTE_TO_OBJECT` (uses VARIANT type)
- `schematizedObjectBehavior = PASS_THROUGH` (native OBJECT type)
- `allTypesBehavior.integerCanBeLarge = true` (NUMBER unlimited)
### ClickHouse
**File:** `destination-clickhouse/src/test-integration/.../ClickhouseBasicFunctionalityTest.kt`
**Parameters:**
- `dedupBehavior = SOFT_DELETE` (ReplacingMergeTree doesn't support DELETE in MERGE)
- `schematizedArrayBehavior = STRINGIFY` (no native typed arrays)
- `allTypesBehavior.integerCanBeLarge = false` (Int64 has limits)
### MySQL
**File:** `destination-mysql/src/test-integration/.../MySQLBasicFunctionalityTest.kt`
**Parameters:**
- `unionBehavior = STRINGIFY`
- `schematizedObjectBehavior = STRINGIFY` (JSON type but limited)
- `commitDataIncrementally = true`
---
## Troubleshooting
### "No bean of type [DestinationDataDumper]"
**Cause:** DataDumper not created in companion object
**Fix:** Verify `createDataDumper()` returns {DB}DataDumper instance
### "Test hangs indefinitely"
**Cause:** Database not responding or deadlock
**Fix:**
- Check database is running (Testcontainers started?)
- Check for locks (previous test didn't cleanup?)
- Add timeout: `@Timeout(5, unit = TimeUnit.MINUTES)`
### "All tests fail with same error"
**Cause:** Setup/cleanup issue
**Fix:** Check DestinationCleaner.cleanup() actually drops tables
### "Data type test fails for one specific type"
**Cause:** Type conversion in DataDumper is wrong
**Fix:** Add logging to see what database returns:
```kotlin
val value = rs.getObject(i)
println("Column $columnName: value=$value, type=${value?.javaClass}")
```
---
## Success Criteria
BasicFunctionalityIntegrationTest is complete when:
**Minimum (Phase 8):**
- ✅ testAppend passes
**Full Feature Set (Phase 13):**
- ✅ testAppend passes
- ✅ testTruncate passes
- ✅ testAppendSchemaEvolution passes
- ✅ testDedupe passes
**Production Ready (Phase 15):**
- ✅ All tests pass
- ✅ All type tests pass
- ✅ CDC tests pass (if supported)
- ✅ No flaky tests
- ✅ Tests run in <15 minutes
---
## Time Estimates
| Task | Time |
|------|------|
| Implement DataDumper | 1-2 hours |
| Implement Cleaner | 30 min |
| Create test class with parameters | 30 min |
| Debug testAppend | 1-2 hours |
| Debug other tests | 2-4 hours |
| **Total** | **5-9 hours** |
**Tip:** Implement tests incrementally:
1. testAppend first (simplest)
2. testTruncate next
3. testAppendSchemaEvolution
4. testDedupe last (most complex)
---
## Summary
BasicFunctionalityIntegrationTest is the **gold standard** for connector validation but has significant complexity:
**Pros:**
- Comprehensive coverage (50+ scenarios)
- Validates edge cases
- Required for production certification
- Catches type handling bugs
**Cons:**
- 13 required parameters
- 5-9 hours to implement and debug
- Complex failure modes
- Slow test execution
**Strategy:**
- Phase 8: Get working connector with ConnectorWiringSuite (fast)
- Phase 15: Add BasicFunctionalityIntegrationTest (comprehensive)
- Balance: Quick iteration early, thorough validation later
The v2 guide gets you to working connector without this complexity, but this guide ensures production readiness!

View File

@@ -0,0 +1,273 @@
# Troubleshooting Reference
**Summary:** Quick reference guide for common errors encountered during connector development. Jump here when you hit a problem, find your error, and get back to coding.
---
## Understanding Test Contexts
**Why this section matters:** Tests pass but Docker fails? This section explains the three different DI contexts your connector runs in.
### The Three DI Contexts
Your connector runs in 3 different dependency injection contexts, each with different catalog loading and bean requirements:
### 1. Component Test Context
**Annotation:** `@MicronautTest(environments = ["component"])`
**What it is:**
- Unit-style tests for connector components
- Fast iteration (< 1 second per test)
- Isolated from real catalog parsing
**Catalog:** MockDestinationCatalog
- CDK provides MockDestinationCatalog bean
- Streams created dynamically by test code
- No JSON catalog parsing
- No TableCatalog auto-instantiation
**Database:** Testcontainers
- Fresh database per test class
- Automatic cleanup
- No manual setup needed
**Tests that run here:**
- TableOperationsSuite (Phases 2-5)
- ConnectorWiringSuite (Phase 8)
**What this catches:**
- Missing @Singleton annotations on Writer, AggregateFactory, Client
- Circular dependencies
- Database connection errors
- SQL syntax errors
- Business logic bugs
**What this DOESN'T catch:**
- Missing name generators (MockDestinationCatalog bypasses TableCatalog)
- Missing application-connector.yml (uses test config)
- Bean registration errors for TableCatalog dependencies
### 2. Integration Test Context
**No special annotation** - spawns actual connector process
**What it is:**
- Integration tests that spawn real connector subprocess
- Same execution path as Docker
- Full catalog parsing
**Catalog:** REAL catalog from JSON
- Parses JSON catalog file
- Auto-instantiates TableCatalog
- Requires name generators (Phase 6)
- Full DI graph validation
**Tests that run here:**
- SpecTest (Phase 1)
- CheckIntegrationTest (Phase 5)
- WriteInitializationTest (Phase 7)
- BasicFunctionalityIntegrationTest (Phases 8+)
**What this catches:**
- **Missing name generators** (TableCatalog fails to instantiate)
- **Missing WriteOperationV2** (write operation can't start)
- **Missing DatabaseInitialStatusGatherer bean** (Writer DI fails)
- All DI errors that would occur in Docker
**What this DOESN'T catch:**
- application-connector.yml errors (test uses test config)
### 3. Docker Runtime Context
**How it runs:** `docker run airbyte/destination-{db}:0.1.0 --write`
**What it is:**
- Production execution environment
- Real Airbyte platform invocation
- Full configuration from platform
**Catalog:** REAL catalog from platform
- Provided by Airbyte platform
- Auto-instantiates TableCatalog
- Requires name generators (Phase 6)
**Configuration:** application-connector.yml
- ⚠️ CRITICAL: Must exist in src/main/resources/
- Provides data-channel configuration
- Provides namespace-mapping-config-path
- Missing file = DI errors
**Common failure:** Tests pass, Docker fails
- Why: Tests use test config, Docker uses application-connector.yml
- Fix: Create application-connector.yml (Phase 0, Step 0.8)
### Test Progression Strategy
```
Phase 2-5: TableOperationsSuite (component tests)
↓ Validates: Database operations work
✓ Fast feedback
Phase 6: Name generators created
↓ Enables: TableCatalog instantiation
Phase 7: WriteInitializationTest (integration test)
↓ Validates: Write operation can initialize with REAL catalog
✓ Catches: Missing name generators, WriteOperationV2, bean registrations
Phase 8: ConnectorWiringSuite (component tests)
↓ Validates: Full write path with MOCK catalog
✓ Fast iteration on business logic
Phase 8+: BasicFunctionalityIntegrationTest
↓ Validates: End-to-end with REAL catalog
✓ Full connector functionality
```
**Best practice:** Run BOTH
```bash
# Fast iteration (component tests)
$ ./gradlew :destination-{db}:componentTest
# Full validation (integration tests)
$ ./gradlew :destination-{db}:integrationTest
```
---
## Common DI Errors & Fixes
**Quick troubleshooting guide for the most common Dependency Injection errors**
### Error: "Error instantiating TableCatalog" or "No bean of type [FinalTableNameGenerator]"
**What it means:**
- TableCatalog requires name generator beans
- Only happens with real catalog parsing
**Fix:** Create name generators (Phase 6)
**File:** `config/{DB}NameGenerators.kt`
```kotlin
@Singleton
class {DB}FinalTableNameGenerator(...) : FinalTableNameGenerator { ... }
@Singleton
class {DB}RawTableNameGenerator(...) : RawTableNameGenerator { ... }
@Singleton
class {DB}ColumnNameGenerator : ColumnNameGenerator { ... }
```
**Also register in BeanFactory:**
```kotlin
@Singleton
fun tempTableNameGenerator(...): TempTableNameGenerator { ... }
```
---
### Error: "No bean of type [DatabaseInitialStatusGatherer]"
**What it means:**
- Class exists but bean registration missing
**Fix:** Add bean registration (Phase 7, Step 7.3)
**File:** `{DB}BeanFactory.kt`
```kotlin
@Singleton
fun initialStatusGatherer(
client: TableOperationsClient,
tempTableNameGenerator: TempTableNameGenerator,
): DatabaseInitialStatusGatherer<DirectLoadInitialStatus> {
return {DB}DirectLoadDatabaseInitialStatusGatherer(client, tempTableNameGenerator)
}
```
---
### Error: "A legal sync requires a declared @Singleton of a type that implements LoadStrategy"
**What it means:**
- Missing WriteOperationV2 bean
**Fix:** Create WriteOperationV2 (Phase 7, Step 7.1)
**File:** `cdk/WriteOperationV2.kt`
```kotlin
@Primary
@Singleton
@Requires(property = Operation.PROPERTY, value = "write")
class WriteOperationV2(private val d: DestinationLifecycle) : Operation {
override fun execute() { d.run() }
}
```
---
### Error: "Failed to inject value for parameter [dataChannelMedium]"
**What it means:**
- Missing application-connector.yml
- **Only happens in Docker, NOT in tests**
**Fix:** Create application-connector.yml (Phase 0, Step 0.8)
**File:** `src/main/resources/application-connector.yml`
```yaml
airbyte:
destination:
core:
data-channel:
medium: STDIO
format: JSONL
mappers:
namespace-mapping-config-path: ""
```
---
### Error: "lateinit property initialStatuses has not been initialized"
**What it means:**
- ConnectorWiringSuite creates dynamic test streams
- Writer needs defensive handling
**Fix:** Make Writer defensive (Phase 8, Step 8.5)
```kotlin
val initialStatus = if (::initialStatuses.isInitialized) {
initialStatuses[stream] ?: DirectLoadInitialStatus(null, null)
} else {
DirectLoadInitialStatus(null, null)
}
```
---
## When to Use This Guide
**Jump here when:**
- Tests pass but Docker fails → Check "Understanding Test Contexts"
- Getting DI errors → Check "Common DI Errors & Fixes"
- Not sure which test to run → Check "Test Progression Strategy"
- Need quick error lookup → Scan error titles
**Return to phase guides when:**
- Building new features
- Following step-by-step implementation
- Need detailed explanations
---
## Additional Resources
- [dataflow-cdk.md](../dataflow-cdk.md) - Architecture overview
- [implementation-reference.md](../implementation-reference.md) - Component reference
- [coding-standards.md](../coding-standards.md) - Best practices