From a960a0f2e34faad6871517cf01484a4bae25f2d2 Mon Sep 17 00:00:00 2001 From: Jimmy Ma Date: Wed, 26 Nov 2025 08:21:05 -0800 Subject: [PATCH] chore: dataflow cdk connector building guide (#70217) --- .../destination/coding-standards.md | 918 +++++++++++++ connector-writer/destination/dataflow-cdk.md | 803 ++++++++++++ .../destination/implementation-reference.md | 1131 +++++++++++++++++ .../destination/preflight-checklist.md | 581 +++++++++ .../step-by-step/0-introduction.md | 252 ++++ .../step-by-step/1-getting-started.md | 614 +++++++++ .../step-by-step/2-database-setup.md | 834 ++++++++++++ .../step-by-step/3-write-infrastructure.md | 596 +++++++++ .../step-by-step/4-write-operations.md | 769 +++++++++++ .../step-by-step/5-advanced-features.md | 756 +++++++++++ .../destination/step-by-step/6-testing.md | 750 +++++++++++ .../step-by-step/7-troubleshooting.md | 273 ++++ 12 files changed, 8277 insertions(+) create mode 100644 connector-writer/destination/coding-standards.md create mode 100644 connector-writer/destination/dataflow-cdk.md create mode 100644 connector-writer/destination/implementation-reference.md create mode 100644 connector-writer/destination/preflight-checklist.md create mode 100644 connector-writer/destination/step-by-step/0-introduction.md create mode 100644 connector-writer/destination/step-by-step/1-getting-started.md create mode 100644 connector-writer/destination/step-by-step/2-database-setup.md create mode 100644 connector-writer/destination/step-by-step/3-write-infrastructure.md create mode 100644 connector-writer/destination/step-by-step/4-write-operations.md create mode 100644 connector-writer/destination/step-by-step/5-advanced-features.md create mode 100644 connector-writer/destination/step-by-step/6-testing.md create mode 100644 connector-writer/destination/step-by-step/7-troubleshooting.md diff --git a/connector-writer/destination/coding-standards.md b/connector-writer/destination/coding-standards.md new file mode 100644 index 00000000000..d37f4828d55 --- /dev/null +++ b/connector-writer/destination/coding-standards.md @@ -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 + + 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() + + 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 diff --git a/connector-writer/destination/dataflow-cdk.md b/connector-writer/destination/dataflow-cdk.md new file mode 100644 index 00000000000..fee3cc43f45 --- /dev/null +++ b/connector-writer/destination/dataflow-cdk.md @@ -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, +) : 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, + private val streamStateStore: StreamStateStore, + private val client: MyAirbyteClient, + private val tempTableNameGenerator: TempTableNameGenerator, +) : DestinationWriter { + private lateinit var initialStatuses: Map + + 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) diff --git a/connector-writer/destination/implementation-reference.md b/connector-writer/destination/implementation-reference.md new file mode 100644 index 00000000000..fa7ac5eee20 --- /dev/null +++ b/connector-writer/destination/implementation-reference.md @@ -0,0 +1,1131 @@ +# Dataflow CDK Implementation Reference + +**Summary:** Quick reference for implementing destination connectors. Covers the 4 core custom components, type mapping, schema evolution, CDC handling, and integration points. Use as a lookup guide during development. + +--- + +## The 4 Core Custom Components + +### 1. SQL Generator + +**Purpose:** Generate all database-specific SQL statements + +**Key Methods:** + +```kotlin +@Singleton +class MySqlGenerator { + fun createNamespace(namespace: String): String + fun createTable(stream, tableName, columnMapping, replace): String + fun dropTable(tableName: TableName): String + fun copyTable(columnMapping, source, target): String + fun upsertTable(stream, columnMapping, source, target): String + fun overwriteTable(source, target): String + fun alterTable(tableName, added, dropped, modified): Set + fun countTable(tableName): String +} +``` + +**Responsibilities:** +- Generate SQL for database dialect +- Handle quoting (quotes, backticks, brackets) +- Generate MERGE/UPSERT for deduplication +- Generate window functions for deduplication +- Handle CDC deletions (DELETE clause in MERGE) +- **Always** call `.andLog()` on generated SQL + +**Example:** + +```kotlin +fun createTable(stream: DestinationStream, tableName: TableName, ...): String { + val columnDeclarations = stream.schema.asColumns() + .map { (name, type) -> + "${name.quote()} ${columnUtils.toDialectType(type)}" + } + .joinToString(",\n") + + return """ + CREATE TABLE ${fullyQualifiedName(tableName)} ( + ${COLUMN_NAME_AB_RAW_ID} VARCHAR NOT NULL, + ${COLUMN_NAME_AB_EXTRACTED_AT} TIMESTAMP NOT NULL, + ${COLUMN_NAME_AB_META} JSON NOT NULL, + ${COLUMN_NAME_AB_GENERATION_ID} INTEGER, + ${columnDeclarations} + ) + """.trimIndent().andLog() +} +``` + +--- + +### 2. Database Client + +**Purpose:** Execute database operations + +**Implements:** `TableOperationsClient` + `TableSchemaEvolutionClient` + +**TableOperationsClient Methods:** + +```kotlin +@Singleton +class MyAirbyteClient( + private val dataSource: DataSource, + private val sqlGenerator: MySqlGenerator, +) : TableOperationsClient, TableSchemaEvolutionClient { + + // Namespace operations + suspend fun createNamespace(namespace: String) + suspend fun namespaceExists(namespace: String): Boolean + + // Table operations + suspend fun createTable(stream, tableName, columnMapping, replace) + suspend fun tableExists(table: TableName): Boolean + suspend fun dropTable(tableName: TableName) + suspend fun countTable(tableName: TableName): Long? // null if not exists + + // Finalization operations + suspend fun overwriteTable(source, target) // SWAP/RENAME for truncate + suspend fun copyTable(columnMapping, source, target) // Copy data + suspend fun upsertTable(stream, columnMapping, source, target) // MERGE for dedupe + + // Metadata + suspend fun getGenerationId(tableName: TableName): Long +} +``` + +**TableSchemaEvolutionClient Methods:** + +```kotlin +// Schema evolution (4 steps) +suspend fun discoverSchema(tableName): TableSchema +fun computeSchema(stream, columnMapping): TableSchema +suspend fun ensureSchemaMatches(stream, tableName, columnMapping) +suspend fun applyChangeset(stream, columnMapping, tableName, expectedColumns, changeset) +``` + +**Pattern:** + +```kotlin +override suspend fun createTable(...) { + execute(sqlGenerator.createTable(...)) +} + +override suspend fun upsertTable(...) { + execute(sqlGenerator.upsertTable(...)) +} + +private suspend fun execute(sql: String) { + dataSource.connection.use { conn -> + conn.createStatement().use { stmt -> + stmt.executeQuery(sql) + } + } +} +``` + +**Key Responsibilities:** +- Delegate SQL generation to SqlGenerator +- Execute SQL via connection/client +- Handle database errors → ConfigErrorException for user errors +- Implement schema evolution (discover → compute → compare → apply) +- Return `null` for expected missing data (table doesn't exist) + +--- + +### 3. Insert Buffer + +**Purpose:** Efficient batch writes to database + +**Custom Implementation (database-specific):** + +```kotlin +class MyInsertBuffer( + private val tableName: TableName, + private val client: MyAirbyteClient, + private val flushLimit: Int = 1000, +) { + private val buffer = mutableListOf>() + private var recordCount = 0 + + fun accumulate(recordFields: Map) { + buffer.add(format(recordFields)) + recordCount++ + + if (recordCount >= flushLimit) { + runBlocking { flush() } + } + } + + suspend fun flush() { + if (buffer.isEmpty()) return + + try { + // Write batch to database + writeBatchToDatabase(tableName, buffer) + } finally { + buffer.clear() + recordCount = 0 + } + } + + private fun format(fields: Map): Map { + // Convert AirbyteValue to database types + } +} +``` + +**Database-Specific Strategies:** + +| Database | Strategy | Details | +|----------|----------|---------| +| **Snowflake** | CSV staging | CSV → GZIP → stage via PUT → COPY INTO | +| **ClickHouse** | Binary rows | Binary format → in-memory → direct insert | +| **Postgres** | COPY | CSV → temp file → COPY FROM file | +| **BigQuery** | JSON | Batch JSON → streaming insert API | +| **MySQL** | Multi-row INSERT | `INSERT INTO ... VALUES (...), (...), (...)` | + +**Key Points:** +- Format records for database (CSV, binary, JSON) +- Buffer in memory or temp files +- Auto-flush at thresholds (count, size, time) +- Clean up resources in `finally` +- **Do NOT** call `upsertTable()` or `overwriteTable()` - StreamLoader does that + +--- + +### 4. Column Utilities + +**Purpose:** Type mapping and column declarations + +**Key Methods:** + +```kotlin +class MyColumnUtils { + fun toDialectType(type: AirbyteType): String + fun columnsAndTypes(columns, columnMapping): List + fun formatColumn(name, type): String +} +``` + +**Type Mapping:** + +| Airbyte Type | Snowflake | ClickHouse | Postgres | BigQuery | +|--------------|-----------|------------|----------|----------| +| `BooleanType` | `BOOLEAN` | `Bool` | `BOOLEAN` | `BOOL` | +| `IntegerType` | `NUMBER(38,0)` | `Int64` | `BIGINT` | `INT64` | +| `NumberType` | `FLOAT` | `Decimal(38,9)` | `DOUBLE PRECISION` | `FLOAT64` | +| `StringType` | `VARCHAR` | `String` | `TEXT` | `STRING` | +| `DateType` | `DATE` | `Date32` | `DATE` | `DATE` | +| `TimestampTypeWithTimezone` | `TIMESTAMP_TZ` | `DateTime64(3)` | `TIMESTAMPTZ` | `TIMESTAMP` | +| `TimestampTypeWithoutTimezone` | `TIMESTAMP_NTZ` | `DateTime64(3)` | `TIMESTAMP` | `DATETIME` | +| `ArrayType` | `ARRAY` | `String` | `JSONB` | `ARRAY` | +| `ObjectType` | `VARIANT` | `String`/`JSON` | `JSONB` | `JSON` | +| `UnionType` | `VARIANT` | `String` | `JSONB` | `JSON` | + +**Implementation:** + +```kotlin +fun AirbyteType.toDialectType(): String = when (this) { + BooleanType -> "BOOLEAN" + IntegerType -> "BIGINT" + NumberType -> "DECIMAL(38, 9)" + StringType -> "VARCHAR" + DateType -> "DATE" + TimestampTypeWithTimezone -> "TIMESTAMP WITH TIME ZONE" + TimestampTypeWithoutTimezone -> "TIMESTAMP" + is ArrayType -> "JSONB" + is ObjectType -> "JSONB" + is UnionType -> "JSONB" + else -> "VARCHAR" // Fallback +} +``` + +**Nullable Handling:** + +```kotlin +// Snowflake: Add NOT NULL suffix +val typeDecl = if (columnType.nullable) { + columnType.type // "VARCHAR" +} else { + "${columnType.type} NOT NULL" // "VARCHAR NOT NULL" +} + +// ClickHouse: Wrap in Nullable() +val typeDecl = if (columnType.nullable) { + "Nullable(${columnType.type})" // "Nullable(String)" +} else { + columnType.type // "String" +} +``` + +--- + +## Sync Mode Decision Tree + +**Selection Logic:** + +```kotlin +when (stream.minimumGenerationId) { + 0L -> when (stream.importType) { + Dedupe -> DirectLoadTableDedupStreamLoader // Temp → MERGE + else -> DirectLoadTableAppendStreamLoader // Direct write + } + stream.generationId -> when (stream.importType) { + Dedupe -> DirectLoadTableDedupTruncateStreamLoader // Temp → dedupe → SWAP + else -> DirectLoadTableAppendTruncateStreamLoader // Temp → SWAP + } +} +``` + +**Temp Table Usage:** + +| StreamLoader | Temp Table? | Finalization | When | +|--------------|-------------|--------------|------| +| Append | No | None | Incremental, no PK | +| Dedupe | Yes | MERGE temp→final | Incremental with PK | +| AppendTruncate | Yes | SWAP temp↔final | Full refresh, no PK | +| DedupTruncate | Yes (sometimes 2) | MERGE temp→temp2, SWAP temp2↔final | Full refresh with PK | + +**Dedupe+Truncate Complexity:** + +When dedupe+truncate and real table doesn't exist or wrong generation: +1. Write to temp1 +2. Create temp2 +3. MERGE temp1 → temp2 (deduplicate) +4. SWAP temp2 ↔ real (atomic replacement) + +Why: Can't MERGE into non-existent table. Can't MERGE then SWAP (two operations, not atomic). + +--- + +## Component Interaction Flow + +### Full Sync Lifecycle + +``` +1. CONFIGURATION + User Config → ConfigFactory → Configuration + BeanFactory creates all singletons + +2. SETUP (Writer.setup()) + - Create all namespaces + - Gather initial table state + +3. STREAM INIT (per stream) + Writer.createStreamLoader() → select appropriate StreamLoader + + StreamLoader.start(): + - tableExists(finalTable) + - If exists: ensureSchemaMatches() [schema evolution] + - If not: createTable(finalTable) + - If dedupe/truncate: createTable(tempTable) + - Store target table in streamStateStore + + AggregateFactory.create(): + - Read tableName from streamStateStore + - Create InsertBuffer(tableName, client) + - Wrap in Aggregate + +4. DATA PROCESSING (automatic) + Pipeline → Aggregate.accept(record): + → InsertBuffer.accumulate(record) + → [auto-flush at threshold] + + Aggregate.flush(): + → InsertBuffer.flush() → write batch to database + +5. FINALIZATION (StreamLoader.close()) + If streamCompleted: + - Dedupe: upsertTable(temp → final) [MERGE] + - Truncate: overwriteTable(temp → final) [SWAP] + - Append: nothing (already in final) + Always: + - dropTable(tempTable) +``` + +### Component Dependencies + +``` +Writer + ├─ depends on: client, statusGatherer, names, streamStateStore + └─ creates: StreamLoaders + +StreamLoader (CDK-provided) + ├─ depends on: client (TableOperationsClient + TableSchemaEvolutionClient) + └─ calls: createTable(), ensureSchemaMatches(), upsertTable(), dropTable() + +AggregateFactory + ├─ depends on: client, streamStateStore + └─ creates: Aggregate + InsertBuffer + +InsertBuffer + ├─ depends on: client, columnUtils + └─ calls: Only insert operations (NOT upsert/merge/swap) + +Client + ├─ depends on: sqlGenerator, dataSource, config + └─ calls: sqlGenerator for SQL, executes via dataSource + +SqlGenerator + ├─ depends on: columnUtils, config + └─ called by: client for SQL generation +``` + +--- + +## Integration Points + +**Where framework calls your code:** + +### 1. Setup Phase + +```kotlin +// Framework: Writer.setup() +override suspend fun setup() { + // Your code: + namespaces.forEach { client.createNamespace(it) } + initialStatuses = gatherer.gatherInitialStatus(names) +} +``` + +### 2. Stream Initialization + +```kotlin +// Framework: Writer.createStreamLoader(stream) +override fun createStreamLoader(stream: DestinationStream): StreamLoader { + // Your code: + return when (stream.minimumGenerationId) { + 0L -> when (stream.importType) { + is Dedupe -> DirectLoadTableDedupStreamLoader(...) + else -> DirectLoadTableAppendStreamLoader(...) + } + stream.generationId -> /* truncate modes */ + } +} + +// Framework: StreamLoader.start() (inside) +if (tableExists(finalTable)) { + client.ensureSchemaMatches(stream, finalTable, columnMapping) // Your code +} else { + client.createTable(stream, finalTable, columnMapping, false) // Your code +} +``` + +### 3. Data Processing + +```kotlin +// Framework: Aggregate.accept(record) +override fun accept(record: RecordDTO) { + buffer.accumulate(record.fields) // Your code (InsertBuffer) +} + +// Framework: Aggregate.flush() +override suspend fun flush() { + buffer.flush() // Your code (InsertBuffer writes batch) +} +``` + +### 4. Finalization + +```kotlin +// Framework: StreamLoader.close() (inside) +if (streamCompleted) { + // Dedupe mode + client.upsertTable(stream, columnMapping, tempTable, finalTable) // Your code + + // Truncate mode + client.overwriteTable(tempTable, finalTable) // Your code +} +client.dropTable(tempTable) // Your code +``` + +--- + +## Schema Evolution + +**Automatic during `StreamLoader.start()` if table exists** + +### 4-Step Process + +**1. Discover Current Schema** + +```kotlin +override suspend fun discoverSchema(tableName): TableSchema { + // Query system catalog: DESCRIBE TABLE, information_schema, etc. + // Return: Map + // Filter out Airbyte metadata columns +} +``` + +**Examples:** +- Snowflake: `DESCRIBE TABLE` +- Postgres: `information_schema.columns` +- ClickHouse: `system.columns` + +**2. Compute Expected Schema** + +```kotlin +override fun computeSchema(stream, columnMapping): TableSchema { + // Map stream.schema to database types + // Use columnUtils.toDialectType() + // Apply column name mapping + // Filter out Airbyte metadata columns +} +``` + +**3. Compare (automatic by CDK)** + +```kotlin +val changeset = ColumnChangeset( + columnsToAdd = expected - actual, + columnsToDrop = actual - expected, + columnsToChange = actual.filter { expected[it.key] != it.value }, +) +``` + +**4. Apply Changes** + +```kotlin +override suspend fun applyChangeset(..., changeset) { + changeset.columnsToAdd.forEach { (name, type) -> + execute("ALTER TABLE $table ADD COLUMN $name $type") + } + changeset.columnsToDrop.forEach { (name, _) -> + execute("ALTER TABLE $table DROP COLUMN $name") + } + // Type changes: temp column approach or table recreation +} +``` + +### Type Change Strategies + +**Safe (widening):** +- `INT → BIGINT`: Direct ALTER (larger range) +- `VARCHAR(50) → VARCHAR(100)`: Direct ALTER (longer) +- `NOT NULL → NULL`: Drop constraint + +**Unsafe (narrowing):** +- `BIGINT → INT`: Temp column + cast + rename +- `VARCHAR → INT`: Temp column + cast + rename +- `NULL → NOT NULL`: Skip (can't enforce if nulls exist) + +**Temp Column Approach (Snowflake):** + +```sql +-- 1. Add temp column +ALTER TABLE t ADD COLUMN col_temp VARCHAR; + +-- 2. Cast and copy +UPDATE t SET col_temp = CAST(col AS VARCHAR); + +-- 3. Rename original to backup +ALTER TABLE t RENAME COLUMN col TO col_backup; + +-- 4. Rename temp to original +ALTER TABLE t RENAME COLUMN col_temp TO col; + +-- 5. Drop backup +ALTER TABLE t DROP COLUMN col_backup; +``` + +**Table Recreation (ClickHouse for PK changes):** + +```sql +-- 1. Create temp with new schema +CREATE TABLE temp (...) ENGINE = ReplacingMergeTree(...); + +-- 2. Copy intersection +INSERT INTO temp SELECT common_columns FROM original; + +-- 3. Atomic swap +EXCHANGE TABLES original AND temp; + +-- 4. Drop old +DROP TABLE temp; +``` + +--- + +## CDC Handling + +**CDC = Change Data Capture (source emits deletions)** + +### Detection + +```kotlin +val hasCdc = stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN) +// CDC_DELETED_AT_COLUMN = "_ab_cdc_deleted_at" +``` + +### Two Modes + +**1. Hard Delete (default)** - Actually delete records + +```sql +MERGE INTO target +USING source +ON target.pk = source.pk +WHEN MATCHED AND source._ab_cdc_deleted_at IS NOT NULL + AND source.cursor > target.cursor THEN DELETE +WHEN MATCHED AND source.cursor > target.cursor THEN UPDATE ... +WHEN NOT MATCHED AND source._ab_cdc_deleted_at IS NULL THEN INSERT ... +``` + +**2. Soft Delete** - Keep tombstone records + +```sql +MERGE INTO target +USING source +ON target.pk = source.pk +-- No DELETE clause +WHEN MATCHED AND source.cursor > target.cursor THEN UPDATE ... +WHEN NOT MATCHED THEN INSERT ... +-- Deleted records upserted with _ab_cdc_deleted_at populated +``` + +### Implementation + +```kotlin +val cdcDeleteClause = if ( + stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN) && + config.cdcDeletionMode == CdcDeletionMode.HARD_DELETE +) { + """ + WHEN MATCHED AND new_record._ab_cdc_deleted_at IS NOT NULL + AND $cursorComparison THEN DELETE + """ +} else { + "" +} + +val cdcSkipInsertClause = if (hasCdc && isHardDelete) { + "AND new_record._ab_cdc_deleted_at IS NULL" +} else { + "" +} + +val mergeStatement = """ + MERGE INTO $target + USING $source + ON $pkMatch + $cdcDeleteClause + WHEN MATCHED AND $cursorComparison THEN UPDATE ... + WHEN NOT MATCHED $cdcSkipInsertClause THEN INSERT ... +""" +``` + +**Key Points:** +- Only applies to **Dedupe** mode (not Append) +- DELETE clause must come **before** UPDATE +- Must check cursor (only delete if deletion is newer) +- Skip INSERT for deleted records + +**Configuration:** + +```kotlin +data class MyConfiguration( + val cdcDeletionMode: CdcDeletionMode = CdcDeletionMode.HARD_DELETE, +) + +enum class CdcDeletionMode(@get:JsonValue val value: String) { + HARD_DELETE("Hard delete"), + SOFT_DELETE("Soft delete"), +} +``` + +--- + +## Generation IDs and Resume Logic + +**Purpose:** Enable detection of interrupted syncs and safe resume. + +**How It Works:** + +Every record includes `_airbyte_generation_id`: +- Incremental modes: minimumGenerationId = 0 (keep all generations) +- Full refresh: minimumGenerationId = generationId (replace old generations) + +**Resume Detection (Truncate Mode):** + +``` +StreamLoader.start(): + tempGenId = getGenerationId(tempTable) // null if doesn't exist + realGenId = getGenerationId(realTable) // null if doesn't exist + + case 1: tempGenId == stream.generationId + → Resume interrupted sync (write to temp) + + case 2: realGenId == stream.generationId + → Sync already completed, STATE lost (write to real, skip finalization) + + case 3: Neither matches + → New sync (drop stale temp if exists, create fresh temp) +``` + +**Why Case 2 Matters:** + +Scenario: Sync completes, SWAP succeeds, STATE emitted, but network error loses STATE. +- Platform thinks sync failed, retries +- Real table already has new data with correct generationId +- No need for temp table - write directly to real +- Avoids duplicate work and disk usage + +--- + +## Airbyte Metadata Columns + +**Always included (framework-managed):** + +| Column | Type | Nullable | Purpose | +|--------|------|----------|---------| +| `_airbyte_raw_id` | UUID/String | NOT NULL | Unique record ID | +| `_airbyte_extracted_at` | Timestamp | NOT NULL | Extraction timestamp | +| `_airbyte_meta` | JSON | NOT NULL | Errors, warnings, metadata | +| `_airbyte_generation_id` | Integer | Yes | Sync generation tracking | + +**Database-Specific Types:** + +**Snowflake:** +```kotlin +"_AIRBYTE_RAW_ID" to "VARCHAR NOT NULL" +"_AIRBYTE_EXTRACTED_AT" to "TIMESTAMP_TZ NOT NULL" +"_AIRBYTE_META" to "VARIANT NOT NULL" +"_AIRBYTE_GENERATION_ID" to "NUMBER(38,0)" +``` + +**ClickHouse:** +```kotlin +"_airbyte_raw_id" to "String NOT NULL" +"_airbyte_extracted_at" to "DateTime64(3) NOT NULL" +"_airbyte_meta" to "String NOT NULL" +"_airbyte_generation_id" to "UInt32 NOT NULL" +``` + +**Important:** +- **Filter out** during schema discovery and computation +- Never in `ColumnChangeset` (managed separately) +- Created first in CREATE TABLE statements +- Case sensitivity varies by database (Snowflake uppercase, ClickHouse lowercase) + +--- + +## Sync Mode Selection + +**Based on `minimumGenerationId` and `importType`:** + +| minimumGenerationId | importType | StreamLoader | Behavior | +|---------------------|------------|--------------|----------| +| 0 | Append | `DirectLoadTableAppendStreamLoader` | Direct insert to final table | +| 0 | Dedupe | `DirectLoadTableDedupStreamLoader` | Temp → MERGE with PK dedup | +| generationId | Append | `DirectLoadTableAppendTruncateStreamLoader` | Temp → SWAP | +| generationId | Dedupe | `DirectLoadTableDedupTruncateStreamLoader` | Temp → dedupe → SWAP | + +**Pattern:** + +```kotlin +override fun createStreamLoader(stream: DestinationStream): StreamLoader { + val initialStatus = initialStatuses[stream]!! + val tableNames = names[stream]!!.tableNames + val columnMapping = names[stream]!!.columnNameMapping + + return when (stream.minimumGenerationId) { + 0L -> when (stream.importType) { + is Dedupe -> DirectLoadTableDedupStreamLoader( + stream, initialStatus, tableNames.finalTableName!!, + tempTableNameGenerator.generate(tableNames.finalTableName!!), + columnMapping, client, client, streamStateStore + ) + else -> DirectLoadTableAppendStreamLoader( + stream, initialStatus, tableNames.finalTableName!!, + tempTableNameGenerator.generate(tableNames.finalTableName!!), + columnMapping, client, client, streamStateStore + ) + } + stream.generationId -> when (stream.importType) { + is Dedupe -> DirectLoadTableDedupTruncateStreamLoader(...) + else -> DirectLoadTableAppendTruncateStreamLoader(...) + } + else -> throw SystemErrorException("Hybrid refresh not supported") + } +} +``` + +--- + +## Common Operations Reference + +### CREATE TABLE + +```kotlin +fun createTable(stream: DestinationStream, tableName: TableName, ...): String { + val columnDeclarations = stream.schema.asColumns() + .map { (name, type) -> formatColumn(name, type) } + .joinToString(",\n") + + return """ + CREATE TABLE ${fullyQualifiedName(tableName)} ( + ${metadataColumns} + ${columnDeclarations} + ) + """.trimIndent().andLog() +} +``` + +### UPSERT (Dedupe) + +```sql +MERGE INTO final_table AS target +USING ( + SELECT * FROM ( + SELECT *, ROW_NUMBER() OVER ( + PARTITION BY primary_key + ORDER BY _airbyte_extracted_at DESC + ) AS rn + FROM temp_table + ) WHERE rn = 1 +) AS source +ON target.pk = source.pk +WHEN MATCHED AND source.cursor > target.cursor THEN UPDATE SET ... +WHEN NOT MATCHED THEN INSERT ... +``` + +### OVERWRITE (Truncate) + +```sql +-- Option 1: SWAP (if database supports) +ALTER TABLE final SWAP WITH temp; +DROP TABLE temp; + +-- Option 2: EXCHANGE (ClickHouse) +EXCHANGE TABLES final AND temp; +DROP TABLE temp; + +-- Option 3: DROP + RENAME +DROP TABLE IF EXISTS final; +ALTER TABLE temp RENAME TO final; +``` + +### ALTER TABLE (Schema Evolution) + +```sql +-- Add column +ALTER TABLE t ADD COLUMN new_col VARCHAR; + +-- Drop column +ALTER TABLE t DROP COLUMN old_col; + +-- Modify type (Postgres) +ALTER TABLE t ALTER COLUMN col TYPE VARCHAR USING col::VARCHAR; + +-- Modify type (ClickHouse) +ALTER TABLE t MODIFY COLUMN col Nullable(String); +``` + +--- + +## Quick Reference Tables + +### Typical SQL Operations + +| Operation | Typical SQL | When Called | +|-----------|------------|-------------| +| Create namespace | `CREATE SCHEMA IF NOT EXISTS` | Writer.setup() | +| Create table | `CREATE TABLE (columns...)` | StreamLoader.start() | +| Drop table | `DROP TABLE IF EXISTS` | StreamLoader.close() | +| Count rows | `SELECT COUNT(*) FROM table` | Initial status gathering | +| Get generation ID | `SELECT _airbyte_generation_id FROM table LIMIT 1` | Initial status gathering | +| Copy table | `INSERT INTO target SELECT * FROM source` | Rarely (append truncate) | +| Upsert | `MERGE INTO target USING source ON pk WHEN MATCHED...` | Dedupe mode finalization | +| Overwrite | `SWAP/EXCHANGE/DROP+RENAME` | Truncate mode finalization | +| Alter table | `ALTER TABLE ADD/DROP/MODIFY COLUMN` | Schema evolution | + +### Error Classification + +| Error Type | When to Use | Example | +|------------|-------------|---------| +| `ConfigErrorException` | User-fixable | Bad credentials, missing permissions, invalid config | +| `TransientErrorException` | Retryable | Network timeout, DB unavailable, connection pool full | +| `SystemErrorException` | Internal | Null pointer, illegal state, unimplemented feature | + +**Read Consistency During Failures:** + +Guarantee: Readers always see consistent state, even during connector failures. + +- Sync fails before finalization: Real table unchanged, readers see old data +- Sync fails during finalization: Database transaction rollback, readers see old data +- Sync succeeds but STATE lost: Real table has new data (correct state) + +**Cleanup:** StreamLoader.close(streamCompleted=false) always drops temp tables. + +### Log Levels + +| Level | When to Use | Example | +|-------|-------------|---------| +| `info` | Normal operations | "Beginning insert into table...", "Finished insert of 1000 rows" | +| `warn` | Unexpected but recoverable | "CSV file path not set", "Falling back to default" | +| `error` | Will fail operation | "Unable to flush data", "Failed to execute query" | +| `debug` | Detailed diagnostics | "Table does not exist (expected)", "Connection attempt 2/3" | + +--- + +## Implementation Checklist + +### Phase 1: Core Components +- [ ] SQL Generator with all operations +- [ ] Database Client implementing both interfaces +- [ ] Insert Buffer with efficient batch writes +- [ ] Column Utilities for type mapping + +### Phase 2: Configuration +- [ ] Specification class with all properties +- [ ] Configuration data class +- [ ] Configuration Factory with validation +- [ ] BeanFactory with DI setup + +### Phase 3: Orchestration +- [ ] Name Generators (table, column, temp) +- [ ] Initial Status Gatherer (usually extend base) +- [ ] Writer with setup() and createStreamLoader() +- [ ] Aggregate (3-line delegation) +- [ ] AggregateFactory + +### Phase 4: Validation +- [ ] Checker with connection test +- [ ] Error handling (ConfigError, TransientError, SystemError) +- [ ] Logging throughout + +### Phase 5: Testing +- [ ] Unit tests for SQL generation +- [ ] Component tests (TableOperationsSuite) +- [ ] Integration tests (BasicFunctionalityIntegrationTest) +- [ ] Test all sync modes (append, dedupe, overwrite) +- [ ] Test schema evolution +- [ ] Test CDC if supported + +### Phase 6: Polish +- [ ] All SQL logged +- [ ] Resources cleaned up in finally +- [ ] Error messages actionable +- [ ] Documentation complete + +--- + +## The Three Operations + +**Every connector must support three operations:** + +| Operation | Trigger | Purpose | Output | Implementation | +|-----------|---------|---------|--------|----------------| +| `--spec` | CLI flag | Return connector capabilities | SPEC message with JSON schema | Automatic (via Specification class) | +| `--check` | CLI flag | Validate connection | CONNECTION_STATUS message | Implement Checker | +| `--write` | CLI flag | Execute sync | STATE messages | Implement Writer, Client, Buffer | + +### Spec Operation + +**Command:** +```bash +destination-{db} --spec +``` + +**What it does:** +- Reads your `{DB}Specification` class +- Generates JSON schema from Jackson annotations +- Adds supported sync modes from `{DB}SpecificationExtension` +- Returns SPEC message to stdout + +**What you implement:** +- `{DB}Specification` class with `@JsonProperty`, `@JsonSchemaTitle`, etc. +- `{DB}SpecificationExtension` declaring supported sync modes +- `application.yml` with documentation URL (optional) + +**Output example:** +```json +{ + "type": "SPEC", + "spec": { + "documentationUrl": "https://docs.airbyte.com/integrations/destinations/{db}", + "connectionSpecification": { + "$schema": "http://json-schema.org/draft-07/schema#", + "type": "object", + "required": ["hostname", "database", "username", "password"], + "properties": { ... } + }, + "supportsIncremental": true, + "supported_destination_sync_modes": ["overwrite", "append", "append_dedup"] + } +} +``` + +**Testing:** +```kotlin +// src/test-integration/kotlin/.../spec/{DB}SpecTest.kt +class {DB}SpecTest : SpecTest() + +// Validates against: src/test-integration/resources/expected-spec-oss.json +``` + +**Covered in:** Phase 0, Steps 0.6-0.12 of step-by-step-guide.md + +### Check Operation + +**Command:** +```bash +destination-{db} --check --config config.json +``` + +**What it does:** +- Validates configuration +- Tests database connection +- Creates test table, inserts record, verifies, cleans up +- Returns CONNECTION_STATUS (SUCCEEDED or FAILED) + +**What you implement:** +- `{DB}Checker` class implementing `DestinationCheckerV2` +- `check()` method that validates connection + +**Output example:** +```json +{ + "type": "CONNECTION_STATUS", + "connectionStatus": { + "status": "SUCCEEDED" + } +} +``` + +**Covered in:** Phase 5, Step 5.9 of step-by-step-guide.md + +### Write Operation + +**Command:** +```bash +destination-{db} --write --config config.json --catalog catalog.json < messages.jsonl +``` + +**What it does:** +- Reads RECORD and STATE messages from stdin +- Processes records through data pipeline +- Writes to database via your InsertBuffer +- Emits STATE messages to stdout +- Handles all sync modes (append, dedupe, overwrite) + +**What you implement:** +- All 4 core components (Client, SqlGenerator, InsertBuffer, ColumnUtils) +- Writer, Aggregate, AggregateFactory +- Name generators + +**Output example:** +```json +{"type":"LOG","log":{"level":"INFO","message":"Beginning sync..."}} +{"type":"LOG","log":{"level":"INFO","message":"Finished insert of 1000 rows"}} +{"type":"STATE","state":{"type":"STREAM","stream":{...},"sourceStats":{"recordCount":1000.0}}} +``` + +**Guarantees:** + +1. **STATE Emission:** Only after database COMMIT completes +2. **Atomicity:** Finalization (MERGE/SWAP) is atomic or skipped +3. **Read Consistency:** Readers see old data or new data, never mixed/partial + +**Error Recovery Scenarios:** + +| Failure Point | Database State | Reader View | Recovery | +|---------------|----------------|-------------|----------| +| Before flush | No changes | Old data | Retry from last STATE | +| During flush | Partial in temp | Old data (real unchanged) | Drop temp, retry | +| Before finalization | Complete in temp | Old data (real unchanged) | Resume, complete finalization | +| During SWAP | Database rolls back | Old data | Retry SWAP | +| After SWAP, before STATE | New data committed | New data (correct!) | Platform retries, detects completion via generationId | + +**Key Insight:** Temp table strategy ensures real table is never partially updated. + +**Covered in:** Phases 1-11 of step-by-step-guide.md + +--- + +## CDK Version Pinning + +### Required Setup + +**File:** `destination-{db}/gradle.properties` + +```properties +# Always pin to a specific version for production +cdkVersion=0.1.76 +``` + +### Pinning Strategy + +**Production connectors (merged to main):** +- ✅ Must use pinned version: `cdkVersion=0.1.76` +- ❌ Never use: `cdkVersion=local` + +**During CDK development:** +- Use `cdkVersion=local` for faster iteration +- Switch back to pinned version before merging + +### How It Works + +**The `airbyte-bulk-connector` plugin:** +1. Reads `cdkVersion` from `gradle.properties` +2. If pinned (e.g., `0.1.76`): Resolves Maven artifacts + - `io.airbyte.bulk-cdk:bulk-cdk-core-load:0.1.76` + - `io.airbyte.bulk-cdk:bulk-cdk-toolkits-load-db:0.1.76` +3. If `local`: Uses project references + - `:airbyte-cdk:bulk:core:load` + - `:airbyte-cdk:bulk:toolkits:load-db` + +### Verify Pinning + +```bash +./gradlew :destination-{db}:dependencies --configuration runtimeClasspath | grep bulk-cdk +``` + +**Expected (pinned):** +``` +io.airbyte.bulk-cdk:bulk-cdk-core-load:0.1.76 +``` + +**Wrong (local):** +``` +project :airbyte-cdk:bulk:core:load +``` + +### Upgrade CDK Version + +**Manual:** +```bash +# Edit gradle.properties +cdkVersion=0.1.76 # Update to new version +``` + +**Automated:** +```bash +./gradlew destination-{db}:upgradeCdk --cdkVersion=0.1.76 +``` + +### Check Latest CDK Version + +```bash +cat airbyte-cdk/bulk/version.properties +``` + +--- + +## Time Estimates + +| Component | Effort | Lines | Time | +|-----------|--------|-------|------| +| SQL Generator | High | 300-500 | 1-2 days | +| Database Client | High | 400-600 | 1-2 days | +| Insert Buffer | Medium | 200-300 | 0.5-1 day | +| Column Utilities | Medium | 100-200 | 0.5 day | +| Configuration | Low | 100-150 | 0.5 day | +| Name Generators | Low | 50-100 | 0.25 day | +| Checker | Low | 50-80 | 0.25 day | +| Writer | Low | 80-120 | 0.25 day | +| Boilerplate | Minimal | 100-150 | 0.5 day | +| Testing | Medium | - | 2-3 days | +| **Total** | - | **~2000-3000** | **~1 week** | + +**Critical Path:** SqlGenerator → Client → InsertBuffer → ColumnUtils diff --git a/connector-writer/destination/preflight-checklist.md b/connector-writer/destination/preflight-checklist.md new file mode 100644 index 00000000000..2361cbdd871 --- /dev/null +++ b/connector-writer/destination/preflight-checklist.md @@ -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 diff --git a/connector-writer/destination/step-by-step/0-introduction.md b/connector-writer/destination/step-by-step/0-introduction.md new file mode 100644 index 00000000000..5cbcc169eb0 --- /dev/null +++ b/connector-writer/destination/step-by-step/0-introduction.md @@ -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) diff --git a/connector-writer/destination/step-by-step/1-getting-started.md b/connector-writer/destination/step-by-step/1-getting-started.md new file mode 100644 index 00000000000..dcb08fb0cf7 --- /dev/null +++ b/connector-writer/destination/step-by-step/1-getting-started.md @@ -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) { + 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. diff --git a/connector-writer/destination/step-by-step/2-database-setup.md b/connector-writer/destination/step-by-step/2-database-setup.md new file mode 100644 index 00000000000..1f4c794c5da --- /dev/null +++ b/connector-writer/destination/step-by-step/2-database-setup.md @@ -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> + ) { + // Implement in Phase 3 + TODO("Implement in Phase 3") + } + + override suspend fun readTable(table: TableName): List> { + // 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> +) { + 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> { + val results = mutableListOf>() + + 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() + 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. diff --git a/connector-writer/destination/step-by-step/3-write-infrastructure.md b/connector-writer/destination/step-by-step/3-write-infrastructure.md new file mode 100644 index 00000000000..14d91bc8594 --- /dev/null +++ b/connector-writer/destination/step-by-step/3-write-infrastructure.md @@ -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 +``` + +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 { + return {DB}DirectLoadDatabaseInitialStatusGatherer(client, tempTableNameGenerator) +} +``` + +⚠️ **CRITICAL:** This bean registration was MISSING in V1 guide! + +**Why this is needed:** +- Writer requires `DatabaseInitialStatusGatherer` 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` +- 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. diff --git a/connector-writer/destination/step-by-step/4-write-operations.md b/connector-writer/destination/step-by-step/4-write-operations.md new file mode 100644 index 00000000000..77870c71a4f --- /dev/null +++ b/connector-writer/destination/step-by-step/4-write-operations.md @@ -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>() + private var recordCount = 0 + + fun accumulate(recordFields: Map) { + 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 + ) { + 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) { + 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, +) : 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, + private val streamStateStore: StreamStateStore, + private val client: {DB}AirbyteClient, + private val tempTableNameGenerator: TempTableNameGenerator, +) : DestinationWriter { + + private lateinit var initialStatuses: Map + + 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 { + // 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. diff --git a/connector-writer/destination/step-by-step/5-advanced-features.md b/connector-writer/destination/step-by-step/5-advanced-features.md new file mode 100644 index 00000000000..cc9d84be70b --- /dev/null +++ b/connector-writer/destination/step-by-step/5-advanced-features.md @@ -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 { + val columns = mutableMapOf() + + 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, + columnsToDrop: Map, + columnsToChange: Map, +): Set { + val statements = mutableSetOf() + + // 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 { + 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 { + 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 { + 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. diff --git a/connector-writer/destination/step-by-step/6-testing.md b/connector-writer/destination/step-by-step/6-testing.md new file mode 100644 index 00000000000..8ac4a59fe4a --- /dev/null +++ b/connector-writer/destination/step-by-step/6-testing.md @@ -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 { + val tableName = stream.descriptor.name // Or use name generator + val namespace = stream.descriptor.namespace ?: "test" + + val records = mutableListOf() + + 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() + + 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() + + 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 | 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! diff --git a/connector-writer/destination/step-by-step/7-troubleshooting.md b/connector-writer/destination/step-by-step/7-troubleshooting.md new file mode 100644 index 00000000000..3b8f58b01e9 --- /dev/null +++ b/connector-writer/destination/step-by-step/7-troubleshooting.md @@ -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 { + 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