Merge branch 'master' into gosusnp/destination-redshift-v2
This commit is contained in:
2
.github/workflows/autodoc.yml
vendored
2
.github/workflows/autodoc.yml
vendored
@@ -104,7 +104,7 @@ jobs:
|
||||
if: steps.check-support-level.outputs.metadata_file == 'true' && steps.check-support-level.outputs.community_support == 'true'
|
||||
env:
|
||||
PROMPT_TEXT: "The commit to review is ${{ github.sha }}. This commit was pushed to master and may contain connector changes that need documentation updates."
|
||||
uses: aaronsteers/devin-action@0d74d6d9ff1b16ada5966dc31af53a9d155759f4 # Pinned to specific commit for security
|
||||
uses: aaronsteers/devin-action@98d15ae93d1848914f5ab8e9ce45341182958d27 # v0.1.7 - Pinned to specific commit for security
|
||||
with:
|
||||
devin-token: ${{ secrets.DEVIN_AI_API_KEY }}
|
||||
github-token: ${{ secrets.GITHUB_TOKEN }}
|
||||
|
||||
@@ -1,3 +1,3 @@
|
||||
testExecutionConcurrency=-1
|
||||
cdkVersion=0.1.82
|
||||
cdkVersion=0.1.91
|
||||
JunitMethodExecutionTimeout=10m
|
||||
|
||||
@@ -6,7 +6,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: destination
|
||||
definitionId: 424892c4-daac-4491-b35d-c6688ba547ba
|
||||
dockerImageTag: 4.0.31
|
||||
dockerImageTag: 4.0.32-rc.1
|
||||
dockerRepository: airbyte/destination-snowflake
|
||||
documentationUrl: https://docs.airbyte.com/integrations/destinations/snowflake
|
||||
githubIssueLabel: destination-snowflake
|
||||
@@ -31,6 +31,8 @@ data:
|
||||
enabled: true
|
||||
releaseStage: generally_available
|
||||
releases:
|
||||
rolloutConfiguration:
|
||||
enableProgressiveRollout: true
|
||||
breakingChanges:
|
||||
2.0.0:
|
||||
message: Remove GCS/S3 loading method support.
|
||||
|
||||
@@ -11,15 +11,18 @@ import io.airbyte.cdk.load.check.CheckOperationV2
|
||||
import io.airbyte.cdk.load.check.DestinationCheckerV2
|
||||
import io.airbyte.cdk.load.config.DataChannelMedium
|
||||
import io.airbyte.cdk.load.dataflow.config.AggregatePublishingConfig
|
||||
import io.airbyte.cdk.load.orchestration.db.DefaultTempTableNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.table.DefaultTempTableNameGenerator
|
||||
import io.airbyte.cdk.load.table.TempTableNameGenerator
|
||||
import io.airbyte.cdk.output.OutputConsumer
|
||||
import io.airbyte.integrations.destination.snowflake.cdk.SnowflakeMigratingConfigurationSpecificationSupplier
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.KeyPairAuthConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfigurationFactory
|
||||
import io.airbyte.integrations.destination.snowflake.spec.UsernamePasswordAuthConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeRawRecordFormatter
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeRecordFormatter
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeSchemaRecordFormatter
|
||||
import io.micronaut.context.annotation.Factory
|
||||
import io.micronaut.context.annotation.Primary
|
||||
import io.micronaut.context.annotation.Requires
|
||||
@@ -204,6 +207,17 @@ class SnowflakeBeanFactory {
|
||||
outputConsumer: OutputConsumer,
|
||||
) = CheckOperationV2(destinationChecker, outputConsumer)
|
||||
|
||||
@Singleton
|
||||
fun snowflakeRecordFormatter(
|
||||
snowflakeConfiguration: SnowflakeConfiguration
|
||||
): SnowflakeRecordFormatter {
|
||||
return if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
SnowflakeRawRecordFormatter()
|
||||
} else {
|
||||
SnowflakeSchemaRecordFormatter()
|
||||
}
|
||||
}
|
||||
|
||||
@Singleton
|
||||
fun aggregatePublishingConfig(dataChannelMedium: DataChannelMedium): AggregatePublishingConfig {
|
||||
// NOT speed mode
|
||||
|
||||
@@ -13,13 +13,17 @@ import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.data.ObjectType
|
||||
import io.airbyte.cdk.load.data.StringType
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.cdk.load.schema.model.StreamTableSchema
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.schema.model.TableNames
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeInsertBuffer
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeSchemaRecordFormatter
|
||||
import jakarta.inject.Singleton
|
||||
import java.time.OffsetDateTime
|
||||
import java.util.UUID
|
||||
@@ -31,7 +35,7 @@ internal const val CHECK_COLUMN_NAME = "test_key"
|
||||
class SnowflakeChecker(
|
||||
private val snowflakeAirbyteClient: SnowflakeAirbyteClient,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
private val columnManager: SnowflakeColumnManager,
|
||||
) : DestinationCheckerV2 {
|
||||
|
||||
override fun check() {
|
||||
@@ -46,11 +50,40 @@ class SnowflakeChecker(
|
||||
Meta.AirbyteMetaFields.GENERATION_ID.fieldName to AirbyteValue.from(0),
|
||||
CHECK_COLUMN_NAME.toSnowflakeCompatibleName() to AirbyteValue.from("test-value")
|
||||
)
|
||||
val outputSchema = snowflakeConfiguration.schema.toSnowflakeCompatibleName()
|
||||
val outputSchema =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
snowflakeConfiguration.schema
|
||||
} else {
|
||||
snowflakeConfiguration.schema.toSnowflakeCompatibleName()
|
||||
}
|
||||
val tableName =
|
||||
"_airbyte_connection_test_${
|
||||
UUID.randomUUID().toString().replace("-".toRegex(), "")}".toSnowflakeCompatibleName()
|
||||
val qualifiedTableName = TableName(namespace = outputSchema, name = tableName)
|
||||
val tableSchema =
|
||||
StreamTableSchema(
|
||||
tableNames =
|
||||
TableNames(
|
||||
finalTableName = qualifiedTableName,
|
||||
tempTableName = qualifiedTableName
|
||||
),
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames =
|
||||
mapOf(
|
||||
CHECK_COLUMN_NAME to CHECK_COLUMN_NAME.toSnowflakeCompatibleName()
|
||||
),
|
||||
finalSchema =
|
||||
mapOf(
|
||||
CHECK_COLUMN_NAME.toSnowflakeCompatibleName() to
|
||||
io.airbyte.cdk.load.component.ColumnType("VARCHAR", false)
|
||||
),
|
||||
inputSchema =
|
||||
mapOf(CHECK_COLUMN_NAME to FieldType(StringType, nullable = false))
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
|
||||
val destinationStream =
|
||||
DestinationStream(
|
||||
unmappedNamespace = outputSchema,
|
||||
@@ -63,7 +96,8 @@ class SnowflakeChecker(
|
||||
generationId = 0L,
|
||||
minimumGenerationId = 0L,
|
||||
syncId = 0L,
|
||||
namespaceMapper = NamespaceMapper()
|
||||
namespaceMapper = NamespaceMapper(),
|
||||
tableSchema = tableSchema
|
||||
)
|
||||
runBlocking {
|
||||
try {
|
||||
@@ -75,14 +109,14 @@ class SnowflakeChecker(
|
||||
replace = true,
|
||||
)
|
||||
|
||||
val columns = snowflakeAirbyteClient.describeTable(qualifiedTableName)
|
||||
val snowflakeInsertBuffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = qualifiedTableName,
|
||||
columns = columns,
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
columnSchema = tableSchema.columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = SnowflakeSchemaRecordFormatter(),
|
||||
)
|
||||
|
||||
snowflakeInsertBuffer.accumulate(data)
|
||||
|
||||
@@ -13,18 +13,16 @@ import io.airbyte.cdk.load.component.TableColumns
|
||||
import io.airbyte.cdk.load.component.TableOperationsClient
|
||||
import io.airbyte.cdk.load.component.TableSchema
|
||||
import io.airbyte.cdk.load.component.TableSchemaEvolutionClient
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.util.deserializeToNode
|
||||
import io.airbyte.integrations.destination.snowflake.db.ColumnDefinition
|
||||
import io.airbyte.integrations.destination.snowflake.db.escapeJsonIdentifier
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.COUNT_TOTAL_ALIAS
|
||||
import io.airbyte.integrations.destination.snowflake.sql.NOT_NULL
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDirectLoadSqlGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.sql.andLog
|
||||
import io.airbyte.integrations.destination.snowflake.sql.escapeJsonIdentifier
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import jakarta.inject.Singleton
|
||||
import java.sql.ResultSet
|
||||
@@ -41,13 +39,10 @@ private val log = KotlinLogging.logger {}
|
||||
class SnowflakeAirbyteClient(
|
||||
private val dataSource: DataSource,
|
||||
private val sqlGenerator: SnowflakeDirectLoadSqlGenerator,
|
||||
private val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val columnManager: SnowflakeColumnManager,
|
||||
) : TableOperationsClient, TableSchemaEvolutionClient {
|
||||
|
||||
private val airbyteColumnNames =
|
||||
snowflakeColumnUtils.getFormattedDefaultColumnNames(false).toSet()
|
||||
|
||||
override suspend fun countTable(tableName: TableName): Long? =
|
||||
try {
|
||||
dataSource.connection.use { connection ->
|
||||
@@ -126,7 +121,7 @@ class SnowflakeAirbyteClient(
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
replace: Boolean
|
||||
) {
|
||||
execute(sqlGenerator.createTable(stream, tableName, columnNameMapping, replace))
|
||||
execute(sqlGenerator.createTable(tableName, stream.tableSchema, replace))
|
||||
execute(sqlGenerator.createSnowflakeStage(tableName))
|
||||
}
|
||||
|
||||
@@ -163,7 +158,15 @@ class SnowflakeAirbyteClient(
|
||||
sourceTableName: TableName,
|
||||
targetTableName: TableName
|
||||
) {
|
||||
execute(sqlGenerator.copyTable(columnNameMapping, sourceTableName, targetTableName))
|
||||
// Get all column names from the mapping (both meta columns and user columns)
|
||||
val columnNames = buildSet {
|
||||
// Add Airbyte meta columns (using uppercase constants)
|
||||
addAll(columnManager.getMetaColumnNames())
|
||||
// Add user columns from mapping
|
||||
addAll(columnNameMapping.values)
|
||||
}
|
||||
|
||||
execute(sqlGenerator.copyTable(columnNames, sourceTableName, targetTableName))
|
||||
}
|
||||
|
||||
override suspend fun upsertTable(
|
||||
@@ -172,9 +175,7 @@ class SnowflakeAirbyteClient(
|
||||
sourceTableName: TableName,
|
||||
targetTableName: TableName
|
||||
) {
|
||||
execute(
|
||||
sqlGenerator.upsertTable(stream, columnNameMapping, sourceTableName, targetTableName)
|
||||
)
|
||||
execute(sqlGenerator.upsertTable(stream.tableSchema, sourceTableName, targetTableName))
|
||||
}
|
||||
|
||||
override suspend fun dropTable(tableName: TableName) {
|
||||
@@ -206,7 +207,7 @@ class SnowflakeAirbyteClient(
|
||||
stream: DestinationStream,
|
||||
columnNameMapping: ColumnNameMapping
|
||||
): TableSchema {
|
||||
return TableSchema(getColumnsFromStream(stream, columnNameMapping))
|
||||
return TableSchema(stream.tableSchema.columnSchema.finalSchema)
|
||||
}
|
||||
|
||||
override suspend fun applyChangeset(
|
||||
@@ -253,7 +254,7 @@ class SnowflakeAirbyteClient(
|
||||
val columnName = escapeJsonIdentifier(rs.getString("name"))
|
||||
|
||||
// Filter out airbyte columns
|
||||
if (airbyteColumnNames.contains(columnName)) {
|
||||
if (columnManager.getMetaColumnNames().contains(columnName)) {
|
||||
continue
|
||||
}
|
||||
val dataType = rs.getString("type").takeWhile { char -> char != '(' }
|
||||
@@ -271,49 +272,6 @@ class SnowflakeAirbyteClient(
|
||||
}
|
||||
}
|
||||
|
||||
internal fun getColumnsFromStream(
|
||||
stream: DestinationStream,
|
||||
columnNameMapping: ColumnNameMapping
|
||||
): Map<String, ColumnType> =
|
||||
snowflakeColumnUtils
|
||||
.columnsAndTypes(stream.schema.asColumns(), columnNameMapping)
|
||||
.filter { column -> column.columnName !in airbyteColumnNames }
|
||||
.associate { column ->
|
||||
// columnsAndTypes returns types as either `FOO` or `FOO NOT NULL`.
|
||||
// so check for that suffix.
|
||||
val nullable = !column.columnType.endsWith(NOT_NULL)
|
||||
val type =
|
||||
column.columnType
|
||||
.takeWhile { char ->
|
||||
// This is to remove any precision parts of the dialect type
|
||||
char != '('
|
||||
}
|
||||
.removeSuffix(NOT_NULL)
|
||||
.trim()
|
||||
|
||||
column.columnName to ColumnType(type, nullable)
|
||||
}
|
||||
|
||||
internal fun generateSchemaChanges(
|
||||
columnsInDb: Set<ColumnDefinition>,
|
||||
columnsInStream: Set<ColumnDefinition>
|
||||
): Triple<Set<ColumnDefinition>, Set<ColumnDefinition>, Set<ColumnDefinition>> {
|
||||
val addedColumns =
|
||||
columnsInStream.filter { it.name !in columnsInDb.map { col -> col.name } }.toSet()
|
||||
val deletedColumns =
|
||||
columnsInDb.filter { it.name !in columnsInStream.map { col -> col.name } }.toSet()
|
||||
val commonColumns =
|
||||
columnsInStream.filter { it.name in columnsInDb.map { col -> col.name } }.toSet()
|
||||
val modifiedColumns =
|
||||
commonColumns
|
||||
.filter {
|
||||
val dbType = columnsInDb.find { column -> it.name == column.name }?.type
|
||||
it.type != dbType
|
||||
}
|
||||
.toSet()
|
||||
return Triple(addedColumns, deletedColumns, modifiedColumns)
|
||||
}
|
||||
|
||||
override suspend fun getGenerationId(tableName: TableName): Long =
|
||||
try {
|
||||
dataSource.connection.use { connection ->
|
||||
@@ -326,7 +284,7 @@ class SnowflakeAirbyteClient(
|
||||
* format. In order to make sure these strings will match any column names
|
||||
* that we have formatted in-memory, re-apply the escaping.
|
||||
*/
|
||||
resultSet.getLong(snowflakeColumnUtils.getGenerationIdColumnName())
|
||||
resultSet.getLong(columnManager.getGenerationIdColumnName())
|
||||
} else {
|
||||
log.warn {
|
||||
"No generation ID found for table ${tableName.toPrettyString()}, returning 0"
|
||||
@@ -351,8 +309,8 @@ class SnowflakeAirbyteClient(
|
||||
execute(sqlGenerator.putInStage(tableName, tempFilePath))
|
||||
}
|
||||
|
||||
fun copyFromStage(tableName: TableName, filename: String) {
|
||||
execute(sqlGenerator.copyFromStage(tableName, filename))
|
||||
fun copyFromStage(tableName: TableName, filename: String, columnNames: List<String>) {
|
||||
execute(sqlGenerator.copyFromStage(tableName, filename, columnNames))
|
||||
}
|
||||
|
||||
fun describeTable(tableName: TableName): LinkedHashMap<String, String> =
|
||||
|
||||
@@ -4,47 +4,41 @@
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.dataflow
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.dataflow.aggregate.Aggregate
|
||||
import io.airbyte.cdk.load.dataflow.aggregate.AggregateFactory
|
||||
import io.airbyte.cdk.load.dataflow.aggregate.StoreKey
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeInsertBuffer
|
||||
import io.micronaut.cache.annotation.CacheConfig
|
||||
import io.micronaut.cache.annotation.Cacheable
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeRecordFormatter
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
@CacheConfig("table-columns")
|
||||
// class has to be open to make the cache stuff work
|
||||
open class SnowflakeAggregateFactory(
|
||||
class SnowflakeAggregateFactory(
|
||||
private val snowflakeClient: SnowflakeAirbyteClient,
|
||||
private val streamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
private val catalog: DestinationCatalog,
|
||||
private val columnManager: SnowflakeColumnManager,
|
||||
private val snowflakeRecordFormatter: SnowflakeRecordFormatter,
|
||||
) : AggregateFactory {
|
||||
override fun create(key: StoreKey): Aggregate {
|
||||
val stream = catalog.getStream(key)
|
||||
val tableName = streamStateStore.get(key)!!.tableName
|
||||
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = getTableColumns(tableName),
|
||||
snowflakeClient = snowflakeClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
columnSchema = stream.tableSchema.columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
)
|
||||
return SnowflakeAggregate(buffer = buffer)
|
||||
}
|
||||
|
||||
// We assume that a table isn't getting altered _during_ a sync.
|
||||
// This allows us to only SHOW COLUMNS once per table per sync,
|
||||
// rather than refetching it on every aggregate.
|
||||
@Cacheable
|
||||
// function has to be open to make caching work
|
||||
internal open fun getTableColumns(tableName: TableName) =
|
||||
snowflakeClient.describeTable(tableName)
|
||||
}
|
||||
|
||||
@@ -1,13 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.db
|
||||
|
||||
/**
|
||||
* Jdbc destination column definition representation
|
||||
*
|
||||
* @param name
|
||||
* @param type
|
||||
*/
|
||||
data class ColumnDefinition(val name: String, val type: String)
|
||||
@@ -4,17 +4,17 @@
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.db
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.component.TableOperationsClient
|
||||
import io.airbyte.cdk.load.orchestration.db.BaseDirectLoadInitialStatusGatherer
|
||||
import io.airbyte.cdk.load.orchestration.db.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.table.BaseDirectLoadInitialStatusGatherer
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
class SnowflakeDirectLoadDatabaseInitialStatusGatherer(
|
||||
tableOperationsClient: TableOperationsClient,
|
||||
tempTableNameGenerator: TempTableNameGenerator,
|
||||
catalog: DestinationCatalog,
|
||||
) :
|
||||
BaseDirectLoadInitialStatusGatherer(
|
||||
tableOperationsClient,
|
||||
tempTableNameGenerator,
|
||||
catalog,
|
||||
)
|
||||
|
||||
@@ -1,95 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.db
|
||||
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.orchestration.db.ColumnNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.FinalTableNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingUtil
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.QUOTE
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
class SnowflakeFinalTableNameGenerator(private val config: SnowflakeConfiguration) :
|
||||
FinalTableNameGenerator {
|
||||
override fun getTableName(streamDescriptor: DestinationStream.Descriptor): TableName {
|
||||
val namespace = streamDescriptor.namespace ?: config.schema
|
||||
return if (!config.legacyRawTablesOnly) {
|
||||
TableName(
|
||||
namespace = namespace.toSnowflakeCompatibleName(),
|
||||
name = streamDescriptor.name.toSnowflakeCompatibleName(),
|
||||
)
|
||||
} else {
|
||||
TableName(
|
||||
namespace = config.internalTableSchema,
|
||||
name =
|
||||
TypingDedupingUtil.concatenateRawTableName(
|
||||
namespace = escapeJsonIdentifier(namespace),
|
||||
name = escapeJsonIdentifier(streamDescriptor.name),
|
||||
),
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class SnowflakeColumnNameGenerator(private val config: SnowflakeConfiguration) :
|
||||
ColumnNameGenerator {
|
||||
override fun getColumnName(column: String): ColumnNameGenerator.ColumnName {
|
||||
return if (!config.legacyRawTablesOnly) {
|
||||
ColumnNameGenerator.ColumnName(
|
||||
column.toSnowflakeCompatibleName(),
|
||||
column.toSnowflakeCompatibleName(),
|
||||
)
|
||||
} else {
|
||||
ColumnNameGenerator.ColumnName(
|
||||
column,
|
||||
column,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Escapes double-quotes in a JSON identifier by doubling them. This shit is legacy -- I don't know
|
||||
* why this would be necessary but no harm in keeping it so I am keeping it.
|
||||
*
|
||||
* @return The escaped identifier.
|
||||
*/
|
||||
fun escapeJsonIdentifier(identifier: String): String {
|
||||
// Note that we don't need to escape backslashes here!
|
||||
// The only special character in an identifier is the double-quote, which needs to be
|
||||
// doubled.
|
||||
return identifier.replace(QUOTE, "$QUOTE$QUOTE")
|
||||
}
|
||||
|
||||
/**
|
||||
* Transforms a string to be compatible with Snowflake table and column names.
|
||||
*
|
||||
* @return The transformed string suitable for Snowflake identifiers.
|
||||
*/
|
||||
fun String.toSnowflakeCompatibleName(): String {
|
||||
var identifier = this
|
||||
|
||||
// Handle empty strings
|
||||
if (identifier.isEmpty()) {
|
||||
throw ConfigErrorException("Empty string is invalid identifier")
|
||||
}
|
||||
|
||||
// Snowflake scripting language does something weird when the `${` bigram shows up in the
|
||||
// script so replace these with something else.
|
||||
// For completeness, if we trigger this, also replace closing curly braces with underscores.
|
||||
if (identifier.contains("\${")) {
|
||||
identifier = identifier.replace("$", "_").replace("{", "_").replace("}", "_")
|
||||
}
|
||||
|
||||
// Escape double quotes
|
||||
identifier = escapeJsonIdentifier(identifier)
|
||||
|
||||
return identifier.uppercase()
|
||||
}
|
||||
@@ -0,0 +1,116 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.schema
|
||||
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SNOWFLAKE_AB_EXTRACTED_AT
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SNOWFLAKE_AB_GENERATION_ID
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SNOWFLAKE_AB_META
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SNOWFLAKE_AB_RAW_ID
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDataType
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
/**
|
||||
* Manages column names and ordering for Snowflake tables based on whether legacy raw tables mode is
|
||||
* enabled.
|
||||
*
|
||||
* TODO: We should add meta column munging and raw table support to the CDK, so this extra layer of
|
||||
* management shouldn't be necessary.
|
||||
*/
|
||||
@Singleton
|
||||
class SnowflakeColumnManager(
|
||||
private val config: SnowflakeConfiguration,
|
||||
) {
|
||||
/**
|
||||
* Get the list of column names for a table in the order they should appear in the CSV file and
|
||||
* COPY INTO statement.
|
||||
*
|
||||
* Warning: MUST match the order defined in SnowflakeRecordFormatter
|
||||
*
|
||||
* @param columnSchema The schema containing column information (ignored in raw mode)
|
||||
* @return List of column names in the correct order
|
||||
*/
|
||||
fun getTableColumnNames(columnSchema: ColumnSchema): List<String> {
|
||||
return buildList {
|
||||
addAll(getMetaColumnNames())
|
||||
addAll(columnSchema.finalSchema.keys)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the list of Airbyte meta column names. In schema mode, these are uppercase. In raw mode,
|
||||
* they are lowercase and included loaded_at
|
||||
*
|
||||
* @return Set of meta column names
|
||||
*/
|
||||
fun getMetaColumnNames(): Set<String> =
|
||||
if (config.legacyRawTablesOnly) {
|
||||
Constants.rawModeMetaColNames
|
||||
} else {
|
||||
Constants.schemaModeMetaColNames
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the Airbyte meta columns as a map of column name to ColumnType. This provides both the
|
||||
* column names and their types for table creation.
|
||||
*
|
||||
* @param columnSchema The user column schema (used to check for CDC columns)
|
||||
* @return Map of meta column names to their types
|
||||
*/
|
||||
fun getMetaColumns(): LinkedHashMap<String, ColumnType> {
|
||||
return if (config.legacyRawTablesOnly) {
|
||||
Constants.rawModeMetaColumns
|
||||
} else {
|
||||
Constants.schemaModeMetaColumns
|
||||
}
|
||||
}
|
||||
|
||||
fun getGenerationIdColumnName(): String {
|
||||
return if (config.legacyRawTablesOnly) {
|
||||
Meta.COLUMN_NAME_AB_GENERATION_ID
|
||||
} else {
|
||||
SNOWFLAKE_AB_GENERATION_ID
|
||||
}
|
||||
}
|
||||
|
||||
object Constants {
|
||||
val rawModeMetaColumns =
|
||||
linkedMapOf(
|
||||
Meta.COLUMN_NAME_AB_RAW_ID to ColumnType(SnowflakeDataType.VARCHAR.typeName, false),
|
||||
Meta.COLUMN_NAME_AB_EXTRACTED_AT to
|
||||
ColumnType(
|
||||
SnowflakeDataType.TIMESTAMP_TZ.typeName,
|
||||
false,
|
||||
),
|
||||
Meta.COLUMN_NAME_AB_META to ColumnType(SnowflakeDataType.VARIANT.typeName, false),
|
||||
Meta.COLUMN_NAME_AB_GENERATION_ID to
|
||||
ColumnType(
|
||||
SnowflakeDataType.NUMBER.typeName,
|
||||
true,
|
||||
),
|
||||
Meta.COLUMN_NAME_AB_LOADED_AT to
|
||||
ColumnType(
|
||||
SnowflakeDataType.TIMESTAMP_TZ.typeName,
|
||||
true,
|
||||
),
|
||||
)
|
||||
|
||||
val schemaModeMetaColumns =
|
||||
linkedMapOf(
|
||||
SNOWFLAKE_AB_RAW_ID to ColumnType(SnowflakeDataType.VARCHAR.typeName, false),
|
||||
SNOWFLAKE_AB_EXTRACTED_AT to
|
||||
ColumnType(SnowflakeDataType.TIMESTAMP_TZ.typeName, false),
|
||||
SNOWFLAKE_AB_META to ColumnType(SnowflakeDataType.VARIANT.typeName, false),
|
||||
SNOWFLAKE_AB_GENERATION_ID to ColumnType(SnowflakeDataType.NUMBER.typeName, true),
|
||||
)
|
||||
|
||||
val rawModeMetaColNames: Set<String> = rawModeMetaColumns.keys
|
||||
|
||||
val schemaModeMetaColNames: Set<String> = schemaModeMetaColumns.keys
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,34 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.schema
|
||||
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.integrations.destination.snowflake.sql.escapeJsonIdentifier
|
||||
|
||||
/**
|
||||
* Transforms a string to be compatible with Snowflake table and column names.
|
||||
*
|
||||
* @return The transformed string suitable for Snowflake identifiers.
|
||||
*/
|
||||
fun String.toSnowflakeCompatibleName(): String {
|
||||
var identifier = this
|
||||
|
||||
// Handle empty strings
|
||||
if (identifier.isEmpty()) {
|
||||
throw ConfigErrorException("Empty string is invalid identifier")
|
||||
}
|
||||
|
||||
// Snowflake scripting language does something weird when the `${` bigram shows up in the
|
||||
// script so replace these with something else.
|
||||
// For completeness, if we trigger this, also replace closing curly braces with underscores.
|
||||
if (identifier.contains("\${")) {
|
||||
identifier = identifier.replace("$", "_").replace("{", "_").replace("}", "_")
|
||||
}
|
||||
|
||||
// Escape double quotes
|
||||
identifier = escapeJsonIdentifier(identifier)
|
||||
|
||||
return identifier.uppercase()
|
||||
}
|
||||
@@ -0,0 +1,121 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.schema
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.data.ArrayType
|
||||
import io.airbyte.cdk.load.data.ArrayTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.BooleanType
|
||||
import io.airbyte.cdk.load.data.DateType
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.data.IntegerType
|
||||
import io.airbyte.cdk.load.data.NumberType
|
||||
import io.airbyte.cdk.load.data.ObjectType
|
||||
import io.airbyte.cdk.load.data.ObjectTypeWithEmptySchema
|
||||
import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.StringType
|
||||
import io.airbyte.cdk.load.data.TimeTypeWithTimezone
|
||||
import io.airbyte.cdk.load.data.TimeTypeWithoutTimezone
|
||||
import io.airbyte.cdk.load.data.TimestampTypeWithTimezone
|
||||
import io.airbyte.cdk.load.data.TimestampTypeWithoutTimezone
|
||||
import io.airbyte.cdk.load.data.UnionType
|
||||
import io.airbyte.cdk.load.data.UnknownType
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.schema.TableSchemaMapper
|
||||
import io.airbyte.cdk.load.schema.model.StreamTableSchema
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.table.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.table.TypingDedupingUtil
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDataType
|
||||
import io.airbyte.integrations.destination.snowflake.sql.escapeJsonIdentifier
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
class SnowflakeTableSchemaMapper(
|
||||
private val config: SnowflakeConfiguration,
|
||||
private val tempTableNameGenerator: TempTableNameGenerator,
|
||||
) : TableSchemaMapper {
|
||||
override fun toFinalTableName(desc: DestinationStream.Descriptor): TableName {
|
||||
val namespace = desc.namespace ?: config.schema
|
||||
return if (!config.legacyRawTablesOnly) {
|
||||
TableName(
|
||||
namespace = namespace.toSnowflakeCompatibleName(),
|
||||
name = desc.name.toSnowflakeCompatibleName(),
|
||||
)
|
||||
} else {
|
||||
TableName(
|
||||
namespace = config.internalTableSchema,
|
||||
name =
|
||||
TypingDedupingUtil.concatenateRawTableName(
|
||||
namespace = escapeJsonIdentifier(namespace),
|
||||
name = escapeJsonIdentifier(desc.name),
|
||||
),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
override fun toTempTableName(tableName: TableName): TableName {
|
||||
return tempTableNameGenerator.generate(tableName)
|
||||
}
|
||||
|
||||
override fun toColumnName(name: String): String {
|
||||
return if (!config.legacyRawTablesOnly) {
|
||||
name.toSnowflakeCompatibleName()
|
||||
} else {
|
||||
// In legacy mode, column names are not transformed
|
||||
name
|
||||
}
|
||||
}
|
||||
|
||||
override fun toColumnType(fieldType: FieldType): ColumnType {
|
||||
val snowflakeType =
|
||||
when (fieldType.type) {
|
||||
// Simple types
|
||||
BooleanType -> SnowflakeDataType.BOOLEAN.typeName
|
||||
IntegerType -> SnowflakeDataType.NUMBER.typeName
|
||||
NumberType -> SnowflakeDataType.FLOAT.typeName
|
||||
StringType -> SnowflakeDataType.VARCHAR.typeName
|
||||
|
||||
// Temporal types
|
||||
DateType -> SnowflakeDataType.DATE.typeName
|
||||
TimeTypeWithTimezone -> SnowflakeDataType.VARCHAR.typeName
|
||||
TimeTypeWithoutTimezone -> SnowflakeDataType.TIME.typeName
|
||||
TimestampTypeWithTimezone -> SnowflakeDataType.TIMESTAMP_TZ.typeName
|
||||
TimestampTypeWithoutTimezone -> SnowflakeDataType.TIMESTAMP_NTZ.typeName
|
||||
|
||||
// Semistructured types
|
||||
is ArrayType,
|
||||
ArrayTypeWithoutSchema -> SnowflakeDataType.ARRAY.typeName
|
||||
is ObjectType,
|
||||
ObjectTypeWithEmptySchema,
|
||||
ObjectTypeWithoutSchema -> SnowflakeDataType.OBJECT.typeName
|
||||
is UnionType -> SnowflakeDataType.VARIANT.typeName
|
||||
is UnknownType -> SnowflakeDataType.VARIANT.typeName
|
||||
}
|
||||
|
||||
return ColumnType(snowflakeType, fieldType.nullable)
|
||||
}
|
||||
|
||||
override fun toFinalSchema(tableSchema: StreamTableSchema): StreamTableSchema {
|
||||
if (!config.legacyRawTablesOnly) {
|
||||
return tableSchema
|
||||
}
|
||||
|
||||
return StreamTableSchema(
|
||||
tableNames = tableSchema.tableNames,
|
||||
columnSchema =
|
||||
tableSchema.columnSchema.copy(
|
||||
finalSchema =
|
||||
mapOf(
|
||||
Meta.COLUMN_NAME_DATA to
|
||||
ColumnType(SnowflakeDataType.OBJECT.typeName, false)
|
||||
)
|
||||
),
|
||||
importType = tableSchema.importType,
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -1,201 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.sql
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting
|
||||
import io.airbyte.cdk.load.data.AirbyteType
|
||||
import io.airbyte.cdk.load.data.ArrayType
|
||||
import io.airbyte.cdk.load.data.ArrayTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.BooleanType
|
||||
import io.airbyte.cdk.load.data.DateType
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.data.IntegerType
|
||||
import io.airbyte.cdk.load.data.NumberType
|
||||
import io.airbyte.cdk.load.data.ObjectType
|
||||
import io.airbyte.cdk.load.data.ObjectTypeWithEmptySchema
|
||||
import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.StringType
|
||||
import io.airbyte.cdk.load.data.TimeTypeWithTimezone
|
||||
import io.airbyte.cdk.load.data.TimeTypeWithoutTimezone
|
||||
import io.airbyte.cdk.load.data.TimestampTypeWithTimezone
|
||||
import io.airbyte.cdk.load.data.TimestampTypeWithoutTimezone
|
||||
import io.airbyte.cdk.load.data.UnionType
|
||||
import io.airbyte.cdk.load.data.UnknownType
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_EXTRACTED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_GENERATION_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_LOADED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_META
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_RAW_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_DATA
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.integrations.destination.snowflake.db.SnowflakeColumnNameGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import jakarta.inject.Singleton
|
||||
import kotlin.collections.component1
|
||||
import kotlin.collections.component2
|
||||
import kotlin.collections.joinToString
|
||||
import kotlin.collections.map
|
||||
import kotlin.collections.plus
|
||||
|
||||
internal const val NOT_NULL = "NOT NULL"
|
||||
|
||||
internal val DEFAULT_COLUMNS =
|
||||
listOf(
|
||||
ColumnAndType(
|
||||
columnName = COLUMN_NAME_AB_RAW_ID,
|
||||
columnType = "${SnowflakeDataType.VARCHAR.typeName} $NOT_NULL"
|
||||
),
|
||||
ColumnAndType(
|
||||
columnName = COLUMN_NAME_AB_EXTRACTED_AT,
|
||||
columnType = "${SnowflakeDataType.TIMESTAMP_TZ.typeName} $NOT_NULL"
|
||||
),
|
||||
ColumnAndType(
|
||||
columnName = COLUMN_NAME_AB_META,
|
||||
columnType = "${SnowflakeDataType.VARIANT.typeName} $NOT_NULL"
|
||||
),
|
||||
ColumnAndType(
|
||||
columnName = COLUMN_NAME_AB_GENERATION_ID,
|
||||
columnType = SnowflakeDataType.NUMBER.typeName
|
||||
),
|
||||
)
|
||||
|
||||
internal val RAW_DATA_COLUMN =
|
||||
ColumnAndType(
|
||||
columnName = COLUMN_NAME_DATA,
|
||||
columnType = "${SnowflakeDataType.VARIANT.typeName} $NOT_NULL"
|
||||
)
|
||||
|
||||
internal val RAW_COLUMNS =
|
||||
listOf(
|
||||
ColumnAndType(
|
||||
columnName = COLUMN_NAME_AB_LOADED_AT,
|
||||
columnType = SnowflakeDataType.TIMESTAMP_TZ.typeName
|
||||
),
|
||||
RAW_DATA_COLUMN
|
||||
)
|
||||
|
||||
@Singleton
|
||||
class SnowflakeColumnUtils(
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val snowflakeColumnNameGenerator: SnowflakeColumnNameGenerator,
|
||||
) {
|
||||
|
||||
@VisibleForTesting
|
||||
internal fun defaultColumns(): List<ColumnAndType> =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
DEFAULT_COLUMNS + RAW_COLUMNS
|
||||
} else {
|
||||
DEFAULT_COLUMNS
|
||||
}
|
||||
|
||||
internal fun formattedDefaultColumns(): List<ColumnAndType> =
|
||||
defaultColumns().map {
|
||||
ColumnAndType(
|
||||
columnName = formatColumnName(it.columnName, false),
|
||||
columnType = it.columnType,
|
||||
)
|
||||
}
|
||||
|
||||
fun getGenerationIdColumnName(): String {
|
||||
return if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
COLUMN_NAME_AB_GENERATION_ID
|
||||
} else {
|
||||
COLUMN_NAME_AB_GENERATION_ID.toSnowflakeCompatibleName()
|
||||
}
|
||||
}
|
||||
|
||||
fun getColumnNames(columnNameMapping: ColumnNameMapping): String =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
getFormattedDefaultColumnNames(true).joinToString(",")
|
||||
} else {
|
||||
(getFormattedDefaultColumnNames(true) +
|
||||
columnNameMapping.map { (_, actualName) -> actualName.quote() })
|
||||
.joinToString(",")
|
||||
}
|
||||
|
||||
fun getFormattedDefaultColumnNames(quote: Boolean = false): List<String> =
|
||||
defaultColumns().map { formatColumnName(it.columnName, quote) }
|
||||
|
||||
fun getFormattedColumnNames(
|
||||
columns: Map<String, FieldType>,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
quote: Boolean = true,
|
||||
): List<String> =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
getFormattedDefaultColumnNames(quote)
|
||||
} else {
|
||||
getFormattedDefaultColumnNames(quote) +
|
||||
columns.map { (fieldName, _) ->
|
||||
val columnName = columnNameMapping[fieldName] ?: fieldName
|
||||
if (quote) columnName.quote() else columnName
|
||||
}
|
||||
}
|
||||
|
||||
fun columnsAndTypes(
|
||||
columns: Map<String, FieldType>,
|
||||
columnNameMapping: ColumnNameMapping
|
||||
): List<ColumnAndType> =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
formattedDefaultColumns()
|
||||
} else {
|
||||
formattedDefaultColumns() +
|
||||
columns.map { (fieldName, type) ->
|
||||
val columnName = columnNameMapping[fieldName] ?: fieldName
|
||||
val typeName = toDialectType(type.type)
|
||||
ColumnAndType(
|
||||
columnName = columnName,
|
||||
columnType = if (type.nullable) typeName else "$typeName $NOT_NULL",
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
fun formatColumnName(
|
||||
columnName: String,
|
||||
quote: Boolean = true,
|
||||
): String {
|
||||
val formattedColumnName =
|
||||
if (columnName == COLUMN_NAME_DATA) columnName
|
||||
else snowflakeColumnNameGenerator.getColumnName(columnName).displayName
|
||||
return if (quote) formattedColumnName.quote() else formattedColumnName
|
||||
}
|
||||
|
||||
fun toDialectType(type: AirbyteType): String =
|
||||
when (type) {
|
||||
// Simple types
|
||||
BooleanType -> SnowflakeDataType.BOOLEAN.typeName
|
||||
IntegerType -> SnowflakeDataType.NUMBER.typeName
|
||||
NumberType -> SnowflakeDataType.FLOAT.typeName
|
||||
StringType -> SnowflakeDataType.VARCHAR.typeName
|
||||
|
||||
// Temporal types
|
||||
DateType -> SnowflakeDataType.DATE.typeName
|
||||
TimeTypeWithTimezone -> SnowflakeDataType.VARCHAR.typeName
|
||||
TimeTypeWithoutTimezone -> SnowflakeDataType.TIME.typeName
|
||||
TimestampTypeWithTimezone -> SnowflakeDataType.TIMESTAMP_TZ.typeName
|
||||
TimestampTypeWithoutTimezone -> SnowflakeDataType.TIMESTAMP_NTZ.typeName
|
||||
|
||||
// Semistructured types
|
||||
is ArrayType,
|
||||
ArrayTypeWithoutSchema -> SnowflakeDataType.ARRAY.typeName
|
||||
is ObjectType,
|
||||
ObjectTypeWithEmptySchema,
|
||||
ObjectTypeWithoutSchema -> SnowflakeDataType.OBJECT.typeName
|
||||
is UnionType -> SnowflakeDataType.VARIANT.typeName
|
||||
is UnknownType -> SnowflakeDataType.VARIANT.typeName
|
||||
}
|
||||
}
|
||||
|
||||
data class ColumnAndType(val columnName: String, val columnType: String) {
|
||||
override fun toString(): String {
|
||||
return "${columnName.quote()} $columnType"
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Surrounds the string instance with double quotation marks (e.g. "some string" -> "\"some
|
||||
* string\"").
|
||||
*/
|
||||
fun String.quote() = "$QUOTE$this$QUOTE"
|
||||
@@ -10,7 +10,7 @@ package io.airbyte.integrations.destination.snowflake.sql
|
||||
*/
|
||||
enum class SnowflakeDataType(val typeName: String) {
|
||||
// Numeric types
|
||||
NUMBER("NUMBER(38,0)"),
|
||||
NUMBER("NUMBER"),
|
||||
FLOAT("FLOAT"),
|
||||
|
||||
// String & binary types
|
||||
|
||||
@@ -4,16 +4,19 @@
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.sql
|
||||
|
||||
import io.airbyte.cdk.load.command.Dedupe
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import com.google.common.annotations.VisibleForTesting
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.component.ColumnTypeChange
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_EXTRACTED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_GENERATION_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_META
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_RAW_ID
|
||||
import io.airbyte.cdk.load.schema.model.StreamTableSchema
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.table.CDC_DELETED_AT_COLUMN
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.util.UUIDGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.CdcDeletionMode
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.CSV_FIELD_SEPARATOR
|
||||
@@ -22,6 +25,14 @@ import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
internal const val COUNT_TOTAL_ALIAS = "TOTAL"
|
||||
internal const val NOT_NULL = "NOT NULL"
|
||||
|
||||
// Snowflake-compatible (uppercase) versions of the Airbyte meta column names
|
||||
internal val SNOWFLAKE_AB_RAW_ID = COLUMN_NAME_AB_RAW_ID.toSnowflakeCompatibleName()
|
||||
internal val SNOWFLAKE_AB_EXTRACTED_AT = COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()
|
||||
internal val SNOWFLAKE_AB_META = COLUMN_NAME_AB_META.toSnowflakeCompatibleName()
|
||||
internal val SNOWFLAKE_AB_GENERATION_ID = COLUMN_NAME_AB_GENERATION_ID.toSnowflakeCompatibleName()
|
||||
internal val SNOWFLAKE_AB_CDC_DELETED_AT_COLUMN = CDC_DELETED_AT_COLUMN.toSnowflakeCompatibleName()
|
||||
|
||||
private val log = KotlinLogging.logger {}
|
||||
|
||||
@@ -36,80 +47,91 @@ fun String.andLog(): String {
|
||||
|
||||
@Singleton
|
||||
class SnowflakeDirectLoadSqlGenerator(
|
||||
private val columnUtils: SnowflakeColumnUtils,
|
||||
private val uuidGenerator: UUIDGenerator,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val snowflakeSqlNameUtils: SnowflakeSqlNameUtils,
|
||||
private val config: SnowflakeConfiguration,
|
||||
private val columnManager: SnowflakeColumnManager,
|
||||
) {
|
||||
fun countTable(tableName: TableName): String {
|
||||
return "SELECT COUNT(*) AS $COUNT_TOTAL_ALIAS FROM ${snowflakeSqlNameUtils.fullyQualifiedName(tableName)}".andLog()
|
||||
return "SELECT COUNT(*) AS $COUNT_TOTAL_ALIAS FROM ${fullyQualifiedName(tableName)}".andLog()
|
||||
}
|
||||
|
||||
fun createNamespace(namespace: String): String {
|
||||
return "CREATE SCHEMA IF NOT EXISTS ${snowflakeSqlNameUtils.fullyQualifiedNamespace(namespace)}".andLog()
|
||||
return "CREATE SCHEMA IF NOT EXISTS ${fullyQualifiedNamespace(namespace)}".andLog()
|
||||
}
|
||||
|
||||
fun createTable(
|
||||
stream: DestinationStream,
|
||||
tableName: TableName,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
tableSchema: StreamTableSchema,
|
||||
replace: Boolean
|
||||
): String {
|
||||
val finalSchema = tableSchema.columnSchema.finalSchema
|
||||
val metaColumns = columnManager.getMetaColumns()
|
||||
|
||||
// Build column declarations from the meta columns and user schema
|
||||
val columnDeclarations =
|
||||
columnUtils
|
||||
.columnsAndTypes(stream.schema.asColumns(), columnNameMapping)
|
||||
.joinToString(",\n")
|
||||
buildList {
|
||||
// Add Airbyte meta columns from the column manager
|
||||
metaColumns.forEach { (columnName, columnType) ->
|
||||
val nullability = if (columnType.nullable) "" else " NOT NULL"
|
||||
add("${columnName.quote()} ${columnType.type}$nullability")
|
||||
}
|
||||
|
||||
// Add user columns from the munged schema
|
||||
finalSchema.forEach { (columnName, columnType) ->
|
||||
val nullability = if (columnType.nullable) "" else " NOT NULL"
|
||||
add("${columnName.quote()} ${columnType.type}$nullability")
|
||||
}
|
||||
}
|
||||
.joinToString(",\n ")
|
||||
|
||||
// Snowflake supports CREATE OR REPLACE TABLE, which is simpler than drop+recreate
|
||||
val createOrReplace = if (replace) "CREATE OR REPLACE" else "CREATE"
|
||||
|
||||
val createTableStatement =
|
||||
"""
|
||||
$createOrReplace TABLE ${snowflakeSqlNameUtils.fullyQualifiedName(tableName)} (
|
||||
$columnDeclarations
|
||||
)
|
||||
""".trimIndent()
|
||||
|$createOrReplace TABLE ${fullyQualifiedName(tableName)} (
|
||||
| $columnDeclarations
|
||||
|)
|
||||
""".trimMargin() // Something was tripping up trimIndent so we opt for trimMargin
|
||||
|
||||
return createTableStatement.andLog()
|
||||
}
|
||||
|
||||
fun showColumns(tableName: TableName): String =
|
||||
"SHOW COLUMNS IN TABLE ${snowflakeSqlNameUtils.fullyQualifiedName(tableName)}".andLog()
|
||||
"SHOW COLUMNS IN TABLE ${fullyQualifiedName(tableName)}".andLog()
|
||||
|
||||
fun copyTable(
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
columnNames: Set<String>,
|
||||
sourceTableName: TableName,
|
||||
targetTableName: TableName
|
||||
): String {
|
||||
val columnNames = columnUtils.getColumnNames(columnNameMapping)
|
||||
val columnList = columnNames.joinToString(", ") { it.quote() }
|
||||
|
||||
return """
|
||||
INSERT INTO ${snowflakeSqlNameUtils.fullyQualifiedName(targetTableName)}
|
||||
INSERT INTO ${fullyQualifiedName(targetTableName)}
|
||||
(
|
||||
$columnNames
|
||||
$columnList
|
||||
)
|
||||
SELECT
|
||||
$columnNames
|
||||
FROM ${snowflakeSqlNameUtils.fullyQualifiedName(sourceTableName)}
|
||||
$columnList
|
||||
FROM ${fullyQualifiedName(sourceTableName)}
|
||||
"""
|
||||
.trimIndent()
|
||||
.andLog()
|
||||
}
|
||||
|
||||
fun upsertTable(
|
||||
stream: DestinationStream,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
tableSchema: StreamTableSchema,
|
||||
sourceTableName: TableName,
|
||||
targetTableName: TableName
|
||||
): String {
|
||||
val importType = stream.importType as Dedupe
|
||||
val finalSchema = tableSchema.columnSchema.finalSchema
|
||||
|
||||
// Build primary key matching condition
|
||||
val pks = tableSchema.getPrimaryKey().flatten()
|
||||
val pkEquivalent =
|
||||
if (importType.primaryKey.isNotEmpty()) {
|
||||
importType.primaryKey.joinToString(" AND ") { fieldPath ->
|
||||
val fieldName = fieldPath.first()
|
||||
val columnName = columnNameMapping[fieldName] ?: fieldName
|
||||
if (pks.isNotEmpty()) {
|
||||
pks.joinToString(" AND ") { columnName ->
|
||||
val targetTableColumnName = "target_table.${columnName.quote()}"
|
||||
val newRecordColumnName = "new_record.${columnName.quote()}"
|
||||
"""($targetTableColumnName = $newRecordColumnName OR ($targetTableColumnName IS NULL AND $newRecordColumnName IS NULL))"""
|
||||
@@ -120,80 +142,62 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
}
|
||||
|
||||
// Build column lists for INSERT and UPDATE
|
||||
val columnList: String =
|
||||
columnUtils
|
||||
.getFormattedColumnNames(
|
||||
columns = stream.schema.asColumns(),
|
||||
columnNameMapping = columnNameMapping,
|
||||
quote = false,
|
||||
)
|
||||
.joinToString(
|
||||
",\n",
|
||||
) {
|
||||
it.quote()
|
||||
}
|
||||
val allColumns = buildList {
|
||||
add(SNOWFLAKE_AB_RAW_ID)
|
||||
add(SNOWFLAKE_AB_EXTRACTED_AT)
|
||||
add(SNOWFLAKE_AB_META)
|
||||
add(SNOWFLAKE_AB_GENERATION_ID)
|
||||
addAll(finalSchema.keys)
|
||||
}
|
||||
|
||||
val columnList: String = allColumns.joinToString(",\n ") { it.quote() }
|
||||
val newRecordColumnList: String =
|
||||
columnUtils
|
||||
.getFormattedColumnNames(
|
||||
columns = stream.schema.asColumns(),
|
||||
columnNameMapping = columnNameMapping,
|
||||
quote = false,
|
||||
)
|
||||
.joinToString(",\n") { "new_record.${it.quote()}" }
|
||||
allColumns.joinToString(",\n ") { "new_record.${it.quote()}" }
|
||||
|
||||
// Get deduped records from source
|
||||
val selectSourceRecords = selectDedupedRecords(stream, sourceTableName, columnNameMapping)
|
||||
val selectSourceRecords = selectDedupedRecords(tableSchema, sourceTableName)
|
||||
|
||||
// Build cursor comparison for determining which record is newer
|
||||
val cursorComparison: String
|
||||
if (importType.cursor.isNotEmpty()) {
|
||||
val cursorFieldName = importType.cursor.first()
|
||||
val cursor = (columnNameMapping[cursorFieldName] ?: cursorFieldName)
|
||||
val cursor = tableSchema.getCursor().firstOrNull()
|
||||
if (cursor != null) {
|
||||
val targetTableCursor = "target_table.${cursor.quote()}"
|
||||
val newRecordCursor = "new_record.${cursor.quote()}"
|
||||
cursorComparison =
|
||||
"""
|
||||
(
|
||||
$targetTableCursor < $newRecordCursor
|
||||
OR ($targetTableCursor = $newRecordCursor AND target_table."${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}" < new_record."${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}")
|
||||
OR ($targetTableCursor IS NULL AND $newRecordCursor IS NULL AND target_table."${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}" < new_record."${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}")
|
||||
OR ($targetTableCursor = $newRecordCursor AND target_table."$SNOWFLAKE_AB_EXTRACTED_AT" < new_record."$SNOWFLAKE_AB_EXTRACTED_AT")
|
||||
OR ($targetTableCursor IS NULL AND $newRecordCursor IS NULL AND target_table."$SNOWFLAKE_AB_EXTRACTED_AT" < new_record."$SNOWFLAKE_AB_EXTRACTED_AT")
|
||||
OR ($targetTableCursor IS NULL AND $newRecordCursor IS $NOT_NULL)
|
||||
)
|
||||
""".trimIndent()
|
||||
} else {
|
||||
// No cursor - use extraction timestamp only
|
||||
cursorComparison =
|
||||
"""target_table."${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}" < new_record."${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}""""
|
||||
"""target_table."$SNOWFLAKE_AB_EXTRACTED_AT" < new_record."$SNOWFLAKE_AB_EXTRACTED_AT""""
|
||||
}
|
||||
|
||||
// Build column assignments for UPDATE
|
||||
val columnAssignments: String =
|
||||
columnUtils
|
||||
.getFormattedColumnNames(
|
||||
columns = stream.schema.asColumns(),
|
||||
columnNameMapping = columnNameMapping,
|
||||
quote = false,
|
||||
)
|
||||
.joinToString(",\n") { column ->
|
||||
"${column.quote()} = new_record.${column.quote()}"
|
||||
}
|
||||
allColumns.joinToString(",\n ") { column ->
|
||||
"${column.quote()} = new_record.${column.quote()}"
|
||||
}
|
||||
|
||||
// Handle CDC deletions based on mode
|
||||
val cdcDeleteClause: String
|
||||
val cdcSkipInsertClause: String
|
||||
if (
|
||||
stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN) &&
|
||||
snowflakeConfiguration.cdcDeletionMode == CdcDeletionMode.HARD_DELETE
|
||||
finalSchema.containsKey(SNOWFLAKE_AB_CDC_DELETED_AT_COLUMN) &&
|
||||
config.cdcDeletionMode == CdcDeletionMode.HARD_DELETE
|
||||
) {
|
||||
// Execute CDC deletions if there's already a record
|
||||
cdcDeleteClause =
|
||||
"WHEN MATCHED AND new_record.\"${CDC_DELETED_AT_COLUMN.toSnowflakeCompatibleName()}\" IS NOT NULL AND $cursorComparison THEN DELETE"
|
||||
"WHEN MATCHED AND new_record.\"${SNOWFLAKE_AB_CDC_DELETED_AT_COLUMN}\" IS NOT NULL AND $cursorComparison THEN DELETE"
|
||||
// And skip insertion entirely if there's no matching record.
|
||||
// (This is possible if a single T+D batch contains both an insertion and deletion for
|
||||
// the same PK)
|
||||
cdcSkipInsertClause =
|
||||
"AND new_record.\"${CDC_DELETED_AT_COLUMN.toSnowflakeCompatibleName()}\" IS NULL"
|
||||
cdcSkipInsertClause = "AND new_record.\"${SNOWFLAKE_AB_CDC_DELETED_AT_COLUMN}\" IS NULL"
|
||||
} else {
|
||||
cdcDeleteClause = ""
|
||||
cdcSkipInsertClause = ""
|
||||
@@ -203,35 +207,35 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
val mergeStatement =
|
||||
if (cdcDeleteClause.isNotEmpty()) {
|
||||
"""
|
||||
MERGE INTO ${snowflakeSqlNameUtils.fullyQualifiedName(targetTableName)} AS target_table
|
||||
USING (
|
||||
$selectSourceRecords
|
||||
) AS new_record
|
||||
ON $pkEquivalent
|
||||
$cdcDeleteClause
|
||||
WHEN MATCHED AND $cursorComparison THEN UPDATE SET
|
||||
$columnAssignments
|
||||
WHEN NOT MATCHED $cdcSkipInsertClause THEN INSERT (
|
||||
$columnList
|
||||
) VALUES (
|
||||
$newRecordColumnList
|
||||
)
|
||||
""".trimIndent()
|
||||
|MERGE INTO ${fullyQualifiedName(targetTableName)} AS target_table
|
||||
|USING (
|
||||
|$selectSourceRecords
|
||||
|) AS new_record
|
||||
|ON $pkEquivalent
|
||||
|$cdcDeleteClause
|
||||
|WHEN MATCHED AND $cursorComparison THEN UPDATE SET
|
||||
| $columnAssignments
|
||||
|WHEN NOT MATCHED $cdcSkipInsertClause THEN INSERT (
|
||||
| $columnList
|
||||
|) VALUES (
|
||||
| $newRecordColumnList
|
||||
|)
|
||||
""".trimMargin()
|
||||
} else {
|
||||
"""
|
||||
MERGE INTO ${snowflakeSqlNameUtils.fullyQualifiedName(targetTableName)} AS target_table
|
||||
USING (
|
||||
$selectSourceRecords
|
||||
) AS new_record
|
||||
ON $pkEquivalent
|
||||
WHEN MATCHED AND $cursorComparison THEN UPDATE SET
|
||||
$columnAssignments
|
||||
WHEN NOT MATCHED THEN INSERT (
|
||||
$columnList
|
||||
) VALUES (
|
||||
$newRecordColumnList
|
||||
)
|
||||
""".trimIndent()
|
||||
|MERGE INTO ${fullyQualifiedName(targetTableName)} AS target_table
|
||||
|USING (
|
||||
|$selectSourceRecords
|
||||
|) AS new_record
|
||||
|ON $pkEquivalent
|
||||
|WHEN MATCHED AND $cursorComparison THEN UPDATE SET
|
||||
| $columnAssignments
|
||||
|WHEN NOT MATCHED THEN INSERT (
|
||||
| $columnList
|
||||
|) VALUES (
|
||||
| $newRecordColumnList
|
||||
|)
|
||||
""".trimMargin()
|
||||
}
|
||||
|
||||
return mergeStatement.andLog()
|
||||
@@ -242,75 +246,66 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
* table. Uses ROW_NUMBER() window function to select the most recent record per primary key.
|
||||
*/
|
||||
private fun selectDedupedRecords(
|
||||
stream: DestinationStream,
|
||||
sourceTableName: TableName,
|
||||
columnNameMapping: ColumnNameMapping
|
||||
tableSchema: StreamTableSchema,
|
||||
sourceTableName: TableName
|
||||
): String {
|
||||
val columnList: String =
|
||||
columnUtils
|
||||
.getFormattedColumnNames(
|
||||
columns = stream.schema.asColumns(),
|
||||
columnNameMapping = columnNameMapping,
|
||||
quote = false,
|
||||
)
|
||||
.joinToString(
|
||||
",\n",
|
||||
) {
|
||||
it.quote()
|
||||
}
|
||||
val importType = stream.importType as Dedupe
|
||||
val allColumns = buildList {
|
||||
add(SNOWFLAKE_AB_RAW_ID)
|
||||
add(SNOWFLAKE_AB_EXTRACTED_AT)
|
||||
add(SNOWFLAKE_AB_META)
|
||||
add(SNOWFLAKE_AB_GENERATION_ID)
|
||||
addAll(tableSchema.columnSchema.finalSchema.keys)
|
||||
}
|
||||
val columnList: String = allColumns.joinToString(",\n ") { it.quote() }
|
||||
|
||||
// Build the primary key list for partitioning
|
||||
val pks = tableSchema.getPrimaryKey().flatten()
|
||||
val pkList =
|
||||
if (importType.primaryKey.isNotEmpty()) {
|
||||
importType.primaryKey.joinToString(",") { fieldPath ->
|
||||
(columnNameMapping[fieldPath.first()] ?: fieldPath.first()).quote()
|
||||
}
|
||||
if (pks.isNotEmpty()) {
|
||||
pks.joinToString(",") { it.quote() }
|
||||
} else {
|
||||
// Should not happen as we check this earlier, but handle it defensively
|
||||
throw IllegalArgumentException("Cannot deduplicate without primary key")
|
||||
}
|
||||
|
||||
// Build cursor order clause for sorting within each partition
|
||||
val cursor = tableSchema.getCursor().firstOrNull()
|
||||
val cursorOrderClause =
|
||||
if (importType.cursor.isNotEmpty()) {
|
||||
val columnName =
|
||||
(columnNameMapping[importType.cursor.first()] ?: importType.cursor.first())
|
||||
.quote()
|
||||
"$columnName DESC NULLS LAST,"
|
||||
if (cursor != null) {
|
||||
"${cursor.quote()} DESC NULLS LAST,"
|
||||
} else {
|
||||
""
|
||||
}
|
||||
|
||||
return """
|
||||
WITH records AS (
|
||||
SELECT
|
||||
$columnList
|
||||
FROM ${snowflakeSqlNameUtils.fullyQualifiedName(sourceTableName)}
|
||||
), numbered_rows AS (
|
||||
SELECT *, ROW_NUMBER() OVER (
|
||||
PARTITION BY $pkList ORDER BY $cursorOrderClause "${COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName()}" DESC
|
||||
) AS row_number
|
||||
FROM records
|
||||
)
|
||||
SELECT $columnList
|
||||
FROM numbered_rows
|
||||
WHERE row_number = 1
|
||||
| WITH records AS (
|
||||
| SELECT
|
||||
| $columnList
|
||||
| FROM ${fullyQualifiedName(sourceTableName)}
|
||||
| ), numbered_rows AS (
|
||||
| SELECT *, ROW_NUMBER() OVER (
|
||||
| PARTITION BY $pkList ORDER BY $cursorOrderClause "$SNOWFLAKE_AB_EXTRACTED_AT" DESC
|
||||
| ) AS row_number
|
||||
| FROM records
|
||||
| )
|
||||
| SELECT $columnList
|
||||
| FROM numbered_rows
|
||||
| WHERE row_number = 1
|
||||
"""
|
||||
.trimIndent()
|
||||
.trimMargin()
|
||||
.andLog()
|
||||
}
|
||||
|
||||
fun dropTable(tableName: TableName): String {
|
||||
return "DROP TABLE IF EXISTS ${snowflakeSqlNameUtils.fullyQualifiedName(tableName)}".andLog()
|
||||
return "DROP TABLE IF EXISTS ${fullyQualifiedName(tableName)}".andLog()
|
||||
}
|
||||
|
||||
fun getGenerationId(
|
||||
tableName: TableName,
|
||||
): String {
|
||||
return """
|
||||
SELECT "${columnUtils.getGenerationIdColumnName()}"
|
||||
FROM ${snowflakeSqlNameUtils.fullyQualifiedName(tableName)}
|
||||
SELECT "${columnManager.getGenerationIdColumnName()}"
|
||||
FROM ${fullyQualifiedName(tableName)}
|
||||
LIMIT 1
|
||||
"""
|
||||
.trimIndent()
|
||||
@@ -318,12 +313,12 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
}
|
||||
|
||||
fun createSnowflakeStage(tableName: TableName): String {
|
||||
val stageName = snowflakeSqlNameUtils.fullyQualifiedStageName(tableName)
|
||||
val stageName = fullyQualifiedStageName(tableName)
|
||||
return "CREATE STAGE IF NOT EXISTS $stageName".andLog()
|
||||
}
|
||||
|
||||
fun putInStage(tableName: TableName, tempFilePath: String): String {
|
||||
val stageName = snowflakeSqlNameUtils.fullyQualifiedStageName(tableName, true)
|
||||
val stageName = fullyQualifiedStageName(tableName, true)
|
||||
return """
|
||||
PUT 'file://$tempFilePath' '@$stageName'
|
||||
AUTO_COMPRESS = FALSE
|
||||
@@ -334,35 +329,45 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
.andLog()
|
||||
}
|
||||
|
||||
fun copyFromStage(tableName: TableName, filename: String): String {
|
||||
val stageName = snowflakeSqlNameUtils.fullyQualifiedStageName(tableName, true)
|
||||
fun copyFromStage(
|
||||
tableName: TableName,
|
||||
filename: String,
|
||||
columnNames: List<String>? = null
|
||||
): String {
|
||||
val stageName = fullyQualifiedStageName(tableName, true)
|
||||
val columnList =
|
||||
columnNames?.let { names -> "(${names.joinToString(", ") { it.quote() }})" } ?: ""
|
||||
|
||||
return """
|
||||
COPY INTO ${snowflakeSqlNameUtils.fullyQualifiedName(tableName)}
|
||||
FROM '@$stageName'
|
||||
FILE_FORMAT = (
|
||||
TYPE = 'CSV'
|
||||
COMPRESSION = GZIP
|
||||
FIELD_DELIMITER = '$CSV_FIELD_SEPARATOR'
|
||||
RECORD_DELIMITER = '$CSV_LINE_DELIMITER'
|
||||
FIELD_OPTIONALLY_ENCLOSED_BY = '"'
|
||||
TRIM_SPACE = TRUE
|
||||
ERROR_ON_COLUMN_COUNT_MISMATCH = FALSE
|
||||
REPLACE_INVALID_CHARACTERS = TRUE
|
||||
ESCAPE = NONE
|
||||
ESCAPE_UNENCLOSED_FIELD = NONE
|
||||
)
|
||||
ON_ERROR = 'ABORT_STATEMENT'
|
||||
PURGE = TRUE
|
||||
files = ('$filename')
|
||||
|COPY INTO ${fullyQualifiedName(tableName)}$columnList
|
||||
|FROM '@$stageName'
|
||||
|FILE_FORMAT = (
|
||||
| TYPE = 'CSV'
|
||||
| COMPRESSION = GZIP
|
||||
| FIELD_DELIMITER = '$CSV_FIELD_SEPARATOR'
|
||||
| RECORD_DELIMITER = '$CSV_LINE_DELIMITER'
|
||||
| FIELD_OPTIONALLY_ENCLOSED_BY = '"'
|
||||
| TRIM_SPACE = TRUE
|
||||
| ERROR_ON_COLUMN_COUNT_MISMATCH = FALSE
|
||||
| REPLACE_INVALID_CHARACTERS = TRUE
|
||||
| ESCAPE = NONE
|
||||
| ESCAPE_UNENCLOSED_FIELD = NONE
|
||||
|)
|
||||
|ON_ERROR = 'ABORT_STATEMENT'
|
||||
|PURGE = TRUE
|
||||
|files = ('$filename')
|
||||
"""
|
||||
.trimIndent()
|
||||
.trimMargin()
|
||||
.andLog()
|
||||
}
|
||||
|
||||
fun swapTableWith(sourceTableName: TableName, targetTableName: TableName): String {
|
||||
return """
|
||||
ALTER TABLE ${snowflakeSqlNameUtils.fullyQualifiedName(sourceTableName)} SWAP WITH ${snowflakeSqlNameUtils.fullyQualifiedName(targetTableName)}
|
||||
ALTER TABLE ${fullyQualifiedName(sourceTableName)} SWAP WITH ${
|
||||
fullyQualifiedName(
|
||||
targetTableName,
|
||||
)
|
||||
}
|
||||
"""
|
||||
.trimIndent()
|
||||
.andLog()
|
||||
@@ -372,7 +377,11 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
// Snowflake RENAME TO only accepts the table name, not a fully qualified name
|
||||
// The renamed table stays in the same schema
|
||||
return """
|
||||
ALTER TABLE ${snowflakeSqlNameUtils.fullyQualifiedName(sourceTableName)} RENAME TO ${snowflakeSqlNameUtils.fullyQualifiedName(targetTableName)}
|
||||
ALTER TABLE ${fullyQualifiedName(sourceTableName)} RENAME TO ${
|
||||
fullyQualifiedName(
|
||||
targetTableName,
|
||||
)
|
||||
}
|
||||
"""
|
||||
.trimIndent()
|
||||
.andLog()
|
||||
@@ -382,7 +391,7 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
schemaName: String,
|
||||
tableName: String,
|
||||
): String =
|
||||
"""DESCRIBE TABLE ${snowflakeSqlNameUtils.fullyQualifiedName(TableName(schemaName, tableName))}""".andLog()
|
||||
"""DESCRIBE TABLE ${fullyQualifiedName(TableName(schemaName, tableName))}""".andLog()
|
||||
|
||||
fun alterTable(
|
||||
tableName: TableName,
|
||||
@@ -391,14 +400,14 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
modifiedColumns: Map<String, ColumnTypeChange>,
|
||||
): Set<String> {
|
||||
val clauses = mutableSetOf<String>()
|
||||
val prettyTableName = snowflakeSqlNameUtils.fullyQualifiedName(tableName)
|
||||
val prettyTableName = fullyQualifiedName(tableName)
|
||||
addedColumns.forEach { (name, columnType) ->
|
||||
clauses.add(
|
||||
// Note that we intentionally don't set NOT NULL.
|
||||
// We're adding a new column, and we don't know what constitutes a reasonable
|
||||
// default value for preexisting records.
|
||||
// So we add the column as nullable.
|
||||
"ALTER TABLE $prettyTableName ADD COLUMN ${name.quote()} ${columnType.type};".andLog()
|
||||
"ALTER TABLE $prettyTableName ADD COLUMN ${name.quote()} ${columnType.type};".andLog(),
|
||||
)
|
||||
}
|
||||
deletedColumns.forEach {
|
||||
@@ -412,35 +421,34 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
val tempColumn = "${name}_${uuidGenerator.v4()}"
|
||||
clauses.add(
|
||||
// As above: we add the column as nullable.
|
||||
"ALTER TABLE $prettyTableName ADD COLUMN ${tempColumn.quote()} ${typeChange.newType.type};".andLog()
|
||||
"ALTER TABLE $prettyTableName ADD COLUMN ${tempColumn.quote()} ${typeChange.newType.type};".andLog(),
|
||||
)
|
||||
clauses.add(
|
||||
"UPDATE $prettyTableName SET ${tempColumn.quote()} = CAST(${name.quote()} AS ${typeChange.newType.type});".andLog()
|
||||
"UPDATE $prettyTableName SET ${tempColumn.quote()} = CAST(${name.quote()} AS ${typeChange.newType.type});".andLog(),
|
||||
)
|
||||
val backupColumn = "${tempColumn}_backup"
|
||||
clauses.add(
|
||||
"""
|
||||
ALTER TABLE $prettyTableName
|
||||
RENAME COLUMN "$name" TO "$backupColumn";
|
||||
""".trimIndent()
|
||||
""".trimIndent(),
|
||||
)
|
||||
clauses.add(
|
||||
"""
|
||||
ALTER TABLE $prettyTableName
|
||||
RENAME COLUMN "$tempColumn" TO "$name";
|
||||
""".trimIndent()
|
||||
""".trimIndent(),
|
||||
)
|
||||
clauses.add(
|
||||
"ALTER TABLE $prettyTableName DROP COLUMN ${backupColumn.quote()};".andLog()
|
||||
"ALTER TABLE $prettyTableName DROP COLUMN ${backupColumn.quote()};".andLog(),
|
||||
)
|
||||
} else if (!typeChange.originalType.nullable && typeChange.newType.nullable) {
|
||||
// If the type is unchanged, we can change a column from NOT NULL to nullable.
|
||||
// But we'll never do the reverse, because there's a decent chance that historical
|
||||
// records
|
||||
// had null values.
|
||||
// records had null values.
|
||||
// Users can always manually ALTER COLUMN ... SET NOT NULL if they want.
|
||||
clauses.add(
|
||||
"""ALTER TABLE $prettyTableName ALTER COLUMN "$name" DROP NOT NULL;""".andLog()
|
||||
"""ALTER TABLE $prettyTableName ALTER COLUMN "$name" DROP NOT NULL;""".andLog(),
|
||||
)
|
||||
} else {
|
||||
log.info {
|
||||
@@ -450,4 +458,45 @@ class SnowflakeDirectLoadSqlGenerator(
|
||||
}
|
||||
return clauses
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
fun fullyQualifiedName(tableName: TableName): String =
|
||||
combineParts(listOf(getDatabaseName(), tableName.namespace, tableName.name))
|
||||
|
||||
@VisibleForTesting
|
||||
fun fullyQualifiedNamespace(namespace: String) =
|
||||
combineParts(listOf(getDatabaseName(), namespace))
|
||||
|
||||
@VisibleForTesting
|
||||
fun fullyQualifiedStageName(tableName: TableName, escape: Boolean = false): String {
|
||||
val currentTableName =
|
||||
if (escape) {
|
||||
tableName.name
|
||||
} else {
|
||||
tableName.name
|
||||
}
|
||||
return combineParts(
|
||||
parts =
|
||||
listOf(
|
||||
getDatabaseName(),
|
||||
tableName.namespace,
|
||||
"$STAGE_NAME_PREFIX$currentTableName",
|
||||
),
|
||||
escape = escape,
|
||||
)
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
internal fun combineParts(parts: List<String>, escape: Boolean = false): String =
|
||||
parts
|
||||
.map { if (escape) sqlEscape(it) else it }
|
||||
.joinToString(separator = ".") {
|
||||
if (!it.startsWith(QUOTE)) {
|
||||
"$QUOTE$it$QUOTE"
|
||||
} else {
|
||||
it
|
||||
}
|
||||
}
|
||||
|
||||
private fun getDatabaseName() = config.database.toSnowflakeCompatibleName()
|
||||
}
|
||||
|
||||
@@ -0,0 +1,29 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.sql
|
||||
|
||||
const val STAGE_NAME_PREFIX = "airbyte_stage_"
|
||||
internal const val QUOTE: String = "\""
|
||||
|
||||
fun sqlEscape(part: String) = part.replace("\\", "\\\\").replace("'", "\\'").replace("\"", "\\\"")
|
||||
|
||||
/**
|
||||
* Surrounds the string instance with double quotation marks (e.g. "some string" -> "\"some
|
||||
* string\"").
|
||||
*/
|
||||
fun String.quote() = "$QUOTE$this$QUOTE"
|
||||
|
||||
/**
|
||||
* Escapes double-quotes in a JSON identifier by doubling them. This is legacy -- I don't know why
|
||||
* this would be necessary but no harm in keeping it, so I am keeping it.
|
||||
*
|
||||
* @return The escaped identifier.
|
||||
*/
|
||||
fun escapeJsonIdentifier(identifier: String): String {
|
||||
// Note that we don't need to escape backslashes here!
|
||||
// The only special character in an identifier is the double-quote, which needs to be
|
||||
// doubled.
|
||||
return identifier.replace(QUOTE, "$QUOTE$QUOTE")
|
||||
}
|
||||
@@ -1,57 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.sql
|
||||
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
const val STAGE_NAME_PREFIX = "airbyte_stage_"
|
||||
internal const val QUOTE: String = "\""
|
||||
|
||||
fun sqlEscape(part: String) = part.replace("\\", "\\\\").replace("'", "\\'").replace("\"", "\\\"")
|
||||
|
||||
@Singleton
|
||||
class SnowflakeSqlNameUtils(
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
) {
|
||||
fun fullyQualifiedName(tableName: TableName): String =
|
||||
combineParts(listOf(getDatabaseName(), tableName.namespace, tableName.name))
|
||||
|
||||
fun fullyQualifiedNamespace(namespace: String) =
|
||||
combineParts(listOf(getDatabaseName(), namespace))
|
||||
|
||||
fun fullyQualifiedStageName(tableName: TableName, escape: Boolean = false): String {
|
||||
val currentTableName =
|
||||
if (escape) {
|
||||
tableName.name
|
||||
} else {
|
||||
tableName.name
|
||||
}
|
||||
return combineParts(
|
||||
parts =
|
||||
listOf(
|
||||
getDatabaseName(),
|
||||
tableName.namespace,
|
||||
"$STAGE_NAME_PREFIX$currentTableName"
|
||||
),
|
||||
escape = escape,
|
||||
)
|
||||
}
|
||||
|
||||
fun combineParts(parts: List<String>, escape: Boolean = false): String =
|
||||
parts
|
||||
.map { if (escape) sqlEscape(it) else it }
|
||||
.joinToString(separator = ".") {
|
||||
if (!it.startsWith(QUOTE)) {
|
||||
"$QUOTE$it$QUOTE"
|
||||
} else {
|
||||
it
|
||||
}
|
||||
}
|
||||
|
||||
private fun getDatabaseName() = snowflakeConfiguration.database.toSnowflakeCompatibleName()
|
||||
}
|
||||
@@ -6,38 +6,39 @@ package io.airbyte.integrations.destination.snowflake.write
|
||||
|
||||
import io.airbyte.cdk.SystemErrorException
|
||||
import io.airbyte.cdk.load.command.Dedupe
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatusGatherer
|
||||
import io.airbyte.cdk.load.orchestration.db.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadInitialStatus
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableAppendStreamLoader
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableAppendTruncateStreamLoader
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableDedupStreamLoader
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableDedupTruncateStreamLoader
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.table.DatabaseInitialStatusGatherer
|
||||
import io.airbyte.cdk.load.table.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadInitialStatus
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableAppendStreamLoader
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableAppendTruncateStreamLoader
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableDedupStreamLoader
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableDedupTruncateStreamLoader
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.write.DestinationWriter
|
||||
import io.airbyte.cdk.load.write.StreamLoader
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.db.escapeJsonIdentifier
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.escapeJsonIdentifier
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
class SnowflakeWriter(
|
||||
private val names: TableCatalog,
|
||||
private val catalog: DestinationCatalog,
|
||||
private val stateGatherer: DatabaseInitialStatusGatherer<DirectLoadInitialStatus>,
|
||||
private val streamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>,
|
||||
private val snowflakeClient: SnowflakeAirbyteClient,
|
||||
private val tempTableNameGenerator: TempTableNameGenerator,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val tempTableNameGenerator: TempTableNameGenerator,
|
||||
) : DestinationWriter {
|
||||
private lateinit var initialStatuses: Map<DestinationStream, DirectLoadInitialStatus>
|
||||
|
||||
override suspend fun setup() {
|
||||
names.values
|
||||
.map { (tableNames, _) -> tableNames.finalTableName!!.namespace }
|
||||
catalog.streams
|
||||
.map { it.tableSchema.tableNames.finalTableName!!.namespace }
|
||||
.toSet()
|
||||
.forEach { snowflakeClient.createNamespace(it) }
|
||||
|
||||
@@ -45,15 +46,15 @@ class SnowflakeWriter(
|
||||
escapeJsonIdentifier(snowflakeConfiguration.internalTableSchema)
|
||||
)
|
||||
|
||||
initialStatuses = stateGatherer.gatherInitialStatus(names)
|
||||
initialStatuses = stateGatherer.gatherInitialStatus()
|
||||
}
|
||||
|
||||
override fun createStreamLoader(stream: DestinationStream): StreamLoader {
|
||||
val initialStatus = initialStatuses[stream]!!
|
||||
val tableNameInfo = names[stream]!!
|
||||
val realTableName = tableNameInfo.tableNames.finalTableName!!
|
||||
val tempTableName = tempTableNameGenerator.generate(realTableName)
|
||||
val columnNameMapping = tableNameInfo.columnNameMapping
|
||||
val realTableName = stream.tableSchema.tableNames.finalTableName!!
|
||||
val tempTableName = stream.tableSchema.tableNames.tempTableName!!
|
||||
val columnNameMapping =
|
||||
ColumnNameMapping(stream.tableSchema.columnSchema.inputToFinalColumnNames)
|
||||
return when (stream.minimumGenerationId) {
|
||||
0L ->
|
||||
when (stream.importType) {
|
||||
|
||||
@@ -9,11 +9,12 @@ import de.siegmar.fastcsv.writer.CsvWriter
|
||||
import de.siegmar.fastcsv.writer.LineDelimiter
|
||||
import de.siegmar.fastcsv.writer.QuoteStrategies
|
||||
import io.airbyte.cdk.load.data.AirbyteValue
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.QUOTE
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.io.File
|
||||
import java.io.OutputStream
|
||||
@@ -36,10 +37,11 @@ private const val CSV_WRITER_BUFFER_SIZE = 1024 * 1024 // 1 MB
|
||||
|
||||
class SnowflakeInsertBuffer(
|
||||
private val tableName: TableName,
|
||||
val columns: LinkedHashMap<String, String>,
|
||||
private val snowflakeClient: SnowflakeAirbyteClient,
|
||||
val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
val columnSchema: ColumnSchema,
|
||||
private val columnManager: SnowflakeColumnManager,
|
||||
private val snowflakeRecordFormatter: SnowflakeRecordFormatter,
|
||||
private val flushLimit: Int = DEFAULT_FLUSH_LIMIT,
|
||||
) {
|
||||
|
||||
@@ -57,12 +59,6 @@ class SnowflakeInsertBuffer(
|
||||
.lineDelimiter(CSV_LINE_DELIMITER)
|
||||
.quoteStrategy(QuoteStrategies.REQUIRED)
|
||||
|
||||
private val snowflakeRecordFormatter: SnowflakeRecordFormatter =
|
||||
when (snowflakeConfiguration.legacyRawTablesOnly) {
|
||||
true -> SnowflakeRawRecordFormatter(columns, snowflakeColumnUtils)
|
||||
else -> SnowflakeSchemaRecordFormatter(columns, snowflakeColumnUtils)
|
||||
}
|
||||
|
||||
fun accumulate(recordFields: Map<String, AirbyteValue>) {
|
||||
if (csvFilePath == null) {
|
||||
val csvFile = createCsvFile()
|
||||
@@ -92,7 +88,9 @@ class SnowflakeInsertBuffer(
|
||||
"Copying staging data into ${tableName.toPrettyString(quote = QUOTE)}..."
|
||||
}
|
||||
// Finally, copy the data from the staging table to the final table
|
||||
snowflakeClient.copyFromStage(tableName, filePath.fileName.toString())
|
||||
// Pass column names to ensure correct mapping even after ALTER TABLE operations
|
||||
val columnNames = columnManager.getTableColumnNames(columnSchema)
|
||||
snowflakeClient.copyFromStage(tableName, filePath.fileName.toString(), columnNames)
|
||||
logger.info {
|
||||
"Finished insert of $recordCount row(s) into ${tableName.toPrettyString(quote = QUOTE)}."
|
||||
}
|
||||
@@ -117,7 +115,9 @@ class SnowflakeInsertBuffer(
|
||||
|
||||
private fun writeToCsvFile(record: Map<String, AirbyteValue>) {
|
||||
csvWriter?.let {
|
||||
it.writeRecord(snowflakeRecordFormatter.format(record).map { col -> col.toString() })
|
||||
it.writeRecord(
|
||||
snowflakeRecordFormatter.format(record, columnSchema).map { col -> col.toString() }
|
||||
)
|
||||
recordCount++
|
||||
if ((recordCount % flushLimit) == 0) {
|
||||
it.flush()
|
||||
|
||||
@@ -8,99 +8,62 @@ import io.airbyte.cdk.load.data.AirbyteValue
|
||||
import io.airbyte.cdk.load.data.NullValue
|
||||
import io.airbyte.cdk.load.data.StringValue
|
||||
import io.airbyte.cdk.load.data.csv.toCsvValue
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_EXTRACTED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_GENERATION_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_LOADED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_META
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_RAW_ID
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.cdk.load.util.Jsons
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
|
||||
interface SnowflakeRecordFormatter {
|
||||
fun format(record: Map<String, AirbyteValue>): List<Any>
|
||||
fun format(record: Map<String, AirbyteValue>, columnSchema: ColumnSchema): List<Any>
|
||||
}
|
||||
|
||||
class SnowflakeSchemaRecordFormatter(
|
||||
private val columns: LinkedHashMap<String, String>,
|
||||
val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
) : SnowflakeRecordFormatter {
|
||||
class SnowflakeSchemaRecordFormatter : SnowflakeRecordFormatter {
|
||||
override fun format(record: Map<String, AirbyteValue>, columnSchema: ColumnSchema): List<Any> {
|
||||
val result = mutableListOf<Any>()
|
||||
val userColumns = columnSchema.finalSchema.keys
|
||||
|
||||
private val airbyteColumnNames =
|
||||
snowflakeColumnUtils.getFormattedDefaultColumnNames(false).toSet()
|
||||
// WARNING: MUST match the order defined in SnowflakeColumnManager#getTableColumnNames
|
||||
//
|
||||
// Why don't we just use that here? Well, unlike the user fields, the meta fields on the
|
||||
// record are not munged for the destination. So we must access the values for those columns
|
||||
// using the original lowercase meta key.
|
||||
result.add(record[COLUMN_NAME_AB_RAW_ID].toCsvValue())
|
||||
result.add(record[COLUMN_NAME_AB_EXTRACTED_AT].toCsvValue())
|
||||
result.add(record[COLUMN_NAME_AB_META].toCsvValue())
|
||||
result.add(record[COLUMN_NAME_AB_GENERATION_ID].toCsvValue())
|
||||
|
||||
override fun format(record: Map<String, AirbyteValue>): List<Any> =
|
||||
columns.map { (columnName, _) ->
|
||||
/*
|
||||
* Meta columns are forced to uppercase for backwards compatibility with previous
|
||||
* versions of the destination. Therefore, convert the column to lowercase so
|
||||
* that it can match the constants, which use the lowercase version of the meta
|
||||
* column names.
|
||||
*/
|
||||
if (airbyteColumnNames.contains(columnName)) {
|
||||
record[columnName.lowercase()].toCsvValue()
|
||||
} else {
|
||||
record.keys
|
||||
// The columns retrieved from Snowflake do not have any escaping applied.
|
||||
// Therefore, re-apply the compatible name escaping to the name of the
|
||||
// columns retrieved from Snowflake. The record keys should already have
|
||||
// been escaped by the CDK before arriving at the aggregate, so no need
|
||||
// to escape again here.
|
||||
.find { it == columnName.toSnowflakeCompatibleName() }
|
||||
?.let { record[it].toCsvValue() }
|
||||
?: ""
|
||||
}
|
||||
}
|
||||
// Add user columns from the final schema
|
||||
userColumns.forEach { columnName -> result.add(record[columnName].toCsvValue()) }
|
||||
|
||||
return result
|
||||
}
|
||||
}
|
||||
|
||||
class SnowflakeRawRecordFormatter(
|
||||
columns: LinkedHashMap<String, String>,
|
||||
val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
) : SnowflakeRecordFormatter {
|
||||
private val columns = columns.keys
|
||||
class SnowflakeRawRecordFormatter : SnowflakeRecordFormatter {
|
||||
|
||||
private val airbyteColumnNames =
|
||||
snowflakeColumnUtils.getFormattedDefaultColumnNames(false).toSet()
|
||||
|
||||
override fun format(record: Map<String, AirbyteValue>): List<Any> =
|
||||
override fun format(record: Map<String, AirbyteValue>, columnSchema: ColumnSchema): List<Any> =
|
||||
toOutputRecord(record.toMutableMap())
|
||||
|
||||
private fun toOutputRecord(record: MutableMap<String, AirbyteValue>): List<Any> {
|
||||
val outputRecord = mutableListOf<Any>()
|
||||
// Copy the Airbyte metadata columns to the raw output, removing each
|
||||
// one from the record to avoid duplicates in the "data" field
|
||||
columns
|
||||
.filter { airbyteColumnNames.contains(it) && it != Meta.COLUMN_NAME_DATA }
|
||||
.forEach { column -> safeAddToOutput(column, record, outputRecord) }
|
||||
val mutableRecord = record.toMutableMap()
|
||||
|
||||
// Add meta columns in order (except _airbyte_data which we handle specially)
|
||||
outputRecord.add(mutableRecord.remove(COLUMN_NAME_AB_RAW_ID)?.toCsvValue() ?: "")
|
||||
outputRecord.add(mutableRecord.remove(COLUMN_NAME_AB_EXTRACTED_AT)?.toCsvValue() ?: "")
|
||||
outputRecord.add(mutableRecord.remove(COLUMN_NAME_AB_META)?.toCsvValue() ?: "")
|
||||
outputRecord.add(mutableRecord.remove(COLUMN_NAME_AB_GENERATION_ID)?.toCsvValue() ?: "")
|
||||
outputRecord.add(mutableRecord.remove(COLUMN_NAME_AB_LOADED_AT)?.toCsvValue() ?: "")
|
||||
|
||||
// Do not output null values in the JSON raw output
|
||||
val filteredRecord = record.filter { (_, v) -> v !is NullValue }
|
||||
// Convert all the remaining columns in the record to a JSON document stored in the "data"
|
||||
// column. Add it in the same position as the _airbyte_data column in the column list to
|
||||
// ensure it is inserted into the proper column in the table.
|
||||
insert(
|
||||
columns.indexOf(Meta.COLUMN_NAME_DATA),
|
||||
StringValue(Jsons.writeValueAsString(filteredRecord)).toCsvValue(),
|
||||
outputRecord
|
||||
)
|
||||
val filteredRecord = mutableRecord.filter { (_, v) -> v !is NullValue }
|
||||
|
||||
// Convert all the remaining columns to a JSON document stored in the "data" column
|
||||
outputRecord.add(StringValue(Jsons.writeValueAsString(filteredRecord)).toCsvValue())
|
||||
|
||||
return outputRecord
|
||||
}
|
||||
|
||||
private fun safeAddToOutput(
|
||||
key: String,
|
||||
record: MutableMap<String, AirbyteValue>,
|
||||
output: MutableList<Any>
|
||||
) {
|
||||
val extractedValue = record.remove(key)
|
||||
// Ensure that the data is inserted into the list at the same position as the column
|
||||
insert(columns.indexOf(key), extractedValue?.toCsvValue() ?: "", output)
|
||||
}
|
||||
|
||||
private fun insert(index: Int, value: Any, list: MutableList<Any>) {
|
||||
/*
|
||||
* Attempt to insert the value into the proper order in the list. If the index
|
||||
* is already present in the list, use the add(index, element) method to insert it
|
||||
* into the proper order and push everything to the right. If the index is at the
|
||||
* end of the list, just use add(element) to insert it at the end. If the index
|
||||
* is further beyond the end of the list, throw an exception as that should not occur.
|
||||
*/
|
||||
if (index < list.size) list.add(index, value)
|
||||
else if (index == list.size || index == list.size + 1) list.add(value)
|
||||
else throw IndexOutOfBoundsException()
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,25 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.write.transform
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.dataflow.transform.ColumnNameMapper
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
class SnowflakeColumnNameMapper(
|
||||
private val catalogInfo: TableCatalog,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
) : ColumnNameMapper {
|
||||
override fun getMappedColumnName(stream: DestinationStream, columnName: String): String {
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly == true) {
|
||||
return columnName
|
||||
} else {
|
||||
return catalogInfo.getMappedColumnName(stream, columnName)!!
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -7,9 +7,11 @@ package io.airbyte.integrations.destination.snowflake.component
|
||||
import io.airbyte.cdk.load.component.TableOperationsFixtures
|
||||
import io.airbyte.cdk.load.component.TableOperationsSuite
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.schema.TableSchemaFactory
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.component.SnowflakeComponentTestFixtures.idTestWithCdcMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.SnowflakeComponentTestFixtures.testMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeComponentTestFixtures.idTestWithCdcMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeComponentTestFixtures.testMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeTestTableOperationsClient
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.parallel.Execution
|
||||
@@ -20,6 +22,7 @@ import org.junit.jupiter.api.parallel.ExecutionMode
|
||||
class SnowflakeTableOperationsTest(
|
||||
override val client: SnowflakeAirbyteClient,
|
||||
override val testClient: SnowflakeTestTableOperationsClient,
|
||||
override val schemaFactory: TableSchemaFactory,
|
||||
) : TableOperationsSuite {
|
||||
override val airbyteMetaColumnMapping = Meta.COLUMN_NAMES.associateWith { it.uppercase() }
|
||||
|
||||
|
||||
@@ -9,13 +9,15 @@ import io.airbyte.cdk.load.component.TableSchemaEvolutionFixtures
|
||||
import io.airbyte.cdk.load.component.TableSchemaEvolutionSuite
|
||||
import io.airbyte.cdk.load.data.StringValue
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.schema.TableSchemaFactory
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.util.serializeToString
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.component.SnowflakeComponentTestFixtures.allTypesColumnNameMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.SnowflakeComponentTestFixtures.allTypesTableSchema
|
||||
import io.airbyte.integrations.destination.snowflake.component.SnowflakeComponentTestFixtures.idAndTestMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.SnowflakeComponentTestFixtures.testMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeComponentTestFixtures.allTypesColumnNameMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeComponentTestFixtures.allTypesTableSchema
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeComponentTestFixtures.idAndTestMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeComponentTestFixtures.testMapping
|
||||
import io.airbyte.integrations.destination.snowflake.component.config.SnowflakeTestTableOperationsClient
|
||||
import io.micronaut.test.extensions.junit5.annotation.MicronautTest
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.parallel.Execution
|
||||
@@ -27,6 +29,7 @@ class SnowflakeTableSchemaEvolutionTest(
|
||||
override val client: SnowflakeAirbyteClient,
|
||||
override val opsClient: SnowflakeAirbyteClient,
|
||||
override val testClient: SnowflakeTestTableOperationsClient,
|
||||
override val schemaFactory: TableSchemaFactory,
|
||||
) : TableSchemaEvolutionSuite {
|
||||
override val airbyteMetaColumnMapping = Meta.COLUMN_NAMES.associateWith { it.uppercase() }
|
||||
|
||||
|
||||
@@ -2,7 +2,7 @@
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.component
|
||||
package io.airbyte.integrations.destination.snowflake.component.config
|
||||
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.component.TableOperationsFixtures
|
||||
@@ -33,6 +33,8 @@ object SnowflakeComponentTestFixtures {
|
||||
"TIME_NTZ" to ColumnType("TIME", true),
|
||||
"ARRAY" to ColumnType("ARRAY", true),
|
||||
"OBJECT" to ColumnType("OBJECT", true),
|
||||
"UNION" to ColumnType("VARIANT", true),
|
||||
"LEGACY_UNION" to ColumnType("VARIANT", true),
|
||||
"UNKNOWN" to ColumnType("VARIANT", true),
|
||||
)
|
||||
)
|
||||
@@ -2,7 +2,7 @@
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.component
|
||||
package io.airbyte.integrations.destination.snowflake.component.config
|
||||
|
||||
import io.airbyte.cdk.load.component.config.TestConfigLoader.loadTestConfig
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
@@ -2,22 +2,24 @@
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.component
|
||||
package io.airbyte.integrations.destination.snowflake.component.config
|
||||
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.component.TestTableOperationsClient
|
||||
import io.airbyte.cdk.load.data.AirbyteValue
|
||||
import io.airbyte.cdk.load.dataflow.state.PartitionKey
|
||||
import io.airbyte.cdk.load.dataflow.transform.RecordDTO
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.util.Jsons
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.client.execute
|
||||
import io.airbyte.integrations.destination.snowflake.dataflow.SnowflakeAggregate
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeSqlNameUtils
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDirectLoadSqlGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.sql.andLog
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeInsertBuffer
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeRecordFormatter
|
||||
import io.micronaut.context.annotation.Requires
|
||||
import jakarta.inject.Singleton
|
||||
import java.time.format.DateTimeFormatter
|
||||
@@ -29,25 +31,40 @@ import net.snowflake.client.jdbc.SnowflakeTimestampWithTimezone
|
||||
class SnowflakeTestTableOperationsClient(
|
||||
private val client: SnowflakeAirbyteClient,
|
||||
private val dataSource: DataSource,
|
||||
private val snowflakeSqlNameUtils: SnowflakeSqlNameUtils,
|
||||
private val snowflakeColumnUtils: SnowflakeColumnUtils,
|
||||
private val sqlGenerator: SnowflakeDirectLoadSqlGenerator,
|
||||
private val snowflakeConfiguration: SnowflakeConfiguration,
|
||||
private val columnManager: SnowflakeColumnManager,
|
||||
private val snowflakeRecordFormatter: SnowflakeRecordFormatter,
|
||||
) : TestTableOperationsClient {
|
||||
override suspend fun dropNamespace(namespace: String) {
|
||||
dataSource.execute(
|
||||
"DROP SCHEMA IF EXISTS ${snowflakeSqlNameUtils.fullyQualifiedNamespace(namespace)}".andLog()
|
||||
"DROP SCHEMA IF EXISTS ${sqlGenerator.fullyQualifiedNamespace(namespace)}".andLog()
|
||||
)
|
||||
}
|
||||
|
||||
override suspend fun insertRecords(table: TableName, records: List<Map<String, AirbyteValue>>) {
|
||||
// TODO: we should just pass a proper column schema
|
||||
// Since we don't pass in a proper column schema, we have to recreate one here
|
||||
// Fetch the columns and filter out the meta columns so we're just looking at user columns
|
||||
val columnTypes =
|
||||
client.describeTable(table).filterNot {
|
||||
columnManager.getMetaColumnNames().contains(it.key)
|
||||
}
|
||||
val columnSchema =
|
||||
io.airbyte.cdk.load.schema.model.ColumnSchema(
|
||||
inputToFinalColumnNames = columnTypes.keys.associateWith { it },
|
||||
finalSchema = columnTypes.mapValues { (_, _) -> ColumnType("", true) },
|
||||
inputSchema = emptyMap() // Not needed for insert buffer
|
||||
)
|
||||
val a =
|
||||
SnowflakeAggregate(
|
||||
SnowflakeInsertBuffer(
|
||||
table,
|
||||
client.describeTable(table),
|
||||
client,
|
||||
snowflakeConfiguration,
|
||||
snowflakeColumnUtils,
|
||||
tableName = table,
|
||||
snowflakeClient = client,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
columnSchema = columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
)
|
||||
)
|
||||
records.forEach { a.accept(RecordDTO(it, PartitionKey(""), 0, 0)) }
|
||||
@@ -7,11 +7,11 @@ package io.airbyte.integrations.destination.snowflake.write
|
||||
import io.airbyte.cdk.load.test.util.DestinationCleaner
|
||||
import io.airbyte.integrations.destination.snowflake.SnowflakeBeanFactory
|
||||
import io.airbyte.integrations.destination.snowflake.cdk.SnowflakeMigratingConfigurationSpecificationSupplier
|
||||
import io.airbyte.integrations.destination.snowflake.db.escapeJsonIdentifier
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfigurationFactory
|
||||
import io.airbyte.integrations.destination.snowflake.sql.STAGE_NAME_PREFIX
|
||||
import io.airbyte.integrations.destination.snowflake.sql.escapeJsonIdentifier
|
||||
import io.airbyte.integrations.destination.snowflake.sql.quote
|
||||
import java.nio.file.Files
|
||||
import java.sql.Connection
|
||||
|
||||
@@ -12,16 +12,22 @@ import io.airbyte.cdk.load.data.ObjectValue
|
||||
import io.airbyte.cdk.load.data.json.toAirbyteValue
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.table.CDC_DELETED_AT_COLUMN
|
||||
import io.airbyte.cdk.load.table.DefaultTempTableNameGenerator
|
||||
import io.airbyte.cdk.load.test.util.DestinationDataDumper
|
||||
import io.airbyte.cdk.load.test.util.OutputRecord
|
||||
import io.airbyte.cdk.load.util.UUIDGenerator
|
||||
import io.airbyte.cdk.load.util.deserializeToNode
|
||||
import io.airbyte.integrations.destination.snowflake.SnowflakeBeanFactory
|
||||
import io.airbyte.integrations.destination.snowflake.db.SnowflakeFinalTableNameGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeTableSchemaMapper
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeSqlNameUtils
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDirectLoadSqlGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.sql.sqlEscape
|
||||
import java.math.BigDecimal
|
||||
import java.sql.Date
|
||||
import java.sql.Time
|
||||
import java.sql.Timestamp
|
||||
import net.snowflake.client.jdbc.SnowflakeTimestampWithTimezone
|
||||
|
||||
private val AIRBYTE_META_COLUMNS = Meta.COLUMN_NAMES + setOf(CDC_DELETED_AT_COLUMN)
|
||||
@@ -34,8 +40,14 @@ class SnowflakeDataDumper(
|
||||
stream: DestinationStream
|
||||
): List<OutputRecord> {
|
||||
val config = configProvider(spec)
|
||||
val sqlUtils = SnowflakeSqlNameUtils(config)
|
||||
val snowflakeFinalTableNameGenerator = SnowflakeFinalTableNameGenerator(config)
|
||||
val snowflakeFinalTableNameGenerator =
|
||||
SnowflakeTableSchemaMapper(
|
||||
config = config,
|
||||
tempTableNameGenerator = DefaultTempTableNameGenerator(),
|
||||
)
|
||||
val snowflakeColumnManager = SnowflakeColumnManager(config)
|
||||
val sqlGenerator =
|
||||
SnowflakeDirectLoadSqlGenerator(UUIDGenerator(), config, snowflakeColumnManager)
|
||||
val dataSource =
|
||||
SnowflakeBeanFactory()
|
||||
.snowflakeDataSource(snowflakeConfiguration = config, airbyteEdition = "COMMUNITY")
|
||||
@@ -46,7 +58,7 @@ class SnowflakeDataDumper(
|
||||
ds.connection.use { connection ->
|
||||
val statement = connection.createStatement()
|
||||
val tableName =
|
||||
snowflakeFinalTableNameGenerator.getTableName(stream.mappedDescriptor)
|
||||
snowflakeFinalTableNameGenerator.toFinalTableName(stream.mappedDescriptor)
|
||||
|
||||
// First check if the table exists
|
||||
val tableExistsQuery =
|
||||
@@ -69,7 +81,7 @@ class SnowflakeDataDumper(
|
||||
|
||||
val resultSet =
|
||||
statement.executeQuery(
|
||||
"SELECT * FROM ${sqlUtils.fullyQualifiedName(tableName)}"
|
||||
"SELECT * FROM ${sqlGenerator.fullyQualifiedName(tableName)}"
|
||||
)
|
||||
|
||||
while (resultSet.next()) {
|
||||
@@ -143,10 +155,10 @@ class SnowflakeDataDumper(
|
||||
private fun convertValue(value: Any?): Any? =
|
||||
when (value) {
|
||||
is BigDecimal -> value.toBigInteger()
|
||||
is java.sql.Date -> value.toLocalDate()
|
||||
is Date -> value.toLocalDate()
|
||||
is SnowflakeTimestampWithTimezone -> value.toZonedDateTime()
|
||||
is java.sql.Time -> value.toLocalTime()
|
||||
is java.sql.Timestamp -> value.toLocalDateTime()
|
||||
is Time -> value.toLocalTime()
|
||||
is Timestamp -> value.toLocalDateTime()
|
||||
else -> value
|
||||
}
|
||||
}
|
||||
|
||||
@@ -15,7 +15,7 @@ import io.airbyte.cdk.load.dataflow.transform.ValidationResult
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.test.util.ExpectedRecordMapper
|
||||
import io.airbyte.cdk.load.test.util.OutputRecord
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.write.transform.SnowflakeValueCoercer
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange.Change
|
||||
|
||||
@@ -5,7 +5,7 @@
|
||||
package io.airbyte.integrations.destination.snowflake.write
|
||||
|
||||
import io.airbyte.cdk.load.test.util.NameMapper
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
|
||||
class SnowflakeNameMapper : NameMapper {
|
||||
override fun mapFieldName(path: List<String>): List<String> =
|
||||
|
||||
@@ -9,13 +9,16 @@ import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.data.StringValue
|
||||
import io.airbyte.cdk.load.data.json.toAirbyteValue
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.table.DefaultTempTableNameGenerator
|
||||
import io.airbyte.cdk.load.test.util.DestinationDataDumper
|
||||
import io.airbyte.cdk.load.test.util.OutputRecord
|
||||
import io.airbyte.cdk.load.util.UUIDGenerator
|
||||
import io.airbyte.cdk.load.util.deserializeToNode
|
||||
import io.airbyte.integrations.destination.snowflake.SnowflakeBeanFactory
|
||||
import io.airbyte.integrations.destination.snowflake.db.SnowflakeFinalTableNameGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeTableSchemaMapper
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeSqlNameUtils
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDirectLoadSqlGenerator
|
||||
|
||||
class SnowflakeRawDataDumper(
|
||||
private val configProvider: (ConfigurationSpecification) -> SnowflakeConfiguration
|
||||
@@ -27,8 +30,18 @@ class SnowflakeRawDataDumper(
|
||||
val output = mutableListOf<OutputRecord>()
|
||||
|
||||
val config = configProvider(spec)
|
||||
val sqlUtils = SnowflakeSqlNameUtils(config)
|
||||
val snowflakeFinalTableNameGenerator = SnowflakeFinalTableNameGenerator(config)
|
||||
val snowflakeColumnManager = SnowflakeColumnManager(config)
|
||||
val sqlGenerator =
|
||||
SnowflakeDirectLoadSqlGenerator(
|
||||
UUIDGenerator(),
|
||||
config,
|
||||
snowflakeColumnManager,
|
||||
)
|
||||
val snowflakeFinalTableNameGenerator =
|
||||
SnowflakeTableSchemaMapper(
|
||||
config = config,
|
||||
tempTableNameGenerator = DefaultTempTableNameGenerator(),
|
||||
)
|
||||
val dataSource =
|
||||
SnowflakeBeanFactory()
|
||||
.snowflakeDataSource(snowflakeConfiguration = config, airbyteEdition = "COMMUNITY")
|
||||
@@ -37,11 +50,11 @@ class SnowflakeRawDataDumper(
|
||||
ds.connection.use { connection ->
|
||||
val statement = connection.createStatement()
|
||||
val tableName =
|
||||
snowflakeFinalTableNameGenerator.getTableName(stream.mappedDescriptor)
|
||||
snowflakeFinalTableNameGenerator.toFinalTableName(stream.mappedDescriptor)
|
||||
|
||||
val resultSet =
|
||||
statement.executeQuery(
|
||||
"SELECT * FROM ${sqlUtils.fullyQualifiedName(tableName)}"
|
||||
"SELECT * FROM ${sqlGenerator.fullyQualifiedName(tableName)}"
|
||||
)
|
||||
|
||||
while (resultSet.next()) {
|
||||
|
||||
@@ -6,7 +6,7 @@ package io.airbyte.integrations.destination.snowflake
|
||||
|
||||
import com.zaxxer.hikari.HikariConfig
|
||||
import com.zaxxer.hikari.HikariDataSource
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.CdcDeletionMode
|
||||
import io.airbyte.integrations.destination.snowflake.spec.KeyPairAuthConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
|
||||
@@ -5,11 +5,9 @@
|
||||
package io.airbyte.integrations.destination.snowflake.check
|
||||
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.DEFAULT_COLUMNS
|
||||
import io.airbyte.integrations.destination.snowflake.sql.RAW_DATA_COLUMN
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.mockk.coEvery
|
||||
import io.mockk.coVerify
|
||||
import io.mockk.every
|
||||
@@ -23,47 +21,31 @@ internal class SnowflakeCheckerTest {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = [true, false])
|
||||
fun testSuccessfulCheck(isLegacyRawTablesOnly: Boolean) {
|
||||
val defaultColumnsMap =
|
||||
if (isLegacyRawTablesOnly) {
|
||||
linkedMapOf<String, String>().also { map ->
|
||||
(DEFAULT_COLUMNS + RAW_DATA_COLUMN).forEach {
|
||||
map[it.columnName] = it.columnType
|
||||
}
|
||||
}
|
||||
} else {
|
||||
linkedMapOf<String, String>().also { map ->
|
||||
(DEFAULT_COLUMNS + RAW_DATA_COLUMN).forEach {
|
||||
map[it.columnName.toSnowflakeCompatibleName()] = it.columnType
|
||||
}
|
||||
}
|
||||
}
|
||||
val defaultColumns = defaultColumnsMap.keys.toMutableList()
|
||||
val snowflakeAirbyteClient: SnowflakeAirbyteClient =
|
||||
mockk(relaxed = true) {
|
||||
coEvery { countTable(any()) } returns 1L
|
||||
coEvery { describeTable(any()) } returns defaultColumnsMap
|
||||
}
|
||||
mockk(relaxed = true) { coEvery { countTable(any()) } returns 1L }
|
||||
|
||||
val testSchema = "test-schema"
|
||||
val snowflakeConfiguration: SnowflakeConfiguration = mockk {
|
||||
every { schema } returns testSchema
|
||||
every { legacyRawTablesOnly } returns isLegacyRawTablesOnly
|
||||
}
|
||||
val snowflakeColumnUtils =
|
||||
mockk<SnowflakeColumnUtils>(relaxUnitFun = true) {
|
||||
every { getFormattedDefaultColumnNames(any()) } returns defaultColumns
|
||||
}
|
||||
|
||||
val columnManager: SnowflakeColumnManager = SnowflakeColumnManager(snowflakeConfiguration)
|
||||
|
||||
val checker =
|
||||
SnowflakeChecker(
|
||||
snowflakeAirbyteClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
columnManager = columnManager,
|
||||
)
|
||||
checker.check()
|
||||
|
||||
coVerify(exactly = 1) {
|
||||
snowflakeAirbyteClient.createNamespace(testSchema.toSnowflakeCompatibleName())
|
||||
if (isLegacyRawTablesOnly) {
|
||||
snowflakeAirbyteClient.createNamespace(testSchema)
|
||||
} else {
|
||||
snowflakeAirbyteClient.createNamespace(testSchema.toSnowflakeCompatibleName())
|
||||
}
|
||||
}
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.createTable(any(), any(), any(), any()) }
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.dropTable(any()) }
|
||||
@@ -72,48 +54,32 @@ internal class SnowflakeCheckerTest {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = [true, false])
|
||||
fun testUnsuccessfulCheck(isLegacyRawTablesOnly: Boolean) {
|
||||
val defaultColumnsMap =
|
||||
if (isLegacyRawTablesOnly) {
|
||||
linkedMapOf<String, String>().also { map ->
|
||||
(DEFAULT_COLUMNS + RAW_DATA_COLUMN).forEach {
|
||||
map[it.columnName] = it.columnType
|
||||
}
|
||||
}
|
||||
} else {
|
||||
linkedMapOf<String, String>().also { map ->
|
||||
(DEFAULT_COLUMNS + RAW_DATA_COLUMN).forEach {
|
||||
map[it.columnName.toSnowflakeCompatibleName()] = it.columnType
|
||||
}
|
||||
}
|
||||
}
|
||||
val defaultColumns = defaultColumnsMap.keys.toMutableList()
|
||||
val snowflakeAirbyteClient: SnowflakeAirbyteClient =
|
||||
mockk(relaxed = true) {
|
||||
coEvery { countTable(any()) } returns 0L
|
||||
coEvery { describeTable(any()) } returns defaultColumnsMap
|
||||
}
|
||||
mockk(relaxed = true) { coEvery { countTable(any()) } returns 0L }
|
||||
|
||||
val testSchema = "test-schema"
|
||||
val snowflakeConfiguration: SnowflakeConfiguration = mockk {
|
||||
every { schema } returns testSchema
|
||||
every { legacyRawTablesOnly } returns isLegacyRawTablesOnly
|
||||
}
|
||||
val snowflakeColumnUtils =
|
||||
mockk<SnowflakeColumnUtils>(relaxUnitFun = true) {
|
||||
every { getFormattedDefaultColumnNames(any()) } returns defaultColumns
|
||||
}
|
||||
|
||||
val columnManager: SnowflakeColumnManager = SnowflakeColumnManager(snowflakeConfiguration)
|
||||
|
||||
val checker =
|
||||
SnowflakeChecker(
|
||||
snowflakeAirbyteClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
columnManager = columnManager,
|
||||
)
|
||||
|
||||
assertThrows<IllegalArgumentException> { checker.check() }
|
||||
|
||||
coVerify(exactly = 1) {
|
||||
snowflakeAirbyteClient.createNamespace(testSchema.toSnowflakeCompatibleName())
|
||||
if (isLegacyRawTablesOnly) {
|
||||
snowflakeAirbyteClient.createNamespace(testSchema)
|
||||
} else {
|
||||
snowflakeAirbyteClient.createNamespace(testSchema.toSnowflakeCompatibleName())
|
||||
}
|
||||
}
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.createTable(any(), any(), any(), any()) }
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.dropTable(any()) }
|
||||
|
||||
@@ -6,24 +6,16 @@ package io.airbyte.integrations.destination.snowflake.client
|
||||
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.command.NamespaceMapper
|
||||
import io.airbyte.cdk.load.command.Overwrite
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.config.NamespaceDefinitionType
|
||||
import io.airbyte.cdk.load.data.AirbyteType
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_GENERATION_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_RAW_ID
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.db.ColumnDefinition
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.COUNT_TOTAL_ALIAS
|
||||
import io.airbyte.integrations.destination.snowflake.sql.ColumnAndType
|
||||
import io.airbyte.integrations.destination.snowflake.sql.DEFAULT_COLUMNS
|
||||
import io.airbyte.integrations.destination.snowflake.sql.QUOTE
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeDirectLoadSqlGenerator
|
||||
import io.mockk.Runs
|
||||
import io.mockk.every
|
||||
@@ -49,31 +41,18 @@ internal class SnowflakeAirbyteClientTest {
|
||||
private lateinit var client: SnowflakeAirbyteClient
|
||||
private lateinit var dataSource: DataSource
|
||||
private lateinit var sqlGenerator: SnowflakeDirectLoadSqlGenerator
|
||||
private lateinit var snowflakeColumnUtils: SnowflakeColumnUtils
|
||||
private lateinit var snowflakeConfiguration: SnowflakeConfiguration
|
||||
private lateinit var columnManager: SnowflakeColumnManager
|
||||
|
||||
@BeforeEach
|
||||
fun setup() {
|
||||
dataSource = mockk()
|
||||
sqlGenerator = mockk(relaxed = true)
|
||||
snowflakeColumnUtils =
|
||||
mockk(relaxed = true) {
|
||||
every { formatColumnName(any()) } answers
|
||||
{
|
||||
firstArg<String>().toSnowflakeCompatibleName()
|
||||
}
|
||||
every { getFormattedDefaultColumnNames(any()) } returns
|
||||
DEFAULT_COLUMNS.map { it.columnName.toSnowflakeCompatibleName() }
|
||||
}
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) { every { database } returns "test_database" }
|
||||
columnManager = mockk(relaxed = true)
|
||||
client =
|
||||
SnowflakeAirbyteClient(
|
||||
dataSource,
|
||||
sqlGenerator,
|
||||
snowflakeColumnUtils,
|
||||
snowflakeConfiguration
|
||||
)
|
||||
SnowflakeAirbyteClient(dataSource, sqlGenerator, snowflakeConfiguration, columnManager)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -231,7 +210,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
@Test
|
||||
fun testCreateTable() {
|
||||
val columnNameMapping = mockk<ColumnNameMapping>(relaxed = true)
|
||||
val stream = mockk<DestinationStream>()
|
||||
val stream = mockk<DestinationStream>(relaxed = true)
|
||||
val tableName = TableName(namespace = "namespace", name = "name")
|
||||
val resultSet = mockk<ResultSet>(relaxed = true)
|
||||
val statement =
|
||||
@@ -254,9 +233,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
columnNameMapping = columnNameMapping,
|
||||
replace = true,
|
||||
)
|
||||
verify(exactly = 1) {
|
||||
sqlGenerator.createTable(stream, tableName, columnNameMapping, true)
|
||||
}
|
||||
verify(exactly = 1) { sqlGenerator.createTable(tableName, any(), true) }
|
||||
verify(exactly = 1) { sqlGenerator.createSnowflakeStage(tableName) }
|
||||
verify(exactly = 2) { mockConnection.close() }
|
||||
}
|
||||
@@ -288,7 +265,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
targetTableName = destinationTableName,
|
||||
)
|
||||
verify(exactly = 1) {
|
||||
sqlGenerator.copyTable(columnNameMapping, sourceTableName, destinationTableName)
|
||||
sqlGenerator.copyTable(any<Set<String>>(), sourceTableName, destinationTableName)
|
||||
}
|
||||
verify(exactly = 1) { mockConnection.close() }
|
||||
}
|
||||
@@ -299,7 +276,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
val columnNameMapping = mockk<ColumnNameMapping>(relaxed = true)
|
||||
val sourceTableName = TableName(namespace = "namespace", name = "source")
|
||||
val destinationTableName = TableName(namespace = "namespace", name = "destination")
|
||||
val stream = mockk<DestinationStream>()
|
||||
val stream = mockk<DestinationStream>(relaxed = true)
|
||||
val resultSet = mockk<ResultSet>(relaxed = true)
|
||||
val statement =
|
||||
mockk<Statement> {
|
||||
@@ -322,12 +299,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
targetTableName = destinationTableName,
|
||||
)
|
||||
verify(exactly = 1) {
|
||||
sqlGenerator.upsertTable(
|
||||
stream,
|
||||
columnNameMapping,
|
||||
sourceTableName,
|
||||
destinationTableName
|
||||
)
|
||||
sqlGenerator.upsertTable(any(), sourceTableName, destinationTableName)
|
||||
}
|
||||
verify(exactly = 1) { mockConnection.close() }
|
||||
}
|
||||
@@ -379,7 +351,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
}
|
||||
|
||||
every { dataSource.connection } returns mockConnection
|
||||
every { snowflakeColumnUtils.getGenerationIdColumnName() } returns generationIdColumnName
|
||||
every { columnManager.getGenerationIdColumnName() } returns generationIdColumnName
|
||||
every { sqlGenerator.getGenerationId(tableName) } returns
|
||||
"SELECT $generationIdColumnName FROM ${tableName.toPrettyString(QUOTE)}"
|
||||
|
||||
@@ -501,8 +473,8 @@ internal class SnowflakeAirbyteClientTest {
|
||||
every { dataSource.connection } returns mockConnection
|
||||
|
||||
runBlocking {
|
||||
client.copyFromStage(tableName, "test.csv.gz")
|
||||
verify(exactly = 1) { sqlGenerator.copyFromStage(tableName, "test.csv.gz") }
|
||||
client.copyFromStage(tableName, "test.csv.gz", listOf())
|
||||
verify(exactly = 1) { sqlGenerator.copyFromStage(tableName, "test.csv.gz", listOf()) }
|
||||
verify(exactly = 1) { mockConnection.close() }
|
||||
}
|
||||
}
|
||||
@@ -556,7 +528,7 @@ internal class SnowflakeAirbyteClientTest {
|
||||
"COL1" andThen
|
||||
COLUMN_NAME_AB_RAW_ID.toSnowflakeCompatibleName() andThen
|
||||
"COL2"
|
||||
every { resultSet.getString("type") } returns "VARCHAR(255)" andThen "NUMBER(38,0)"
|
||||
every { resultSet.getString("type") } returns "VARCHAR(255)" andThen "NUMBER"
|
||||
every { resultSet.getString("null?") } returns "Y" andThen "N" andThen "N"
|
||||
|
||||
val statement =
|
||||
@@ -571,6 +543,10 @@ internal class SnowflakeAirbyteClientTest {
|
||||
|
||||
every { dataSource.connection } returns connection
|
||||
|
||||
// Mock the columnManager to return the correct set of meta columns
|
||||
every { columnManager.getMetaColumnNames() } returns
|
||||
setOf(COLUMN_NAME_AB_RAW_ID.toSnowflakeCompatibleName())
|
||||
|
||||
val result = client.getColumnsFromDb(tableName)
|
||||
|
||||
val expectedColumns =
|
||||
@@ -582,81 +558,6 @@ internal class SnowflakeAirbyteClientTest {
|
||||
assertEquals(expectedColumns, result)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun `getColumnsFromStream should return correct column definitions`() {
|
||||
val schema = mockk<AirbyteType>()
|
||||
val stream =
|
||||
DestinationStream(
|
||||
unmappedNamespace = "test_namespace",
|
||||
unmappedName = "test_stream",
|
||||
importType = Overwrite,
|
||||
schema = schema,
|
||||
generationId = 1,
|
||||
minimumGenerationId = 1,
|
||||
syncId = 1,
|
||||
namespaceMapper = NamespaceMapper(NamespaceDefinitionType.DESTINATION)
|
||||
)
|
||||
val columnNameMapping =
|
||||
ColumnNameMapping(
|
||||
mapOf(
|
||||
"col1" to "COL1_MAPPED",
|
||||
"col2" to "COL2_MAPPED",
|
||||
)
|
||||
)
|
||||
|
||||
val col1FieldType = mockk<FieldType>()
|
||||
every { col1FieldType.type } returns mockk()
|
||||
|
||||
val col2FieldType = mockk<FieldType>()
|
||||
every { col2FieldType.type } returns mockk()
|
||||
|
||||
every { schema.asColumns() } returns
|
||||
linkedMapOf("col1" to col1FieldType, "col2" to col2FieldType)
|
||||
every { snowflakeColumnUtils.toDialectType(col1FieldType.type) } returns "VARCHAR(255)"
|
||||
every { snowflakeColumnUtils.toDialectType(col2FieldType.type) } returns "NUMBER(38,0)"
|
||||
every { snowflakeColumnUtils.columnsAndTypes(any(), any()) } returns
|
||||
listOf(ColumnAndType("COL1_MAPPED", "VARCHAR"), ColumnAndType("COL2_MAPPED", "NUMBER"))
|
||||
every { snowflakeColumnUtils.formatColumnName(any(), false) } answers
|
||||
{
|
||||
firstArg<String>().toSnowflakeCompatibleName()
|
||||
}
|
||||
|
||||
val result = client.getColumnsFromStream(stream, columnNameMapping)
|
||||
|
||||
val expectedColumns =
|
||||
mapOf(
|
||||
"COL1_MAPPED" to ColumnType("VARCHAR", true),
|
||||
"COL2_MAPPED" to ColumnType("NUMBER", true),
|
||||
)
|
||||
|
||||
assertEquals(expectedColumns, result)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun `generateSchemaChanges should correctly identify changes`() {
|
||||
val columnsInDb =
|
||||
setOf(
|
||||
ColumnDefinition("COL1", "VARCHAR"),
|
||||
ColumnDefinition("COL2", "NUMBER"),
|
||||
ColumnDefinition("COL3", "BOOLEAN")
|
||||
)
|
||||
val columnsInStream =
|
||||
setOf(
|
||||
ColumnDefinition("COL1", "VARCHAR"), // Unchanged
|
||||
ColumnDefinition("COL3", "TEXT"), // Modified
|
||||
ColumnDefinition("COL4", "DATE") // Added
|
||||
)
|
||||
|
||||
val (added, deleted, modified) = client.generateSchemaChanges(columnsInDb, columnsInStream)
|
||||
|
||||
assertEquals(1, added.size)
|
||||
assertEquals("COL4", added.first().name)
|
||||
assertEquals(1, deleted.size)
|
||||
assertEquals("COL2", deleted.first().name)
|
||||
assertEquals(1, modified.size)
|
||||
assertEquals("COL3", modified.first().name)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testCreateNamespaceWithNetworkFailure() {
|
||||
val namespace = "test_namespace"
|
||||
|
||||
@@ -4,14 +4,19 @@
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.dataflow
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.command.DestinationStream.Descriptor
|
||||
import io.airbyte.cdk.load.dataflow.aggregate.StoreKey
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeRawRecordFormatter
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeRecordFormatter
|
||||
import io.airbyte.integrations.destination.snowflake.write.load.SnowflakeSchemaRecordFormatter
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
@@ -22,27 +27,33 @@ internal class SnowflakeAggregateFactoryTest {
|
||||
@Test
|
||||
fun testCreatingAggregateWithRawBuffer() {
|
||||
val descriptor = Descriptor(namespace = "namespace", name = "name")
|
||||
val directLoadTableExecutionConfig =
|
||||
DirectLoadTableExecutionConfig(
|
||||
tableName =
|
||||
TableName(
|
||||
namespace = descriptor.namespace!!,
|
||||
name = descriptor.name,
|
||||
)
|
||||
val tableName =
|
||||
TableName(
|
||||
namespace = descriptor.namespace!!,
|
||||
name = descriptor.name,
|
||||
)
|
||||
val directLoadTableExecutionConfig = DirectLoadTableExecutionConfig(tableName = tableName)
|
||||
val key = StoreKey(namespace = descriptor.namespace!!, name = descriptor.name)
|
||||
val streamStore = StreamStateStore<DirectLoadTableExecutionConfig>()
|
||||
streamStore.put(descriptor, directLoadTableExecutionConfig)
|
||||
streamStore.put(key, directLoadTableExecutionConfig)
|
||||
|
||||
val stream = mockk<DestinationStream>(relaxed = true)
|
||||
val catalog = mockk<DestinationCatalog> { every { getStream(key) } returns stream }
|
||||
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val snowflakeConfiguration =
|
||||
mockk<SnowflakeConfiguration> { every { legacyRawTablesOnly } returns true }
|
||||
val snowflakeColumnUtils = mockk<SnowflakeColumnUtils>(relaxed = true)
|
||||
val columnManager = SnowflakeColumnManager(snowflakeConfiguration)
|
||||
val snowflakeRecordFormatter: SnowflakeRecordFormatter = SnowflakeRawRecordFormatter()
|
||||
|
||||
val factory =
|
||||
SnowflakeAggregateFactory(
|
||||
snowflakeClient = snowflakeClient,
|
||||
streamStateStore = streamStore,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
catalog = catalog,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
)
|
||||
val aggregate = factory.create(key)
|
||||
assertNotNull(aggregate)
|
||||
@@ -52,26 +63,33 @@ internal class SnowflakeAggregateFactoryTest {
|
||||
@Test
|
||||
fun testCreatingAggregateWithStagingBuffer() {
|
||||
val descriptor = Descriptor(namespace = "namespace", name = "name")
|
||||
val directLoadTableExecutionConfig =
|
||||
DirectLoadTableExecutionConfig(
|
||||
tableName =
|
||||
TableName(
|
||||
namespace = descriptor.namespace!!,
|
||||
name = descriptor.name,
|
||||
)
|
||||
val tableName =
|
||||
TableName(
|
||||
namespace = descriptor.namespace!!,
|
||||
name = descriptor.name,
|
||||
)
|
||||
val directLoadTableExecutionConfig = DirectLoadTableExecutionConfig(tableName = tableName)
|
||||
val key = StoreKey(namespace = descriptor.namespace!!, name = descriptor.name)
|
||||
val streamStore = StreamStateStore<DirectLoadTableExecutionConfig>()
|
||||
streamStore.put(descriptor, directLoadTableExecutionConfig)
|
||||
streamStore.put(key, directLoadTableExecutionConfig)
|
||||
|
||||
val stream = mockk<DestinationStream>(relaxed = true)
|
||||
val catalog = mockk<DestinationCatalog> { every { getStream(key) } returns stream }
|
||||
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val snowflakeConfiguration = mockk<SnowflakeConfiguration>(relaxed = true)
|
||||
val snowflakeColumnUtils = mockk<SnowflakeColumnUtils>(relaxed = true)
|
||||
val snowflakeConfiguration =
|
||||
mockk<SnowflakeConfiguration> { every { legacyRawTablesOnly } returns false }
|
||||
val columnManager = SnowflakeColumnManager(snowflakeConfiguration)
|
||||
val snowflakeRecordFormatter: SnowflakeRecordFormatter = SnowflakeSchemaRecordFormatter()
|
||||
|
||||
val factory =
|
||||
SnowflakeAggregateFactory(
|
||||
snowflakeClient = snowflakeClient,
|
||||
streamStateStore = streamStore,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
catalog = catalog,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
)
|
||||
val aggregate = factory.create(key)
|
||||
assertNotNull(aggregate)
|
||||
|
||||
@@ -1,23 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.db
|
||||
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
internal class SnowflakeColumnNameGeneratorTest {
|
||||
|
||||
@Test
|
||||
fun testGetColumnName() {
|
||||
val column = "test-column"
|
||||
val generator =
|
||||
SnowflakeColumnNameGenerator(mockk { every { legacyRawTablesOnly } returns false })
|
||||
val columnName = generator.getColumnName(column)
|
||||
assertEquals(column.toSnowflakeCompatibleName(), columnName.displayName)
|
||||
assertEquals(column.toSnowflakeCompatibleName(), columnName.canonicalName)
|
||||
}
|
||||
}
|
||||
@@ -1,72 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.db
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
internal class SnowflakeFinalTableNameGeneratorTest {
|
||||
|
||||
@Test
|
||||
fun testGetTableNameWithInternalNamespace() {
|
||||
val configuration =
|
||||
mockk<SnowflakeConfiguration> {
|
||||
every { internalTableSchema } returns "test-internal-namespace"
|
||||
every { legacyRawTablesOnly } returns true
|
||||
}
|
||||
val generator = SnowflakeFinalTableNameGenerator(config = configuration)
|
||||
val streamName = "test-stream-name"
|
||||
val streamNamespace = "test-stream-namespace"
|
||||
val streamDescriptor =
|
||||
mockk<DestinationStream.Descriptor> {
|
||||
every { namespace } returns streamNamespace
|
||||
every { name } returns streamName
|
||||
}
|
||||
val tableName = generator.getTableName(streamDescriptor)
|
||||
assertEquals("test-stream-namespace_raw__stream_test-stream-name", tableName.name)
|
||||
assertEquals("test-internal-namespace", tableName.namespace)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetTableNameWithNamespace() {
|
||||
val configuration =
|
||||
mockk<SnowflakeConfiguration> { every { legacyRawTablesOnly } returns false }
|
||||
val generator = SnowflakeFinalTableNameGenerator(config = configuration)
|
||||
val streamName = "test-stream-name"
|
||||
val streamNamespace = "test-stream-namespace"
|
||||
val streamDescriptor =
|
||||
mockk<DestinationStream.Descriptor> {
|
||||
every { namespace } returns streamNamespace
|
||||
every { name } returns streamName
|
||||
}
|
||||
val tableName = generator.getTableName(streamDescriptor)
|
||||
assertEquals("TEST-STREAM-NAME", tableName.name)
|
||||
assertEquals("TEST-STREAM-NAMESPACE", tableName.namespace)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetTableNameWithDefaultNamespace() {
|
||||
val defaultNamespace = "test-default-namespace"
|
||||
val configuration =
|
||||
mockk<SnowflakeConfiguration> {
|
||||
every { schema } returns defaultNamespace
|
||||
every { legacyRawTablesOnly } returns false
|
||||
}
|
||||
val generator = SnowflakeFinalTableNameGenerator(config = configuration)
|
||||
val streamName = "test-stream-name"
|
||||
val streamDescriptor =
|
||||
mockk<DestinationStream.Descriptor> {
|
||||
every { namespace } returns null
|
||||
every { name } returns streamName
|
||||
}
|
||||
val tableName = generator.getTableName(streamDescriptor)
|
||||
assertEquals("TEST-STREAM-NAME", tableName.name)
|
||||
assertEquals("TEST-DEFAULT-NAMESPACE", tableName.namespace)
|
||||
}
|
||||
}
|
||||
@@ -1,27 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.db
|
||||
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.CsvSource
|
||||
|
||||
internal class SnowflakeNameGeneratorsTest {
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource(
|
||||
value =
|
||||
[
|
||||
"test-name,TEST-NAME",
|
||||
"1-test-name,1-TEST-NAME",
|
||||
"test-name!!!,TEST-NAME!!!",
|
||||
"test\${name,TEST__NAME",
|
||||
"test\"name,TEST\"\"NAME",
|
||||
]
|
||||
)
|
||||
fun testToSnowflakeCompatibleName(name: String, expected: String) {
|
||||
assertEquals(expected, name.toSnowflakeCompatibleName())
|
||||
}
|
||||
}
|
||||
@@ -1,358 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.sql
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import io.airbyte.cdk.load.data.ArrayType
|
||||
import io.airbyte.cdk.load.data.ArrayTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.BooleanType
|
||||
import io.airbyte.cdk.load.data.DateType
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.data.IntegerType
|
||||
import io.airbyte.cdk.load.data.NumberType
|
||||
import io.airbyte.cdk.load.data.ObjectType
|
||||
import io.airbyte.cdk.load.data.ObjectTypeWithEmptySchema
|
||||
import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.StringType
|
||||
import io.airbyte.cdk.load.data.TimeTypeWithTimezone
|
||||
import io.airbyte.cdk.load.data.TimeTypeWithoutTimezone
|
||||
import io.airbyte.cdk.load.data.TimestampTypeWithTimezone
|
||||
import io.airbyte.cdk.load.data.TimestampTypeWithoutTimezone
|
||||
import io.airbyte.cdk.load.data.UnionType
|
||||
import io.airbyte.cdk.load.data.UnknownType
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_DATA
|
||||
import io.airbyte.cdk.load.orchestration.db.ColumnNameGenerator
|
||||
import io.airbyte.cdk.load.table.CDC_DELETED_AT_COLUMN
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.integrations.destination.snowflake.db.SnowflakeColumnNameGenerator
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import kotlin.collections.LinkedHashMap
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.CsvSource
|
||||
|
||||
internal class SnowflakeColumnUtilsTest {
|
||||
|
||||
private lateinit var snowflakeConfiguration: SnowflakeConfiguration
|
||||
private lateinit var snowflakeColumnUtils: SnowflakeColumnUtils
|
||||
private lateinit var snowflakeColumnNameGenerator: SnowflakeColumnNameGenerator
|
||||
|
||||
@BeforeEach
|
||||
fun setup() {
|
||||
snowflakeConfiguration = mockk(relaxed = true)
|
||||
snowflakeColumnNameGenerator =
|
||||
mockk(relaxed = true) {
|
||||
every { getColumnName(any()) } answers
|
||||
{
|
||||
val displayName =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) firstArg<String>()
|
||||
else firstArg<String>().toSnowflakeCompatibleName()
|
||||
val canonicalName =
|
||||
if (snowflakeConfiguration.legacyRawTablesOnly) firstArg<String>()
|
||||
else firstArg<String>().toSnowflakeCompatibleName()
|
||||
ColumnNameGenerator.ColumnName(
|
||||
displayName = displayName,
|
||||
canonicalName = canonicalName,
|
||||
)
|
||||
}
|
||||
}
|
||||
snowflakeColumnUtils =
|
||||
SnowflakeColumnUtils(snowflakeConfiguration, snowflakeColumnNameGenerator)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testDefaultColumns() {
|
||||
val expectedDefaultColumns = DEFAULT_COLUMNS
|
||||
assertEquals(expectedDefaultColumns, snowflakeColumnUtils.defaultColumns())
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testDefaultRawColumns() {
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
|
||||
val expectedDefaultColumns = DEFAULT_COLUMNS + RAW_COLUMNS
|
||||
|
||||
assertEquals(expectedDefaultColumns, snowflakeColumnUtils.defaultColumns())
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetFormattedDefaultColumnNames() {
|
||||
val expectedDefaultColumnNames =
|
||||
DEFAULT_COLUMNS.map { it.columnName.toSnowflakeCompatibleName() }
|
||||
val defaultColumnNames = snowflakeColumnUtils.getFormattedDefaultColumnNames()
|
||||
assertEquals(expectedDefaultColumnNames, defaultColumnNames)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetFormattedDefaultColumnNamesQuoted() {
|
||||
val expectedDefaultColumnNames =
|
||||
DEFAULT_COLUMNS.map { it.columnName.toSnowflakeCompatibleName().quote() }
|
||||
val defaultColumnNames = snowflakeColumnUtils.getFormattedDefaultColumnNames(true)
|
||||
assertEquals(expectedDefaultColumnNames, defaultColumnNames)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetColumnName() {
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("original" to "actual"))
|
||||
val columnNames = snowflakeColumnUtils.getColumnNames(columnNameMapping)
|
||||
val expectedColumnNames =
|
||||
(DEFAULT_COLUMNS.map { it.columnName.toSnowflakeCompatibleName() } + listOf("actual"))
|
||||
.joinToString(",") { it.quote() }
|
||||
assertEquals(expectedColumnNames, columnNames)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetRawColumnName() {
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("original" to "actual"))
|
||||
val columnNames = snowflakeColumnUtils.getColumnNames(columnNameMapping)
|
||||
val expectedColumnNames =
|
||||
(DEFAULT_COLUMNS.map { it.columnName } + RAW_COLUMNS.map { it.columnName })
|
||||
.joinToString(",") { it.quote() }
|
||||
assertEquals(expectedColumnNames, columnNames)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetRawFormattedColumnNames() {
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("original" to "actual"))
|
||||
val schemaColumns =
|
||||
mapOf(
|
||||
"column_one" to FieldType(StringType, true),
|
||||
"column_two" to FieldType(IntegerType, true),
|
||||
"original" to FieldType(StringType, true),
|
||||
CDC_DELETED_AT_COLUMN to FieldType(TimestampTypeWithTimezone, true)
|
||||
)
|
||||
val expectedColumnNames =
|
||||
DEFAULT_COLUMNS.map { it.columnName.quote() } +
|
||||
RAW_COLUMNS.map { it.columnName.quote() }
|
||||
|
||||
val columnNames =
|
||||
snowflakeColumnUtils.getFormattedColumnNames(
|
||||
columns = schemaColumns,
|
||||
columnNameMapping = columnNameMapping
|
||||
)
|
||||
assertEquals(expectedColumnNames.size, columnNames.size)
|
||||
assertEquals(expectedColumnNames.sorted(), columnNames.sorted())
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetFormattedColumnNames() {
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("original" to "actual"))
|
||||
val schemaColumns =
|
||||
mapOf(
|
||||
"column_one" to FieldType(StringType, true),
|
||||
"column_two" to FieldType(IntegerType, true),
|
||||
"original" to FieldType(StringType, true),
|
||||
CDC_DELETED_AT_COLUMN to FieldType(TimestampTypeWithTimezone, true)
|
||||
)
|
||||
val expectedColumnNames =
|
||||
listOf(
|
||||
"actual",
|
||||
"column_one",
|
||||
"column_two",
|
||||
CDC_DELETED_AT_COLUMN,
|
||||
)
|
||||
.map { it.quote() } +
|
||||
DEFAULT_COLUMNS.map { it.columnName.toSnowflakeCompatibleName().quote() }
|
||||
val columnNames =
|
||||
snowflakeColumnUtils.getFormattedColumnNames(
|
||||
columns = schemaColumns,
|
||||
columnNameMapping = columnNameMapping
|
||||
)
|
||||
assertEquals(expectedColumnNames.size, columnNames.size)
|
||||
assertEquals(expectedColumnNames.sorted(), columnNames.sorted())
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetFormattedColumnNamesNoQuotes() {
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("original" to "actual"))
|
||||
val schemaColumns =
|
||||
mapOf(
|
||||
"column_one" to FieldType(StringType, true),
|
||||
"column_two" to FieldType(IntegerType, true),
|
||||
"original" to FieldType(StringType, true),
|
||||
CDC_DELETED_AT_COLUMN to FieldType(TimestampTypeWithTimezone, true)
|
||||
)
|
||||
val expectedColumnNames =
|
||||
listOf(
|
||||
"actual",
|
||||
"column_one",
|
||||
"column_two",
|
||||
CDC_DELETED_AT_COLUMN,
|
||||
) + DEFAULT_COLUMNS.map { it.columnName.toSnowflakeCompatibleName() }
|
||||
val columnNames =
|
||||
snowflakeColumnUtils.getFormattedColumnNames(
|
||||
columns = schemaColumns,
|
||||
columnNameMapping = columnNameMapping,
|
||||
quote = false
|
||||
)
|
||||
assertEquals(expectedColumnNames.size, columnNames.size)
|
||||
assertEquals(expectedColumnNames.sorted(), columnNames.sorted())
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGeneratingRawTableColumnsAndTypesNoColumnMapping() {
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
|
||||
val columns =
|
||||
snowflakeColumnUtils.columnsAndTypes(
|
||||
columns = emptyMap(),
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
assertEquals(DEFAULT_COLUMNS.size + RAW_COLUMNS.size, columns.size)
|
||||
assertEquals(
|
||||
"${SnowflakeDataType.VARIANT.typeName} $NOT_NULL",
|
||||
columns.find { it.columnName == RAW_DATA_COLUMN.columnName }?.columnType
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGeneratingColumnsAndTypesNoColumnMapping() {
|
||||
val columnName = "test-column"
|
||||
val fieldType = FieldType(StringType, false)
|
||||
val declaredColumns = mapOf(columnName to fieldType)
|
||||
|
||||
val columns =
|
||||
snowflakeColumnUtils.columnsAndTypes(
|
||||
columns = declaredColumns,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
assertEquals(DEFAULT_COLUMNS.size + 1, columns.size)
|
||||
assertEquals(
|
||||
"${SnowflakeDataType.VARCHAR.typeName} $NOT_NULL",
|
||||
columns.find { it.columnName == columnName }?.columnType
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGeneratingColumnsAndTypesWithColumnMapping() {
|
||||
val columnName = "test-column"
|
||||
val mappedColumnName = "mapped-column-name"
|
||||
val fieldType = FieldType(StringType, false)
|
||||
val declaredColumns = mapOf(columnName to fieldType)
|
||||
val columnNameMapping = ColumnNameMapping(mapOf(columnName to mappedColumnName))
|
||||
|
||||
val columns =
|
||||
snowflakeColumnUtils.columnsAndTypes(
|
||||
columns = declaredColumns,
|
||||
columnNameMapping = columnNameMapping
|
||||
)
|
||||
assertEquals(DEFAULT_COLUMNS.size + 1, columns.size)
|
||||
assertEquals(
|
||||
"${SnowflakeDataType.VARCHAR.typeName} $NOT_NULL",
|
||||
columns.find { it.columnName == mappedColumnName }?.columnType
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testToDialectType() {
|
||||
assertEquals(
|
||||
SnowflakeDataType.BOOLEAN.typeName,
|
||||
snowflakeColumnUtils.toDialectType(BooleanType)
|
||||
)
|
||||
assertEquals(SnowflakeDataType.DATE.typeName, snowflakeColumnUtils.toDialectType(DateType))
|
||||
assertEquals(
|
||||
SnowflakeDataType.NUMBER.typeName,
|
||||
snowflakeColumnUtils.toDialectType(IntegerType)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.FLOAT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(NumberType)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.VARCHAR.typeName,
|
||||
snowflakeColumnUtils.toDialectType(StringType)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.VARCHAR.typeName,
|
||||
snowflakeColumnUtils.toDialectType(TimeTypeWithTimezone)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.TIME.typeName,
|
||||
snowflakeColumnUtils.toDialectType(TimeTypeWithoutTimezone)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.TIMESTAMP_TZ.typeName,
|
||||
snowflakeColumnUtils.toDialectType(TimestampTypeWithTimezone)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.TIMESTAMP_NTZ.typeName,
|
||||
snowflakeColumnUtils.toDialectType(TimestampTypeWithoutTimezone)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.ARRAY.typeName,
|
||||
snowflakeColumnUtils.toDialectType(ArrayType(items = FieldType(StringType, false)))
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.ARRAY.typeName,
|
||||
snowflakeColumnUtils.toDialectType(ArrayTypeWithoutSchema)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.OBJECT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(
|
||||
ObjectType(
|
||||
properties = LinkedHashMap(),
|
||||
additionalProperties = false,
|
||||
)
|
||||
)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.OBJECT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(ObjectTypeWithEmptySchema)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.OBJECT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(ObjectTypeWithoutSchema)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.VARIANT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(
|
||||
UnionType(
|
||||
options = setOf(StringType),
|
||||
isLegacyUnion = true,
|
||||
)
|
||||
)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.VARIANT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(
|
||||
UnionType(
|
||||
options = emptySet(),
|
||||
isLegacyUnion = false,
|
||||
)
|
||||
)
|
||||
)
|
||||
assertEquals(
|
||||
SnowflakeDataType.VARIANT.typeName,
|
||||
snowflakeColumnUtils.toDialectType(UnknownType(schema = mockk<JsonNode>()))
|
||||
)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource(
|
||||
value =
|
||||
[
|
||||
"$COLUMN_NAME_DATA, true, \"$COLUMN_NAME_DATA\"",
|
||||
"some-other_Column, true, \"SOME-OTHER_COLUMN\"",
|
||||
"$COLUMN_NAME_DATA, false, $COLUMN_NAME_DATA",
|
||||
"some-other_Column, false, SOME-OTHER_COLUMN",
|
||||
"$COLUMN_NAME_DATA, true, \"$COLUMN_NAME_DATA\"",
|
||||
"some-other_Column, true, \"SOME-OTHER_COLUMN\"",
|
||||
]
|
||||
)
|
||||
fun testFormatColumnName(columnName: String, quote: Boolean, expectedFormattedName: String) {
|
||||
assertEquals(
|
||||
expectedFormattedName,
|
||||
snowflakeColumnUtils.formatColumnName(columnName, quote)
|
||||
)
|
||||
}
|
||||
}
|
||||
File diff suppressed because it is too large
Load Diff
@@ -1,97 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.sql
|
||||
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
internal class SnowflakeSqlNameUtilsTest {
|
||||
|
||||
private lateinit var snowflakeConfiguration: SnowflakeConfiguration
|
||||
private lateinit var snowflakeSqlNameUtils: SnowflakeSqlNameUtils
|
||||
|
||||
@BeforeEach
|
||||
fun setUp() {
|
||||
snowflakeConfiguration = mockk(relaxed = true)
|
||||
snowflakeSqlNameUtils =
|
||||
SnowflakeSqlNameUtils(snowflakeConfiguration = snowflakeConfiguration)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFullyQualifiedName() {
|
||||
val databaseName = "test-database"
|
||||
val namespace = "test-namespace"
|
||||
val name = "test=name"
|
||||
val tableName = TableName(namespace = namespace, name = name)
|
||||
every { snowflakeConfiguration.database } returns databaseName
|
||||
|
||||
val expectedName =
|
||||
snowflakeSqlNameUtils.combineParts(
|
||||
listOf(
|
||||
databaseName.toSnowflakeCompatibleName(),
|
||||
tableName.namespace,
|
||||
tableName.name
|
||||
)
|
||||
)
|
||||
val fullyQualifiedName = snowflakeSqlNameUtils.fullyQualifiedName(tableName)
|
||||
assertEquals(expectedName, fullyQualifiedName)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFullyQualifiedNamespace() {
|
||||
val databaseName = "test-database"
|
||||
val namespace = "test-namespace"
|
||||
every { snowflakeConfiguration.database } returns databaseName
|
||||
|
||||
val fullyQualifiedNamespace = snowflakeSqlNameUtils.fullyQualifiedNamespace(namespace)
|
||||
assertEquals("\"TEST-DATABASE\".\"test-namespace\"", fullyQualifiedNamespace)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFullyQualifiedStageName() {
|
||||
val databaseName = "test-database"
|
||||
val namespace = "test-namespace"
|
||||
val name = "test=name"
|
||||
val tableName = TableName(namespace = namespace, name = name)
|
||||
every { snowflakeConfiguration.database } returns databaseName
|
||||
|
||||
val expectedName =
|
||||
snowflakeSqlNameUtils.combineParts(
|
||||
listOf(
|
||||
databaseName.toSnowflakeCompatibleName(),
|
||||
namespace,
|
||||
"$STAGE_NAME_PREFIX$name"
|
||||
)
|
||||
)
|
||||
val fullyQualifiedName = snowflakeSqlNameUtils.fullyQualifiedStageName(tableName)
|
||||
assertEquals(expectedName, fullyQualifiedName)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFullyQualifiedStageNameWithEscape() {
|
||||
val databaseName = "test-database"
|
||||
val namespace = "test-namespace"
|
||||
val name = "test=\"\"\'name"
|
||||
val tableName = TableName(namespace = namespace, name = name)
|
||||
every { snowflakeConfiguration.database } returns databaseName
|
||||
|
||||
val expectedName =
|
||||
snowflakeSqlNameUtils.combineParts(
|
||||
listOf(
|
||||
databaseName.toSnowflakeCompatibleName(),
|
||||
namespace,
|
||||
"$STAGE_NAME_PREFIX${sqlEscape(name)}"
|
||||
)
|
||||
)
|
||||
val fullyQualifiedName = snowflakeSqlNameUtils.fullyQualifiedStageName(tableName, true)
|
||||
assertEquals(expectedName, fullyQualifiedName)
|
||||
}
|
||||
}
|
||||
@@ -6,20 +6,21 @@ package io.airbyte.integrations.destination.snowflake.write
|
||||
|
||||
import io.airbyte.cdk.SystemErrorException
|
||||
import io.airbyte.cdk.load.command.Append
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatusGatherer
|
||||
import io.airbyte.cdk.load.orchestration.db.TableNames
|
||||
import io.airbyte.cdk.load.orchestration.db.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadInitialStatus
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableAppendStreamLoader
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableAppendTruncateStreamLoader
|
||||
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableStatus
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableNameInfo
|
||||
import io.airbyte.cdk.load.table.ColumnNameMapping
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.cdk.load.schema.model.StreamTableSchema
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.cdk.load.schema.model.TableNames
|
||||
import io.airbyte.cdk.load.table.DatabaseInitialStatusGatherer
|
||||
import io.airbyte.cdk.load.table.TempTableNameGenerator
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadInitialStatus
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableAppendStreamLoader
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableAppendTruncateStreamLoader
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableExecutionConfig
|
||||
import io.airbyte.cdk.load.table.directload.DirectLoadTableStatus
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.mockk.coEvery
|
||||
import io.mockk.coVerify
|
||||
import io.mockk.every
|
||||
@@ -34,55 +35,93 @@ internal class SnowflakeWriterTest {
|
||||
@Test
|
||||
fun testSetup() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
val stream = mockk<DestinationStream>()
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val tempTableName = TableName(namespace = "test-namespace", name = "test-name-temp")
|
||||
val tableNames = TableNames(finalTableName = tableName, tempTableName = tempTableName)
|
||||
val stream =
|
||||
mockk<DestinationStream> {
|
||||
every { tableSchema } returns
|
||||
StreamTableSchema(
|
||||
tableNames = tableNames,
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = emptyMap(),
|
||||
finalSchema = emptyMap(),
|
||||
inputSchema = emptyMap()
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
every { mappedDescriptor } returns
|
||||
DestinationStream.Descriptor(
|
||||
namespace = tableName.namespace,
|
||||
name = tableName.name
|
||||
)
|
||||
every { importType } returns Append
|
||||
}
|
||||
val catalog = DestinationCatalog(listOf(stream))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer =
|
||||
mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>> {
|
||||
coEvery { gatherInitialStatus(catalog) } returns emptyMap()
|
||||
coEvery { gatherInitialStatus() } returns
|
||||
mapOf(
|
||||
stream to
|
||||
DirectLoadInitialStatus(
|
||||
realTable = DirectLoadTableStatus(false),
|
||||
tempTable = null
|
||||
)
|
||||
)
|
||||
}
|
||||
val streamStateStore = mockk<StreamStateStore<DirectLoadTableExecutionConfig>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
catalog = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
streamStateStore = streamStateStore,
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = mockk(),
|
||||
snowflakeConfiguration = mockk(relaxed = true),
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) {
|
||||
every { internalTableSchema } returns "internal_schema"
|
||||
},
|
||||
)
|
||||
|
||||
runBlocking { writer.setup() }
|
||||
|
||||
coVerify(exactly = 1) { snowflakeClient.createNamespace(tableName.namespace) }
|
||||
coVerify(exactly = 1) { stateGatherer.gatherInitialStatus(catalog) }
|
||||
coVerify(exactly = 1) { stateGatherer.gatherInitialStatus() }
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testCreateStreamLoaderFirstGeneration() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
val tempTableName = TableName(namespace = "test-namespace", name = "test-name-temp")
|
||||
val tableNames = TableNames(finalTableName = tableName, tempTableName = tempTableName)
|
||||
val stream =
|
||||
mockk<DestinationStream> {
|
||||
every { minimumGenerationId } returns 0L
|
||||
every { generationId } returns 0L
|
||||
every { importType } returns Append
|
||||
every { tableSchema } returns
|
||||
StreamTableSchema(
|
||||
tableNames = tableNames,
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = emptyMap(),
|
||||
finalSchema = emptyMap(),
|
||||
inputSchema = emptyMap()
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
every { mappedDescriptor } returns
|
||||
DestinationStream.Descriptor(
|
||||
namespace = tableName.namespace,
|
||||
name = tableName.name
|
||||
)
|
||||
}
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val catalog = DestinationCatalog(listOf(stream))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer =
|
||||
mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>> {
|
||||
coEvery { gatherInitialStatus(catalog) } returns
|
||||
coEvery { gatherInitialStatus() } returns
|
||||
mapOf(
|
||||
stream to
|
||||
DirectLoadInitialStatus(
|
||||
@@ -93,14 +132,18 @@ internal class SnowflakeWriterTest {
|
||||
}
|
||||
val tempTableNameGenerator =
|
||||
mockk<TempTableNameGenerator> { every { generate(any()) } answers { firstArg() } }
|
||||
val streamStateStore = mockk<StreamStateStore<DirectLoadTableExecutionConfig>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
catalog = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
streamStateStore = streamStateStore,
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = tempTableNameGenerator,
|
||||
snowflakeConfiguration = mockk(relaxed = true),
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) {
|
||||
every { internalTableSchema } returns "internal_schema"
|
||||
},
|
||||
)
|
||||
|
||||
runBlocking {
|
||||
@@ -113,23 +156,35 @@ internal class SnowflakeWriterTest {
|
||||
@Test
|
||||
fun testCreateStreamLoaderNotFirstGeneration() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
val tempTableName = TableName(namespace = "test-namespace", name = "test-name-temp")
|
||||
val tableNames = TableNames(finalTableName = tableName, tempTableName = tempTableName)
|
||||
val stream =
|
||||
mockk<DestinationStream> {
|
||||
every { minimumGenerationId } returns 1L
|
||||
every { generationId } returns 1L
|
||||
every { importType } returns Append
|
||||
every { tableSchema } returns
|
||||
StreamTableSchema(
|
||||
tableNames = tableNames,
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = emptyMap(),
|
||||
finalSchema = emptyMap(),
|
||||
inputSchema = emptyMap()
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
every { mappedDescriptor } returns
|
||||
DestinationStream.Descriptor(
|
||||
namespace = tableName.namespace,
|
||||
name = tableName.name
|
||||
)
|
||||
}
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val catalog = DestinationCatalog(listOf(stream))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer =
|
||||
mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>> {
|
||||
coEvery { gatherInitialStatus(catalog) } returns
|
||||
coEvery { gatherInitialStatus() } returns
|
||||
mapOf(
|
||||
stream to
|
||||
DirectLoadInitialStatus(
|
||||
@@ -140,14 +195,18 @@ internal class SnowflakeWriterTest {
|
||||
}
|
||||
val tempTableNameGenerator =
|
||||
mockk<TempTableNameGenerator> { every { generate(any()) } answers { firstArg() } }
|
||||
val streamStateStore = mockk<StreamStateStore<DirectLoadTableExecutionConfig>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
catalog = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
streamStateStore = streamStateStore,
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = tempTableNameGenerator,
|
||||
snowflakeConfiguration = mockk(relaxed = true),
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) {
|
||||
every { internalTableSchema } returns "internal_schema"
|
||||
},
|
||||
)
|
||||
|
||||
runBlocking {
|
||||
@@ -160,22 +219,35 @@ internal class SnowflakeWriterTest {
|
||||
@Test
|
||||
fun testCreateStreamLoaderHybrid() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
val tempTableName = TableName(namespace = "test-namespace", name = "test-name-temp")
|
||||
val tableNames = TableNames(finalTableName = tableName, tempTableName = tempTableName)
|
||||
val stream =
|
||||
mockk<DestinationStream> {
|
||||
every { minimumGenerationId } returns 1L
|
||||
every { generationId } returns 2L
|
||||
every { importType } returns Append
|
||||
every { tableSchema } returns
|
||||
StreamTableSchema(
|
||||
tableNames = tableNames,
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = emptyMap(),
|
||||
finalSchema = emptyMap(),
|
||||
inputSchema = emptyMap()
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
every { mappedDescriptor } returns
|
||||
DestinationStream.Descriptor(
|
||||
namespace = tableName.namespace,
|
||||
name = tableName.name
|
||||
)
|
||||
}
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val catalog = DestinationCatalog(listOf(stream))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer =
|
||||
mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>> {
|
||||
coEvery { gatherInitialStatus(catalog) } returns
|
||||
coEvery { gatherInitialStatus() } returns
|
||||
mapOf(
|
||||
stream to
|
||||
DirectLoadInitialStatus(
|
||||
@@ -186,14 +258,18 @@ internal class SnowflakeWriterTest {
|
||||
}
|
||||
val tempTableNameGenerator =
|
||||
mockk<TempTableNameGenerator> { every { generate(any()) } answers { firstArg() } }
|
||||
val streamStateStore = mockk<StreamStateStore<DirectLoadTableExecutionConfig>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
catalog = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
streamStateStore = streamStateStore,
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = tempTableNameGenerator,
|
||||
snowflakeConfiguration = mockk(relaxed = true),
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) {
|
||||
every { internalTableSchema } returns "internal_schema"
|
||||
},
|
||||
)
|
||||
|
||||
runBlocking {
|
||||
@@ -203,169 +279,126 @@ internal class SnowflakeWriterTest {
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testSetupWithNamespaceCreationFailure() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
val stream = mockk<DestinationStream>()
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>()
|
||||
val stateGatherer = mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = mockk(),
|
||||
snowflakeConfiguration = mockk(),
|
||||
)
|
||||
|
||||
// Simulate network failure during namespace creation
|
||||
coEvery {
|
||||
snowflakeClient.createNamespace(tableName.namespace.toSnowflakeCompatibleName())
|
||||
} throws RuntimeException("Network connection failed")
|
||||
|
||||
assertThrows(RuntimeException::class.java) { runBlocking { writer.setup() } }
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testSetupWithInitialStatusGatheringFailure() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
val stream = mockk<DestinationStream>()
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer = mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = mockk(),
|
||||
snowflakeConfiguration = mockk(),
|
||||
)
|
||||
|
||||
// Simulate failure while gathering initial status
|
||||
coEvery { stateGatherer.gatherInitialStatus(catalog) } throws
|
||||
RuntimeException("Failed to query table status")
|
||||
|
||||
assertThrows(RuntimeException::class.java) { runBlocking { writer.setup() } }
|
||||
|
||||
// Verify namespace creation was still attempted
|
||||
coVerify(exactly = 1) { snowflakeClient.createNamespace(tableName.namespace) }
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testCreateStreamLoaderWithMissingInitialStatus() {
|
||||
val tableName = TableName(namespace = "test-namespace", name = "test-name")
|
||||
val tableNames = TableNames(rawTableName = null, finalTableName = tableName)
|
||||
fun testCreateStreamLoaderNamespaceLegacy() {
|
||||
val namespace = "test-namespace"
|
||||
val name = "test-name"
|
||||
val tableName = TableName(namespace = namespace, name = name)
|
||||
val tempTableName = TableName(namespace = namespace, name = "${name}-temp")
|
||||
val tableNames = TableNames(finalTableName = tableName, tempTableName = tempTableName)
|
||||
val stream =
|
||||
mockk<DestinationStream> {
|
||||
every { minimumGenerationId } returns 0L
|
||||
every { generationId } returns 0L
|
||||
every { importType } returns Append
|
||||
every { tableSchema } returns
|
||||
StreamTableSchema(
|
||||
tableNames = tableNames,
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = emptyMap(),
|
||||
finalSchema = emptyMap(),
|
||||
inputSchema = emptyMap()
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
every { mappedDescriptor } returns
|
||||
DestinationStream.Descriptor(
|
||||
namespace = tableName.namespace,
|
||||
name = tableName.name
|
||||
)
|
||||
}
|
||||
val missingStream =
|
||||
mockk<DestinationStream> {
|
||||
every { minimumGenerationId } returns 0L
|
||||
every { generationId } returns 0L
|
||||
}
|
||||
val tableInfo =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream to tableInfo))
|
||||
val catalog = DestinationCatalog(listOf(stream))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer =
|
||||
mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>> {
|
||||
coEvery { gatherInitialStatus(catalog) } returns
|
||||
coEvery { gatherInitialStatus() } returns
|
||||
mapOf(
|
||||
stream to
|
||||
DirectLoadInitialStatus(
|
||||
realTable = DirectLoadTableStatus(false),
|
||||
tempTable = null,
|
||||
tempTable = null
|
||||
)
|
||||
)
|
||||
}
|
||||
val streamStateStore = mockk<StreamStateStore<DirectLoadTableExecutionConfig>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
catalog = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
streamStateStore = streamStateStore,
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = mockk(),
|
||||
snowflakeConfiguration = mockk(relaxed = true),
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) {
|
||||
every { legacyRawTablesOnly } returns true
|
||||
every { internalTableSchema } returns "internal_schema"
|
||||
},
|
||||
)
|
||||
|
||||
runBlocking {
|
||||
writer.setup()
|
||||
// Try to create loader for a stream that wasn't in initial status
|
||||
assertThrows(NullPointerException::class.java) {
|
||||
writer.createStreamLoader(missingStream)
|
||||
runBlocking { writer.setup() }
|
||||
|
||||
coVerify(exactly = 1) { snowflakeClient.createNamespace(tableName.namespace) }
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testCreateStreamLoaderNamespaceNonLegacy() {
|
||||
val namespace = "test-namespace"
|
||||
val name = "test-name"
|
||||
val tableName = TableName(namespace = namespace, name = name)
|
||||
val tempTableName = TableName(namespace = namespace, name = "${name}-temp")
|
||||
val tableNames = TableNames(finalTableName = tableName, tempTableName = tempTableName)
|
||||
val stream =
|
||||
mockk<DestinationStream> {
|
||||
every { minimumGenerationId } returns 0L
|
||||
every { generationId } returns 0L
|
||||
every { importType } returns Append
|
||||
every { tableSchema } returns
|
||||
StreamTableSchema(
|
||||
tableNames = tableNames,
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = emptyMap(),
|
||||
finalSchema = emptyMap(),
|
||||
inputSchema = emptyMap()
|
||||
),
|
||||
importType = Append
|
||||
)
|
||||
every { mappedDescriptor } returns
|
||||
DestinationStream.Descriptor(
|
||||
namespace = tableName.namespace,
|
||||
name = tableName.name
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testCreateStreamLoaderWithNullFinalTableName() {
|
||||
// TableNames constructor throws IllegalStateException when both names are null
|
||||
assertThrows(IllegalStateException::class.java) {
|
||||
TableNames(rawTableName = null, finalTableName = null)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testSetupWithMultipleNamespaceFailuresPartial() {
|
||||
val tableName1 = TableName(namespace = "namespace1", name = "table1")
|
||||
val tableName2 = TableName(namespace = "namespace2", name = "table2")
|
||||
val tableNames1 = TableNames(rawTableName = null, finalTableName = tableName1)
|
||||
val tableNames2 = TableNames(rawTableName = null, finalTableName = tableName2)
|
||||
val stream1 = mockk<DestinationStream>()
|
||||
val stream2 = mockk<DestinationStream>()
|
||||
val tableInfo1 =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames1,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val tableInfo2 =
|
||||
TableNameInfo(
|
||||
tableNames = tableNames2,
|
||||
columnNameMapping = ColumnNameMapping(emptyMap())
|
||||
)
|
||||
val catalog = TableCatalog(mapOf(stream1 to tableInfo1, stream2 to tableInfo2))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>()
|
||||
val stateGatherer = mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>>()
|
||||
val catalog = DestinationCatalog(listOf(stream))
|
||||
val snowflakeClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val stateGatherer =
|
||||
mockk<DatabaseInitialStatusGatherer<DirectLoadInitialStatus>> {
|
||||
coEvery { gatherInitialStatus() } returns
|
||||
mapOf(
|
||||
stream to
|
||||
DirectLoadInitialStatus(
|
||||
realTable = DirectLoadTableStatus(false),
|
||||
tempTable = null
|
||||
)
|
||||
)
|
||||
}
|
||||
val streamStateStore = mockk<StreamStateStore<DirectLoadTableExecutionConfig>>()
|
||||
val writer =
|
||||
SnowflakeWriter(
|
||||
names = catalog,
|
||||
catalog = catalog,
|
||||
stateGatherer = stateGatherer,
|
||||
streamStateStore = mockk(),
|
||||
streamStateStore = streamStateStore,
|
||||
snowflakeClient = snowflakeClient,
|
||||
tempTableNameGenerator = mockk(),
|
||||
snowflakeConfiguration = mockk(),
|
||||
snowflakeConfiguration =
|
||||
mockk(relaxed = true) {
|
||||
every { legacyRawTablesOnly } returns false
|
||||
every { internalTableSchema } returns "internal_schema"
|
||||
},
|
||||
)
|
||||
|
||||
// First namespace succeeds, second fails (namespaces are uppercased by
|
||||
// toSnowflakeCompatibleName)
|
||||
coEvery { snowflakeClient.createNamespace("namespace1") } returns Unit
|
||||
coEvery { snowflakeClient.createNamespace("namespace2") } throws
|
||||
RuntimeException("Connection timeout")
|
||||
runBlocking { writer.setup() }
|
||||
|
||||
assertThrows(RuntimeException::class.java) { runBlocking { writer.setup() } }
|
||||
|
||||
// Verify both namespace creations were attempted
|
||||
coVerify(exactly = 1) { snowflakeClient.createNamespace("namespace1") }
|
||||
coVerify(exactly = 1) { snowflakeClient.createNamespace("namespace2") }
|
||||
coVerify(exactly = 1) { snowflakeClient.createNamespace(namespace) }
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,218 +4,220 @@
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.write.load
|
||||
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.data.AirbyteValue
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.data.IntegerValue
|
||||
import io.airbyte.cdk.load.data.NullValue
|
||||
import io.airbyte.cdk.load.data.StringType
|
||||
import io.airbyte.cdk.load.data.StringValue
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.table.TableName
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.cdk.load.schema.model.TableName
|
||||
import io.airbyte.integrations.destination.snowflake.client.SnowflakeAirbyteClient
|
||||
import io.airbyte.integrations.destination.snowflake.schema.SnowflakeColumnManager
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.mockk.coVerify
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import java.io.BufferedReader
|
||||
import java.io.File
|
||||
import java.io.InputStreamReader
|
||||
import java.util.zip.GZIPInputStream
|
||||
import kotlin.io.path.exists
|
||||
import kotlinx.coroutines.runBlocking
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Assertions.assertNotNull
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
internal class SnowflakeInsertBufferTest {
|
||||
|
||||
private lateinit var snowflakeConfiguration: SnowflakeConfiguration
|
||||
private lateinit var snowflakeColumnUtils: SnowflakeColumnUtils
|
||||
private lateinit var columnManager: SnowflakeColumnManager
|
||||
private lateinit var columnSchema: ColumnSchema
|
||||
private lateinit var snowflakeRecordFormatter: SnowflakeRecordFormatter
|
||||
|
||||
@BeforeEach
|
||||
fun setUp() {
|
||||
snowflakeConfiguration = mockk(relaxed = true)
|
||||
snowflakeColumnUtils = mockk(relaxed = true)
|
||||
snowflakeRecordFormatter = SnowflakeSchemaRecordFormatter()
|
||||
columnManager =
|
||||
mockk(relaxed = true) {
|
||||
every { getMetaColumns() } returns
|
||||
linkedMapOf(
|
||||
"_AIRBYTE_RAW_ID" to ColumnType("VARCHAR", false),
|
||||
"_AIRBYTE_EXTRACTED_AT" to ColumnType("TIMESTAMP_TZ", false),
|
||||
"_AIRBYTE_META" to ColumnType("VARIANT", false),
|
||||
"_AIRBYTE_GENERATION_ID" to ColumnType("NUMBER", true)
|
||||
)
|
||||
every { getTableColumnNames(any()) } returns
|
||||
listOf(
|
||||
"_AIRBYTE_RAW_ID",
|
||||
"_AIRBYTE_EXTRACTED_AT",
|
||||
"_AIRBYTE_META",
|
||||
"_AIRBYTE_GENERATION_ID",
|
||||
"columnName"
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testAccumulate() {
|
||||
val tableName = mockk<TableName>(relaxed = true)
|
||||
val tableName = TableName(namespace = "test", name = "table")
|
||||
val column = "columnName"
|
||||
val columns = linkedMapOf(column to "NUMBER(38,0)")
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = mapOf(column to column.uppercase()),
|
||||
finalSchema = mapOf(column.uppercase() to ColumnType("NUMBER", true)),
|
||||
inputSchema = mapOf(column to FieldType(StringType, nullable = true))
|
||||
)
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val record = createRecord(column)
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = columns,
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
flushLimit = 1,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
columnSchema = columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
)
|
||||
|
||||
buffer.accumulate(record)
|
||||
|
||||
assertEquals(true, buffer.csvFilePath?.exists())
|
||||
assertEquals(0, buffer.recordCount)
|
||||
runBlocking { buffer.accumulate(record) }
|
||||
assertEquals(1, buffer.recordCount)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testAccumulateRaw() {
|
||||
val tableName = mockk<TableName>(relaxed = true)
|
||||
fun testFlushToStaging() {
|
||||
val tableName = TableName(namespace = "test", name = "table")
|
||||
val column = "columnName"
|
||||
val columns = linkedMapOf(column to "NUMBER(38,0)")
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = mapOf(column to column.uppercase()),
|
||||
finalSchema = mapOf(column.uppercase() to ColumnType("NUMBER", true)),
|
||||
inputSchema = mapOf(column to FieldType(StringType, nullable = true))
|
||||
)
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val record = createRecord(column)
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = columns,
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
columnSchema = columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
flushLimit = 1,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
)
|
||||
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
|
||||
buffer.accumulate(record)
|
||||
|
||||
assertEquals(true, buffer.csvFilePath?.exists())
|
||||
assertEquals(1, buffer.recordCount)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFlush() {
|
||||
val tableName = mockk<TableName>(relaxed = true)
|
||||
val column = "columnName"
|
||||
val columns = linkedMapOf(column to "NUMBER(38,0)")
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val record = createRecord(column)
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = columns,
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
flushLimit = 1,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
val expectedColumnNames =
|
||||
listOf(
|
||||
"_AIRBYTE_RAW_ID",
|
||||
"_AIRBYTE_EXTRACTED_AT",
|
||||
"_AIRBYTE_META",
|
||||
"_AIRBYTE_GENERATION_ID",
|
||||
"columnName"
|
||||
)
|
||||
|
||||
runBlocking {
|
||||
buffer.accumulate(record)
|
||||
buffer.flush()
|
||||
}
|
||||
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.putInStage(tableName, any()) }
|
||||
coVerify(exactly = 1) {
|
||||
snowflakeAirbyteClient.copyFromStage(
|
||||
tableName,
|
||||
match { it.endsWith("$CSV_FILE_EXTENSION$FILE_SUFFIX") }
|
||||
)
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.putInStage(tableName, any()) }
|
||||
coVerify(exactly = 1) {
|
||||
snowflakeAirbyteClient.copyFromStage(tableName, any(), expectedColumnNames)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFlushRaw() {
|
||||
val tableName = mockk<TableName>(relaxed = true)
|
||||
fun testFlushToNoStaging() {
|
||||
val tableName = TableName(namespace = "test", name = "table")
|
||||
val column = "columnName"
|
||||
val columns = linkedMapOf(column to "NUMBER(38,0)")
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = mapOf(column to column.uppercase()),
|
||||
finalSchema = mapOf(column.uppercase() to ColumnType("NUMBER", true)),
|
||||
inputSchema = mapOf(column to FieldType(StringType, nullable = true))
|
||||
)
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
val record = createRecord(column)
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
columnSchema = columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
flushLimit = 1,
|
||||
)
|
||||
val expectedColumnNames =
|
||||
listOf(
|
||||
"_AIRBYTE_RAW_ID",
|
||||
"_AIRBYTE_EXTRACTED_AT",
|
||||
"_AIRBYTE_META",
|
||||
"_AIRBYTE_GENERATION_ID",
|
||||
"columnName"
|
||||
)
|
||||
runBlocking {
|
||||
buffer.accumulate(record)
|
||||
buffer.flush()
|
||||
// In legacy raw mode, it still uses staging
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.putInStage(tableName, any()) }
|
||||
coVerify(exactly = 1) {
|
||||
snowflakeAirbyteClient.copyFromStage(tableName, any(), expectedColumnNames)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFileCreation() {
|
||||
val tableName = TableName(namespace = "test", name = "table")
|
||||
val column = "columnName"
|
||||
columnSchema =
|
||||
ColumnSchema(
|
||||
inputToFinalColumnNames = mapOf(column to column.uppercase()),
|
||||
finalSchema = mapOf(column.uppercase() to ColumnType("NUMBER", true)),
|
||||
inputSchema = mapOf(column to FieldType(StringType, nullable = true))
|
||||
)
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val record = createRecord(column)
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = columns,
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
columnSchema = columnSchema,
|
||||
columnManager = columnManager,
|
||||
snowflakeRecordFormatter = snowflakeRecordFormatter,
|
||||
flushLimit = 1,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
)
|
||||
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
|
||||
runBlocking {
|
||||
buffer.accumulate(record)
|
||||
buffer.flush()
|
||||
}
|
||||
|
||||
coVerify(exactly = 1) { snowflakeAirbyteClient.putInStage(tableName, any()) }
|
||||
coVerify(exactly = 1) {
|
||||
snowflakeAirbyteClient.copyFromStage(
|
||||
tableName,
|
||||
match { it.endsWith("$CSV_FILE_EXTENSION$FILE_SUFFIX") }
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testMissingFields() {
|
||||
val tableName = mockk<TableName>(relaxed = true)
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val record = createRecord("COLUMN1")
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = linkedMapOf("COLUMN1" to "NUMBER(38,0)", "COLUMN2" to "NUMBER(38,0)"),
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
flushLimit = 1,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
)
|
||||
|
||||
runBlocking {
|
||||
buffer.accumulate(record)
|
||||
buffer.csvWriter?.flush()
|
||||
// The csvFilePath is internal, we can access it for testing
|
||||
val filepath = buffer.csvFilePath
|
||||
assertNotNull(filepath)
|
||||
val file = filepath!!.toFile()
|
||||
assert(file.exists())
|
||||
// Close the writer to ensure all data is flushed
|
||||
buffer.csvWriter?.close()
|
||||
assertEquals(
|
||||
"test-value$CSV_FIELD_SEPARATOR$CSV_LINE_DELIMITER",
|
||||
readFromCsvFile(buffer.csvFilePath!!.toFile())
|
||||
)
|
||||
val lines = mutableListOf<String>()
|
||||
GZIPInputStream(file.inputStream()).use { gzip ->
|
||||
BufferedReader(InputStreamReader(gzip)).use { bufferedReader ->
|
||||
bufferedReader.forEachLine { line -> lines.add(line) }
|
||||
}
|
||||
}
|
||||
assertEquals(1, lines.size)
|
||||
file.delete()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testMissingFieldsRaw() {
|
||||
val tableName = mockk<TableName>(relaxed = true)
|
||||
val snowflakeAirbyteClient = mockk<SnowflakeAirbyteClient>(relaxed = true)
|
||||
val record = createRecord("COLUMN1")
|
||||
val buffer =
|
||||
SnowflakeInsertBuffer(
|
||||
tableName = tableName,
|
||||
columns = linkedMapOf("COLUMN1" to "NUMBER(38,0)", "COLUMN2" to "NUMBER(38,0)"),
|
||||
snowflakeClient = snowflakeAirbyteClient,
|
||||
snowflakeConfiguration = snowflakeConfiguration,
|
||||
flushLimit = 1,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils,
|
||||
)
|
||||
|
||||
every { snowflakeConfiguration.legacyRawTablesOnly } returns true
|
||||
|
||||
runBlocking {
|
||||
buffer.accumulate(record)
|
||||
buffer.csvWriter?.flush()
|
||||
buffer.csvWriter?.close()
|
||||
assertEquals(
|
||||
"test-value$CSV_FIELD_SEPARATOR$CSV_LINE_DELIMITER",
|
||||
readFromCsvFile(buffer.csvFilePath!!.toFile())
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
private fun readFromCsvFile(file: File) =
|
||||
GZIPInputStream(file.inputStream()).use { input ->
|
||||
val reader = BufferedReader(InputStreamReader(input))
|
||||
reader.readText()
|
||||
}
|
||||
|
||||
private fun createRecord(columnName: String) =
|
||||
mapOf(
|
||||
columnName to AirbyteValue.from("test-value"),
|
||||
Meta.COLUMN_NAME_AB_EXTRACTED_AT to IntegerValue(System.currentTimeMillis()),
|
||||
Meta.COLUMN_NAME_AB_RAW_ID to StringValue("raw-id"),
|
||||
Meta.COLUMN_NAME_AB_GENERATION_ID to IntegerValue(1223),
|
||||
Meta.COLUMN_NAME_AB_META to StringValue("{\"changes\":[],\"syncId\":43}"),
|
||||
"${columnName}Null" to NullValue
|
||||
private fun createRecord(column: String): Map<String, AirbyteValue> {
|
||||
return mapOf(
|
||||
column to IntegerValue(value = 42),
|
||||
Meta.COLUMN_NAME_AB_GENERATION_ID to NullValue,
|
||||
Meta.COLUMN_NAME_AB_RAW_ID to StringValue("raw-id-1"),
|
||||
Meta.COLUMN_NAME_AB_EXTRACTED_AT to IntegerValue(1234567890),
|
||||
Meta.COLUMN_NAME_AB_META to StringValue("meta-data-foo"),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -15,12 +15,8 @@ import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_LOADED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_META
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_RAW_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_DATA
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import kotlin.collections.plus
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
private val AIRBYTE_COLUMN_TYPES_MAP =
|
||||
@@ -58,28 +54,16 @@ private fun createExpected(
|
||||
|
||||
internal class SnowflakeRawRecordFormatterTest {
|
||||
|
||||
private lateinit var snowflakeColumnUtils: SnowflakeColumnUtils
|
||||
|
||||
@BeforeEach
|
||||
fun setup() {
|
||||
snowflakeColumnUtils = mockk {
|
||||
every { getFormattedDefaultColumnNames(any()) } returns
|
||||
AIRBYTE_COLUMN_TYPES_MAP.keys.toList()
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFormatting() {
|
||||
val columnName = "test-column-name"
|
||||
val columnValue = "test-column-value"
|
||||
val columns = AIRBYTE_COLUMN_TYPES_MAP
|
||||
val record = createRecord(columnName = columnName, columnValue = columnValue)
|
||||
val formatter =
|
||||
SnowflakeRawRecordFormatter(
|
||||
columns = AIRBYTE_COLUMN_TYPES_MAP,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils
|
||||
)
|
||||
val formattedValue = formatter.format(record)
|
||||
val formatter = SnowflakeRawRecordFormatter()
|
||||
// RawRecordFormatter doesn't use columnSchema but still needs one per interface
|
||||
val dummyColumnSchema = ColumnSchema(emptyMap(), emptyMap(), emptyMap())
|
||||
val formattedValue = formatter.format(record, dummyColumnSchema)
|
||||
val expectedValue =
|
||||
createExpected(
|
||||
record = record,
|
||||
@@ -93,33 +77,28 @@ internal class SnowflakeRawRecordFormatterTest {
|
||||
fun testFormattingMigratedFromPreviousVersion() {
|
||||
val columnName = "test-column-name"
|
||||
val columnValue = "test-column-value"
|
||||
val columnsMap =
|
||||
linkedMapOf(
|
||||
COLUMN_NAME_AB_EXTRACTED_AT to "TIMESTAMP_TZ(9)",
|
||||
COLUMN_NAME_AB_LOADED_AT to "TIMESTAMP_TZ(9)",
|
||||
COLUMN_NAME_AB_META to "VARIANT",
|
||||
COLUMN_NAME_DATA to "VARIANT",
|
||||
COLUMN_NAME_AB_RAW_ID to "VARCHAR(16777216)",
|
||||
COLUMN_NAME_AB_GENERATION_ID to "NUMBER(38,0)",
|
||||
)
|
||||
val record = createRecord(columnName = columnName, columnValue = columnValue)
|
||||
val formatter =
|
||||
SnowflakeRawRecordFormatter(
|
||||
columns = columnsMap,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils
|
||||
)
|
||||
val formattedValue = formatter.format(record)
|
||||
val formatter = SnowflakeRawRecordFormatter()
|
||||
// RawRecordFormatter doesn't use columnSchema but still needs one per interface
|
||||
val dummyColumnSchema = ColumnSchema(emptyMap(), emptyMap(), emptyMap())
|
||||
val formattedValue = formatter.format(record, dummyColumnSchema)
|
||||
|
||||
// The formatter outputs in a fixed order regardless of input column order:
|
||||
// 1. AB_RAW_ID
|
||||
// 2. AB_EXTRACTED_AT
|
||||
// 3. AB_META
|
||||
// 4. AB_GENERATION_ID
|
||||
// 5. AB_LOADED_AT
|
||||
// 6. DATA (JSON with remaining columns)
|
||||
val expectedValue =
|
||||
createExpected(
|
||||
record = record,
|
||||
columns = columnsMap,
|
||||
airbyteColumns = columnsMap.keys.toList(),
|
||||
)
|
||||
.toMutableList()
|
||||
expectedValue.add(
|
||||
columnsMap.keys.indexOf(COLUMN_NAME_DATA),
|
||||
"{\"$columnName\":\"$columnValue\"}"
|
||||
)
|
||||
listOf(
|
||||
record[COLUMN_NAME_AB_RAW_ID]!!.toCsvValue(),
|
||||
record[COLUMN_NAME_AB_EXTRACTED_AT]!!.toCsvValue(),
|
||||
record[COLUMN_NAME_AB_META]!!.toCsvValue(),
|
||||
record[COLUMN_NAME_AB_GENERATION_ID]!!.toCsvValue(),
|
||||
record[COLUMN_NAME_AB_LOADED_AT]!!.toCsvValue(),
|
||||
"{\"$columnName\":\"$columnValue\"}"
|
||||
)
|
||||
assertEquals(expectedValue, formattedValue)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,66 +4,58 @@
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.write.load
|
||||
|
||||
import io.airbyte.cdk.load.component.ColumnType
|
||||
import io.airbyte.cdk.load.data.AirbyteValue
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
import io.airbyte.cdk.load.data.IntegerValue
|
||||
import io.airbyte.cdk.load.data.NullValue
|
||||
import io.airbyte.cdk.load.data.StringType
|
||||
import io.airbyte.cdk.load.data.StringValue
|
||||
import io.airbyte.cdk.load.data.csv.toCsvValue
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_EXTRACTED_AT
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_GENERATION_ID
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_META
|
||||
import io.airbyte.cdk.load.message.Meta.Companion.COLUMN_NAME_AB_RAW_ID
|
||||
import io.airbyte.integrations.destination.snowflake.db.toSnowflakeCompatibleName
|
||||
import io.airbyte.integrations.destination.snowflake.sql.SnowflakeColumnUtils
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import java.util.AbstractMap
|
||||
import kotlin.collections.component1
|
||||
import kotlin.collections.component2
|
||||
import io.airbyte.cdk.load.schema.model.ColumnSchema
|
||||
import io.airbyte.integrations.destination.snowflake.schema.toSnowflakeCompatibleName
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
private val AIRBYTE_COLUMN_TYPES_MAP =
|
||||
linkedMapOf(
|
||||
COLUMN_NAME_AB_RAW_ID to "VARCHAR(16777216)",
|
||||
COLUMN_NAME_AB_EXTRACTED_AT to "TIMESTAMP_TZ(9)",
|
||||
COLUMN_NAME_AB_META to "VARIANT",
|
||||
COLUMN_NAME_AB_GENERATION_ID to "NUMBER(38,0)",
|
||||
)
|
||||
.mapKeys { it.key.toSnowflakeCompatibleName() }
|
||||
|
||||
internal class SnowflakeSchemaRecordFormatterTest {
|
||||
|
||||
private lateinit var snowflakeColumnUtils: SnowflakeColumnUtils
|
||||
private fun createColumnSchema(userColumns: Map<String, String>): ColumnSchema {
|
||||
val finalSchema = linkedMapOf<String, ColumnType>()
|
||||
val inputToFinalColumnNames = mutableMapOf<String, String>()
|
||||
val inputSchema = mutableMapOf<String, FieldType>()
|
||||
|
||||
@BeforeEach
|
||||
fun setup() {
|
||||
snowflakeColumnUtils = mockk {
|
||||
every { getFormattedDefaultColumnNames(any()) } returns
|
||||
AIRBYTE_COLUMN_TYPES_MAP.keys.toList()
|
||||
// Add user columns
|
||||
userColumns.forEach { (name, type) ->
|
||||
val finalName = name.toSnowflakeCompatibleName()
|
||||
finalSchema[finalName] = ColumnType(type, true)
|
||||
inputToFinalColumnNames[name] = finalName
|
||||
inputSchema[name] = FieldType(StringType, nullable = true)
|
||||
}
|
||||
|
||||
return ColumnSchema(
|
||||
inputToFinalColumnNames = inputToFinalColumnNames,
|
||||
finalSchema = finalSchema,
|
||||
inputSchema = inputSchema
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testFormatting() {
|
||||
val columnName = "test-column-name"
|
||||
val columnValue = "test-column-value"
|
||||
val columns =
|
||||
(AIRBYTE_COLUMN_TYPES_MAP + linkedMapOf(columnName to "VARCHAR(16777216)")).mapKeys {
|
||||
it.key.toSnowflakeCompatibleName()
|
||||
}
|
||||
val userColumns = mapOf(columnName to "VARCHAR(16777216)")
|
||||
val columnSchema = createColumnSchema(userColumns)
|
||||
val record = createRecord(columnName, columnValue)
|
||||
val formatter =
|
||||
SnowflakeSchemaRecordFormatter(
|
||||
columns = columns as LinkedHashMap<String, String>,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils
|
||||
)
|
||||
val formattedValue = formatter.format(record)
|
||||
val formatter = SnowflakeSchemaRecordFormatter()
|
||||
val formattedValue = formatter.format(record, columnSchema)
|
||||
val expectedValue =
|
||||
createExpected(
|
||||
record = record,
|
||||
columns = columns,
|
||||
columnSchema = columnSchema,
|
||||
)
|
||||
assertEquals(expectedValue, formattedValue)
|
||||
}
|
||||
@@ -72,21 +64,15 @@ internal class SnowflakeSchemaRecordFormatterTest {
|
||||
fun testFormattingVariant() {
|
||||
val columnName = "test-column-name"
|
||||
val columnValue = "{\"test\": \"test-value\"}"
|
||||
val columns =
|
||||
(AIRBYTE_COLUMN_TYPES_MAP + linkedMapOf(columnName to "VARIANT")).mapKeys {
|
||||
it.key.toSnowflakeCompatibleName()
|
||||
}
|
||||
val userColumns = mapOf(columnName to "VARIANT")
|
||||
val columnSchema = createColumnSchema(userColumns)
|
||||
val record = createRecord(columnName, columnValue)
|
||||
val formatter =
|
||||
SnowflakeSchemaRecordFormatter(
|
||||
columns = columns as LinkedHashMap<String, String>,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils
|
||||
)
|
||||
val formattedValue = formatter.format(record)
|
||||
val formatter = SnowflakeSchemaRecordFormatter()
|
||||
val formattedValue = formatter.format(record, columnSchema)
|
||||
val expectedValue =
|
||||
createExpected(
|
||||
record = record,
|
||||
columns = columns,
|
||||
columnSchema = columnSchema,
|
||||
)
|
||||
assertEquals(expectedValue, formattedValue)
|
||||
}
|
||||
@@ -95,23 +81,16 @@ internal class SnowflakeSchemaRecordFormatterTest {
|
||||
fun testFormattingMissingColumn() {
|
||||
val columnName = "test-column-name"
|
||||
val columnValue = "test-column-value"
|
||||
val columns =
|
||||
AIRBYTE_COLUMN_TYPES_MAP +
|
||||
linkedMapOf(
|
||||
columnName to "VARCHAR(16777216)",
|
||||
"missing-column" to "VARCHAR(16777216)"
|
||||
)
|
||||
val userColumns =
|
||||
mapOf(columnName to "VARCHAR(16777216)", "missing-column" to "VARCHAR(16777216)")
|
||||
val columnSchema = createColumnSchema(userColumns)
|
||||
val record = createRecord(columnName, columnValue)
|
||||
val formatter =
|
||||
SnowflakeSchemaRecordFormatter(
|
||||
columns = columns as LinkedHashMap<String, String>,
|
||||
snowflakeColumnUtils = snowflakeColumnUtils
|
||||
)
|
||||
val formattedValue = formatter.format(record)
|
||||
val formatter = SnowflakeSchemaRecordFormatter()
|
||||
val formattedValue = formatter.format(record, columnSchema)
|
||||
val expectedValue =
|
||||
createExpected(
|
||||
record = record,
|
||||
columns = columns,
|
||||
columnSchema = columnSchema,
|
||||
filterMissing = false,
|
||||
)
|
||||
assertEquals(expectedValue, formattedValue)
|
||||
@@ -128,16 +107,37 @@ internal class SnowflakeSchemaRecordFormatterTest {
|
||||
|
||||
private fun createExpected(
|
||||
record: Map<String, AirbyteValue>,
|
||||
columns: Map<String, String>,
|
||||
columnSchema: ColumnSchema,
|
||||
filterMissing: Boolean = true,
|
||||
) =
|
||||
record.entries
|
||||
.associate { entry -> entry.key.toSnowflakeCompatibleName() to entry.value }
|
||||
.map { entry -> AbstractMap.SimpleEntry(entry.key, entry.value.toCsvValue()) }
|
||||
.sortedBy { entry ->
|
||||
if (columns.keys.indexOf(entry.key) > -1) columns.keys.indexOf(entry.key)
|
||||
else Int.MAX_VALUE
|
||||
): List<Any> {
|
||||
val columns = columnSchema.finalSchema.keys.toList()
|
||||
val result = mutableListOf<Any>()
|
||||
|
||||
// Add meta columns first in the expected order
|
||||
result.add(record[COLUMN_NAME_AB_RAW_ID]?.toCsvValue() ?: "")
|
||||
result.add(record[COLUMN_NAME_AB_EXTRACTED_AT]?.toCsvValue() ?: "")
|
||||
result.add(record[COLUMN_NAME_AB_META]?.toCsvValue() ?: "")
|
||||
result.add(record[COLUMN_NAME_AB_GENERATION_ID]?.toCsvValue() ?: "")
|
||||
|
||||
// Add user columns
|
||||
val userColumns =
|
||||
columns.filterNot { col ->
|
||||
listOf(
|
||||
COLUMN_NAME_AB_RAW_ID.toSnowflakeCompatibleName(),
|
||||
COLUMN_NAME_AB_EXTRACTED_AT.toSnowflakeCompatibleName(),
|
||||
COLUMN_NAME_AB_META.toSnowflakeCompatibleName(),
|
||||
COLUMN_NAME_AB_GENERATION_ID.toSnowflakeCompatibleName()
|
||||
)
|
||||
.contains(col)
|
||||
}
|
||||
.filter { (k, _) -> if (filterMissing) columns.contains(k) else true }
|
||||
.map { it.value }
|
||||
|
||||
userColumns.forEach { columnName ->
|
||||
val value = record[columnName] ?: if (!filterMissing) NullValue else null
|
||||
if (value != null || !filterMissing) {
|
||||
result.add(value?.toCsvValue() ?: "")
|
||||
}
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
}
|
||||
|
||||
@@ -1,45 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.snowflake.write.transform
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog
|
||||
import io.airbyte.integrations.destination.snowflake.spec.SnowflakeConfiguration
|
||||
import io.mockk.every
|
||||
import io.mockk.mockk
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
internal class SnowflakeColumnNameMapperTest {
|
||||
|
||||
@Test
|
||||
fun testGetMappedColumnName() {
|
||||
val columnName = "tést-column-name"
|
||||
val expectedName = "test-column-name"
|
||||
val stream = mockk<DestinationStream>()
|
||||
val tableCatalog = mockk<TableCatalog>()
|
||||
val snowflakeConfiguration = mockk<SnowflakeConfiguration>(relaxed = true)
|
||||
|
||||
// Configure the mock to return the expected mapped column name
|
||||
every { tableCatalog.getMappedColumnName(stream, columnName) } returns expectedName
|
||||
|
||||
val mapper = SnowflakeColumnNameMapper(tableCatalog, snowflakeConfiguration)
|
||||
val result = mapper.getMappedColumnName(stream = stream, columnName = columnName)
|
||||
assertEquals(expectedName, result)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetMappedColumnNameRawFormat() {
|
||||
val columnName = "tést-column-name"
|
||||
val stream = mockk<DestinationStream>()
|
||||
val tableCatalog = mockk<TableCatalog>()
|
||||
val snowflakeConfiguration =
|
||||
mockk<SnowflakeConfiguration> { every { legacyRawTablesOnly } returns true }
|
||||
|
||||
val mapper = SnowflakeColumnNameMapper(tableCatalog, snowflakeConfiguration)
|
||||
val result = mapper.getMappedColumnName(stream = stream, columnName = columnName)
|
||||
assertEquals(columnName, result)
|
||||
}
|
||||
}
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-100ms
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.0@sha256:9c0ea900af7e20d119da0446fc1679ca4015c5c5f7a3b022d321071570a78749
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 41bf83df-e5e4-44e3-b9c0-6933a26a4410
|
||||
dockerImageTag: 0.0.15
|
||||
dockerImageTag: 0.0.16
|
||||
dockerRepository: airbyte/source-100ms
|
||||
githubIssueLabel: source-100ms
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-adobe-commerce-magento
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.0@sha256:9c0ea900af7e20d119da0446fc1679ca4015c5c5f7a3b022d321071570a78749
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 2a97bcea-a59e-4fe1-89e0-1663f11ec646
|
||||
dockerImageTag: 0.0.5
|
||||
dockerImageTag: 0.0.6
|
||||
dockerRepository: airbyte/source-adobe-commerce-magento
|
||||
githubIssueLabel: source-adobe-commerce-magento
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-airbyte
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 284f6466-3004-4d83-a9b2-e4b36cbbbd41
|
||||
dockerImageTag: 0.1.13
|
||||
dockerImageTag: 0.1.14
|
||||
dockerRepository: airbyte/source-airbyte
|
||||
githubIssueLabel: source-airbyte
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-alpaca-broker-api
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 9065ae98-3c65-4283-b858-d22772b6fab5
|
||||
dockerImageTag: 0.0.17
|
||||
dockerImageTag: 0.0.18
|
||||
dockerRepository: airbyte/source-alpaca-broker-api
|
||||
githubIssueLabel: source-alpaca-broker-api
|
||||
icon: icon.svg
|
||||
|
||||
@@ -7,11 +7,11 @@ data:
|
||||
- amplitude.com
|
||||
- analytics.eu.amplitude.com
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.0@sha256:9c0ea900af7e20d119da0446fc1679ca4015c5c5f7a3b022d321071570a78749
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: fa9f58c6-2d03-4237-aaa4-07d75e0c1396
|
||||
dockerImageTag: 0.7.25
|
||||
dockerImageTag: 0.7.26
|
||||
dockerRepository: airbyte/source-amplitude
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/amplitude
|
||||
githubIssueLabel: source-amplitude
|
||||
|
||||
@@ -3,11 +3,11 @@ data:
|
||||
hosts:
|
||||
- api.apify.com
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 47f17145-fe20-4ef5-a548-e29b048adf84
|
||||
dockerImageTag: 2.2.34
|
||||
dockerImageTag: 2.2.35
|
||||
dockerRepository: airbyte/source-apify-dataset
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/apify-dataset
|
||||
githubIssueLabel: source-apify-dataset
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-appcues
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 3bab735a-e108-4c94-ab3f-414e3447b409
|
||||
dockerImageTag: 0.0.36
|
||||
dockerImageTag: 0.0.37
|
||||
dockerRepository: airbyte/source-appcues
|
||||
githubIssueLabel: source-appcues
|
||||
icon: icon.svg
|
||||
|
||||
@@ -14,7 +14,7 @@ data:
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: b4375641-e270-41d3-9c20-4f9cecad87a8
|
||||
dockerImageTag: 1.1.34
|
||||
dockerImageTag: 1.1.35
|
||||
dockerRepository: airbyte/source-appfollow
|
||||
githubIssueLabel: source-appfollow
|
||||
icon: appfollow.svg
|
||||
@@ -48,7 +48,7 @@ data:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
externalDocumentationUrls:
|
||||
- title: API documentation
|
||||
url: https://appfollow.docs.apiary.io/
|
||||
|
||||
@@ -2,7 +2,7 @@ data:
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: e59c8416-c2fa-4bd3-9e95-52677ea281c1
|
||||
dockerImageTag: 1.0.3
|
||||
dockerImageTag: 1.0.4
|
||||
dockerRepository: airbyte/source-apple-search-ads
|
||||
githubIssueLabel: source-apple-search-ads
|
||||
icon: icon.svg
|
||||
@@ -47,7 +47,7 @@ data:
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
supportLevel: community
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
externalDocumentationUrls:
|
||||
- title: API reference
|
||||
url: https://developer.apple.com/documentation/apple_search_ads
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-apptivo
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: eb41169b-b293-4690-b690-a1046cca0e3b
|
||||
dockerImageTag: 0.0.36
|
||||
dockerImageTag: 0.0.37
|
||||
dockerRepository: airbyte/source-apptivo
|
||||
githubIssueLabel: source-apptivo
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-assemblyai
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: dcb2f998-6493-4078-8390-e320487157a0
|
||||
dockerImageTag: 0.0.18
|
||||
dockerImageTag: 0.0.19
|
||||
dockerRepository: airbyte/source-assemblyai
|
||||
githubIssueLabel: source-assemblyai
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-aviationstack
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 874cbc8d-6b04-4da6-9f88-881da2b0febe
|
||||
dockerImageTag: 0.0.18
|
||||
dockerImageTag: 0.0.19
|
||||
dockerRepository: airbyte/source-aviationstack
|
||||
githubIssueLabel: source-aviationstack
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-awin-advertiser
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 807cabe0-9c76-4132-8863-60f669dea534
|
||||
dockerImageTag: 0.0.13
|
||||
dockerImageTag: 0.0.14
|
||||
dockerRepository: airbyte/source-awin-advertiser
|
||||
githubIssueLabel: source-awin-advertiser
|
||||
icon: icon.svg
|
||||
|
||||
@@ -12,7 +12,7 @@ data:
|
||||
connectorSubtype: file
|
||||
connectorType: source
|
||||
definitionId: fdaaba68-4875-4ed9-8fcd-4ae1e0a25093
|
||||
dockerImageTag: 0.8.7
|
||||
dockerImageTag: 0.8.8
|
||||
dockerRepository: airbyte/source-azure-blob-storage
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/azure-blob-storage
|
||||
externalDocumentationUrls:
|
||||
|
||||
@@ -2,13 +2,13 @@
|
||||
|
||||
[[package]]
|
||||
name = "airbyte-cdk"
|
||||
version = "7.5.1"
|
||||
version = "7.6.1"
|
||||
description = "A framework for writing Airbyte Connectors."
|
||||
optional = false
|
||||
python-versions = "<3.14,>=3.10"
|
||||
files = [
|
||||
{file = "airbyte_cdk-7.5.1-py3-none-any.whl", hash = "sha256:ab80a6ca0c50c24247a37476d03355fe421b55212fc57fd838412ba5f98695df"},
|
||||
{file = "airbyte_cdk-7.5.1.tar.gz", hash = "sha256:9690309d8573791f94d82de92fca66cebbc0429ab31266abe03463df53835c21"},
|
||||
{file = "airbyte_cdk-7.6.1-py3-none-any.whl", hash = "sha256:64292ec636b2be6751b59f2bc1fd2cc450f950472348a1de115df22d7a58d488"},
|
||||
{file = "airbyte_cdk-7.6.1.tar.gz", hash = "sha256:1ba27db6ebd27b03d9eccdfb67f20ba2c163765784ae794ca3ff2e5cbfd0d63e"},
|
||||
]
|
||||
|
||||
[package.dependencies]
|
||||
@@ -154,13 +154,13 @@ files = [
|
||||
|
||||
[[package]]
|
||||
name = "azure-core"
|
||||
version = "1.36.0"
|
||||
version = "1.37.0"
|
||||
description = "Microsoft Azure Core Library for Python"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "azure_core-1.36.0-py3-none-any.whl", hash = "sha256:fee9923a3a753e94a259563429f3644aaf05c486d45b1215d098115102d91d3b"},
|
||||
{file = "azure_core-1.36.0.tar.gz", hash = "sha256:22e5605e6d0bf1d229726af56d9e92bc37b6e726b141a18be0b4d424131741b7"},
|
||||
{file = "azure_core-1.37.0-py3-none-any.whl", hash = "sha256:b3abe2c59e7d6bb18b38c275a5029ff80f98990e7c90a5e646249a56630fcc19"},
|
||||
{file = "azure_core-1.37.0.tar.gz", hash = "sha256:7064f2c11e4b97f340e8e8c6d923b822978be3016e46b7bc4aa4b337cfb48aee"},
|
||||
]
|
||||
|
||||
[package.dependencies]
|
||||
@@ -248,13 +248,13 @@ files = [
|
||||
|
||||
[[package]]
|
||||
name = "cachetools"
|
||||
version = "6.2.2"
|
||||
version = "6.2.4"
|
||||
description = "Extensible memoizing collections and decorators"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "cachetools-6.2.2-py3-none-any.whl", hash = "sha256:6c09c98183bf58560c97b2abfcedcbaf6a896a490f534b031b661d3723b45ace"},
|
||||
{file = "cachetools-6.2.2.tar.gz", hash = "sha256:8e6d266b25e539df852251cfd6f990b4bc3a141db73b939058d809ebd2590fc6"},
|
||||
{file = "cachetools-6.2.4-py3-none-any.whl", hash = "sha256:69a7a52634fed8b8bf6e24a050fb60bff1c9bd8f6d24572b99c32d4e71e62a51"},
|
||||
{file = "cachetools-6.2.4.tar.gz", hash = "sha256:82c5c05585e70b6ba2d3ae09ea60b79548872185d2f24ae1f2709d37299fd607"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -977,13 +977,13 @@ grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
|
||||
|
||||
[[package]]
|
||||
name = "google-auth"
|
||||
version = "2.43.0"
|
||||
version = "2.45.0"
|
||||
description = "Google Authentication Library"
|
||||
optional = false
|
||||
python-versions = ">=3.7"
|
||||
files = [
|
||||
{file = "google_auth-2.43.0-py2.py3-none-any.whl", hash = "sha256:af628ba6fa493f75c7e9dbe9373d148ca9f4399b5ea29976519e0a3848eddd16"},
|
||||
{file = "google_auth-2.43.0.tar.gz", hash = "sha256:88228eee5fc21b62a1b5fe773ca15e67778cb07dc8363adcb4a8827b52d81483"},
|
||||
{file = "google_auth-2.45.0-py2.py3-none-any.whl", hash = "sha256:82344e86dc00410ef5382d99be677c6043d72e502b625aa4f4afa0bdacca0f36"},
|
||||
{file = "google_auth-2.45.0.tar.gz", hash = "sha256:90d3f41b6b72ea72dd9811e765699ee491ab24139f34ebf1ca2b9cc0c38708f3"},
|
||||
]
|
||||
|
||||
[package.dependencies]
|
||||
@@ -993,6 +993,7 @@ rsa = ">=3.1.4,<5"
|
||||
|
||||
[package.extras]
|
||||
aiohttp = ["aiohttp (>=3.6.2,<4.0.0)", "requests (>=2.20.0,<3.0.0)"]
|
||||
cryptography = ["cryptography (<39.0.0)", "cryptography (>=38.0.3)"]
|
||||
enterprise-cert = ["cryptography", "pyopenssl"]
|
||||
pyjwt = ["cryptography (<39.0.0)", "cryptography (>=38.0.3)", "pyjwt (>=2.0)"]
|
||||
pyopenssl = ["cryptography (<39.0.0)", "cryptography (>=38.0.3)", "pyopenssl (>=20.0.0)"]
|
||||
@@ -1207,13 +1208,13 @@ i18n = ["Babel (>=2.7)"]
|
||||
|
||||
[[package]]
|
||||
name = "joblib"
|
||||
version = "1.5.2"
|
||||
version = "1.5.3"
|
||||
description = "Lightweight pipelining with Python functions"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "joblib-1.5.2-py3-none-any.whl", hash = "sha256:4e1f0bdbb987e6d843c70cf43714cb276623def372df3c22fe5266b2670bc241"},
|
||||
{file = "joblib-1.5.2.tar.gz", hash = "sha256:3faa5c39054b2f03ca547da9b2f52fde67c06240c31853f306aea97f13647b55"},
|
||||
{file = "joblib-1.5.3-py3-none-any.whl", hash = "sha256:5fc3c5039fc5ca8c0276333a188bbd59d6b7ab37fe6632daa76bc7f9ec18e713"},
|
||||
{file = "joblib-1.5.3.tar.gz", hash = "sha256:8561a3269e6801106863fd0d6d84bb737be9e7631e33aaed3fb9ce5953688da3"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -1718,98 +1719,98 @@ files = [
|
||||
|
||||
[[package]]
|
||||
name = "orjson"
|
||||
version = "3.11.4"
|
||||
version = "3.11.5"
|
||||
description = "Fast, correct Python JSON library supporting dataclasses, datetimes, and numpy"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "orjson-3.11.4-cp310-cp310-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:e3aa2118a3ece0d25489cbe48498de8a5d580e42e8d9979f65bf47900a15aba1"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a69ab657a4e6733133a3dca82768f2f8b884043714e8d2b9ba9f52b6efef5c44"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3740bffd9816fc0326ddc406098a3a8f387e42223f5f455f2a02a9f834ead80c"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:65fd2f5730b1bf7f350c6dc896173d3460d235c4be007af73986d7cd9a2acd23"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9fdc3ae730541086158d549c97852e2eea6820665d4faf0f41bf99df41bc11ea"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e10b4d65901da88845516ce9f7f9736f9638d19a1d483b3883dc0182e6e5edba"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fb6a03a678085f64b97f9d4a9ae69376ce91a3a9e9b56a82b1580d8e1d501aff"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:2c82e4f0b1c712477317434761fbc28b044c838b6b1240d895607441412371ac"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:d58c166a18f44cc9e2bad03a327dc2d1a3d2e85b847133cfbafd6bfc6719bd79"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:94f206766bf1ea30e1382e4890f763bd1eefddc580e08fec1ccdc20ddd95c827"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:41bf25fb39a34cf8edb4398818523277ee7096689db352036a9e8437f2f3ee6b"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-win32.whl", hash = "sha256:fa9627eba4e82f99ca6d29bc967f09aba446ee2b5a1ea728949ede73d313f5d3"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-win_amd64.whl", hash = "sha256:23ef7abc7fca96632d8174ac115e668c1e931b8fe4dde586e92a500bf1914dcc"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:5e59d23cd93ada23ec59a96f215139753fbfe3a4d989549bcb390f8c00370b39"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-macosx_15_0_arm64.whl", hash = "sha256:5c3aedecfc1beb988c27c79d52ebefab93b6c3921dbec361167e6559aba2d36d"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da9e5301f1c2caa2a9a4a303480d79c9ad73560b2e7761de742ab39fe59d9175"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8873812c164a90a79f65368f8f96817e59e35d0cc02786a5356f0e2abed78040"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5d7feb0741ebb15204e748f26c9638e6665a5fa93c37a2c73d64f1669b0ddc63"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:01ee5487fefee21e6910da4c2ee9eef005bee568a0879834df86f888d2ffbdd9"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3d40d46f348c0321df01507f92b95a377240c4ec31985225a6668f10e2676f9a"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:95713e5fc8af84d8edc75b785d2386f653b63d62b16d681687746734b4dfc0be"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:ad73ede24f9083614d6c4ca9a85fe70e33be7bf047ec586ee2363bc7418fe4d7"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:842289889de515421f3f224ef9c1f1efb199a32d76d8d2ca2706fa8afe749549"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:3b2427ed5791619851c52a1261b45c233930977e7de8cf36de05636c708fa905"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:3c36e524af1d29982e9b190573677ea02781456b2e537d5840e4538a5ec41907"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-win32.whl", hash = "sha256:87255b88756eab4a68ec61837ca754e5d10fa8bc47dc57f75cedfeaec358d54c"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-win_amd64.whl", hash = "sha256:e2d5d5d798aba9a0e1fede8d853fa899ce2cb930ec0857365f700dffc2c7af6a"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-win_arm64.whl", hash = "sha256:6bb6bb41b14c95d4f2702bce9975fda4516f1db48e500102fc4d8119032ff045"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:d4371de39319d05d3f482f372720b841c841b52f5385bd99c61ed69d55d9ab50"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-macosx_15_0_arm64.whl", hash = "sha256:e41fd3b3cac850eaae78232f37325ed7d7436e11c471246b87b2cd294ec94853"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:600e0e9ca042878c7fdf189cf1b028fe2c1418cc9195f6cb9824eb6ed99cb938"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7bbf9b333f1568ef5da42bc96e18bf30fd7f8d54e9ae066d711056add508e415"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4806363144bb6e7297b8e95870e78d30a649fdc4e23fc84daa80c8ebd366ce44"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad355e8308493f527d41154e9053b86a5be892b3b359a5c6d5d95cda23601cb2"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c8a7517482667fb9f0ff1b2f16fe5829296ed7a655d04d68cd9711a4d8a4e708"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:97eb5942c7395a171cbfecc4ef6701fc3c403e762194683772df4c54cfbb2210"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:149d95d5e018bdd822e3f38c103b1a7c91f88d38a88aada5c4e9b3a73a244241"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:624f3951181eb46fc47dea3d221554e98784c823e7069edb5dbd0dc826ac909b"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:03bfa548cf35e3f8b3a96c4e8e41f753c686ff3d8e182ce275b1751deddab58c"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:525021896afef44a68148f6ed8a8bf8375553d6066c7f48537657f64823565b9"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-win32.whl", hash = "sha256:b58430396687ce0f7d9eeb3dd47761ca7d8fda8e9eb92b3077a7a353a75efefa"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-win_amd64.whl", hash = "sha256:c6dbf422894e1e3c80a177133c0dda260f81428f9de16d61041949f6a2e5c140"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-win_arm64.whl", hash = "sha256:d38d2bc06d6415852224fcc9c0bfa834c25431e466dc319f0edd56cca81aa96e"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:2d6737d0e616a6e053c8b4acc9eccea6b6cce078533666f32d140e4f85002534"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-macosx_15_0_arm64.whl", hash = "sha256:afb14052690aa328cc118a8e09f07c651d301a72e44920b887c519b313d892ff"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:38aa9e65c591febb1b0aed8da4d469eba239d434c218562df179885c94e1a3ad"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:f2cf4dfaf9163b0728d061bebc1e08631875c51cd30bf47cb9e3293bfbd7dcd5"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:89216ff3dfdde0e4070932e126320a1752c9d9a758d6a32ec54b3b9334991a6a"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9daa26ca8e97fae0ce8aa5d80606ef8f7914e9b129b6b5df9104266f764ce436"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5c8b2769dc31883c44a9cd126560327767f848eb95f99c36c9932f51090bfce9"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1469d254b9884f984026bd9b0fa5bbab477a4bfe558bba6848086f6d43eb5e73"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:68e44722541983614e37117209a194e8c3ad07838ccb3127d96863c95ec7f1e0"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:8e7805fda9672c12be2f22ae124dcd7b03928d6c197544fe12174b86553f3196"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:04b69c14615fb4434ab867bf6f38b2d649f6f300af30a6705397e895f7aec67a"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:639c3735b8ae7f970066930e58cf0ed39a852d417c24acd4a25fc0b3da3c39a6"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-win32.whl", hash = "sha256:6c13879c0d2964335491463302a6ca5ad98105fc5db3565499dcb80b1b4bd839"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-win_amd64.whl", hash = "sha256:09bf242a4af98732db9f9a1ec57ca2604848e16f132e3f72edfd3c5c96de009a"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-win_arm64.whl", hash = "sha256:a85f0adf63319d6c1ba06fb0dbf997fced64a01179cf17939a6caca662bf92de"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:42d43a1f552be1a112af0b21c10a5f553983c2a0938d2bbb8ecd8bc9fb572803"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-macosx_15_0_arm64.whl", hash = "sha256:26a20f3fbc6c7ff2cb8e89c4c5897762c9d88cf37330c6a117312365d6781d54"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6e3f20be9048941c7ffa8fc523ccbd17f82e24df1549d1d1fe9317712d19938e"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:aac364c758dc87a52e68e349924d7e4ded348dedff553889e4d9f22f74785316"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d5c54a6d76e3d741dcc3f2707f8eeb9ba2a791d3adbf18f900219b62942803b1"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f28485bdca8617b79d44627f5fb04336897041dfd9fa66d383a49d09d86798bc"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bfc2a484cad3585e4ba61985a6062a4c2ed5c7925db6d39f1fa267c9d166487f"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e34dbd508cb91c54f9c9788923daca129fe5b55c5b4eebe713bf5ed3791280cf"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:b13c478fa413d4b4ee606ec8e11c3b2e52683a640b006bb586b3041c2ca5f606"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:724ca721ecc8a831b319dcd72cfa370cc380db0bf94537f08f7edd0a7d4e1780"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:977c393f2e44845ce1b540e19a786e9643221b3323dae190668a98672d43fb23"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:1e539e382cf46edec157ad66b0b0872a90d829a6b71f17cb633d6c160a223155"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-win32.whl", hash = "sha256:d63076d625babab9db5e7836118bdfa086e60f37d8a174194ae720161eb12394"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-win_amd64.whl", hash = "sha256:0a54d6635fa3aaa438ae32e8570b9f0de36f3f6562c308d2a2a452e8b0592db1"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-win_arm64.whl", hash = "sha256:78b999999039db3cf58f6d230f524f04f75f129ba3d1ca2ed121f8657e575d3d"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:405261b0a8c62bcbd8e2931c26fdc08714faf7025f45531541e2b29e544b545b"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:af02ff34059ee9199a3546f123a6ab4c86caf1708c79042caf0820dc290a6d4f"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:0b2eba969ea4203c177c7b38b36c69519e6067ee68c34dc37081fac74c796e10"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0baa0ea43cfa5b008a28d3c07705cf3ada40e5d347f0f44994a64b1b7b4b5350"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80fd082f5dcc0e94657c144f1b2a3a6479c44ad50be216cf0c244e567f5eae19"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1e3704d35e47d5bee811fb1cbd8599f0b4009b14d451c4c57be5a7e25eb89a13"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:caa447f2b5356779d914658519c874cf3b7629e99e63391ed519c28c8aea4919"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:bba5118143373a86f91dadb8df41d9457498226698ebdf8e11cbb54d5b0e802d"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:622463ab81d19ef3e06868b576551587de8e4d518892d1afab71e0fbc1f9cffc"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:3e0a700c4b82144b72946b6629968df9762552ee1344bfdb767fecdd634fbd5a"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:6e18a5c15e764e5f3fc569b47872450b4bcea24f2a6354c0a0e95ad21045d5a9"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-win32.whl", hash = "sha256:fb1c37c71cad991ef4d89c7a634b5ffb4447dbd7ae3ae13e8f5ee7f1775e7ab1"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-win_amd64.whl", hash = "sha256:e2985ce8b8c42d00492d0ed79f2bd2b6460d00f2fa671dfde4bf2e02f49bf5c6"},
|
||||
{file = "orjson-3.11.4.tar.gz", hash = "sha256:39485f4ab4c9b30a3943cfe99e1a213c4776fb69e8abd68f66b83d5a0b0fdc6d"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:df9eadb2a6386d5ea2bfd81309c505e125cfc9ba2b1b99a97e60985b0b3665d1"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ccc70da619744467d8f1f49a8cadae5ec7bbe054e5232d95f92ed8737f8c5870"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:073aab025294c2f6fc0807201c76fdaed86f8fc4be52c440fb78fbb759a1ac09"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:835f26fa24ba0bb8c53ae2a9328d1706135b74ec653ed933869b74b6909e63fd"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:667c132f1f3651c14522a119e4dd631fad98761fa960c55e8e7430bb2a1ba4ac"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:42e8961196af655bb5e63ce6c60d25e8798cd4dfbc04f4203457fa3869322c2e"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75412ca06e20904c19170f8a24486c4e6c7887dea591ba18a1ab572f1300ee9f"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6af8680328c69e15324b5af3ae38abbfcf9cbec37b5346ebfd52339c3d7e8a18"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:a86fe4ff4ea523eac8f4b57fdac319faf037d3c1be12405e6a7e86b3fbc4756a"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e607b49b1a106ee2086633167033afbd63f76f2999e9236f638b06b112b24ea7"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:7339f41c244d0eea251637727f016b3d20050636695bc78345cce9029b189401"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-win32.whl", hash = "sha256:8be318da8413cdbbce77b8c5fac8d13f6eb0f0db41b30bb598631412619572e8"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-win_amd64.whl", hash = "sha256:b9f86d69ae822cabc2a0f6c099b43e8733dda788405cba2665595b7e8dd8d167"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:9c8494625ad60a923af6b2b0bd74107146efe9b55099e20d7740d995f338fcd8"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-macosx_15_0_arm64.whl", hash = "sha256:7bb2ce0b82bc9fd1168a513ddae7a857994b780b2945a8c51db4ab1c4b751ebc"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:67394d3becd50b954c4ecd24ac90b5051ee7c903d167459f93e77fc6f5b4c968"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:298d2451f375e5f17b897794bcc3e7b821c0f32b4788b9bcae47ada24d7f3cf7"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:aa5e4244063db8e1d87e0f54c3f7522f14b2dc937e65d5241ef0076a096409fd"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1db2088b490761976c1b2e956d5d4e6409f3732e9d79cfa69f876c5248d1baf9"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c2ed66358f32c24e10ceea518e16eb3549e34f33a9d51f99ce23b0251776a1ef"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c2021afda46c1ed64d74b555065dbd4c2558d510d8cec5ea6a53001b3e5e82a9"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:b42ffbed9128e547a1647a3e50bc88ab28ae9daa61713962e0d3dd35e820c125"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:8d5f16195bb671a5dd3d1dbea758918bada8f6cc27de72bd64adfbd748770814"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:c0e5d9f7a0227df2927d343a6e3859bebf9208b427c79bd31949abcc2fa32fa5"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:23d04c4543e78f724c4dfe656b3791b5f98e4c9253e13b2636f1af5d90e4a880"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-win32.whl", hash = "sha256:c404603df4865f8e0afe981aa3c4b62b406e6d06049564d58934860b62b7f91d"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-win_amd64.whl", hash = "sha256:9645ef655735a74da4990c24ffbd6894828fbfa117bc97c1edd98c282ecb52e1"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-win_arm64.whl", hash = "sha256:1cbf2735722623fcdee8e712cbaaab9e372bbcb0c7924ad711b261c2eccf4a5c"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:334e5b4bff9ad101237c2d799d9fd45737752929753bf4faf4b207335a416b7d"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-macosx_15_0_arm64.whl", hash = "sha256:ff770589960a86eae279f5d8aa536196ebda8273a2a07db2a54e82b93bc86626"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ed24250e55efbcb0b35bed7caaec8cedf858ab2f9f2201f17b8938c618c8ca6f"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:a66d7769e98a08a12a139049aac2f0ca3adae989817f8c43337455fbc7669b85"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:86cfc555bfd5794d24c6a1903e558b50644e5e68e6471d66502ce5cb5fdef3f9"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a230065027bc2a025e944f9d4714976a81e7ecfa940923283bca7bbc1f10f626"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b29d36b60e606df01959c4b982729c8845c69d1963f88686608be9ced96dbfaa"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c74099c6b230d4261fdc3169d50efc09abf38ace1a42ea2f9994b1d79153d477"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:e697d06ad57dd0c7a737771d470eedc18e68dfdefcdd3b7de7f33dfda5b6212e"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:e08ca8a6c851e95aaecc32bc44a5aa75d0ad26af8cdac7c77e4ed93acf3d5b69"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:e8b5f96c05fce7d0218df3fdfeb962d6b8cfff7e3e20264306b46dd8b217c0f3"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ddbfdb5099b3e6ba6d6ea818f61997bb66de14b411357d24c4612cf1ebad08ca"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-win32.whl", hash = "sha256:9172578c4eb09dbfcf1657d43198de59b6cef4054de385365060ed50c458ac98"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-win_amd64.whl", hash = "sha256:2b91126e7b470ff2e75746f6f6ee32b9ab67b7a93c8ba1d15d3a0caaf16ec875"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-win_arm64.whl", hash = "sha256:acbc5fac7e06777555b0722b8ad5f574739e99ffe99467ed63da98f97f9ca0fe"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:3b01799262081a4c47c035dd77c1301d40f568f77cc7ec1bb7db5d63b0a01629"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-macosx_15_0_arm64.whl", hash = "sha256:61de247948108484779f57a9f406e4c84d636fa5a59e411e6352484985e8a7c3"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:894aea2e63d4f24a7f04a1908307c738d0dce992e9249e744b8f4e8dd9197f39"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ddc21521598dbe369d83d4d40338e23d4101dad21dae0e79fa20465dbace019f"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7cce16ae2f5fb2c53c3eafdd1706cb7b6530a67cc1c17abe8ec747f5cd7c0c51"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e46c762d9f0e1cfb4ccc8515de7f349abbc95b59cb5a2bd68df5973fdef913f8"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d7345c759276b798ccd6d77a87136029e71e66a8bbf2d2755cbdde1d82e78706"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75bc2e59e6a2ac1dd28901d07115abdebc4563b5b07dd612bf64260a201b1c7f"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:54aae9b654554c3b4edd61896b978568c6daa16af96fa4681c9b5babd469f863"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:4bdd8d164a871c4ec773f9de0f6fe8769c2d6727879c37a9666ba4183b7f8228"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:a261fef929bcf98a60713bf5e95ad067cea16ae345d9a35034e73c3990e927d2"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:c028a394c766693c5c9909dec76b24f37e6a1b91999e8d0c0d5feecbe93c3e05"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-win32.whl", hash = "sha256:2cc79aaad1dfabe1bd2d50ee09814a1253164b3da4c00a78c458d82d04b3bdef"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-win_amd64.whl", hash = "sha256:ff7877d376add4e16b274e35a3f58b7f37b362abf4aa31863dadacdd20e3a583"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-win_arm64.whl", hash = "sha256:59ac72ea775c88b163ba8d21b0177628bd015c5dd060647bbab6e22da3aad287"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:e446a8ea0a4c366ceafc7d97067bfd55292969143b57e3c846d87fc701e797a0"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-macosx_15_0_arm64.whl", hash = "sha256:53deb5addae9c22bbe3739298f5f2196afa881ea75944e7720681c7080909a81"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:82cd00d49d6063d2b8791da5d4f9d20539c5951f965e45ccf4e96d33505ce68f"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3fd15f9fc8c203aeceff4fda211157fad114dde66e92e24097b3647a08f4ee9e"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9df95000fbe6777bf9820ae82ab7578e8662051bb5f83d71a28992f539d2cda7"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:92a8d676748fca47ade5bc3da7430ed7767afe51b2f8100e3cd65e151c0eaceb"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:aa0f513be38b40234c77975e68805506cad5d57b3dfd8fe3baa7f4f4051e15b4"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa1863e75b92891f553b7922ce4ee10ed06db061e104f2b7815de80cdcb135ad"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:d4be86b58e9ea262617b8ca6251a2f0d63cc132a6da4b5fcc8e0a4128782c829"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:b923c1c13fa02084eb38c9c065afd860a5cff58026813319a06949c3af5732ac"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:1b6bd351202b2cd987f35a13b5e16471cf4d952b42a73c391cc537974c43ef6d"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:bb150d529637d541e6af06bbe3d02f5498d628b7f98267ff87647584293ab439"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-win32.whl", hash = "sha256:9cc1e55c884921434a84a0c3dd2699eb9f92e7b441d7f53f3941079ec6ce7499"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-win_amd64.whl", hash = "sha256:a4f3cb2d874e03bc7767c8f88adaa1a9a05cecea3712649c3b58589ec7317310"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-win_arm64.whl", hash = "sha256:38b22f476c351f9a1c43e5b07d8b5a02eb24a6ab8e75f700f7d479d4568346a5"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:1b280e2d2d284a6713b0cfec7b08918ebe57df23e3f76b27586197afca3cb1e9"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3c8d8a112b274fae8c5f0f01954cb0480137072c271f3f4958127b010dfefaec"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:5f0a2ae6f09ac7bd47d2d5a5305c1d9ed08ac057cda55bb0a49fa506f0d2da00"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c0d87bd1896faac0d10b4f849016db81a63e4ec5df38757ffae84d45ab38aa71"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:801a821e8e6099b8c459ac7540b3c32dba6013437c57fdcaec205b169754f38c"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:69a0f6ac618c98c74b7fbc8c0172ba86f9e01dbf9f62aa0b1776c2231a7bffe5"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fea7339bdd22e6f1060c55ac31b6a755d86a5b2ad3657f2669ec243f8e3b2bdb"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:4dad582bc93cef8f26513e12771e76385a7e6187fd713157e971c784112aad56"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:0522003e9f7fba91982e83a97fec0708f5a714c96c4209db7104e6b9d132f111"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:7403851e430a478440ecc1258bcbacbfbd8175f9ac1e39031a7121dd0de05ff8"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:5f691263425d3177977c8d1dd896cde7b98d93cbf390b2544a090675e83a6a0a"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-win32.whl", hash = "sha256:61026196a1c4b968e1b1e540563e277843082e9e97d78afa03eb89315af531f1"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-win_amd64.whl", hash = "sha256:09b94b947ac08586af635ef922d69dc9bc63321527a3a04647f4986a73f4bd30"},
|
||||
{file = "orjson-3.11.5.tar.gz", hash = "sha256:82393ab47b4fe44ffd0a7659fa9cfaacc717eb617c93cde83795f14af5c2e9d5"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -2052,13 +2053,13 @@ xmp = ["defusedxml"]
|
||||
|
||||
[[package]]
|
||||
name = "platformdirs"
|
||||
version = "4.5.0"
|
||||
version = "4.5.1"
|
||||
description = "A small Python package for determining appropriate platform-specific dirs, e.g. a `user data dir`."
|
||||
optional = false
|
||||
python-versions = ">=3.10"
|
||||
files = [
|
||||
{file = "platformdirs-4.5.0-py3-none-any.whl", hash = "sha256:e578a81bb873cbb89a41fcc904c7ef523cc18284b7e3b3ccf06aca1403b7ebd3"},
|
||||
{file = "platformdirs-4.5.0.tar.gz", hash = "sha256:70ddccdd7c99fc5942e9fc25636a8b34d04c24b335100223152c2803e4063312"},
|
||||
{file = "platformdirs-4.5.1-py3-none-any.whl", hash = "sha256:d03afa3963c806a9bed9d5125c8f4cb2fdaf74a55ab60e5d59b3fde758104d31"},
|
||||
{file = "platformdirs-4.5.1.tar.gz", hash = "sha256:61d5cdcc6065745cdd94f0f878977f8de9437be93de97c1c12f853c9c0cdcbda"},
|
||||
]
|
||||
|
||||
[package.extras]
|
||||
@@ -2100,21 +2101,21 @@ testing = ["google-api-core (>=1.31.5)"]
|
||||
|
||||
[[package]]
|
||||
name = "protobuf"
|
||||
version = "6.33.1"
|
||||
version = "6.33.2"
|
||||
description = ""
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "protobuf-6.33.1-cp310-abi3-win32.whl", hash = "sha256:f8d3fdbc966aaab1d05046d0240dd94d40f2a8c62856d41eaa141ff64a79de6b"},
|
||||
{file = "protobuf-6.33.1-cp310-abi3-win_amd64.whl", hash = "sha256:923aa6d27a92bf44394f6abf7ea0500f38769d4b07f4be41cb52bd8b1123b9ed"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:fe34575f2bdde76ac429ec7b570235bf0c788883e70aee90068e9981806f2490"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:f8adba2e44cde2d7618996b3fc02341f03f5bc3f2748be72dc7b063319276178"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-manylinux2014_s390x.whl", hash = "sha256:0f4cf01222c0d959c2b399142deb526de420be8236f22c71356e2a544e153c53"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:8fd7d5e0eb08cd5b87fd3df49bc193f5cfd778701f47e11d127d0afc6c39f1d1"},
|
||||
{file = "protobuf-6.33.1-cp39-cp39-win32.whl", hash = "sha256:023af8449482fa884d88b4563d85e83accab54138ae098924a985bcbb734a213"},
|
||||
{file = "protobuf-6.33.1-cp39-cp39-win_amd64.whl", hash = "sha256:df051de4fd7e5e4371334e234c62ba43763f15ab605579e04c7008c05735cd82"},
|
||||
{file = "protobuf-6.33.1-py3-none-any.whl", hash = "sha256:d595a9fd694fdeb061a62fbe10eb039cc1e444df81ec9bb70c7fc59ebcb1eafa"},
|
||||
{file = "protobuf-6.33.1.tar.gz", hash = "sha256:97f65757e8d09870de6fd973aeddb92f85435607235d20b2dfed93405d00c85b"},
|
||||
{file = "protobuf-6.33.2-cp310-abi3-win32.whl", hash = "sha256:87eb388bd2d0f78febd8f4c8779c79247b26a5befad525008e49a6955787ff3d"},
|
||||
{file = "protobuf-6.33.2-cp310-abi3-win_amd64.whl", hash = "sha256:fc2a0e8b05b180e5fc0dd1559fe8ebdae21a27e81ac77728fb6c42b12c7419b4"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:d9b19771ca75935b3a4422957bc518b0cecb978b31d1dd12037b088f6bcc0e43"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:b5d3b5625192214066d99b2b605f5783483575656784de223f00a8d00754fc0e"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-manylinux2014_s390x.whl", hash = "sha256:8cd7640aee0b7828b6d03ae518b5b4806fdfc1afe8de82f79c3454f8aef29872"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:1f8017c48c07ec5859106533b682260ba3d7c5567b1ca1f24297ce03384d1b4f"},
|
||||
{file = "protobuf-6.33.2-cp39-cp39-win32.whl", hash = "sha256:7109dcc38a680d033ffb8bf896727423528db9163be1b6a02d6a49606dcadbfe"},
|
||||
{file = "protobuf-6.33.2-cp39-cp39-win_amd64.whl", hash = "sha256:2981c58f582f44b6b13173e12bb8656711189c2a70250845f264b877f00b1913"},
|
||||
{file = "protobuf-6.33.2-py3-none-any.whl", hash = "sha256:7636aad9bb01768870266de5dc009de2d1b936771b38a793f73cbbf279c91c5c"},
|
||||
{file = "protobuf-6.33.2.tar.gz", hash = "sha256:56dc370c91fbb8ac85bc13582c9e373569668a290aa2e66a590c2a0d35ddb9e4"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -3466,13 +3467,13 @@ typing-extensions = ">=4.12.0"
|
||||
|
||||
[[package]]
|
||||
name = "tzdata"
|
||||
version = "2025.2"
|
||||
version = "2025.3"
|
||||
description = "Provider of IANA time zone data"
|
||||
optional = false
|
||||
python-versions = ">=2"
|
||||
files = [
|
||||
{file = "tzdata-2025.2-py2.py3-none-any.whl", hash = "sha256:1a403fada01ff9221ca8044d701868fa132215d84beb92242d9acd2147f667a8"},
|
||||
{file = "tzdata-2025.2.tar.gz", hash = "sha256:b60a638fcc0daffadf82fe0f57e53d06bdec2f36c4df66280ae79bce6bd6f2b9"},
|
||||
{file = "tzdata-2025.3-py2.py3-none-any.whl", hash = "sha256:06a47e5700f3081aab02b2e513160914ff0694bce9947d6b76ebd6bf57cfc5d1"},
|
||||
{file = "tzdata-2025.3.tar.gz", hash = "sha256:de39c2ca5dc7b0344f2eba86f49d614019d29f060fc4ebc8a417896a620b56a7"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -3617,20 +3618,20 @@ dev = ["mypy", "pre-commit", "pytest", "pytest-cov", "pytest-socket", "ruff"]
|
||||
|
||||
[[package]]
|
||||
name = "urllib3"
|
||||
version = "2.5.0"
|
||||
version = "2.6.2"
|
||||
description = "HTTP library with thread-safe connection pooling, file post, and more."
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "urllib3-2.5.0-py3-none-any.whl", hash = "sha256:e6b01673c0fa6a13e374b50871808eb3bf7046c4b125b216f6bf1cc604cff0dc"},
|
||||
{file = "urllib3-2.5.0.tar.gz", hash = "sha256:3fc47733c7e419d4bc3f6b3dc2b4f890bb743906a30d56ba4a5bfa4bbff92760"},
|
||||
{file = "urllib3-2.6.2-py3-none-any.whl", hash = "sha256:ec21cddfe7724fc7cb4ba4bea7aa8e2ef36f607a4bab81aa6ce42a13dc3f03dd"},
|
||||
{file = "urllib3-2.6.2.tar.gz", hash = "sha256:016f9c98bb7e98085cb2b4b17b87d2c702975664e4f060c6532e64d1c1a5e797"},
|
||||
]
|
||||
|
||||
[package.extras]
|
||||
brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)"]
|
||||
brotli = ["brotli (>=1.2.0)", "brotlicffi (>=1.2.0.0)"]
|
||||
h2 = ["h2 (>=4,<5)"]
|
||||
socks = ["pysocks (>=1.5.6,!=1.5.7,<2.0)"]
|
||||
zstd = ["zstandard (>=0.18.0)"]
|
||||
zstd = ["backports-zstd (>=1.0.0)"]
|
||||
|
||||
[[package]]
|
||||
name = "wcmatch"
|
||||
|
||||
@@ -3,7 +3,7 @@ requires = [ "poetry-core>=1.0.0",]
|
||||
build-backend = "poetry.core.masonry.api"
|
||||
|
||||
[tool.poetry]
|
||||
version = "0.8.7"
|
||||
version = "0.8.8"
|
||||
name = "source-azure-blob-storage"
|
||||
description = "Source implementation for Azure Blob Storage."
|
||||
authors = [ "Airbyte <contact@airbyte.io>",]
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-basecamp
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 0abc340d-772c-4fa1-8d1d-ebf3be72ab51
|
||||
dockerImageTag: 0.0.27
|
||||
dockerImageTag: 0.0.28
|
||||
dockerRepository: airbyte/source-basecamp
|
||||
githubIssueLabel: source-basecamp
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-beamer
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: b928158d-4d2a-4ea6-a9c6-efa90f5c1e5d
|
||||
dockerImageTag: 0.0.35
|
||||
dockerImageTag: 0.0.36
|
||||
dockerRepository: airbyte/source-beamer
|
||||
githubIssueLabel: source-beamer
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-bigmailer
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: c448df04-e5ce-43e0-85cb-a4cfff936c3d
|
||||
dockerImageTag: 0.0.37
|
||||
dockerImageTag: 0.0.38
|
||||
dockerRepository: airbyte/source-bigmailer
|
||||
githubIssueLabel: source-bigmailer
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-bitly
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 3631f862-646b-4abf-abde-dc37acf3847c
|
||||
dockerImageTag: 0.0.34
|
||||
dockerImageTag: 0.0.35
|
||||
dockerRepository: airbyte/source-bitly
|
||||
githubIssueLabel: source-bitly
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-blogger
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: c6c4d8ae-60e9-49b4-9b48-e3a5857455fe
|
||||
dockerImageTag: 0.0.34
|
||||
dockerImageTag: 0.0.35
|
||||
dockerRepository: airbyte/source-blogger
|
||||
githubIssueLabel: source-blogger
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-bluetally
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: d4154c0b-7737-42c3-92dc-7a8e046e6456
|
||||
dockerImageTag: 0.0.17
|
||||
dockerImageTag: 0.0.18
|
||||
dockerRepository: airbyte/source-bluetally
|
||||
githubIssueLabel: source-bluetally
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-boldsign
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: b013676a-2286-4f7a-bca4-811477207761
|
||||
dockerImageTag: 0.0.19
|
||||
dockerImageTag: 0.0.20
|
||||
dockerRepository: airbyte/source-boldsign
|
||||
githubIssueLabel: source-boldsign
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-breezy-hr
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: bc2c2e4f-41a1-40e3-9e82-eea19cf958ff
|
||||
dockerImageTag: 0.0.35
|
||||
dockerImageTag: 0.0.36
|
||||
dockerRepository: airbyte/source-breezy-hr
|
||||
githubIssueLabel: source-breezy-hr
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-brevo
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: e2276f19-1c19-4d4e-ae6c-7df3c9c4ad49
|
||||
dockerImageTag: 0.2.22
|
||||
dockerImageTag: 0.2.23
|
||||
dockerRepository: airbyte/source-brevo
|
||||
githubIssueLabel: source-brevo
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-brex
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 7d605ca7-f2b8-41c4-96a7-3d5637e58d6d
|
||||
dockerImageTag: 0.0.32
|
||||
dockerImageTag: 0.0.33
|
||||
dockerRepository: airbyte/source-brex
|
||||
githubIssueLabel: source-brex
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-bugsnag
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: fa6c7629-0556-4b29-9f2c-7f6f4e54e997
|
||||
dockerImageTag: 0.0.35
|
||||
dockerImageTag: 0.0.36
|
||||
dockerRepository: airbyte/source-bugsnag
|
||||
githubIssueLabel: source-bugsnag
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-buildkite
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 9b0d6e72-5c85-40a3-af88-3a8f7c65746f
|
||||
dockerImageTag: 0.0.37
|
||||
dockerImageTag: 0.0.38
|
||||
dockerRepository: airbyte/source-buildkite
|
||||
githubIssueLabel: source-buildkite
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-bunny-inc
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 6aa362e3-6107-4fdc-9209-130a60e8725c
|
||||
dockerImageTag: 0.0.33
|
||||
dockerImageTag: 0.0.34
|
||||
dockerRepository: airbyte/source-bunny-inc
|
||||
githubIssueLabel: source-bunny-inc
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-buzzsprout
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 6ad23bfc-cb11-4faa-a243-f9ccdb0145cc
|
||||
dockerImageTag: 0.0.38
|
||||
dockerImageTag: 0.0.39
|
||||
dockerRepository: airbyte/source-buzzsprout
|
||||
githubIssueLabel: source-buzzsprout
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-cal-com
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 3db8a652-88f7-41ee-91a3-2f745322d9ae
|
||||
dockerImageTag: 0.0.40
|
||||
dockerImageTag: 0.0.41
|
||||
dockerRepository: airbyte/source-cal-com
|
||||
githubIssueLabel: source-cal-com
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-calendly
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: b8f2cbee-b073-4dd8-9b80-97d7bae967a4
|
||||
dockerImageTag: 0.1.28
|
||||
dockerImageTag: 0.1.29
|
||||
dockerRepository: airbyte/source-calendly
|
||||
githubIssueLabel: source-calendly
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-campaign-monitor
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 9d350ec7-2860-4106-a331-7d9403dd9a02
|
||||
dockerImageTag: 0.0.35
|
||||
dockerImageTag: 0.0.36
|
||||
dockerRepository: airbyte/source-campaign-monitor
|
||||
githubIssueLabel: source-campaign-monitor
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-campayn
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 5674ae5a-511e-4093-9da9-086bf5f0b568
|
||||
dockerImageTag: 0.0.36
|
||||
dockerImageTag: 0.0.37
|
||||
dockerRepository: airbyte/source-campayn
|
||||
githubIssueLabel: source-campayn
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-capsule-crm
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 4f1b8b9c-fa2c-4fc2-b041-e3bde27a0cb1
|
||||
dockerImageTag: 0.0.33
|
||||
dockerImageTag: 0.0.34
|
||||
dockerRepository: airbyte/source-capsule-crm
|
||||
githubIssueLabel: source-capsule-crm
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-care-quality-commission
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 2366b7bf-b83e-471c-b4a0-1405887fdf6e
|
||||
dockerImageTag: 0.0.40
|
||||
dockerImageTag: 0.0.41
|
||||
dockerRepository: airbyte/source-care-quality-commission
|
||||
githubIssueLabel: source-care-quality-commission
|
||||
icon: icon.svg
|
||||
|
||||
@@ -7,7 +7,7 @@ data:
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: bb1a6d31-6879-4819-a2bd-3eed299ea8e2
|
||||
dockerImageTag: 0.3.38
|
||||
dockerImageTag: 0.3.39
|
||||
dockerRepository: airbyte/source-cart
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/cart
|
||||
githubIssueLabel: source-cart
|
||||
|
||||
245
airbyte-integrations/connectors/source-cart/poetry.lock
generated
245
airbyte-integrations/connectors/source-cart/poetry.lock
generated
@@ -2,13 +2,13 @@
|
||||
|
||||
[[package]]
|
||||
name = "airbyte-cdk"
|
||||
version = "7.5.1"
|
||||
version = "7.6.1"
|
||||
description = "A framework for writing Airbyte Connectors."
|
||||
optional = false
|
||||
python-versions = "<3.14,>=3.10"
|
||||
files = [
|
||||
{file = "airbyte_cdk-7.5.1-py3-none-any.whl", hash = "sha256:ab80a6ca0c50c24247a37476d03355fe421b55212fc57fd838412ba5f98695df"},
|
||||
{file = "airbyte_cdk-7.5.1.tar.gz", hash = "sha256:9690309d8573791f94d82de92fca66cebbc0429ab31266abe03463df53835c21"},
|
||||
{file = "airbyte_cdk-7.6.1-py3-none-any.whl", hash = "sha256:64292ec636b2be6751b59f2bc1fd2cc450f950472348a1de115df22d7a58d488"},
|
||||
{file = "airbyte_cdk-7.6.1.tar.gz", hash = "sha256:1ba27db6ebd27b03d9eccdfb67f20ba2c163765784ae794ca3ff2e5cbfd0d63e"},
|
||||
]
|
||||
|
||||
[package.dependencies]
|
||||
@@ -150,13 +150,13 @@ files = [
|
||||
|
||||
[[package]]
|
||||
name = "cachetools"
|
||||
version = "6.2.2"
|
||||
version = "6.2.4"
|
||||
description = "Extensible memoizing collections and decorators"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "cachetools-6.2.2-py3-none-any.whl", hash = "sha256:6c09c98183bf58560c97b2abfcedcbaf6a896a490f534b031b661d3723b45ace"},
|
||||
{file = "cachetools-6.2.2.tar.gz", hash = "sha256:8e6d266b25e539df852251cfd6f990b4bc3a141db73b939058d809ebd2590fc6"},
|
||||
{file = "cachetools-6.2.4-py3-none-any.whl", hash = "sha256:69a7a52634fed8b8bf6e24a050fb60bff1c9bd8f6d24572b99c32d4e71e62a51"},
|
||||
{file = "cachetools-6.2.4.tar.gz", hash = "sha256:82c5c05585e70b6ba2d3ae09ea60b79548872185d2f24ae1f2709d37299fd607"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -610,13 +610,13 @@ grpcio-gcp = ["grpcio-gcp (>=0.2.2,<1.0.0)"]
|
||||
|
||||
[[package]]
|
||||
name = "google-auth"
|
||||
version = "2.43.0"
|
||||
version = "2.45.0"
|
||||
description = "Google Authentication Library"
|
||||
optional = false
|
||||
python-versions = ">=3.7"
|
||||
files = [
|
||||
{file = "google_auth-2.43.0-py2.py3-none-any.whl", hash = "sha256:af628ba6fa493f75c7e9dbe9373d148ca9f4399b5ea29976519e0a3848eddd16"},
|
||||
{file = "google_auth-2.43.0.tar.gz", hash = "sha256:88228eee5fc21b62a1b5fe773ca15e67778cb07dc8363adcb4a8827b52d81483"},
|
||||
{file = "google_auth-2.45.0-py2.py3-none-any.whl", hash = "sha256:82344e86dc00410ef5382d99be677c6043d72e502b625aa4f4afa0bdacca0f36"},
|
||||
{file = "google_auth-2.45.0.tar.gz", hash = "sha256:90d3f41b6b72ea72dd9811e765699ee491ab24139f34ebf1ca2b9cc0c38708f3"},
|
||||
]
|
||||
|
||||
[package.dependencies]
|
||||
@@ -626,6 +626,7 @@ rsa = ">=3.1.4,<5"
|
||||
|
||||
[package.extras]
|
||||
aiohttp = ["aiohttp (>=3.6.2,<4.0.0)", "requests (>=2.20.0,<3.0.0)"]
|
||||
cryptography = ["cryptography (<39.0.0)", "cryptography (>=38.0.3)"]
|
||||
enterprise-cert = ["cryptography", "pyopenssl"]
|
||||
pyjwt = ["cryptography (<39.0.0)", "cryptography (>=38.0.3)", "pyjwt (>=2.0)"]
|
||||
pyopenssl = ["cryptography (<39.0.0)", "cryptography (>=38.0.3)", "pyopenssl (>=20.0.0)"]
|
||||
@@ -840,13 +841,13 @@ i18n = ["Babel (>=2.7)"]
|
||||
|
||||
[[package]]
|
||||
name = "joblib"
|
||||
version = "1.5.2"
|
||||
version = "1.5.3"
|
||||
description = "Lightweight pipelining with Python functions"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "joblib-1.5.2-py3-none-any.whl", hash = "sha256:4e1f0bdbb987e6d843c70cf43714cb276623def372df3c22fe5266b2670bc241"},
|
||||
{file = "joblib-1.5.2.tar.gz", hash = "sha256:3faa5c39054b2f03ca547da9b2f52fde67c06240c31853f306aea97f13647b55"},
|
||||
{file = "joblib-1.5.3-py3-none-any.whl", hash = "sha256:5fc3c5039fc5ca8c0276333a188bbd59d6b7ab37fe6632daa76bc7f9ec18e713"},
|
||||
{file = "joblib-1.5.3.tar.gz", hash = "sha256:8561a3269e6801106863fd0d6d84bb737be9e7631e33aaed3fb9ce5953688da3"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -1201,98 +1202,98 @@ files = [
|
||||
|
||||
[[package]]
|
||||
name = "orjson"
|
||||
version = "3.11.4"
|
||||
version = "3.11.5"
|
||||
description = "Fast, correct Python JSON library supporting dataclasses, datetimes, and numpy"
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "orjson-3.11.4-cp310-cp310-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:e3aa2118a3ece0d25489cbe48498de8a5d580e42e8d9979f65bf47900a15aba1"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:a69ab657a4e6733133a3dca82768f2f8b884043714e8d2b9ba9f52b6efef5c44"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3740bffd9816fc0326ddc406098a3a8f387e42223f5f455f2a02a9f834ead80c"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:65fd2f5730b1bf7f350c6dc896173d3460d235c4be007af73986d7cd9a2acd23"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9fdc3ae730541086158d549c97852e2eea6820665d4faf0f41bf99df41bc11ea"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:e10b4d65901da88845516ce9f7f9736f9638d19a1d483b3883dc0182e6e5edba"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fb6a03a678085f64b97f9d4a9ae69376ce91a3a9e9b56a82b1580d8e1d501aff"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:2c82e4f0b1c712477317434761fbc28b044c838b6b1240d895607441412371ac"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:d58c166a18f44cc9e2bad03a327dc2d1a3d2e85b847133cfbafd6bfc6719bd79"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:94f206766bf1ea30e1382e4890f763bd1eefddc580e08fec1ccdc20ddd95c827"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:41bf25fb39a34cf8edb4398818523277ee7096689db352036a9e8437f2f3ee6b"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-win32.whl", hash = "sha256:fa9627eba4e82f99ca6d29bc967f09aba446ee2b5a1ea728949ede73d313f5d3"},
|
||||
{file = "orjson-3.11.4-cp310-cp310-win_amd64.whl", hash = "sha256:23ef7abc7fca96632d8174ac115e668c1e931b8fe4dde586e92a500bf1914dcc"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:5e59d23cd93ada23ec59a96f215139753fbfe3a4d989549bcb390f8c00370b39"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-macosx_15_0_arm64.whl", hash = "sha256:5c3aedecfc1beb988c27c79d52ebefab93b6c3921dbec361167e6559aba2d36d"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:da9e5301f1c2caa2a9a4a303480d79c9ad73560b2e7761de742ab39fe59d9175"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:8873812c164a90a79f65368f8f96817e59e35d0cc02786a5356f0e2abed78040"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:5d7feb0741ebb15204e748f26c9638e6665a5fa93c37a2c73d64f1669b0ddc63"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:01ee5487fefee21e6910da4c2ee9eef005bee568a0879834df86f888d2ffbdd9"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:3d40d46f348c0321df01507f92b95a377240c4ec31985225a6668f10e2676f9a"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:95713e5fc8af84d8edc75b785d2386f653b63d62b16d681687746734b4dfc0be"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:ad73ede24f9083614d6c4ca9a85fe70e33be7bf047ec586ee2363bc7418fe4d7"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:842289889de515421f3f224ef9c1f1efb199a32d76d8d2ca2706fa8afe749549"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:3b2427ed5791619851c52a1261b45c233930977e7de8cf36de05636c708fa905"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:3c36e524af1d29982e9b190573677ea02781456b2e537d5840e4538a5ec41907"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-win32.whl", hash = "sha256:87255b88756eab4a68ec61837ca754e5d10fa8bc47dc57f75cedfeaec358d54c"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-win_amd64.whl", hash = "sha256:e2d5d5d798aba9a0e1fede8d853fa899ce2cb930ec0857365f700dffc2c7af6a"},
|
||||
{file = "orjson-3.11.4-cp311-cp311-win_arm64.whl", hash = "sha256:6bb6bb41b14c95d4f2702bce9975fda4516f1db48e500102fc4d8119032ff045"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:d4371de39319d05d3f482f372720b841c841b52f5385bd99c61ed69d55d9ab50"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-macosx_15_0_arm64.whl", hash = "sha256:e41fd3b3cac850eaae78232f37325ed7d7436e11c471246b87b2cd294ec94853"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:600e0e9ca042878c7fdf189cf1b028fe2c1418cc9195f6cb9824eb6ed99cb938"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:7bbf9b333f1568ef5da42bc96e18bf30fd7f8d54e9ae066d711056add508e415"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:4806363144bb6e7297b8e95870e78d30a649fdc4e23fc84daa80c8ebd366ce44"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:ad355e8308493f527d41154e9053b86a5be892b3b359a5c6d5d95cda23601cb2"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c8a7517482667fb9f0ff1b2f16fe5829296ed7a655d04d68cd9711a4d8a4e708"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:97eb5942c7395a171cbfecc4ef6701fc3c403e762194683772df4c54cfbb2210"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:149d95d5e018bdd822e3f38c103b1a7c91f88d38a88aada5c4e9b3a73a244241"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:624f3951181eb46fc47dea3d221554e98784c823e7069edb5dbd0dc826ac909b"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:03bfa548cf35e3f8b3a96c4e8e41f753c686ff3d8e182ce275b1751deddab58c"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:525021896afef44a68148f6ed8a8bf8375553d6066c7f48537657f64823565b9"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-win32.whl", hash = "sha256:b58430396687ce0f7d9eeb3dd47761ca7d8fda8e9eb92b3077a7a353a75efefa"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-win_amd64.whl", hash = "sha256:c6dbf422894e1e3c80a177133c0dda260f81428f9de16d61041949f6a2e5c140"},
|
||||
{file = "orjson-3.11.4-cp312-cp312-win_arm64.whl", hash = "sha256:d38d2bc06d6415852224fcc9c0bfa834c25431e466dc319f0edd56cca81aa96e"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:2d6737d0e616a6e053c8b4acc9eccea6b6cce078533666f32d140e4f85002534"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-macosx_15_0_arm64.whl", hash = "sha256:afb14052690aa328cc118a8e09f07c651d301a72e44920b887c519b313d892ff"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:38aa9e65c591febb1b0aed8da4d469eba239d434c218562df179885c94e1a3ad"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:f2cf4dfaf9163b0728d061bebc1e08631875c51cd30bf47cb9e3293bfbd7dcd5"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:89216ff3dfdde0e4070932e126320a1752c9d9a758d6a32ec54b3b9334991a6a"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:9daa26ca8e97fae0ce8aa5d80606ef8f7914e9b129b6b5df9104266f764ce436"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:5c8b2769dc31883c44a9cd126560327767f848eb95f99c36c9932f51090bfce9"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:1469d254b9884f984026bd9b0fa5bbab477a4bfe558bba6848086f6d43eb5e73"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:68e44722541983614e37117209a194e8c3ad07838ccb3127d96863c95ec7f1e0"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:8e7805fda9672c12be2f22ae124dcd7b03928d6c197544fe12174b86553f3196"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:04b69c14615fb4434ab867bf6f38b2d649f6f300af30a6705397e895f7aec67a"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:639c3735b8ae7f970066930e58cf0ed39a852d417c24acd4a25fc0b3da3c39a6"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-win32.whl", hash = "sha256:6c13879c0d2964335491463302a6ca5ad98105fc5db3565499dcb80b1b4bd839"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-win_amd64.whl", hash = "sha256:09bf242a4af98732db9f9a1ec57ca2604848e16f132e3f72edfd3c5c96de009a"},
|
||||
{file = "orjson-3.11.4-cp313-cp313-win_arm64.whl", hash = "sha256:a85f0adf63319d6c1ba06fb0dbf997fced64a01179cf17939a6caca662bf92de"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:42d43a1f552be1a112af0b21c10a5f553983c2a0938d2bbb8ecd8bc9fb572803"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-macosx_15_0_arm64.whl", hash = "sha256:26a20f3fbc6c7ff2cb8e89c4c5897762c9d88cf37330c6a117312365d6781d54"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:6e3f20be9048941c7ffa8fc523ccbd17f82e24df1549d1d1fe9317712d19938e"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:aac364c758dc87a52e68e349924d7e4ded348dedff553889e4d9f22f74785316"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:d5c54a6d76e3d741dcc3f2707f8eeb9ba2a791d3adbf18f900219b62942803b1"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:f28485bdca8617b79d44627f5fb04336897041dfd9fa66d383a49d09d86798bc"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:bfc2a484cad3585e4ba61985a6062a4c2ed5c7925db6d39f1fa267c9d166487f"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:e34dbd508cb91c54f9c9788923daca129fe5b55c5b4eebe713bf5ed3791280cf"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:b13c478fa413d4b4ee606ec8e11c3b2e52683a640b006bb586b3041c2ca5f606"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:724ca721ecc8a831b319dcd72cfa370cc380db0bf94537f08f7edd0a7d4e1780"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:977c393f2e44845ce1b540e19a786e9643221b3323dae190668a98672d43fb23"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:1e539e382cf46edec157ad66b0b0872a90d829a6b71f17cb633d6c160a223155"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-win32.whl", hash = "sha256:d63076d625babab9db5e7836118bdfa086e60f37d8a174194ae720161eb12394"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-win_amd64.whl", hash = "sha256:0a54d6635fa3aaa438ae32e8570b9f0de36f3f6562c308d2a2a452e8b0592db1"},
|
||||
{file = "orjson-3.11.4-cp314-cp314-win_arm64.whl", hash = "sha256:78b999999039db3cf58f6d230f524f04f75f129ba3d1ca2ed121f8657e575d3d"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:405261b0a8c62bcbd8e2931c26fdc08714faf7025f45531541e2b29e544b545b"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:af02ff34059ee9199a3546f123a6ab4c86caf1708c79042caf0820dc290a6d4f"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:0b2eba969ea4203c177c7b38b36c69519e6067ee68c34dc37081fac74c796e10"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:0baa0ea43cfa5b008a28d3c07705cf3ada40e5d347f0f44994a64b1b7b4b5350"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:80fd082f5dcc0e94657c144f1b2a3a6479c44ad50be216cf0c244e567f5eae19"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:1e3704d35e47d5bee811fb1cbd8599f0b4009b14d451c4c57be5a7e25eb89a13"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:caa447f2b5356779d914658519c874cf3b7629e99e63391ed519c28c8aea4919"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:bba5118143373a86f91dadb8df41d9457498226698ebdf8e11cbb54d5b0e802d"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:622463ab81d19ef3e06868b576551587de8e4d518892d1afab71e0fbc1f9cffc"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:3e0a700c4b82144b72946b6629968df9762552ee1344bfdb767fecdd634fbd5a"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:6e18a5c15e764e5f3fc569b47872450b4bcea24f2a6354c0a0e95ad21045d5a9"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-win32.whl", hash = "sha256:fb1c37c71cad991ef4d89c7a634b5ffb4447dbd7ae3ae13e8f5ee7f1775e7ab1"},
|
||||
{file = "orjson-3.11.4-cp39-cp39-win_amd64.whl", hash = "sha256:e2985ce8b8c42d00492d0ed79f2bd2b6460d00f2fa671dfde4bf2e02f49bf5c6"},
|
||||
{file = "orjson-3.11.4.tar.gz", hash = "sha256:39485f4ab4c9b30a3943cfe99e1a213c4776fb69e8abd68f66b83d5a0b0fdc6d"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:df9eadb2a6386d5ea2bfd81309c505e125cfc9ba2b1b99a97e60985b0b3665d1"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ccc70da619744467d8f1f49a8cadae5ec7bbe054e5232d95f92ed8737f8c5870"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:073aab025294c2f6fc0807201c76fdaed86f8fc4be52c440fb78fbb759a1ac09"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:835f26fa24ba0bb8c53ae2a9328d1706135b74ec653ed933869b74b6909e63fd"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:667c132f1f3651c14522a119e4dd631fad98761fa960c55e8e7430bb2a1ba4ac"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:42e8961196af655bb5e63ce6c60d25e8798cd4dfbc04f4203457fa3869322c2e"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75412ca06e20904c19170f8a24486c4e6c7887dea591ba18a1ab572f1300ee9f"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_aarch64.whl", hash = "sha256:6af8680328c69e15324b5af3ae38abbfcf9cbec37b5346ebfd52339c3d7e8a18"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_armv7l.whl", hash = "sha256:a86fe4ff4ea523eac8f4b57fdac319faf037d3c1be12405e6a7e86b3fbc4756a"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_i686.whl", hash = "sha256:e607b49b1a106ee2086633167033afbd63f76f2999e9236f638b06b112b24ea7"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-musllinux_1_2_x86_64.whl", hash = "sha256:7339f41c244d0eea251637727f016b3d20050636695bc78345cce9029b189401"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-win32.whl", hash = "sha256:8be318da8413cdbbce77b8c5fac8d13f6eb0f0db41b30bb598631412619572e8"},
|
||||
{file = "orjson-3.11.5-cp310-cp310-win_amd64.whl", hash = "sha256:b9f86d69ae822cabc2a0f6c099b43e8733dda788405cba2665595b7e8dd8d167"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:9c8494625ad60a923af6b2b0bd74107146efe9b55099e20d7740d995f338fcd8"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-macosx_15_0_arm64.whl", hash = "sha256:7bb2ce0b82bc9fd1168a513ddae7a857994b780b2945a8c51db4ab1c4b751ebc"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:67394d3becd50b954c4ecd24ac90b5051ee7c903d167459f93e77fc6f5b4c968"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:298d2451f375e5f17b897794bcc3e7b821c0f32b4788b9bcae47ada24d7f3cf7"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:aa5e4244063db8e1d87e0f54c3f7522f14b2dc937e65d5241ef0076a096409fd"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:1db2088b490761976c1b2e956d5d4e6409f3732e9d79cfa69f876c5248d1baf9"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:c2ed66358f32c24e10ceea518e16eb3549e34f33a9d51f99ce23b0251776a1ef"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c2021afda46c1ed64d74b555065dbd4c2558d510d8cec5ea6a53001b3e5e82a9"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_aarch64.whl", hash = "sha256:b42ffbed9128e547a1647a3e50bc88ab28ae9daa61713962e0d3dd35e820c125"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_armv7l.whl", hash = "sha256:8d5f16195bb671a5dd3d1dbea758918bada8f6cc27de72bd64adfbd748770814"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_i686.whl", hash = "sha256:c0e5d9f7a0227df2927d343a6e3859bebf9208b427c79bd31949abcc2fa32fa5"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-musllinux_1_2_x86_64.whl", hash = "sha256:23d04c4543e78f724c4dfe656b3791b5f98e4c9253e13b2636f1af5d90e4a880"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-win32.whl", hash = "sha256:c404603df4865f8e0afe981aa3c4b62b406e6d06049564d58934860b62b7f91d"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-win_amd64.whl", hash = "sha256:9645ef655735a74da4990c24ffbd6894828fbfa117bc97c1edd98c282ecb52e1"},
|
||||
{file = "orjson-3.11.5-cp311-cp311-win_arm64.whl", hash = "sha256:1cbf2735722623fcdee8e712cbaaab9e372bbcb0c7924ad711b261c2eccf4a5c"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:334e5b4bff9ad101237c2d799d9fd45737752929753bf4faf4b207335a416b7d"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-macosx_15_0_arm64.whl", hash = "sha256:ff770589960a86eae279f5d8aa536196ebda8273a2a07db2a54e82b93bc86626"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:ed24250e55efbcb0b35bed7caaec8cedf858ab2f9f2201f17b8938c618c8ca6f"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:a66d7769e98a08a12a139049aac2f0ca3adae989817f8c43337455fbc7669b85"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:86cfc555bfd5794d24c6a1903e558b50644e5e68e6471d66502ce5cb5fdef3f9"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:a230065027bc2a025e944f9d4714976a81e7ecfa940923283bca7bbc1f10f626"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:b29d36b60e606df01959c4b982729c8845c69d1963f88686608be9ced96dbfaa"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:c74099c6b230d4261fdc3169d50efc09abf38ace1a42ea2f9994b1d79153d477"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_aarch64.whl", hash = "sha256:e697d06ad57dd0c7a737771d470eedc18e68dfdefcdd3b7de7f33dfda5b6212e"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_armv7l.whl", hash = "sha256:e08ca8a6c851e95aaecc32bc44a5aa75d0ad26af8cdac7c77e4ed93acf3d5b69"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_i686.whl", hash = "sha256:e8b5f96c05fce7d0218df3fdfeb962d6b8cfff7e3e20264306b46dd8b217c0f3"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-musllinux_1_2_x86_64.whl", hash = "sha256:ddbfdb5099b3e6ba6d6ea818f61997bb66de14b411357d24c4612cf1ebad08ca"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-win32.whl", hash = "sha256:9172578c4eb09dbfcf1657d43198de59b6cef4054de385365060ed50c458ac98"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-win_amd64.whl", hash = "sha256:2b91126e7b470ff2e75746f6f6ee32b9ab67b7a93c8ba1d15d3a0caaf16ec875"},
|
||||
{file = "orjson-3.11.5-cp312-cp312-win_arm64.whl", hash = "sha256:acbc5fac7e06777555b0722b8ad5f574739e99ffe99467ed63da98f97f9ca0fe"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:3b01799262081a4c47c035dd77c1301d40f568f77cc7ec1bb7db5d63b0a01629"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-macosx_15_0_arm64.whl", hash = "sha256:61de247948108484779f57a9f406e4c84d636fa5a59e411e6352484985e8a7c3"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:894aea2e63d4f24a7f04a1908307c738d0dce992e9249e744b8f4e8dd9197f39"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:ddc21521598dbe369d83d4d40338e23d4101dad21dae0e79fa20465dbace019f"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:7cce16ae2f5fb2c53c3eafdd1706cb7b6530a67cc1c17abe8ec747f5cd7c0c51"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:e46c762d9f0e1cfb4ccc8515de7f349abbc95b59cb5a2bd68df5973fdef913f8"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:d7345c759276b798ccd6d77a87136029e71e66a8bbf2d2755cbdde1d82e78706"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:75bc2e59e6a2ac1dd28901d07115abdebc4563b5b07dd612bf64260a201b1c7f"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_aarch64.whl", hash = "sha256:54aae9b654554c3b4edd61896b978568c6daa16af96fa4681c9b5babd469f863"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_armv7l.whl", hash = "sha256:4bdd8d164a871c4ec773f9de0f6fe8769c2d6727879c37a9666ba4183b7f8228"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_i686.whl", hash = "sha256:a261fef929bcf98a60713bf5e95ad067cea16ae345d9a35034e73c3990e927d2"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-musllinux_1_2_x86_64.whl", hash = "sha256:c028a394c766693c5c9909dec76b24f37e6a1b91999e8d0c0d5feecbe93c3e05"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-win32.whl", hash = "sha256:2cc79aaad1dfabe1bd2d50ee09814a1253164b3da4c00a78c458d82d04b3bdef"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-win_amd64.whl", hash = "sha256:ff7877d376add4e16b274e35a3f58b7f37b362abf4aa31863dadacdd20e3a583"},
|
||||
{file = "orjson-3.11.5-cp313-cp313-win_arm64.whl", hash = "sha256:59ac72ea775c88b163ba8d21b0177628bd015c5dd060647bbab6e22da3aad287"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:e446a8ea0a4c366ceafc7d97067bfd55292969143b57e3c846d87fc701e797a0"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-macosx_15_0_arm64.whl", hash = "sha256:53deb5addae9c22bbe3739298f5f2196afa881ea75944e7720681c7080909a81"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:82cd00d49d6063d2b8791da5d4f9d20539c5951f965e45ccf4e96d33505ce68f"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:3fd15f9fc8c203aeceff4fda211157fad114dde66e92e24097b3647a08f4ee9e"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:9df95000fbe6777bf9820ae82ab7578e8662051bb5f83d71a28992f539d2cda7"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:92a8d676748fca47ade5bc3da7430ed7767afe51b2f8100e3cd65e151c0eaceb"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:aa0f513be38b40234c77975e68805506cad5d57b3dfd8fe3baa7f4f4051e15b4"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fa1863e75b92891f553b7922ce4ee10ed06db061e104f2b7815de80cdcb135ad"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_aarch64.whl", hash = "sha256:d4be86b58e9ea262617b8ca6251a2f0d63cc132a6da4b5fcc8e0a4128782c829"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_armv7l.whl", hash = "sha256:b923c1c13fa02084eb38c9c065afd860a5cff58026813319a06949c3af5732ac"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_i686.whl", hash = "sha256:1b6bd351202b2cd987f35a13b5e16471cf4d952b42a73c391cc537974c43ef6d"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-musllinux_1_2_x86_64.whl", hash = "sha256:bb150d529637d541e6af06bbe3d02f5498d628b7f98267ff87647584293ab439"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-win32.whl", hash = "sha256:9cc1e55c884921434a84a0c3dd2699eb9f92e7b441d7f53f3941079ec6ce7499"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-win_amd64.whl", hash = "sha256:a4f3cb2d874e03bc7767c8f88adaa1a9a05cecea3712649c3b58589ec7317310"},
|
||||
{file = "orjson-3.11.5-cp314-cp314-win_arm64.whl", hash = "sha256:38b22f476c351f9a1c43e5b07d8b5a02eb24a6ab8e75f700f7d479d4568346a5"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-macosx_10_15_x86_64.macosx_11_0_arm64.macosx_10_15_universal2.whl", hash = "sha256:1b280e2d2d284a6713b0cfec7b08918ebe57df23e3f76b27586197afca3cb1e9"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_aarch64.manylinux2014_aarch64.whl", hash = "sha256:3c8d8a112b274fae8c5f0f01954cb0480137072c271f3f4958127b010dfefaec"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_armv7l.manylinux2014_armv7l.whl", hash = "sha256:5f0a2ae6f09ac7bd47d2d5a5305c1d9ed08ac057cda55bb0a49fa506f0d2da00"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_i686.manylinux2014_i686.whl", hash = "sha256:c0d87bd1896faac0d10b4f849016db81a63e4ec5df38757ffae84d45ab38aa71"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_ppc64le.manylinux2014_ppc64le.whl", hash = "sha256:801a821e8e6099b8c459ac7540b3c32dba6013437c57fdcaec205b169754f38c"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_s390x.manylinux2014_s390x.whl", hash = "sha256:69a0f6ac618c98c74b7fbc8c0172ba86f9e01dbf9f62aa0b1776c2231a7bffe5"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-manylinux_2_17_x86_64.manylinux2014_x86_64.whl", hash = "sha256:fea7339bdd22e6f1060c55ac31b6a755d86a5b2ad3657f2669ec243f8e3b2bdb"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_aarch64.whl", hash = "sha256:4dad582bc93cef8f26513e12771e76385a7e6187fd713157e971c784112aad56"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_armv7l.whl", hash = "sha256:0522003e9f7fba91982e83a97fec0708f5a714c96c4209db7104e6b9d132f111"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_i686.whl", hash = "sha256:7403851e430a478440ecc1258bcbacbfbd8175f9ac1e39031a7121dd0de05ff8"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-musllinux_1_2_x86_64.whl", hash = "sha256:5f691263425d3177977c8d1dd896cde7b98d93cbf390b2544a090675e83a6a0a"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-win32.whl", hash = "sha256:61026196a1c4b968e1b1e540563e277843082e9e97d78afa03eb89315af531f1"},
|
||||
{file = "orjson-3.11.5-cp39-cp39-win_amd64.whl", hash = "sha256:09b94b947ac08586af635ef922d69dc9bc63321527a3a04647f4986a73f4bd30"},
|
||||
{file = "orjson-3.11.5.tar.gz", hash = "sha256:82393ab47b4fe44ffd0a7659fa9cfaacc717eb617c93cde83795f14af5c2e9d5"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -1474,13 +1475,13 @@ test = ["time-machine (>=2.6.0)"]
|
||||
|
||||
[[package]]
|
||||
name = "platformdirs"
|
||||
version = "4.5.0"
|
||||
version = "4.5.1"
|
||||
description = "A small Python package for determining appropriate platform-specific dirs, e.g. a `user data dir`."
|
||||
optional = false
|
||||
python-versions = ">=3.10"
|
||||
files = [
|
||||
{file = "platformdirs-4.5.0-py3-none-any.whl", hash = "sha256:e578a81bb873cbb89a41fcc904c7ef523cc18284b7e3b3ccf06aca1403b7ebd3"},
|
||||
{file = "platformdirs-4.5.0.tar.gz", hash = "sha256:70ddccdd7c99fc5942e9fc25636a8b34d04c24b335100223152c2803e4063312"},
|
||||
{file = "platformdirs-4.5.1-py3-none-any.whl", hash = "sha256:d03afa3963c806a9bed9d5125c8f4cb2fdaf74a55ab60e5d59b3fde758104d31"},
|
||||
{file = "platformdirs-4.5.1.tar.gz", hash = "sha256:61d5cdcc6065745cdd94f0f878977f8de9437be93de97c1c12f853c9c0cdcbda"},
|
||||
]
|
||||
|
||||
[package.extras]
|
||||
@@ -1522,21 +1523,21 @@ testing = ["google-api-core (>=1.31.5)"]
|
||||
|
||||
[[package]]
|
||||
name = "protobuf"
|
||||
version = "6.33.1"
|
||||
version = "6.33.2"
|
||||
description = ""
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "protobuf-6.33.1-cp310-abi3-win32.whl", hash = "sha256:f8d3fdbc966aaab1d05046d0240dd94d40f2a8c62856d41eaa141ff64a79de6b"},
|
||||
{file = "protobuf-6.33.1-cp310-abi3-win_amd64.whl", hash = "sha256:923aa6d27a92bf44394f6abf7ea0500f38769d4b07f4be41cb52bd8b1123b9ed"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:fe34575f2bdde76ac429ec7b570235bf0c788883e70aee90068e9981806f2490"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:f8adba2e44cde2d7618996b3fc02341f03f5bc3f2748be72dc7b063319276178"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-manylinux2014_s390x.whl", hash = "sha256:0f4cf01222c0d959c2b399142deb526de420be8236f22c71356e2a544e153c53"},
|
||||
{file = "protobuf-6.33.1-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:8fd7d5e0eb08cd5b87fd3df49bc193f5cfd778701f47e11d127d0afc6c39f1d1"},
|
||||
{file = "protobuf-6.33.1-cp39-cp39-win32.whl", hash = "sha256:023af8449482fa884d88b4563d85e83accab54138ae098924a985bcbb734a213"},
|
||||
{file = "protobuf-6.33.1-cp39-cp39-win_amd64.whl", hash = "sha256:df051de4fd7e5e4371334e234c62ba43763f15ab605579e04c7008c05735cd82"},
|
||||
{file = "protobuf-6.33.1-py3-none-any.whl", hash = "sha256:d595a9fd694fdeb061a62fbe10eb039cc1e444df81ec9bb70c7fc59ebcb1eafa"},
|
||||
{file = "protobuf-6.33.1.tar.gz", hash = "sha256:97f65757e8d09870de6fd973aeddb92f85435607235d20b2dfed93405d00c85b"},
|
||||
{file = "protobuf-6.33.2-cp310-abi3-win32.whl", hash = "sha256:87eb388bd2d0f78febd8f4c8779c79247b26a5befad525008e49a6955787ff3d"},
|
||||
{file = "protobuf-6.33.2-cp310-abi3-win_amd64.whl", hash = "sha256:fc2a0e8b05b180e5fc0dd1559fe8ebdae21a27e81ac77728fb6c42b12c7419b4"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-macosx_10_9_universal2.whl", hash = "sha256:d9b19771ca75935b3a4422957bc518b0cecb978b31d1dd12037b088f6bcc0e43"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-manylinux2014_aarch64.whl", hash = "sha256:b5d3b5625192214066d99b2b605f5783483575656784de223f00a8d00754fc0e"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-manylinux2014_s390x.whl", hash = "sha256:8cd7640aee0b7828b6d03ae518b5b4806fdfc1afe8de82f79c3454f8aef29872"},
|
||||
{file = "protobuf-6.33.2-cp39-abi3-manylinux2014_x86_64.whl", hash = "sha256:1f8017c48c07ec5859106533b682260ba3d7c5567b1ca1f24297ce03384d1b4f"},
|
||||
{file = "protobuf-6.33.2-cp39-cp39-win32.whl", hash = "sha256:7109dcc38a680d033ffb8bf896727423528db9163be1b6a02d6a49606dcadbfe"},
|
||||
{file = "protobuf-6.33.2-cp39-cp39-win_amd64.whl", hash = "sha256:2981c58f582f44b6b13173e12bb8656711189c2a70250845f264b877f00b1913"},
|
||||
{file = "protobuf-6.33.2-py3-none-any.whl", hash = "sha256:7636aad9bb01768870266de5dc009de2d1b936771b38a793f73cbbf279c91c5c"},
|
||||
{file = "protobuf-6.33.2.tar.gz", hash = "sha256:56dc370c91fbb8ac85bc13582c9e373569668a290aa2e66a590c2a0d35ddb9e4"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -2600,13 +2601,13 @@ typing-extensions = ">=4.12.0"
|
||||
|
||||
[[package]]
|
||||
name = "tzdata"
|
||||
version = "2025.2"
|
||||
version = "2025.3"
|
||||
description = "Provider of IANA time zone data"
|
||||
optional = false
|
||||
python-versions = ">=2"
|
||||
files = [
|
||||
{file = "tzdata-2025.2-py2.py3-none-any.whl", hash = "sha256:1a403fada01ff9221ca8044d701868fa132215d84beb92242d9acd2147f667a8"},
|
||||
{file = "tzdata-2025.2.tar.gz", hash = "sha256:b60a638fcc0daffadf82fe0f57e53d06bdec2f36c4df66280ae79bce6bd6f2b9"},
|
||||
{file = "tzdata-2025.3-py2.py3-none-any.whl", hash = "sha256:06a47e5700f3081aab02b2e513160914ff0694bce9947d6b76ebd6bf57cfc5d1"},
|
||||
{file = "tzdata-2025.3.tar.gz", hash = "sha256:de39c2ca5dc7b0344f2eba86f49d614019d29f060fc4ebc8a417896a620b56a7"},
|
||||
]
|
||||
|
||||
[[package]]
|
||||
@@ -2656,20 +2657,20 @@ dev = ["mypy", "pre-commit", "pytest", "pytest-cov", "pytest-socket", "ruff"]
|
||||
|
||||
[[package]]
|
||||
name = "urllib3"
|
||||
version = "2.5.0"
|
||||
version = "2.6.2"
|
||||
description = "HTTP library with thread-safe connection pooling, file post, and more."
|
||||
optional = false
|
||||
python-versions = ">=3.9"
|
||||
files = [
|
||||
{file = "urllib3-2.5.0-py3-none-any.whl", hash = "sha256:e6b01673c0fa6a13e374b50871808eb3bf7046c4b125b216f6bf1cc604cff0dc"},
|
||||
{file = "urllib3-2.5.0.tar.gz", hash = "sha256:3fc47733c7e419d4bc3f6b3dc2b4f890bb743906a30d56ba4a5bfa4bbff92760"},
|
||||
{file = "urllib3-2.6.2-py3-none-any.whl", hash = "sha256:ec21cddfe7724fc7cb4ba4bea7aa8e2ef36f607a4bab81aa6ce42a13dc3f03dd"},
|
||||
{file = "urllib3-2.6.2.tar.gz", hash = "sha256:016f9c98bb7e98085cb2b4b17b87d2c702975664e4f060c6532e64d1c1a5e797"},
|
||||
]
|
||||
|
||||
[package.extras]
|
||||
brotli = ["brotli (>=1.0.9)", "brotlicffi (>=0.8.0)"]
|
||||
brotli = ["brotli (>=1.2.0)", "brotlicffi (>=1.2.0.0)"]
|
||||
h2 = ["h2 (>=4,<5)"]
|
||||
socks = ["pysocks (>=1.5.6,!=1.5.7,<2.0)"]
|
||||
zstd = ["zstandard (>=0.18.0)"]
|
||||
zstd = ["backports-zstd (>=1.0.0)"]
|
||||
|
||||
[[package]]
|
||||
name = "wcmatch"
|
||||
|
||||
@@ -3,7 +3,7 @@ requires = [ "poetry-core>=1.0.0",]
|
||||
build-backend = "poetry.core.masonry.api"
|
||||
|
||||
[tool.poetry]
|
||||
version = "0.3.38"
|
||||
version = "0.3.39"
|
||||
name = "source-cart"
|
||||
description = "Source implementation for Cart."
|
||||
authors = [ "Airbyte <contact@airbyte.io>",]
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-castor-edc
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 2cb45514-7c10-439c-a198-aeb1ddab02cb
|
||||
dockerImageTag: 0.0.38
|
||||
dockerImageTag: 0.0.39
|
||||
dockerRepository: airbyte/source-castor-edc
|
||||
githubIssueLabel: source-castor-edc
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-chameleon
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 64a0240a-81a4-4e40-8002-e063b17cfbbe
|
||||
dockerImageTag: 0.1.37
|
||||
dockerImageTag: 0.1.38
|
||||
dockerRepository: airbyte/source-chameleon
|
||||
githubIssueLabel: source-chameleon
|
||||
icon: icon.svg
|
||||
|
||||
@@ -6,11 +6,11 @@ data:
|
||||
hosts:
|
||||
- "*.chargebee.com"
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 686473f1-76d9-4994-9cc7-9b13da46147c
|
||||
dockerImageTag: 0.10.24
|
||||
dockerImageTag: 0.10.25
|
||||
dockerRepository: airbyte/source-chargebee
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/chargebee
|
||||
externalDocumentationUrls:
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-chargedesk
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: cd803254-3d2c-4613-a870-20d205ee6267
|
||||
dockerImageTag: 0.0.44
|
||||
dockerImageTag: 0.0.45
|
||||
dockerRepository: airbyte/source-chargedesk
|
||||
githubIssueLabel: source-chargedesk
|
||||
icon: icon.svg
|
||||
|
||||
@@ -18,11 +18,11 @@ data:
|
||||
# Please update to the latest version of the connector base image.
|
||||
# https://hub.docker.com/r/airbyte/python-connector-base
|
||||
# Please use the full address with sha256 hash to guarantee build reproducibility.
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: b6604cbd-1b12-4c08-8767-e140d0fb0877
|
||||
dockerImageTag: 1.1.38
|
||||
dockerImageTag: 1.1.39
|
||||
dockerRepository: airbyte/source-chartmogul
|
||||
githubIssueLabel: source-chartmogul
|
||||
icon: chartmogul.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-churnkey
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 0f1a0659-0239-48a7-bf34-072a5bf2713a
|
||||
dockerImageTag: 0.0.17
|
||||
dockerImageTag: 0.0.18
|
||||
dockerRepository: airbyte/source-churnkey
|
||||
githubIssueLabel: source-churnkey
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-cimis
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: d169ef9b-6741-4af6-b4c8-7ec4410d7f0e
|
||||
dockerImageTag: 0.0.44
|
||||
dockerImageTag: 0.0.45
|
||||
dockerRepository: airbyte/source-cimis
|
||||
githubIssueLabel: source-cimis
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,7 +13,7 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-cin7
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorTestSuitesOptions:
|
||||
- suite: acceptanceTests
|
||||
testSecrets:
|
||||
@@ -25,7 +25,7 @@ data:
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: ff48bf15-f917-4fff-ba28-cbab56ef892f
|
||||
dockerImageTag: 0.3.24
|
||||
dockerImageTag: 0.3.25
|
||||
dockerRepository: airbyte/source-cin7
|
||||
githubIssueLabel: source-cin7
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-circa
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 751184ec-3b11-4084-b1b7-8064dde1e76e
|
||||
dockerImageTag: 0.0.41
|
||||
dockerImageTag: 0.0.42
|
||||
dockerRepository: airbyte/source-circa
|
||||
githubIssueLabel: source-circa
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-cisco-meraki
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 65468f53-6682-4fa4-8627-1dfa4b3dc560
|
||||
dockerImageTag: 0.0.23
|
||||
dockerImageTag: 0.0.24
|
||||
dockerRepository: airbyte/source-cisco-meraki
|
||||
githubIssueLabel: source-cisco-meraki
|
||||
icon: icon.svg
|
||||
|
||||
@@ -13,11 +13,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-clarif-ai
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 7fbeaeea-2d0d-4f13-8200-fa228494d91c
|
||||
dockerImageTag: 0.0.43
|
||||
dockerImageTag: 0.0.44
|
||||
dockerRepository: airbyte/source-clarif-ai
|
||||
githubIssueLabel: source-clarif-ai
|
||||
icon: icon.svg
|
||||
|
||||
@@ -12,11 +12,11 @@ data:
|
||||
enabled: false
|
||||
packageName: airbyte-source-clazar
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: d7df7b64-6266-45b5-ad83-e1515578f371
|
||||
dockerImageTag: 0.4.44
|
||||
dockerImageTag: 0.4.45
|
||||
dockerRepository: airbyte/source-clazar
|
||||
githubIssueLabel: source-clazar
|
||||
icon: clazar.svg
|
||||
|
||||
@@ -5,7 +5,7 @@ data:
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: 311a7a27-3fb5-4f7e-8265-5e4afe258b66
|
||||
dockerImageTag: 0.3.38
|
||||
dockerImageTag: 0.3.39
|
||||
dockerRepository: airbyte/source-clickup-api
|
||||
githubIssueLabel: source-clickup-api
|
||||
icon: clickup.svg
|
||||
@@ -43,7 +43,7 @@ data:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
externalDocumentationUrls:
|
||||
- title: ClickUp API reference
|
||||
url: https://clickup.com/api/
|
||||
|
||||
@@ -6,11 +6,11 @@ data:
|
||||
hosts:
|
||||
- api.clockify.me
|
||||
connectorBuildOptions:
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.5.1@sha256:8da9d362c184e2e46532ab94f6f9968a74835c0882d6a4a2f9f9c9e5b972f2a1
|
||||
baseImage: docker.io/airbyte/source-declarative-manifest:7.6.1@sha256:b433ec72f88e69bd836f3c7c815c21bfeeeb32501aeabf586bc421875289e5e2
|
||||
connectorSubtype: api
|
||||
connectorType: source
|
||||
definitionId: e71aae8a-5143-11ed-bdc3-0242ac120002
|
||||
dockerImageTag: 0.4.40
|
||||
dockerImageTag: 0.4.41
|
||||
dockerRepository: airbyte/source-clockify
|
||||
documentationUrl: https://docs.airbyte.com/integrations/sources/clockify
|
||||
githubIssueLabel: source-clockify
|
||||
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user