1818
1919package org .apache .hudi .io .storage ;
2020
21+ import org .apache .hadoop .conf .Configuration ;
2122import org .apache .hadoop .fs .Path ;
23+
2224import org .apache .hudi .common .fs .FSUtils ;
2325import org .apache .hudi .common .fs .HoodieWrapperFileSystem ;
2426import org .apache .hudi .common .util .VisibleForTesting ;
27+
28+ import org .apache .parquet .column .ParquetProperties ;
2529import org .apache .parquet .hadoop .ParquetFileWriter ;
2630import org .apache .parquet .hadoop .ParquetWriter ;
2731import org .apache .parquet .hadoop .api .WriteSupport ;
2832
33+ import java .io .Closeable ;
2934import java .io .IOException ;
35+ import java .lang .reflect .InvocationTargetException ;
3036import java .util .concurrent .atomic .AtomicLong ;
3137
32- import static org .apache .parquet .column .ParquetProperties .DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK ;
33- import static org .apache .parquet .column .ParquetProperties .DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK ;
38+ import java .lang .reflect .Method ;
3439
3540/**
3641 * Base class of Hudi's custom {@link ParquetWriter} implementations
3742 *
3843 * @param <R> target type of the object being written into Parquet files (for ex,
39- * {@code IndexedRecord}, {@code InternalRow})
44+ * {@code IndexedRecord}, {@code InternalRow})
4045 */
41- public abstract class HoodieBaseParquetWriter <R > extends ParquetWriter < R > {
46+ public abstract class HoodieBaseParquetWriter <R > implements Closeable {
4247
4348 private final AtomicLong writtenRecordCount = new AtomicLong (0 );
4449 private final long maxFileSize ;
4550 private long recordCountForNextSizeCheck ;
51+ private final ParquetWriter parquetWriter ;
52+ public static final String BLOOM_FILTER_EXPECTED_NDV = "parquet.bloom.filter.expected.ndv" ;
53+ public static final String BLOOM_FILTER_ENABLED = "parquet.bloom.filter.enabled" ;
4654
4755 public HoodieBaseParquetWriter (Path file ,
4856 HoodieParquetConfig <? extends WriteSupport <R >> parquetConfig ) throws IOException {
49- super (HoodieWrapperFileSystem .convertToHoodiePath (file , parquetConfig .getHadoopConf ()),
50- ParquetFileWriter .Mode .CREATE ,
51- parquetConfig .getWriteSupport (),
52- parquetConfig .getCompressionCodecName (),
53- parquetConfig .getBlockSize (),
54- parquetConfig .getPageSize (),
55- parquetConfig .getPageSize (),
56- parquetConfig .dictionaryEnabled (),
57- DEFAULT_IS_VALIDATING_ENABLED ,
58- DEFAULT_WRITER_VERSION ,
59- FSUtils .registerFileSystem (file , parquetConfig .getHadoopConf ()));
57+ ParquetWriter .Builder parquetWriterbuilder = new ParquetWriter .Builder (HoodieWrapperFileSystem .convertToHoodiePath (file , parquetConfig .getHadoopConf ())) {
58+ @ Override
59+ protected ParquetWriter .Builder self () {
60+ return this ;
61+ }
62+
63+ @ Override
64+ protected WriteSupport getWriteSupport (Configuration conf ) {
65+ return parquetConfig .getWriteSupport ();
66+ }
67+ };
68+
69+ parquetWriterbuilder .withWriteMode (ParquetFileWriter .Mode .CREATE );
70+ parquetWriterbuilder .withCompressionCodec (parquetConfig .getCompressionCodecName ());
71+ parquetWriterbuilder .withRowGroupSize (parquetConfig .getBlockSize ());
72+ parquetWriterbuilder .withPageSize (parquetConfig .getPageSize ());
73+ parquetWriterbuilder .withDictionaryPageSize (parquetConfig .getPageSize ());
74+ parquetWriterbuilder .withDictionaryEncoding (parquetConfig .dictionaryEnabled ());
75+ parquetWriterbuilder .withValidation (ParquetWriter .DEFAULT_IS_VALIDATING_ENABLED );
76+ parquetWriterbuilder .withWriterVersion (ParquetWriter .DEFAULT_WRITER_VERSION );
77+ parquetWriterbuilder .withConf (FSUtils .registerFileSystem (file , parquetConfig .getHadoopConf ()));
78+ handleParquetBloomFilters (parquetWriterbuilder , parquetConfig .getHadoopConf ());
6079
80+ parquetWriter = parquetWriterbuilder .build ();
6181 // We cannot accurately measure the snappy compressed output file size. We are choosing a
6282 // conservative 10%
6383 // TODO - compute this compression ratio dynamically by looking at the bytes written to the
6484 // stream and the actual file size reported by HDFS
6585 this .maxFileSize = parquetConfig .getMaxFileSize ()
6686 + Math .round (parquetConfig .getMaxFileSize () * parquetConfig .getCompressionRatio ());
67- this .recordCountForNextSizeCheck = DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK ;
87+ this .recordCountForNextSizeCheck = ParquetProperties .DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK ;
88+ }
89+
90+ protected void handleParquetBloomFilters (ParquetWriter .Builder parquetWriterbuilder , Configuration hadoopConf ) {
91+ // inspired from https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java#L458-L464
92+ hadoopConf .forEach (conf -> {
93+ String key = conf .getKey ();
94+ if (key .startsWith (BLOOM_FILTER_ENABLED )) {
95+ String column = key .substring (BLOOM_FILTER_ENABLED .length () + 1 , key .length ());
96+ try {
97+ Method method = parquetWriterbuilder .getClass ().getDeclaredMethod ("withBloomFilterEnabled" );
98+ method .invoke (column , Boolean .valueOf (conf .getValue ()));
99+ } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e ) {
100+ // skip
101+ }
102+ }
103+ if (key .startsWith (BLOOM_FILTER_EXPECTED_NDV )) {
104+ String column = key .substring (BLOOM_FILTER_EXPECTED_NDV .length () + 1 , key .length ());
105+ try {
106+ Method method = parquetWriterbuilder .getClass ().getDeclaredMethod ("withBloomFilterNDV" );
107+ method .invoke (column , Long .valueOf (conf .getValue (), -1 ));
108+ } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e ) {
109+ // skip
110+ }
111+ }
112+ });
68113 }
69114
70115 public boolean canWrite () {
@@ -82,15 +127,18 @@ public boolean canWrite() {
82127 }
83128 recordCountForNextSizeCheck = writtenCount + Math .min (
84129 // Do check it in the halfway
85- Math .max (DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK , (maxFileSize / avgRecordSize - writtenCount ) / 2 ),
86- DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK );
130+ Math .max (ParquetProperties . DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK , (maxFileSize / avgRecordSize - writtenCount ) / 2 ),
131+ ParquetProperties . DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK );
87132 }
88133 return true ;
89134 }
90135
91- @ Override
136+ public long getDataSize () {
137+ return this .parquetWriter .getDataSize ();
138+ }
139+
92140 public void write (R object ) throws IOException {
93- super .write (object );
141+ this . parquetWriter .write (object );
94142 writtenRecordCount .incrementAndGet ();
95143 }
96144
@@ -102,4 +150,9 @@ protected long getWrittenRecordCount() {
102150 protected long getRecordCountForNextSizeCheck () {
103151 return recordCountForNextSizeCheck ;
104152 }
153+
154+ @ Override
155+ public void close () throws IOException {
156+ this .parquetWriter .close ();
157+ }
105158}
0 commit comments