1
0

[HUDI-2385] Make parquet dictionary encoding configurable (#3578)

Co-authored-by: leesf <leesf@apache.org>
This commit is contained in:
Shawy Geng
2021-09-24 13:33:34 +08:00
committed by GitHub
parent f52cb32f5f
commit 06c2cc2c8b
6 changed files with 39 additions and 13 deletions

View File

@@ -110,6 +110,11 @@ public class HoodieStorageConfig extends HoodieConfig {
.defaultValue("gzip") .defaultValue("gzip")
.withDocumentation("Compression Codec for parquet files"); .withDocumentation("Compression Codec for parquet files");
public static final ConfigProperty<Boolean> PARQUET_DICTIONARY_ENABLED = ConfigProperty
.key("hoodie.parquet.dictionary.enabled")
.defaultValue(true)
.withDocumentation("Whether to use dictionary encoding");
public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM_NAME = ConfigProperty
.key("hoodie.hfile.compression.algorithm") .key("hoodie.hfile.compression.algorithm")
.defaultValue("GZ") .defaultValue("GZ")

View File

@@ -1403,10 +1403,6 @@ public class HoodieWriteConfig extends HoodieConfig {
return getInt(HoodieStorageConfig.LOGFILE_DATA_BLOCK_MAX_SIZE); return getInt(HoodieStorageConfig.LOGFILE_DATA_BLOCK_MAX_SIZE);
} }
public long getLogFileMaxSize() {
return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE);
}
public double getParquetCompressionRatio() { public double getParquetCompressionRatio() {
return getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION); return getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION);
} }
@@ -1415,6 +1411,14 @@ public class HoodieWriteConfig extends HoodieConfig {
return CompressionCodecName.fromConf(getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME)); return CompressionCodecName.fromConf(getString(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME));
} }
public boolean parquetDictionaryEnabled() {
return getBoolean(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED);
}
public long getLogFileMaxSize() {
return getLong(HoodieStorageConfig.LOGFILE_MAX_SIZE);
}
public double getLogFileToParquetCompressionRatio() { public double getLogFileToParquetCompressionRatio() {
return getDouble(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION); return getDouble(HoodieStorageConfig.LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION);
} }

View File

@@ -33,4 +33,10 @@ public class HoodieAvroParquetConfig extends HoodieBaseParquetConfig<HoodieAvroW
double compressionRatio) { double compressionRatio) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio); super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
} }
public HoodieAvroParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
double compressionRatio, boolean directoryEnabled) {
super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, directoryEnabled);
}
} }

View File

@@ -27,15 +27,21 @@ import org.apache.parquet.hadoop.metadata.CompressionCodecName;
*/ */
public class HoodieBaseParquetConfig<T> { public class HoodieBaseParquetConfig<T> {
private final T writeSupport; private final T writeSupport;
private CompressionCodecName compressionCodecName; private final CompressionCodecName compressionCodecName;
private int blockSize; private final int blockSize;
private int pageSize; private final int pageSize;
private long maxFileSize; private final long maxFileSize;
private Configuration hadoopConf; private final Configuration hadoopConf;
private double compressionRatio; private final double compressionRatio;
private final boolean dictionaryEnabled;
public HoodieBaseParquetConfig(T writeSupport, CompressionCodecName compressionCodecName, int blockSize, public HoodieBaseParquetConfig(T writeSupport, CompressionCodecName compressionCodecName, int blockSize,
int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio) { int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio) {
this(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio, false);
}
public HoodieBaseParquetConfig(T writeSupport, CompressionCodecName compressionCodecName, int blockSize,
int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio, boolean dictionaryEnabled) {
this.writeSupport = writeSupport; this.writeSupport = writeSupport;
this.compressionCodecName = compressionCodecName; this.compressionCodecName = compressionCodecName;
this.blockSize = blockSize; this.blockSize = blockSize;
@@ -43,6 +49,7 @@ public class HoodieBaseParquetConfig<T> {
this.maxFileSize = maxFileSize; this.maxFileSize = maxFileSize;
this.hadoopConf = hadoopConf; this.hadoopConf = hadoopConf;
this.compressionRatio = compressionRatio; this.compressionRatio = compressionRatio;
this.dictionaryEnabled = dictionaryEnabled;
} }
public CompressionCodecName getCompressionCodecName() { public CompressionCodecName getCompressionCodecName() {
@@ -72,4 +79,8 @@ public class HoodieBaseParquetConfig<T> {
public T getWriteSupport() { public T getWriteSupport() {
return writeSupport; return writeSupport;
} }
public boolean dictionaryEnabled() {
return dictionaryEnabled;
}
} }

View File

@@ -71,7 +71,7 @@ public class HoodieFileWriterFactory {
HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(), HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig(writeSupport, config.getParquetCompressionCodec(),
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(), config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio()); hoodieTable.getHadoopConf(), config.getParquetCompressionRatio(), config.parquetDictionaryEnabled());
return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier, populateMetaFields); return new HoodieParquetWriter<>(instantTime, path, parquetConfig, schema, taskContextSupplier, populateMetaFields);
} }

View File

@@ -56,7 +56,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()), super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(), ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(), parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED, parquetConfig.dictionaryEnabled(), DEFAULT_IS_VALIDATING_ENABLED,
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf())); DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()); this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = this.fs =