You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/03/15 23:33:30 UTC

[GitHub] [hudi] vinothchandar commented on a change in pull request #5004: [HUDI-1180] Upgrade HBase to 2.4.9

vinothchandar commented on a change in pull request #5004:
URL: https://github.com/apache/hudi/pull/5004#discussion_r827483332



##########
File path: docker/hoodie/hadoop/pom.xml
##########
@@ -57,7 +57,7 @@
     <docker.spark.version>2.4.4</docker.spark.version>
     <docker.hive.version>2.3.3</docker.hive.version>
     <docker.hadoop.version>2.8.4</docker.hadoop.version>
-    <docker.presto.version>0.268</docker.presto.version>
+    <docker.presto.version>0.271</docker.presto.version>

Review comment:
       why is this needed for this PR

##########
File path: packaging/hudi-flink-bundle/pom.xml
##########
@@ -191,6 +216,74 @@
                   <pattern>com.fasterxml.jackson.</pattern>
                   <shadedPattern>${flink.bundle.shade.prefix}com.fasterxml.jackson.</shadedPattern>
                 </relocation>
+                <!-- The classes below in org.apache.hadoop.metrics2 package come from

Review comment:
       why

##########
File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileConfig.java
##########
@@ -42,12 +42,12 @@
   private final boolean dropBehindCacheCompaction;
   private final Configuration hadoopConf;
   private final BloomFilter bloomFilter;
-  private final KeyValue.KVComparator hfileComparator;
+  private final CellComparator hfileComparator;

Review comment:
       are these backwards and forward compatible . i.e KVComparator is written into the HFile footer?

##########
File path: hudi-common/pom.xml
##########
@@ -221,14 +221,13 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-client</artifactId>
       <version>${hbase.version}</version>
-      <scope>test</scope>

Review comment:
       this may be ok per se, given the hbase server is 2.x. it may no longer work against a HBase 1.x server and we should call this out in the release notes

##########
File path: packaging/hudi-flink-bundle/pom.xml
##########
@@ -162,6 +171,22 @@
                   <pattern>org.apache.avro.</pattern>
                   <shadedPattern>${flink.bundle.shade.prefix}org.apache.avro.</shadedPattern>
                 </relocation>
+                <relocation>
+                  <pattern>org.apache.commons.io.</pattern>

Review comment:
       this reminds me. older bootstrap index files may have an unshaded key comparator class saved within the HFile. Does that cause any issues? ie can we read such files? 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -80,45 +85,49 @@
   public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
     this.conf = configuration;
     this.path = path;
-    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, true, conf);
   }
 
   public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException {
     this.conf = configuration;
     this.path = path;
     this.fsDataInputStream = fs.open(path);
-    this.reader = HFile.createReader(fs, path, cacheConfig, configuration);
+    this.reader = HFile.createReader(fs, path, cacheConfig, true, configuration);

Review comment:
       pull this `true` into a constant explaining why its being used everywhere?

##########
File path: packaging/hudi-flink-bundle/pom.xml
##########
@@ -147,10 +148,18 @@
 
                   <include>org.apache.hbase:hbase-common</include>
                   <include>org.apache.hbase:hbase-client</include>
+                  <include>org.apache.hbase:hbase-hadoop-compat</include>

Review comment:
       is the absolute minimal set of artifacts needed 

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -80,45 +85,49 @@
   public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
     this.conf = configuration;
     this.path = path;
-    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, true, conf);
   }
 
   public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException {
     this.conf = configuration;
     this.path = path;
     this.fsDataInputStream = fs.open(path);
-    this.reader = HFile.createReader(fs, path, cacheConfig, configuration);
+    this.reader = HFile.createReader(fs, path, cacheConfig, true, configuration);
   }
 
   public HoodieHFileReader(byte[] content) throws IOException {
     Configuration conf = new Configuration();
     Path path = new Path("hoodie");
     SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
     FSDataInputStream fsdis = new FSDataInputStream(bis);
-    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
-        content.length, new CacheConfig(conf), conf);
+    FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis);
+    FileSystem fs = FSUtils.getFs("hoodie", conf);
+    HFileSystem hfs = (fs instanceof HFileSystem) ? (HFileSystem) fs : new HFileSystem(fs);
+    ReaderContext context = new ReaderContextBuilder()
+        .withFilePath(path)
+        .withInputStreamWrapper(stream)
+        .withFileSize(content.length)
+        .withFileSystem(hfs)
+        .withPrimaryReplicaReader(true)
+        .withReaderType(ReaderContext.ReaderType.STREAM)
+        .build();
+    HFileInfo fileInfo = new HFileInfo(context, conf);
+    this.reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf);
+    fileInfo.initMetaAndIndex(reader);
   }
 
   @Override
   public String[] readMinMaxRecordKeys() {
-    try {
-      Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
-      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
-          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
-    } catch (IOException e) {
-      throw new HoodieException("Could not read min/max record key out of file information block correctly from path", e);
-    }
+    HFileInfo fileInfo = reader.getHFileInfo();

Review comment:
       Can we UT this?

##########
File path: packaging/hudi-hadoop-mr-bundle/pom.xml
##########
@@ -110,6 +135,74 @@
                   <pattern>com.google.common.</pattern>
                   <shadedPattern>org.apache.hudi.com.google.common.</shadedPattern>
                 </relocation>
+                <!-- The classes below in org.apache.hadoop.metrics2 package come from

Review comment:
       I assume all of this is repeated? does maven offer a way to reuse the include? I am wondering if we can build a `hudi-hbase-shaded` package and simply include that everywhere. will be easier to maintain?

##########
File path: hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -80,45 +85,49 @@
   public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
     this.conf = configuration;
     this.path = path;
-    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
+    this.reader = HFile.createReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, true, conf);
   }
 
   public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException {
     this.conf = configuration;
     this.path = path;
     this.fsDataInputStream = fs.open(path);
-    this.reader = HFile.createReader(fs, path, cacheConfig, configuration);
+    this.reader = HFile.createReader(fs, path, cacheConfig, true, configuration);
   }
 
   public HoodieHFileReader(byte[] content) throws IOException {
     Configuration conf = new Configuration();
     Path path = new Path("hoodie");
     SeekableByteArrayInputStream bis = new SeekableByteArrayInputStream(content);
     FSDataInputStream fsdis = new FSDataInputStream(bis);
-    this.reader = HFile.createReader(FSUtils.getFs("hoodie", conf), path, new FSDataInputStreamWrapper(fsdis),
-        content.length, new CacheConfig(conf), conf);
+    FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis);
+    FileSystem fs = FSUtils.getFs("hoodie", conf);
+    HFileSystem hfs = (fs instanceof HFileSystem) ? (HFileSystem) fs : new HFileSystem(fs);
+    ReaderContext context = new ReaderContextBuilder()
+        .withFilePath(path)
+        .withInputStreamWrapper(stream)
+        .withFileSize(content.length)
+        .withFileSystem(hfs)
+        .withPrimaryReplicaReader(true)
+        .withReaderType(ReaderContext.ReaderType.STREAM)
+        .build();
+    HFileInfo fileInfo = new HFileInfo(context, conf);
+    this.reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf);
+    fileInfo.initMetaAndIndex(reader);
   }
 
   @Override
   public String[] readMinMaxRecordKeys() {
-    try {
-      Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
-      return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
-          new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
-    } catch (IOException e) {

Review comment:
       Good practice to wrap this in a HoodieException no?




-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org