parisni commented on code in PR #8716:
URL: https://github.com/apache/hudi/pull/8716#discussion_r1197707302
##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieBaseParquetWriter.java:
##########
@@ -18,46 +18,68 @@
package org.apache.hudi.io.storage;
+import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.util.VisibleForTesting;
+
import org.apache.parquet.hadoop.ParquetFileWriter;
+import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.WriteSupport;
+import java.io.Closeable;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicLong;
import static
org.apache.parquet.column.ParquetProperties.DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK;
import static
org.apache.parquet.column.ParquetProperties.DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK;
+import static
org.apache.parquet.hadoop.ParquetOutputFormat.BLOOM_FILTER_ENABLED;
+import static
org.apache.parquet.hadoop.ParquetOutputFormat.BLOOM_FILTER_EXPECTED_NDV;
+import static
org.apache.parquet.hadoop.ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED;
+import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_WRITER_VERSION;
/**
* Base class of Hudi's custom {@link ParquetWriter} implementations
*
* @param <R> target type of the object being written into Parquet files (for
ex,
- * {@code IndexedRecord}, {@code InternalRow})
+ * {@code IndexedRecord}, {@code InternalRow})
*/
-public abstract class HoodieBaseParquetWriter<R> extends ParquetWriter<R> {
+public abstract class HoodieBaseParquetWriter<R> implements Closeable {
private final AtomicLong writtenRecordCount = new AtomicLong(0);
private final long maxFileSize;
private long recordCountForNextSizeCheck;
+ private final ParquetWriter parquetWriter;
public HoodieBaseParquetWriter(Path file,
HoodieParquetConfig<? extends
WriteSupport<R>> parquetConfig) throws IOException {
- super(HoodieWrapperFileSystem.convertToHoodiePath(file,
parquetConfig.getHadoopConf()),
- ParquetFileWriter.Mode.CREATE,
- parquetConfig.getWriteSupport(),
- parquetConfig.getCompressionCodecName(),
- parquetConfig.getBlockSize(),
- parquetConfig.getPageSize(),
- parquetConfig.getPageSize(),
- parquetConfig.dictionaryEnabled(),
- DEFAULT_IS_VALIDATING_ENABLED,
- DEFAULT_WRITER_VERSION,
- FSUtils.registerFileSystem(file, parquetConfig.getHadoopConf()));
+ ParquetWriter.Builder parquetWriterbuilder = new
ParquetWriter.Builder(HoodieWrapperFileSystem.convertToHoodiePath(file,
parquetConfig.getHadoopConf())) {
+ @Override
+ protected ParquetWriter.Builder self() {
Review Comment:
"Delta" do you mean MOR logs files configured in parquet format ? If you
mean delta-lake then yes, as well as iceberg they rely spark writer so benefit
from bloom.
By regression do you mean performance regression ? Each column configures
with bloom will introduce overhead at write time, but faster subsequent reads
with predicates on the column. I haven't benchmarked that but I can say blooms
will faster reads significantly by skipping lot of parquet scan that hudi stats
index won't cover. ie uuids, strings, high cardinality dictionaries
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]