[HUDI-4319] Fixed Parquet's PLAIN_DICTIONARY encoding not being applied when bulk-inserting (#5966)
* Fixed Dictionary encoding config not being properly propagated to Parquet writer (making it unable to apply it, substantially bloating the storage footprint)
This commit is contained in:
@@ -23,12 +23,11 @@ import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport
|
||||
import org.apache.hudi.client.SparkTaskContextSupplier
|
||||
import org.apache.hudi.common.HoodieJsonPayload
|
||||
import org.apache.hudi.common.bloom.{BloomFilter, BloomFilterFactory}
|
||||
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
|
||||
import org.apache.hudi.common.util.BaseFileUtils
|
||||
import org.apache.hudi.config.{HoodieIndexConfig, HoodieStorageConfig}
|
||||
import org.apache.hudi.io.storage.{HoodieAvroParquetConfig, HoodieAvroParquetWriter}
|
||||
import org.apache.hudi.io.storage.{HoodieAvroParquetWriter, HoodieParquetConfig}
|
||||
import org.apache.parquet.avro.AvroSchemaConverter
|
||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext}
|
||||
@@ -45,7 +44,7 @@ object SparkHelpers {
|
||||
val filter: BloomFilter = BloomFilterFactory.createBloomFilter(HoodieIndexConfig.BLOOM_FILTER_NUM_ENTRIES_VALUE.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_FPP_VALUE.defaultValue.toDouble,
|
||||
HoodieIndexConfig.BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES.defaultValue.toInt, HoodieIndexConfig.BLOOM_FILTER_TYPE.defaultValue);
|
||||
val writeSupport: HoodieAvroWriteSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(fs.getConf).convert(schema), schema, org.apache.hudi.common.util.Option.of(filter))
|
||||
val parquetConfig: HoodieAvroParquetConfig = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.toDouble)
|
||||
val parquetConfig: HoodieParquetConfig[HoodieAvroWriteSupport] = new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, HoodieStorageConfig.PARQUET_BLOCK_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_PAGE_SIZE.defaultValue.toInt, HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.defaultValue.toInt, fs.getConf, HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue.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