-
Notifications
You must be signed in to change notification settings - Fork 4.2k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Bulk Load CDK: Mapper Pipeline (#48371)
- Loading branch information
1 parent
87f514d
commit 2e3023a
Showing
13 changed files
with
223 additions
and
45 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
39 changes: 39 additions & 0 deletions
39
airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/MapperPipeline.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,39 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.load.data | ||
|
||
import io.airbyte.cdk.load.command.DestinationStream | ||
import io.airbyte.cdk.load.message.DestinationRecord.Change | ||
|
||
class MapperPipeline( | ||
inputSchema: AirbyteType, | ||
schemaValueMapperPairs: List<Pair<AirbyteSchemaMapper, AirbyteValueMapper>>, | ||
) { | ||
private val schemasWithMappers: List<Pair<AirbyteType, AirbyteValueMapper>> | ||
|
||
val finalSchema: AirbyteType | ||
|
||
init { | ||
val (schemaMappers, valueMappers) = schemaValueMapperPairs.unzip() | ||
val schemas = | ||
schemaMappers.runningFold(inputSchema) { schema, mapper -> mapper.map(schema) } | ||
schemasWithMappers = schemas.zip(valueMappers) | ||
finalSchema = schemas.last() | ||
} | ||
|
||
fun map(data: AirbyteValue): Pair<AirbyteValue, List<Change>> { | ||
val results = | ||
schemasWithMappers.runningFold(data) { value, (schema, mapper) -> | ||
mapper.map(value, schema) | ||
} | ||
val changesFlattened = | ||
schemasWithMappers.flatMap { it.second.collectedChanges }.toSet().toList() | ||
return results.last() to changesFlattened | ||
} | ||
} | ||
|
||
interface MapperPipelineFactory { | ||
fun create(stream: DestinationStream): MapperPipeline | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
117 changes: 117 additions & 0 deletions
117
airbyte-cdk/bulk/core/load/src/test/kotlin/io/airbyte/cdk/load/data/MapperPipelineTest.kt
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,117 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.load.data | ||
|
||
import io.airbyte.cdk.load.test.util.Root | ||
import io.airbyte.cdk.load.test.util.SchemaRecordBuilder | ||
import io.airbyte.cdk.load.test.util.ValueTestBuilder | ||
import org.junit.jupiter.api.Assertions | ||
import org.junit.jupiter.api.Test | ||
|
||
class MapperPipelineTest { | ||
class TurnSchemalessObjectTypesIntoIntegers : AirbyteSchemaIdentityMapper { | ||
override fun mapObjectWithoutSchema(schema: ObjectTypeWithoutSchema): AirbyteType = | ||
IntegerType | ||
} | ||
|
||
class TurnSchemalessObjectsIntoIntegers : AirbyteValueIdentityMapper() { | ||
override fun mapObjectWithoutSchema( | ||
value: ObjectValue, | ||
schema: ObjectTypeWithoutSchema, | ||
path: List<String> | ||
): AirbyteValue { | ||
if (value.values.size == 1) { | ||
throw IllegalStateException("Arbitrarily reject 1") | ||
} | ||
return IntegerValue(value.values.size.toLong()) | ||
} | ||
} | ||
|
||
class TurnIntegerTypesIntoStrings : AirbyteSchemaIdentityMapper { | ||
override fun mapInteger(schema: IntegerType): AirbyteType = StringType | ||
} | ||
|
||
class TurnIntegersIntoStrings : AirbyteValueIdentityMapper() { | ||
override fun mapInteger(value: IntegerValue, path: List<String>): AirbyteValue { | ||
if (value.value == 2L) { | ||
throw IllegalStateException("Arbitrarily reject 2") | ||
} | ||
return StringValue(value.value.toString()) | ||
} | ||
} | ||
|
||
private fun makePipeline(schema: AirbyteType) = | ||
MapperPipeline( | ||
schema, | ||
listOf( | ||
TurnIntegerTypesIntoStrings() to TurnIntegersIntoStrings(), | ||
TurnSchemalessObjectTypesIntoIntegers() to TurnSchemalessObjectsIntoIntegers(), | ||
) | ||
) | ||
|
||
@Test | ||
fun testSuccessfulPipeline() { | ||
val (inputSchema, expectedSchema) = | ||
SchemaRecordBuilder<Root>() | ||
.with(ObjectTypeWithoutSchema, IntegerType) | ||
.with(IntegerType, StringType) | ||
.withRecord() | ||
.with(IntegerType, StringType) | ||
.with(BooleanType, BooleanType) // expect unchanged | ||
.endRecord() | ||
.build() | ||
|
||
val pipeline = makePipeline(inputSchema) | ||
Assertions.assertEquals( | ||
expectedSchema, | ||
pipeline.finalSchema, | ||
"final schema matches expected transformed schema" | ||
) | ||
} | ||
|
||
@Test | ||
fun testRecordMapping() { | ||
val (inputValue, inputSchema, expectedOutput) = | ||
ValueTestBuilder<Root>() | ||
.with( | ||
ObjectValue(linkedMapOf("a" to IntegerValue(1), "b" to IntegerValue(2))), | ||
ObjectTypeWithoutSchema, | ||
IntegerValue(2) | ||
) | ||
.with(IntegerValue(1), IntegerType, StringValue("1")) | ||
.withRecord() | ||
.with(IntegerValue(3), IntegerType, StringValue("3")) | ||
.with(BooleanValue(true), BooleanType, BooleanValue(true)) // expect unchanged | ||
.endRecord() | ||
.build() | ||
val pipeline = makePipeline(inputSchema) | ||
val (result, changes) = pipeline.map(inputValue) | ||
|
||
Assertions.assertEquals(0, changes.size, "no changes were captured") | ||
Assertions.assertEquals(expectedOutput, result, "data was transformed as expected") | ||
} | ||
|
||
@Test | ||
fun testFailedMapping() { | ||
val (inputValue, inputSchema, _) = | ||
ValueTestBuilder<Root>() | ||
.with( | ||
ObjectValue(linkedMapOf("a" to IntegerValue(1))), | ||
ObjectTypeWithoutSchema, | ||
NullValue, | ||
nullable = true | ||
) // fail: reject size==1 | ||
.with(IntegerValue(1), IntegerType, StringValue("1")) | ||
.withRecord() | ||
.with(IntegerValue(2), IntegerType, NullValue, nullable = true) // fail: reject 2 | ||
.with(BooleanValue(true), BooleanType, BooleanValue(true)) // expect unchanged | ||
.endRecord() | ||
.build() | ||
val pipeline = makePipeline(inputSchema) | ||
val (_, changes) = pipeline.map(inputValue) | ||
|
||
Assertions.assertEquals(2, changes.size, "two failures were captured") | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.