[HUDI-1013] Adding Bulk Insert V2 implementation (#1834)
- Adding ability to use native spark row writing for bulk_insert - Controlled by `ENABLE_ROW_WRITER_OPT_KEY` datasource write option - Introduced KeyGeneratorInterface in hudi-client, moved KeyGenerator back to hudi-spark - Simplified the new API additions to just two new methods : getRecordKey(row), getPartitionPath(row) - Fixed all built-in key generators with new APIs - Made the field position map lazily created upon the first call to row based apis - Implemented native row based key generators for CustomKeyGenerator - Fixed all the tests, with these new APIs Co-authored-by: Balaji Varadarajan <varadarb@uber.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
committed by
GitHub
parent
8d04268264
commit
379cf0786f
@@ -28,7 +28,7 @@ import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.common.util.ParquetUtils
|
||||
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
|
||||
import org.apache.hudi.io.storage.{HoodieParquetConfig, HoodieParquetWriter}
|
||||
import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieParquetWriter}
|
||||
import org.apache.parquet.avro.AvroSchemaConverter
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
@@ -45,7 +45,7 @@ object SparkHelpers {
|
||||
val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.DEFAULT_BLOOM_FILTER_NUM_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_FILTER_FPP.toDouble,
|
||||
HoodieIndexConfig.DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.toInt, HoodieIndexConfig.DEFAULT_BLOOM_INDEX_FILTER_TYPE);
|
||||
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter)
|
||||
val parquetConfig: HoodieParquetConfig = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
|
||||
val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.DEFAULT_PARQUET_BLOCK_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_PAGE_SIZE_BYTES.toInt, HoodieStorageConfig.DEFAULT_PARQUET_FILE_MAX_BYTES.toInt, fs.getConf, HoodieStorageConfig.DEFAULT_STREAM_COMPRESSION_RATIO.toDouble)
|
||||
|
||||
// Add current classLoad for config, if not will throw classNotFound of 'HoodieWrapperFileSystem'.
|
||||
parquetConfig.getHadoopConf().setClassLoader(Thread.currentThread.getContextClassLoader)
|
||||
|
||||
Reference in New Issue
Block a user