3030
3131import org .apache .avro .Schema ;
3232import org .apache .avro .generic .IndexedRecord ;
33+ import org .apache .hadoop .conf .Configuration ;
3334import org .apache .hadoop .fs .Path ;
3435import org .apache .parquet .avro .AvroSchemaConverter ;
3536
@@ -53,39 +54,42 @@ public static <T extends HoodieRecordPayload, R extends IndexedRecord, I, K, O>
5354 return newParquetFileWriter (instantTime , path , config , schema , hoodieTable , taskContextSupplier , config .populateMetaFields ());
5455 }
5556 if (HFILE .getFileExtension ().equals (extension )) {
56- return newHFileFileWriter (instantTime , path , config , schema , hoodieTable , taskContextSupplier );
57+ return newHFileFileWriter (
58+ instantTime , path , config , schema , hoodieTable .getHadoopConf (), taskContextSupplier );
5759 }
5860 if (ORC .getFileExtension ().equals (extension )) {
59- return newOrcFileWriter (instantTime , path , config , schema , hoodieTable , taskContextSupplier );
61+ return newOrcFileWriter (
62+ instantTime , path , config , schema , hoodieTable .getHadoopConf (), taskContextSupplier );
6063 }
6164 throw new UnsupportedOperationException (extension + " format not supported yet." );
6265 }
6366
6467 private static <T extends HoodieRecordPayload , R extends IndexedRecord > HoodieFileWriter <R > newParquetFileWriter (
6568 String instantTime , Path path , HoodieWriteConfig config , Schema schema , HoodieTable hoodieTable ,
6669 TaskContextSupplier taskContextSupplier , boolean populateMetaFields ) throws IOException {
67- return newParquetFileWriter (instantTime , path , config , schema , hoodieTable , taskContextSupplier , populateMetaFields , populateMetaFields );
70+ return newParquetFileWriter (instantTime , path , config , schema , hoodieTable .getHadoopConf (),
71+ taskContextSupplier , populateMetaFields , populateMetaFields );
6872 }
6973
7074 private static <T extends HoodieRecordPayload , R extends IndexedRecord > HoodieFileWriter <R > newParquetFileWriter (
71- String instantTime , Path path , HoodieWriteConfig config , Schema schema , HoodieTable hoodieTable ,
75+ String instantTime , Path path , HoodieWriteConfig config , Schema schema , Configuration conf ,
7276 TaskContextSupplier taskContextSupplier , boolean populateMetaFields , boolean enableBloomFilter ) throws IOException {
7377 Option <BloomFilter > filter = enableBloomFilter ? Option .of (createBloomFilter (config )) : Option .empty ();
74- HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport (new AvroSchemaConverter (hoodieTable . getHadoopConf () ).convert (schema ), schema , filter );
78+ HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport (new AvroSchemaConverter (conf ).convert (schema ), schema , filter );
7579
7680 HoodieAvroParquetConfig parquetConfig = new HoodieAvroParquetConfig (writeSupport , config .getParquetCompressionCodec (),
7781 config .getParquetBlockSize (), config .getParquetPageSize (), config .getParquetMaxFileSize (),
78- hoodieTable . getHadoopConf () , config .getParquetCompressionRatio (), config .parquetDictionaryEnabled ());
82+ conf , config .getParquetCompressionRatio (), config .parquetDictionaryEnabled ());
7983
8084 return new HoodieParquetWriter <>(instantTime , path , parquetConfig , schema , taskContextSupplier , populateMetaFields );
8185 }
8286
83- private static <T extends HoodieRecordPayload , R extends IndexedRecord > HoodieFileWriter <R > newHFileFileWriter (
84- String instantTime , Path path , HoodieWriteConfig config , Schema schema , HoodieTable hoodieTable ,
87+ static <T extends HoodieRecordPayload , R extends IndexedRecord > HoodieFileWriter <R > newHFileFileWriter (
88+ String instantTime , Path path , HoodieWriteConfig config , Schema schema , Configuration conf ,
8589 TaskContextSupplier taskContextSupplier ) throws IOException {
8690
8791 BloomFilter filter = createBloomFilter (config );
88- HoodieHFileConfig hfileConfig = new HoodieHFileConfig (hoodieTable . getHadoopConf () ,
92+ HoodieHFileConfig hfileConfig = new HoodieHFileConfig (conf ,
8993 config .getHFileCompressionAlgorithm (), config .getHFileBlockSize (), config .getHFileMaxFileSize (),
9094 HoodieHFileReader .KEY_FIELD_NAME , PREFETCH_ON_OPEN , CACHE_DATA_IN_L1 , DROP_BEHIND_CACHE_COMPACTION ,
9195 filter , HFILE_COMPARATOR );
@@ -94,10 +98,10 @@ private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieFi
9498 }
9599
96100 private static <T extends HoodieRecordPayload , R extends IndexedRecord > HoodieFileWriter <R > newOrcFileWriter (
97- String instantTime , Path path , HoodieWriteConfig config , Schema schema , HoodieTable hoodieTable ,
101+ String instantTime , Path path , HoodieWriteConfig config , Schema schema , Configuration conf ,
98102 TaskContextSupplier taskContextSupplier ) throws IOException {
99103 BloomFilter filter = createBloomFilter (config );
100- HoodieOrcConfig orcConfig = new HoodieOrcConfig (hoodieTable . getHadoopConf () , config .getOrcCompressionCodec (),
104+ HoodieOrcConfig orcConfig = new HoodieOrcConfig (conf , config .getOrcCompressionCodec (),
101105 config .getOrcStripeSize (), config .getOrcBlockSize (), config .getOrcMaxFileSize (), filter );
102106 return new HoodieOrcWriter <>(instantTime , path , orcConfig , schema , taskContextSupplier );
103107 }
0 commit comments