yihua commented on code in PR #12866:
URL: https://github.com/apache/hudi/pull/12866#discussion_r2105296367


##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -101,25 +104,25 @@ public HoodieAvroHFileWriter(String instantTime, 
StoragePath file, HoodieHFileCo
     this.taskContextSupplier = taskContextSupplier;
     this.populateMetaFields = populateMetaFields;
 
-    HFileContext context = new 
HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
-        .withCompression(hfileConfig.getCompressionAlgorithm())
-        .withCellComparator(hfileConfig.getHFileComparator())
+    HFileContext context = new HFileContext.Builder()
+        .blockSize(hfileConfig.getBlockSize())
+        .compressionCodec(hfileConfig.getCompressionCodec())
         .build();
 
     conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY,
         String.valueOf(hfileConfig.shouldPrefetchBlocksOnOpen()));
     conf.set(HColumnDescriptor.CACHE_DATA_IN_L1, 
String.valueOf(hfileConfig.shouldCacheDataInL1()));
     conf.set(DROP_BEHIND_CACHE_COMPACTION_KEY,
         String.valueOf(hfileConfig.shouldDropBehindCacheCompaction()));
-    CacheConfig cacheConfig = new CacheConfig(conf);
-    this.writer = HFile.getWriterFactory(conf, cacheConfig)
-        .withPath(fs, this.file)
-        .withFileContext(context)
-        .create();
-
-    
writer.appendFileInfo(getUTF8Bytes(HoodieAvroHFileReaderImplBase.SCHEMA_KEY),
-        getUTF8Bytes(schema.toString()));
+
+    StorageConfiguration<Configuration> storageConf = new 
HadoopStorageConfiguration(conf);
+    StoragePath filePath = new StoragePath(this.file.toUri());

Review Comment:
   Let's get rid of `Path` usage in `this.file` which is no longer required.  
Use `StoragePath` to store `filePath` as a member variable.



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -101,25 +104,25 @@ public HoodieAvroHFileWriter(String instantTime, 
StoragePath file, HoodieHFileCo
     this.taskContextSupplier = taskContextSupplier;
     this.populateMetaFields = populateMetaFields;
 
-    HFileContext context = new 
HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
-        .withCompression(hfileConfig.getCompressionAlgorithm())
-        .withCellComparator(hfileConfig.getHFileComparator())
+    HFileContext context = new HFileContext.Builder()
+        .blockSize(hfileConfig.getBlockSize())
+        .compressionCodec(hfileConfig.getCompressionCodec())
         .build();
 
     conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY,

Review Comment:
   Could we get rid of these configs, which are HBase configs only and do not 
take effect anymore?  Let's also track a ticket to revisit these optimizations 
(whether we should implement similar optimizations in the HFile writer).



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -101,25 +104,25 @@ public HoodieAvroHFileWriter(String instantTime, 
StoragePath file, HoodieHFileCo
     this.taskContextSupplier = taskContextSupplier;
     this.populateMetaFields = populateMetaFields;
 
-    HFileContext context = new 
HFileContextBuilder().withBlockSize(hfileConfig.getBlockSize())
-        .withCompression(hfileConfig.getCompressionAlgorithm())
-        .withCellComparator(hfileConfig.getHFileComparator())
+    HFileContext context = new HFileContext.Builder()
+        .blockSize(hfileConfig.getBlockSize())
+        .compressionCodec(hfileConfig.getCompressionCodec())
         .build();
 
     conf.set(CacheConfig.PREFETCH_BLOCKS_ON_OPEN_KEY,
         String.valueOf(hfileConfig.shouldPrefetchBlocksOnOpen()));
     conf.set(HColumnDescriptor.CACHE_DATA_IN_L1, 
String.valueOf(hfileConfig.shouldCacheDataInL1()));
     conf.set(DROP_BEHIND_CACHE_COMPACTION_KEY,
         String.valueOf(hfileConfig.shouldDropBehindCacheCompaction()));
-    CacheConfig cacheConfig = new CacheConfig(conf);
-    this.writer = HFile.getWriterFactory(conf, cacheConfig)
-        .withPath(fs, this.file)
-        .withFileContext(context)
-        .create();
-
-    
writer.appendFileInfo(getUTF8Bytes(HoodieAvroHFileReaderImplBase.SCHEMA_KEY),
-        getUTF8Bytes(schema.toString()));
+
+    StorageConfiguration<Configuration> storageConf = new 
HadoopStorageConfiguration(conf);
+    StoragePath filePath = new StoragePath(this.file.toUri());

Review Comment:
   Also, let's check if the FS-related operation in the constructor is needed. 



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -38,16 +44,12 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;

Review Comment:
   If it's easy, do it in this PR.



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieHFileConfig.java:
##########
@@ -47,12 +47,12 @@ public class HoodieHFileConfig {
   private final CellComparator hfileComparator;
   private final String keyFieldName;
 
-  public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm 
compressionAlgorithm, int blockSize,
-                           long maxFileSize, String keyFieldName,
+  public HoodieHFileConfig(StorageConfiguration storageConf, CompressionCodec 
compressionCodec,
+                           int blockSize, long maxFileSize, String 
keyFieldName,
                            boolean prefetchBlocksOnOpen, boolean 
cacheDataInL1, boolean dropBehindCacheCompaction,
                            BloomFilter bloomFilter, CellComparator 
hfileComparator) {
-    this.hadoopConf = hadoopConf;
-    this.compressionAlgorithm = compressionAlgorithm;
+    this.hadoopConf = (Configuration) 
storageConf.unwrapAs(Configuration.class);

Review Comment:
   This class can directly store `storageConf` now instead of converting to 
`hadoopConf`?



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieHFileConfig.java:
##########
@@ -47,12 +47,12 @@ public class HoodieHFileConfig {
   private final CellComparator hfileComparator;
   private final String keyFieldName;
 
-  public HoodieHFileConfig(Configuration hadoopConf, Compression.Algorithm 
compressionAlgorithm, int blockSize,
-                           long maxFileSize, String keyFieldName,
+  public HoodieHFileConfig(StorageConfiguration storageConf, CompressionCodec 
compressionCodec,

Review Comment:
   Similarly, see if we can get rid of all HBase class usage in this class.



##########
hudi-hadoop-common/src/main/java/org/apache/hudi/io/hadoop/HoodieAvroHFileWriter.java:
##########
@@ -38,16 +44,12 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileContext;
-import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;

Review Comment:
   Let's file a ticket to get rid of other Hadoop dependencies in this class 
and move the class to `hudi-common` module.



-- 
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]

Reply via email to