[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:
@@ -47,11 +47,11 @@ public class HoodieAvroParquetWriter<R extends IndexedRecord>
|
||||
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
public HoodieAvroParquetWriter(Path file,
|
||||
HoodieAvroParquetConfig parquetConfig,
|
||||
HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig,
|
||||
String instantTime,
|
||||
TaskContextSupplier taskContextSupplier,
|
||||
boolean populateMetaFields) throws IOException {
|
||||
super(file, (HoodieBaseParquetConfig) parquetConfig);
|
||||
super(file, (HoodieParquetConfig) parquetConfig);
|
||||
this.fileName = file.getName();
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
this.instantTime = instantTime;
|
||||
|
||||
@@ -43,7 +43,7 @@ public abstract class HoodieBaseParquetWriter<R> extends ParquetWriter<R> {
|
||||
private long lastCachedDataSize = -1;
|
||||
|
||||
public HoodieBaseParquetWriter(Path file,
|
||||
HoodieBaseParquetConfig<? extends WriteSupport<R>> parquetConfig) throws IOException {
|
||||
HoodieParquetConfig<? extends WriteSupport<R>> parquetConfig) throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE,
|
||||
parquetConfig.getWriteSupport(),
|
||||
|
||||
@@ -77,7 +77,7 @@ public class HoodieFileWriterFactory {
|
||||
Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter(conf).convert(schema), schema, filter);
|
||||
|
||||
HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(),
|
||||
HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig = new HoodieParquetConfig<>(writeSupport, config.getParquetCompressionCodec(),
|
||||
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
|
||||
conf, config.getParquetCompressionRatio(), config.parquetDictionaryEnabled());
|
||||
|
||||
|
||||
@@ -19,6 +19,12 @@
|
||||
|
||||
package org.apache.hudi.testutils;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
@@ -38,18 +44,11 @@ import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieStorageConfig;
|
||||
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
|
||||
import org.apache.hudi.io.storage.HoodieAvroParquetWriter;
|
||||
import org.apache.hudi.io.storage.HoodieParquetConfig;
|
||||
import org.apache.hudi.io.storage.HoodieOrcConfig;
|
||||
import org.apache.hudi.io.storage.HoodieOrcWriter;
|
||||
import org.apache.hudi.io.storage.HoodieAvroParquetWriter;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.orc.CompressionKind;
|
||||
@@ -110,7 +109,7 @@ public class HoodieWriteableTestTable extends HoodieMetadataTestTable {
|
||||
if (HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().equals(HoodieFileFormat.PARQUET)) {
|
||||
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
|
||||
new AvroSchemaConverter().convert(schema), schema, Option.of(filter));
|
||||
HoodieAvroParquetConfig config = new HoodieAvroParquetConfig(writeSupport, CompressionCodecName.GZIP,
|
||||
HoodieParquetConfig<HoodieAvroWriteSupport> config = new HoodieParquetConfig<>(writeSupport, CompressionCodecName.GZIP,
|
||||
ParquetWriter.DEFAULT_BLOCK_SIZE, ParquetWriter.DEFAULT_PAGE_SIZE, 120 * 1024 * 1024,
|
||||
new Configuration(), Double.parseDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION.defaultValue()));
|
||||
try (HoodieAvroParquetWriter writer = new HoodieAvroParquetWriter<>(
|
||||
|
||||
Reference in New Issue
Block a user