Skip to content

Commit 7c0a6c5

Browse files
convert destination-snowflake to Kotlin CDK (#36910)
not only bringing snowflake to the latest CDK but also: 1) Bringing the `SourceOperation` into production code from the test code. There's really no reason those improvements should stay out of production (and they're present in the source-snowflake) 2) adding `putTimestamp` into the `SourceOperation`, so that snowflake doesn't throw an exception at every call, which implies it also creates a new thread 3) make use of the newly added ability to filter orphan thread on shutdown. We filter all the threads created during calls to `SFStatement.close()` 4) don't always take a lock when deleting destinationStates. We now check if there's any states to delete by doing a `SELECT` (and not taking any table lock) before issuing the `DELETE` (the old behavior was causing test contention, and it's a bad idea in general) 5) only execute `airbyte_internal._airbyte_destination_state`
1 parent f23c2e6 commit 7c0a6c5

File tree

31 files changed

+338
-228
lines changed

31 files changed

+338
-228
lines changed

airbyte-cdk/java/airbyte-cdk/README.md

+3-1
Original file line numberDiff line numberDiff line change
@@ -173,7 +173,9 @@ corresponds to that version.
173173
### Java CDK
174174

175175
| Version | Date | Pull Request | Subject |
176-
|:--------|:-----------| :--------------------------------------------------------- |:---------------------------------------------------------------------------------------------------------------------------------------------------------------|
176+
| :------ | :--------- | :--------------------------------------------------------- | :------------------------------------------------------------------------------------------------------------------------------------------------------------- |
177+
| 0.31.7 | 2024-05-02 | [\#36910](https://github.com/airbytehq/airbyte/pull/36910) | changes for destination-snowflake |
178+
| 0.31.6 | 2024-05-02 | [\#37746](https://github.com/airbytehq/airbyte/pull/37746) | debuggability improvements. |
177179
| 0.31.5 | 2024-04-30 | [\#37758](https://github.com/airbytehq/airbyte/pull/37758) | Set debezium max retries to zero |
178180
| 0.31.4 | 2024-04-30 | [\#37754](https://github.com/airbytehq/airbyte/pull/37754) | Add DebeziumEngine notification log |
179181
| 0.31.3 | 2024-04-30 | [\#37726](https://github.com/airbytehq/airbyte/pull/37726) | Remove debezium retries |

airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/db/jdbc/JdbcDatabase.kt

+3
Original file line numberDiff line numberDiff line change
@@ -15,6 +15,8 @@ import java.util.function.Consumer
1515
import java.util.function.Function
1616
import java.util.stream.Stream
1717
import java.util.stream.StreamSupport
18+
import org.slf4j.Logger
19+
import org.slf4j.LoggerFactory
1820

1921
/** Database object for interacting with a JDBC connection. */
2022
abstract class JdbcDatabase(protected val sourceOperations: JdbcCompatibleSourceOperations<*>?) :
@@ -211,6 +213,7 @@ abstract class JdbcDatabase(protected val sourceOperations: JdbcCompatibleSource
211213
abstract fun <T> executeMetadataQuery(query: Function<DatabaseMetaData?, T>): T
212214

213215
companion object {
216+
private val LOGGER: Logger = LoggerFactory.getLogger(JdbcDatabase::class.java)
214217
/**
215218
* Map records returned in a result set. It is an "unsafe" stream because the stream must be
216219
* manually closed. Otherwise, there will be a database connection leak.

airbyte-cdk/java/airbyte-cdk/core/src/main/kotlin/io/airbyte/cdk/integrations/base/IntegrationRunner.kt

+2-2
Original file line numberDiff line numberDiff line change
@@ -370,8 +370,8 @@ internal constructor(
370370
}
371371

372372
@JvmStatic
373-
fun getThreadCreationInfo(thread: Thread): ThreadCreationInfo {
374-
return getMethod.invoke(threadCreationInfo, thread) as ThreadCreationInfo
373+
fun getThreadCreationInfo(thread: Thread): ThreadCreationInfo? {
374+
return getMethod.invoke(threadCreationInfo, thread) as ThreadCreationInfo?
375375
}
376376

377377
/**
Original file line numberDiff line numberDiff line change
@@ -1 +1 @@
1-
version=0.31.6
1+
version=0.31.7

airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/JdbcSqlOperations.kt

+1-1
Original file line numberDiff line numberDiff line change
@@ -204,7 +204,7 @@ abstract class JdbcSqlOperations : SqlOperations {
204204
}
205205
}
206206

207-
fun dropTableIfExistsQuery(schemaName: String?, tableName: String?): String {
207+
open fun dropTableIfExistsQuery(schemaName: String?, tableName: String?): String {
208208
return String.format("DROP TABLE IF EXISTS %s.%s;\n", schemaName, tableName)
209209
}
210210

airbyte-cdk/java/airbyte-cdk/db-destinations/src/main/kotlin/io/airbyte/cdk/integrations/destination/jdbc/typing_deduping/JdbcDestinationHandler.kt

+35-24
Original file line numberDiff line numberDiff line change
@@ -356,7 +356,7 @@ abstract class JdbcDestinationHandler<DestinationState>(
356356
existingTable.columns[JavaBaseConstants.COLUMN_NAME_AB_META]!!.type
357357
}
358358

359-
private fun existingSchemaMatchesStreamConfig(
359+
open protected fun existingSchemaMatchesStreamConfig(
360360
stream: StreamConfig?,
361361
existingTable: TableDefinition
362362
): Boolean {
@@ -400,6 +400,29 @@ abstract class JdbcDestinationHandler<DestinationState>(
400400
return actualColumns == intendedColumns
401401
}
402402

403+
protected open fun getDeleteStatesSql(
404+
destinationStates: Map<StreamId, DestinationState>
405+
): String {
406+
return dslContext
407+
.deleteFrom(table(quotedName(rawTableSchemaName, DESTINATION_STATE_TABLE_NAME)))
408+
.where(
409+
destinationStates.keys
410+
.stream()
411+
.map { streamId: StreamId ->
412+
field(quotedName(DESTINATION_STATE_TABLE_COLUMN_NAME))
413+
.eq(streamId.originalName)
414+
.and(
415+
field(quotedName(DESTINATION_STATE_TABLE_COLUMN_NAMESPACE))
416+
.eq(streamId.originalNamespace)
417+
)
418+
}
419+
.reduce(DSL.falseCondition()) { obj: Condition, arg2: Condition? ->
420+
obj.or(arg2)
421+
}
422+
)
423+
.getSQL(ParamType.INLINED)
424+
}
425+
403426
@Throws(Exception::class)
404427
override fun commitDestinationStates(destinationStates: Map<StreamId, DestinationState>) {
405428
try {
@@ -408,25 +431,7 @@ abstract class JdbcDestinationHandler<DestinationState>(
408431
}
409432

410433
// Delete all state records where the stream name+namespace match one of our states
411-
val deleteStates =
412-
dslContext
413-
.deleteFrom(table(quotedName(rawTableSchemaName, DESTINATION_STATE_TABLE_NAME)))
414-
.where(
415-
destinationStates.keys
416-
.stream()
417-
.map { streamId: StreamId ->
418-
field(quotedName(DESTINATION_STATE_TABLE_COLUMN_NAME))
419-
.eq(streamId.originalName)
420-
.and(
421-
field(quotedName(DESTINATION_STATE_TABLE_COLUMN_NAMESPACE))
422-
.eq(streamId.originalNamespace)
423-
)
424-
}
425-
.reduce(DSL.falseCondition()) { obj: Condition, arg2: Condition? ->
426-
obj.or(arg2)
427-
}
428-
)
429-
.getSQL(ParamType.INLINED)
434+
var deleteStates = getDeleteStatesSql(destinationStates)
430435

431436
// Reinsert all of our states
432437
var insertStatesStep =
@@ -461,12 +466,17 @@ abstract class JdbcDestinationHandler<DestinationState>(
461466
}
462467
val insertStates = insertStatesStep.getSQL(ParamType.INLINED)
463468

464-
jdbcDatabase.executeWithinTransaction(listOf(deleteStates, insertStates))
469+
executeWithinTransaction(listOf(deleteStates, insertStates))
465470
} catch (e: Exception) {
466471
LOGGER.warn("Failed to commit destination states", e)
467472
}
468473
}
469474

475+
@Throws(Exception::class)
476+
protected open fun executeWithinTransaction(statements: List<String>) {
477+
jdbcDatabase.executeWithinTransaction(statements)
478+
}
479+
470480
/**
471481
* Convert to the TYPE_NAME retrieved from [java.sql.DatabaseMetaData.getColumns]
472482
*
@@ -479,9 +489,9 @@ abstract class JdbcDestinationHandler<DestinationState>(
479489

480490
companion object {
481491
private val LOGGER: Logger = LoggerFactory.getLogger(JdbcDestinationHandler::class.java)
482-
private const val DESTINATION_STATE_TABLE_NAME = "_airbyte_destination_state"
483-
private const val DESTINATION_STATE_TABLE_COLUMN_NAME = "name"
484-
private const val DESTINATION_STATE_TABLE_COLUMN_NAMESPACE = "namespace"
492+
protected const val DESTINATION_STATE_TABLE_NAME = "_airbyte_destination_state"
493+
protected const val DESTINATION_STATE_TABLE_COLUMN_NAME = "name"
494+
protected const val DESTINATION_STATE_TABLE_COLUMN_NAMESPACE = "namespace"
485495
private const val DESTINATION_STATE_TABLE_COLUMN_STATE = "destination_state"
486496
private const val DESTINATION_STATE_TABLE_COLUMN_UPDATED_AT = "updated_at"
487497

@@ -542,6 +552,7 @@ abstract class JdbcDestinationHandler<DestinationState>(
542552
return Optional.of(TableDefinition(retrievedColumnDefns))
543553
}
544554

555+
@JvmStatic
545556
fun fromIsNullableIsoString(isNullable: String?): Boolean {
546557
return "YES".equals(isNullable, ignoreCase = true)
547558
}

airbyte-cdk/java/airbyte-cdk/db-destinations/src/testFixtures/kotlin/io/airbyte/cdk/integrations/standardtest/destination/LocalAirbyteDestination.kt

+1-1
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,7 @@ class LocalAirbyteDestination(private val dest: Destination) : AirbyteDestinatio
6161
return isClosed
6262
}
6363

64-
override val exitValue = 0
64+
override var exitValue = 0
6565

6666
override fun attemptRead(): Optional<io.airbyte.protocol.models.AirbyteMessage> {
6767
return Optional.empty()

airbyte-cdk/java/airbyte-cdk/dependencies/src/testFixtures/kotlin/io/airbyte/workers/internal/AirbyteDestination.kt

+1-1
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ interface AirbyteDestination : CheckedConsumer<AirbyteMessage, Exception>, AutoC
7272
* @return exit code of the destination process
7373
* @throws IllegalStateException if the destination process has not exited
7474
*/
75-
abstract val exitValue: Int
75+
val exitValue: Int
7676

7777
/**
7878
* Attempts to read an AirbyteMessage from the Destination.

airbyte-cdk/java/airbyte-cdk/typing-deduping/src/main/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseDestinationV1V2Migrator.kt

+1-1
Original file line numberDiff line numberDiff line change
@@ -171,7 +171,7 @@ abstract class BaseDestinationV1V2Migrator<DialectTableDefinition> : Destination
171171
* @return whether it exists and is in the correct format
172172
*/
173173
@Throws(Exception::class)
174-
protected fun doesValidV1RawTableExist(namespace: String?, tableName: String?): Boolean {
174+
protected open fun doesValidV1RawTableExist(namespace: String?, tableName: String?): Boolean {
175175
val existingV1RawTable = getTableIfExists(namespace, tableName)
176176
return existingV1RawTable.isPresent &&
177177
doesV1RawTableMatchExpectedSchema(existingV1RawTable.get())

airbyte-cdk/java/airbyte-cdk/typing-deduping/src/testFixtures/kotlin/io/airbyte/integrations/base/destination/typing_deduping/BaseSqlGeneratorIntegrationTest.kt

+8-5
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestina
8080
* Subclasses should override this method if they need to make changes to the stream ID. For
8181
* example, you could upcase the final table name here.
8282
*/
83-
protected fun buildStreamId(
83+
open protected fun buildStreamId(
8484
namespace: String,
8585
finalTableName: String,
8686
rawTableName: String
@@ -149,7 +149,7 @@ abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestina
149149
/** Identical to [BaseTypingDedupingTest.getRawMetadataColumnNames]. */
150150
get() = HashMap()
151151

152-
protected val finalMetadataColumnNames: Map<String, String>
152+
open protected val finalMetadataColumnNames: Map<String, String>
153153
/** Identical to [BaseTypingDedupingTest.getFinalMetadataColumnNames]. */
154154
get() = HashMap()
155155

@@ -728,7 +728,7 @@ abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestina
728728
*/
729729
@Test
730730
@Throws(Exception::class)
731-
fun ignoreOldRawRecords() {
731+
open fun ignoreOldRawRecords() {
732732
createRawTable(streamId)
733733
createFinalTable(incrementalAppendStream, "")
734734
insertRawTableRecords(
@@ -1519,7 +1519,10 @@ abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestina
15191519
executeSoftReset(generator, destinationHandler, incrementalAppendStream)
15201520
}
15211521

1522-
protected fun migrationAssertions(v1RawRecords: List<JsonNode>, v2RawRecords: List<JsonNode>) {
1522+
protected open fun migrationAssertions(
1523+
v1RawRecords: List<JsonNode>,
1524+
v2RawRecords: List<JsonNode>
1525+
) {
15231526
val v2RecordMap =
15241527
v2RawRecords
15251528
.stream()
@@ -1570,7 +1573,7 @@ abstract class BaseSqlGeneratorIntegrationTest<DestinationState : MinimumDestina
15701573
}
15711574

15721575
@Throws(Exception::class)
1573-
protected fun dumpV1RawTableRecords(streamId: StreamId): List<JsonNode> {
1576+
open protected fun dumpV1RawTableRecords(streamId: StreamId): List<JsonNode> {
15741577
return dumpRawTableRecords(streamId)
15751578
}
15761579

airbyte-integrations/connectors/destination-snowflake/build.gradle

+1-1
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,7 @@ plugins {
33
}
44

55
airbyteJavaConnector {
6-
cdkVersionRequired = '0.27.7'
6+
cdkVersionRequired = '0.31.7'
77
features = ['db-destinations', 's3-destinations', 'typing-deduping']
88
useLocalCdk = false
99
}
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
11
# currently limit the number of parallel threads until further investigation into the issues \
22
# where Snowflake will fail to login using config credentials
3-
testExecutionConcurrency=4
3+
testExecutionConcurrency=-1
44
JunitMethodExecutionTimeout=15 m

airbyte-integrations/connectors/destination-snowflake/metadata.yaml

+1-1
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@ data:
55
connectorSubtype: database
66
connectorType: destination
77
definitionId: 424892c4-daac-4491-b35d-c6688ba547ba
8-
dockerImageTag: 3.7.0
8+
dockerImageTag: 3.7.1
99
dockerRepository: airbyte/destination-snowflake
1010
documentationUrl: https://docs.airbyte.com/integrations/destinations/snowflake
1111
githubIssueLabel: destination-snowflake

airbyte-integrations/connectors/destination-snowflake/src/main/java/io/airbyte/integrations/destination/snowflake/SnowflakeDatabase.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -197,7 +197,7 @@ private static String getAccessTokenUsingRefreshToken(final String hostName,
197197
}
198198

199199
public static JdbcDatabase getDatabase(final DataSource dataSource) {
200-
return new DefaultJdbcDatabase(dataSource);
200+
return new DefaultJdbcDatabase(dataSource, new SnowflakeSourceOperations());
201201
}
202202

203203
private static Runnable getRefreshTokenTask(final HikariDataSource dataSource) {

airbyte-integrations/connectors/destination-snowflake/src/main/java/io/airbyte/integrations/destination/snowflake/SnowflakeDestinationRunner.java

+14
Original file line numberDiff line numberDiff line change
@@ -7,12 +7,26 @@
77
import static io.airbyte.integrations.destination.snowflake.SnowflakeDestination.SCHEDULED_EXECUTOR_SERVICE;
88

99
import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler;
10+
import io.airbyte.cdk.integrations.base.IntegrationRunner;
1011
import io.airbyte.cdk.integrations.base.adaptive.AdaptiveDestinationRunner;
12+
import net.snowflake.client.core.SFSession;
13+
import net.snowflake.client.core.SFStatement;
1114
import net.snowflake.client.jdbc.SnowflakeSQLException;
1215

1316
public class SnowflakeDestinationRunner {
1417

1518
public static void main(final String[] args) throws Exception {
19+
IntegrationRunner.addOrphanedThreadFilter((Thread t) -> {
20+
for (StackTraceElement stackTraceElement : IntegrationRunner.getThreadCreationInfo(t).getStack()) {
21+
String stackClassName = stackTraceElement.getClassName();
22+
String stackMethodName = stackTraceElement.getMethodName();
23+
if (SFStatement.class.getCanonicalName().equals(stackClassName) && "close".equals(stackMethodName) ||
24+
SFSession.class.getCanonicalName().equals(stackClassName) && "callHeartBeatWithQueryTimeout".equals(stackMethodName)) {
25+
return false;
26+
}
27+
}
28+
return true;
29+
});
1630
AirbyteExceptionHandler.addThrowableForDeinterpolation(SnowflakeSQLException.class);
1731
AdaptiveDestinationRunner.baseOnEnv()
1832
.withOssDestination(() -> new SnowflakeDestination(OssCloudEnvVarConsts.AIRBYTE_OSS))

airbyte-integrations/connectors/destination-snowflake/src/main/java/io/airbyte/integrations/destination/snowflake/SnowflakeInternalStagingDestination.java

+3-2
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@
1010
import io.airbyte.cdk.db.jdbc.JdbcUtils;
1111
import io.airbyte.cdk.integrations.base.Destination;
1212
import io.airbyte.cdk.integrations.base.JavaBaseConstants;
13+
import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns;
1314
import io.airbyte.cdk.integrations.base.SerializedAirbyteMessageConsumer;
1415
import io.airbyte.cdk.integrations.base.TypingAndDedupingFlag;
1516
import io.airbyte.cdk.integrations.destination.NamingConventionTransformer;
@@ -132,7 +133,7 @@ public JsonNode toJdbcConfig(final JsonNode config) {
132133
}
133134

134135
@Override
135-
protected JdbcSqlGenerator getSqlGenerator() {
136+
protected JdbcSqlGenerator getSqlGenerator(final JsonNode config) {
136137
throw new UnsupportedOperationException("Snowflake does not yet use the native JDBC DV2 interface");
137138
}
138139

@@ -209,7 +210,7 @@ public SerializedAirbyteMessageConsumer getSerializedMessageConsumer(final JsonN
209210
typerDeduper,
210211
parsedCatalog,
211212
defaultNamespace,
212-
true)
213+
DestinationColumns.V2_WITHOUT_META)
213214
.setBufferMemoryLimit(Optional.of(getSnowflakeBufferMemoryLimit()))
214215
.setOptimalBatchSizeBytes(
215216
// The per stream size limit is following recommendations from:
Original file line numberDiff line numberDiff line change
@@ -8,12 +8,26 @@
88
import static io.airbyte.cdk.db.jdbc.DateTimeConverter.putJavaSQLTime;
99

1010
import com.fasterxml.jackson.databind.node.ObjectNode;
11+
import io.airbyte.cdk.db.DataTypeUtils;
1112
import io.airbyte.cdk.db.jdbc.JdbcSourceOperations;
1213
import io.airbyte.commons.json.Jsons;
1314
import java.sql.ResultSet;
1415
import java.sql.SQLException;
16+
import java.time.OffsetDateTime;
17+
import java.time.format.DateTimeFormatter;
18+
import java.time.format.DateTimeFormatterBuilder;
1519

16-
public class SnowflakeTestSourceOperations extends JdbcSourceOperations {
20+
public class SnowflakeSourceOperations extends JdbcSourceOperations {
21+
22+
private static final DateTimeFormatter SNOWFLAKE_TIMESTAMPTZ_FORMATTER = new DateTimeFormatterBuilder()
23+
.parseCaseInsensitive()
24+
.append(DateTimeFormatter.ISO_LOCAL_DATE)
25+
.appendLiteral(' ')
26+
.append(DateTimeFormatter.ISO_LOCAL_TIME)
27+
.optionalStart()
28+
.appendLiteral(' ')
29+
.append(DateTimeFormatter.ofPattern("XX"))
30+
.toFormatter();
1731

1832
@Override
1933
public void copyToJsonField(final ResultSet resultSet, final int colIndex, final ObjectNode json) throws SQLException {
@@ -45,4 +59,18 @@ protected void putTime(final ObjectNode node,
4559
putJavaSQLTime(node, columnName, resultSet, index);
4660
}
4761

62+
@Override
63+
protected void putTimestampWithTimezone(final ObjectNode node, final String columnName, final ResultSet resultSet, final int index)
64+
throws SQLException {
65+
final String timestampAsString = resultSet.getString(index);
66+
OffsetDateTime timestampWithOffset = OffsetDateTime.parse(timestampAsString, SNOWFLAKE_TIMESTAMPTZ_FORMATTER);
67+
node.put(columnName, timestampWithOffset.format(DataTypeUtils.TIMESTAMPTZ_FORMATTER));
68+
}
69+
70+
protected void putTimestamp(final ObjectNode node, final String columnName, final ResultSet resultSet, final int index) throws SQLException {
71+
// for backward compatibility
72+
var instant = resultSet.getTimestamp(index).toInstant();
73+
node.put(columnName, DataTypeUtils.toISO8601StringWithMicroseconds(instant));
74+
}
75+
4876
}

airbyte-integrations/connectors/destination-snowflake/src/main/java/io/airbyte/integrations/destination/snowflake/SnowflakeSqlOperations.java

+3-3
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,7 @@
88
import io.airbyte.cdk.db.jdbc.JdbcDatabase;
99
import io.airbyte.cdk.integrations.base.DestinationConfig;
1010
import io.airbyte.cdk.integrations.base.JavaBaseConstants;
11-
import io.airbyte.cdk.integrations.destination.async.partial_messages.PartialAirbyteMessage;
11+
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage;
1212
import io.airbyte.cdk.integrations.destination.jdbc.JdbcSqlOperations;
1313
import io.airbyte.cdk.integrations.destination.jdbc.SqlOperations;
1414
import io.airbyte.cdk.integrations.destination.jdbc.SqlOperationsUtils;
@@ -37,10 +37,10 @@ public class SnowflakeSqlOperations extends JdbcSqlOperations implements SqlOper
3737
@Override
3838
public void createSchemaIfNotExists(final JdbcDatabase database, final String schemaName) throws Exception {
3939
try {
40-
if (!schemaSet.contains(schemaName) && !isSchemaExists(database, schemaName)) {
40+
if (!getSchemaSet().contains(schemaName) && !isSchemaExists(database, schemaName)) {
4141
// 1s1t is assuming a lowercase airbyte_internal schema name, so we need to quote it
4242
database.execute(String.format("CREATE SCHEMA IF NOT EXISTS \"%s\";", schemaName));
43-
schemaSet.add(schemaName);
43+
getSchemaSet().add(schemaName);
4444
}
4545
} catch (final Exception e) {
4646
throw checkForKnownConfigExceptions(e).orElseThrow(() -> e);

0 commit comments

Comments
 (0)