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



##########
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:
       I need to check that.  This is another compatibility test.

##########
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:
       Make sense.  I created a new class, `HoodieHFileUtils`, to hold the 
default `USE_PRIMARY_REPLICA_READER` as true, and hide the usage with 
`HoodieHFileUtils::createHFileReader`.

##########
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:
       Yeah, that's good to have.  The problem as @codope pointed out is that 
such a process is time-consuming.  For now, what I can say is that the newly 
added artifacts are necessary, since I started with the old pom, incrementally 
added new artifacts as I saw NoClassDef exception until every test can pass.
   
   One thing we may try later is to add and trim hudi-hbase-shaded by excluding 
transitives and only depend on hudi-hbase-shaded here.

##########
File path: 
hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java
##########
@@ -121,21 +123,24 @@ public void testSimpleInlineFileSystem() throws 
IOException {
 
     Set<Integer> rowIdsToSearch = getRandomValidRowIds(10);
     for (int rowId : rowIdsToSearch) {
-      assertEquals(0, 
scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(rowId))),
+      KeyValue keyValue = new KeyValue.KeyOnlyKeyValue(getSomeKey(rowId));
+      assertEquals(0, scanner.seekTo(keyValue),
           "location lookup failed");
       // read the key and see if it matches
-      ByteBuffer readKey = scanner.getKey();
-      assertArrayEquals(getSomeKey(rowId), Bytes.toBytes(readKey), "seeked key 
does not match");
-      scanner.seekTo(KeyValue.createKeyValueFromKey(getSomeKey(rowId)));
+      Cell cell = scanner.getCell();
+      byte[] key = Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), 
cell.getRowOffset() + cell.getRowLength());
+      assertArrayEquals(Arrays.copyOfRange(keyValue.getRowArray(), 
keyValue.getRowOffset(), keyValue.getRowOffset() + keyValue.getRowLength()), 
key,

Review comment:
       Fixed.

##########
File path: 
hudi-common/src/test/java/org/apache/hudi/common/fs/inline/TestInLineFileSystemHFileInLining.java
##########
@@ -56,11 +58,12 @@
  */
 public class TestInLineFileSystemHFileInLining {
 
+  private static final String LOCAL_FORMATTER = "%010d";
+  private static final String VALUE_PREFIX = "value";
+  private static final int MIN_BLOCK_SIZE = 1024;

Review comment:
       Fixed.

##########
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);

Review comment:
       Makes sense, I use either basePath or log file path now based on what's 
available.

##########
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:
       There are classes from `hbase-hadoop2-compat` and `hbase-hadoop-compat` 
implementof `org.apache.hadoop.metrics2` classes in `hadoop-common`.  We cannot 
shade all classes in `org.apache.hadoop.metrics2` which cause NoClassDef of 
hadoop classes.  So have to pick and choose here for HBase specific ones.

##########
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:
       Yes, we can have `hudi-hbase-shaded` as a separate module.  I'll see if 
this can be done soon; if not, I'll take it as a follow-up.

##########
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);

Review comment:
       Yes, I added `HoodieHFileUtils::createHFileReader` for this.

##########
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:
       Generally, we should wrap it with HoodieException.

##########
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:
       Yeah, the new API `getHFileInfo()` returns the stored file info directly 
and there is no exception we can catch here now.
   ```
   HFileReaderImpl
     @Override
     public HFileInfo getHFileInfo() {
       return this.fileInfo;
     }
   ```

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -250,7 +259,7 @@ public BloomFilter readBloomFilter() {
    */
   public List<Pair<String, R>> readRecords(List<String> keys, Schema schema) 
throws IOException {
     this.schema = schema;
-    reader.loadFileInfo();
+    reader.getHFileInfo();

Review comment:
       Looks like we can remove this.  After digging more, even for HBase 
1.2.3, loadFileInfo() returns cached `fileInfo` object.




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