[HUDI-4038] Avoid calling getDataSize after every record written (#5497)
- getDataSize has non-trivial overhead in the current ParquetWriter impl, requiring traversal of already composed Column Groups in memory. Instead we can sample these calls to getDataSize to amortize its cost. Co-authored-by: sivabalan <n.siva.b@gmail.com>
This commit is contained in:
@@ -19,11 +19,7 @@
|
||||
package org.apache.hudi.io.storage.row;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
|
||||
|
||||
import org.apache.parquet.hadoop.ParquetFileWriter;
|
||||
import org.apache.parquet.hadoop.ParquetWriter;
|
||||
import org.apache.hudi.io.storage.HoodieBaseParquetWriter;
|
||||
import org.apache.spark.sql.catalyst.InternalRow;
|
||||
|
||||
import java.io.IOException;
|
||||
@@ -31,32 +27,16 @@ import java.io.IOException;
|
||||
/**
|
||||
* Parquet's impl of {@link HoodieInternalRowFileWriter} to write {@link InternalRow}s.
|
||||
*/
|
||||
public class HoodieInternalRowParquetWriter extends ParquetWriter<InternalRow>
|
||||
public class HoodieInternalRowParquetWriter extends HoodieBaseParquetWriter<InternalRow>
|
||||
implements HoodieInternalRowFileWriter {
|
||||
|
||||
private final Path file;
|
||||
private final HoodieWrapperFileSystem fs;
|
||||
private final long maxFileSize;
|
||||
private final HoodieRowParquetWriteSupport writeSupport;
|
||||
|
||||
public HoodieInternalRowParquetWriter(Path file, HoodieRowParquetConfig parquetConfig)
|
||||
throws IOException {
|
||||
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
|
||||
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
|
||||
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
|
||||
DEFAULT_IS_DICTIONARY_ENABLED, DEFAULT_IS_VALIDATING_ENABLED,
|
||||
DEFAULT_WRITER_VERSION, FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
|
||||
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
|
||||
this.fs = (HoodieWrapperFileSystem) this.file.getFileSystem(FSUtils.registerFileSystem(file,
|
||||
parquetConfig.getHadoopConf()));
|
||||
this.maxFileSize = parquetConfig.getMaxFileSize()
|
||||
+ Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
}
|
||||
super(file, parquetConfig);
|
||||
|
||||
@Override
|
||||
public boolean canWrite() {
|
||||
return getDataSize() < maxFileSize;
|
||||
this.writeSupport = parquetConfig.getWriteSupport();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -69,9 +49,4 @@ public class HoodieInternalRowParquetWriter extends ParquetWriter<InternalRow>
|
||||
public void writeRow(InternalRow row) throws IOException {
|
||||
super.write(row);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -49,7 +49,7 @@ public class TestHoodieFileWriterFactory extends HoodieClientTestBase {
|
||||
SparkTaskContextSupplier supplier = new SparkTaskContextSupplier();
|
||||
HoodieFileWriter<IndexedRecord> parquetWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
|
||||
parquetPath, table, cfg, HoodieTestDataGenerator.AVRO_SCHEMA, supplier);
|
||||
assertTrue(parquetWriter instanceof HoodieParquetWriter);
|
||||
assertTrue(parquetWriter instanceof HoodieAvroParquetWriter);
|
||||
|
||||
// hfile format.
|
||||
final Path hfilePath = new Path(basePath + "/partition/path/f1_1-0-1_000.hfile");
|
||||
|
||||
@@ -419,7 +419,7 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
|
||||
List<HoodieRecord> records = new ArrayList<>();
|
||||
// Approx 1150 records are written for block size of 64KB
|
||||
for (int i = 0; i < 2000; i++) {
|
||||
for (int i = 0; i < 2050; i++) {
|
||||
String recordStr = "{\"_row_key\":\"" + UUID.randomUUID().toString()
|
||||
+ "\",\"time\":\"2016-01-31T03:16:41.415Z\",\"number\":" + i + "}";
|
||||
RawTripTestPayload rowChange = new RawTripTestPayload(recordStr);
|
||||
@@ -441,7 +441,8 @@ public class TestCopyOnWriteActionExecutor extends HoodieClientTestBase {
|
||||
counts++;
|
||||
}
|
||||
}
|
||||
assertEquals(5, counts, "If the number of records are more than 1150, then there should be a new file");
|
||||
// we check canWrite only once every 1000 records. and so 2 files with 1000 records and 3rd file with 50 records.
|
||||
assertEquals(3, counts, "If the number of records are more than 1150, then there should be a new file");
|
||||
}
|
||||
|
||||
@Test
|
||||
|
||||
Reference in New Issue
Block a user