-
Notifications
You must be signed in to change notification settings - Fork 4.1k
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: Parquet toolkit and S3V2 Support (#47122)
- Loading branch information
1 parent
7faa1dc
commit 7ec23bb
Showing
14 changed files
with
185 additions
and
29 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
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
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,17 @@ | ||
dependencies { | ||
implementation project(':airbyte-cdk:bulk:core:bulk-cdk-core-base') | ||
implementation project(':airbyte-cdk:bulk:core:bulk-cdk-core-load') | ||
api project(':airbyte-cdk:bulk:toolkits:bulk-cdk-toolkit-load-avro') | ||
|
||
api ('org.apache.hadoop:hadoop-common:3.4.0') { | ||
exclude group: 'org.apache.zookeeper' | ||
exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-common' | ||
} | ||
api ('org.apache.hadoop:hadoop-mapreduce-client-core:3.4.0') { | ||
exclude group: 'org.apache.zookeeper' | ||
exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-common' | ||
} | ||
api 'org.apache.parquet:parquet-avro:1.14.2' | ||
|
||
testFixturesImplementation testFixtures(project(":airbyte-cdk:bulk:core:bulk-cdk-core-load")) | ||
} |
50 changes: 50 additions & 0 deletions
50
...k/toolkits/load-parquet/src/main/kotlin/io/airbyte/cdk/load/file/parquet/ParquetReader.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,50 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.load.file.parquet | ||
|
||
import java.io.Closeable | ||
import java.io.File | ||
import java.io.InputStream | ||
import kotlin.io.path.outputStream | ||
import org.apache.avro.Schema | ||
import org.apache.avro.generic.GenericRecord | ||
import org.apache.hadoop.fs.Path | ||
import org.apache.parquet.avro.AvroParquetReader | ||
import org.apache.parquet.avro.AvroReadSupport | ||
import org.apache.parquet.hadoop.ParquetReader as ApacheParquetReader | ||
|
||
class ParquetReader( | ||
private val reader: ApacheParquetReader<GenericRecord>, | ||
private val tmpFile: File | ||
) : Closeable { | ||
private fun read(): GenericRecord? { | ||
return reader.read() | ||
} | ||
|
||
fun recordSequence(): Sequence<GenericRecord> = generateSequence { read() } | ||
|
||
override fun close() { | ||
reader.close() | ||
tmpFile.delete() | ||
} | ||
} | ||
|
||
fun InputStream.toParquetReader(avroSchema: Schema): ParquetReader { | ||
|
||
val tmpFile = | ||
kotlin.io.path.createTempFile( | ||
prefix = "${avroSchema.namespace}.${avroSchema.name}", | ||
suffix = ".avro" | ||
) | ||
tmpFile.outputStream().use { outputStream -> this.copyTo(outputStream) } | ||
val reader = | ||
AvroParquetReader.builder<GenericRecord>( | ||
AvroReadSupport(), | ||
Path(tmpFile.toAbsolutePath().toString()) | ||
) | ||
.build() | ||
|
||
return ParquetReader(reader, tmpFile.toFile()) | ||
} |
55 changes: 55 additions & 0 deletions
55
...k/toolkits/load-parquet/src/main/kotlin/io/airbyte/cdk/load/file/parquet/ParquetWriter.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,55 @@ | ||
/* | ||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved. | ||
*/ | ||
|
||
package io.airbyte.cdk.load.file.parquet | ||
|
||
import java.io.Closeable | ||
import java.io.OutputStream | ||
import org.apache.avro.Schema | ||
import org.apache.avro.generic.GenericRecord | ||
import org.apache.hadoop.conf.Configuration | ||
import org.apache.parquet.avro.AvroParquetWriter | ||
import org.apache.parquet.hadoop.ParquetWriter as ApacheParquetWriter | ||
import org.apache.parquet.io.OutputFile | ||
import org.apache.parquet.io.PositionOutputStream | ||
|
||
class ParquetWriter(private val writer: ApacheParquetWriter<GenericRecord>) : Closeable { | ||
fun write(record: GenericRecord) = writer.write(record) | ||
override fun close() = writer.close() | ||
} | ||
|
||
fun OutputStream.toParquetWriter(avroSchema: Schema): ParquetWriter { | ||
// Custom OutputFile implementation wrapping the OutputStream | ||
val outputFile = | ||
object : OutputFile { | ||
var position: Long = 0 | ||
override fun create(blockSizeHint: Long) = | ||
object : PositionOutputStream() { | ||
override fun write(b: Int) { | ||
position += 1 | ||
this@toParquetWriter.write(b) | ||
} | ||
override fun write(bytes: ByteArray, off: Int, len: Int) { | ||
position += len | ||
this@toParquetWriter.write(bytes, off, len) | ||
} | ||
override fun flush() = this@toParquetWriter.flush() | ||
override fun close() = this@toParquetWriter.close() | ||
override fun getPos() = position | ||
} | ||
|
||
override fun createOrOverwrite(blockSizeHint: Long) = create(blockSizeHint) | ||
override fun supportsBlockSize() = false | ||
override fun defaultBlockSize() = 0L | ||
} | ||
|
||
// Initialize AvroParquetWriter with the custom OutputFile | ||
val writer = | ||
AvroParquetWriter.builder<GenericRecord>(outputFile) | ||
.withSchema(avroSchema) | ||
.withConf(Configuration()) | ||
.build() | ||
|
||
return ParquetWriter(writer) | ||
} |
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
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