prashantwason commented on a change in pull request #1804:
URL: https://github.com/apache/hudi/pull/1804#discussion_r458355188



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReader.java
##########
@@ -34,7 +35,17 @@
 
   public Set<String> filterRowKeys(Set<String> candidateRowKeys);
 
-  public Iterator<R> getRecordIterator(Schema schema) throws IOException;
+  public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
+
+  default Iterator<R> getRecordIterator() throws IOException {
+    return getRecordIterator(getSchema());
+  }
+
+  public Option<R> getRecordByKey(String key, Schema readerSchema) throws 
IOException;

Review comment:
       We can introduce that config but how will it be used? 
   
   I feel the lookup by key is only useful for internal features (like RFC-15 
or RFC-08) rather than a generic API for HUDI. HUDI record keys tend to be 
UUDIs which are large and looking them up is not a common usecase.
   
   
    

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.io.storage;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements 
HoodieFileReader {
+  private static final Logger LOG = 
LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  public static final String KEY_SCHEMA = "schema";
+  public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
+  public static final String KEY_BLOOM_FILTER_TYPE_CODE = 
"bloomFilterTypeCode";
+  public static final String KEY_MIN_RECORD = "minRecordKey";
+  public static final String KEY_MAX_RECORD = "maxRecordKey";
+
+  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);
+  }
+
+  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);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;

Review comment:
       Updated.

##########
File path: 
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.io.storage;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PositionedReadable;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
+import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+import org.apache.hadoop.hbase.io.hfile.HFile;
+import org.apache.hadoop.hbase.io.hfile.HFileScanner;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.bloom.BloomFilterFactory;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+public class HoodieHFileReader<R extends IndexedRecord> implements 
HoodieFileReader {
+  private static final Logger LOG = 
LogManager.getLogger(HoodieHFileReader.class);
+  private Path path;
+  private Configuration conf;
+  private HFile.Reader reader;
+  private Schema schema;
+
+  public static final String KEY_SCHEMA = "schema";
+  public static final String KEY_BLOOM_FILTER_META_BLOCK = "bloomFilter";
+  public static final String KEY_BLOOM_FILTER_TYPE_CODE = 
"bloomFilterTypeCode";
+  public static final String KEY_MIN_RECORD = "minRecordKey";
+  public static final String KEY_MAX_RECORD = "maxRecordKey";
+
+  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);
+  }
+
+  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);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      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);
+    }
+  }
+
+  @Override
+  public Schema getSchema() {
+    if (schema == null) {
+      try {
+        Map<byte[], byte[]> fileInfo = reader.loadFileInfo();
+        schema = new Schema.Parser().parse(new 
String(fileInfo.get(KEY_SCHEMA.getBytes())));
+      } catch (IOException e) {
+        throw new HoodieException("Could not read schema of file from path", 
e);
+      }
+    }
+
+    return schema;
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    Map<byte[], byte[]> fileInfo;
+    try {
+      fileInfo = reader.loadFileInfo();
+      ByteBuffer serializedFilter = 
reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+      byte[] filterBytes = new byte[serializedFilter.remaining()];
+      serializedFilter.get(filterBytes); // read the bytes that were written
+      return BloomFilterFactory.fromString(new String(filterBytes),
+          new String(fileInfo.get(KEY_BLOOM_FILTER_TYPE_CODE.getBytes())));
+    } catch (IOException e) {
+      throw new HoodieException("Could not read bloom filter from " + path, e);
+    }
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set candidateRowKeys) {

Review comment:
       I have added a comment to capture the potential for optimization.




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

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


Reply via email to