1
0

Adding config for parquet compression ratio

This commit is contained in:
Nishith Agarwal
2018-03-23 21:50:11 -07:00
committed by vinoth chandar
parent 48643795b8
commit 1b756db221
7 changed files with 28 additions and 7 deletions

View File

@@ -28,16 +28,18 @@ public class HoodieParquetConfig {
private int pageSize;
private long maxFileSize;
private Configuration hadoopConf;
private double compressionRatio;
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
CompressionCodecName compressionCodecName, int blockSize, int pageSize, long maxFileSize,
Configuration hadoopConf) {
Configuration hadoopConf, double compressionRatio) {
this.writeSupport = writeSupport;
this.compressionCodecName = compressionCodecName;
this.blockSize = blockSize;
this.pageSize = pageSize;
this.maxFileSize = maxFileSize;
this.hadoopConf = hadoopConf;
this.compressionRatio = compressionRatio;
}
public HoodieAvroWriteSupport getWriteSupport() {
@@ -63,4 +65,8 @@ public class HoodieParquetConfig {
public Configuration getHadoopConf() {
return hadoopConf;
}
public double getCompressionRatio() {
return compressionRatio;
}
}

View File

@@ -39,10 +39,8 @@ import org.apache.spark.TaskContext;
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
private static double STREAM_COMPRESSION_RATIO = 0.1;
private static AtomicLong recordIndex = new AtomicLong(1);
private final Path file;
private final HoodieWrapperFileSystem fs;
private final long maxFileSize;
@@ -75,7 +73,7 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
// We cannot accurately measure the snappy compressed output file size. We are choosing a conservative 10%
// TODO - compute this compression ratio dynamically by looking at the bytes written to the stream and the actual file size reported by HDFS
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
.round(parquetConfig.getMaxFileSize() * STREAM_COMPRESSION_RATIO);
.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
this.writeSupport = parquetConfig.getWriteSupport();
this.commitTime = commitTime;
this.schema = schema;

View File

@@ -50,7 +50,8 @@ public class HoodieStorageWriterFactory {
HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP,
config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf());
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(),
config.getParquetCompressionRatio());
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
}