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


##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java:
##########
@@ -7,205 +7,136 @@
  * "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
+ *   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.
+ * 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 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.model.HoodieAvroIndexedRecord;
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.model.HoodieRecordLocation;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.VisibleForTesting;
 import org.apache.hudi.common.util.collection.ClosableIterator;
 import org.apache.hudi.common.util.collection.CloseableMappingIterator;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
 import org.apache.hudi.exception.HoodieException;
 import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.hfile.HFileReader;
+import org.apache.hudi.io.hfile.HFileReaderImpl;
+import org.apache.hudi.io.hfile.KeyValue;
+import org.apache.hudi.io.hfile.UTF8StringKey;
 import org.apache.hudi.util.Lazy;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
 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.hfile.CacheConfig;
-import org.apache.hadoop.hbase.io.hfile.HFile;
-import org.apache.hadoop.hbase.io.hfile.HFileInfo;
-import org.apache.hadoop.hbase.io.hfile.HFileScanner;
-import org.apache.hadoop.hbase.nio.ByteBuff;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.nio.ByteBuffer;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Objects;
 import java.util.Set;
-import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.stream.Collectors;
 
-import static org.apache.hudi.common.util.CollectionUtils.toStream;
-import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes;
+import static org.apache.hudi.common.util.StringUtils.getStringFromUTF8Bytes;
 import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
+import static org.apache.hudi.io.hfile.HFileUtils.isPrefixOfKey;
 
 /**
- * NOTE: PLEASE READ DOCS & COMMENTS CAREFULLY BEFORE MAKING CHANGES
- * <p>
- * {@link HoodieFileReader} implementation allowing to read from {@link HFile}.
+ * An implementation of {@link BaseHoodieAvroHFileReader} using built-in 
{@link HFileReader}.
  */
-public class HoodieAvroHFileReader extends HoodieAvroFileReaderBase implements 
HoodieSeekingFileReader<IndexedRecord> {
-
-  // TODO HoodieHFileReader right now tightly coupled to MT, we should break 
that coupling
-  public static final String SCHEMA_KEY = "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_FIELD_NAME = "key";
-  public static final String KEY_MIN_RECORD = "minRecordKey";
-  public static final String KEY_MAX_RECORD = "maxRecordKey";
-
+public class HoodieAvroHFileReader extends BaseHoodieAvroHFileReader {
   private static final Logger LOG = 
LoggerFactory.getLogger(HoodieAvroHFileReader.class);
 
-  private final Path path;
-  private final FileSystem fs;
-  private final Configuration hadoopConf;
-  private final CacheConfig config;
-  private final Option<byte[]> content;
+  private final Configuration conf;
+  private final Option<Path> path;
+  private final Option<byte[]> bytesContent;
+  private Option<HFileReader> sharedHFileReader;
   private final Lazy<Schema> schema;
 
-  // NOTE: Reader is ONLY THREAD-SAFE for {@code Scanner} operating in 
Positional Read ("pread")
-  //       mode (ie created w/ "pread = true")
-  // Common reader is not used for the iterators since they can be closed 
independently.
-  // Use {@link getSharedReader()} instead of accessing directly.
-  private Option<HFile.Reader> sharedReader;
-  // NOTE: Scanner caches read blocks, therefore it's important to re-use 
scanner
-  //       wherever possible
-  private Option<HFileScanner> sharedScanner;
-
-  private final Object sharedLock = new Object();
-
-  public HoodieAvroHFileReader(Configuration hadoopConf, Path path, 
CacheConfig cacheConfig) throws IOException {
-    this(path, FSUtils.getFs(path.toString(), hadoopConf), hadoopConf, 
cacheConfig, Option.empty());
-  }
-
-  public HoodieAvroHFileReader(Configuration hadoopConf, Path path, 
CacheConfig cacheConfig, FileSystem fs, Option<Schema> schemaOpt) throws 
IOException {
-    this(path, fs, hadoopConf, cacheConfig, schemaOpt);
-  }
-
-  public HoodieAvroHFileReader(Configuration hadoopConf, Path path, 
CacheConfig cacheConfig, FileSystem fs, byte[] content, Option<Schema> 
schemaOpt) throws IOException {
-    this(path, fs, hadoopConf, cacheConfig, schemaOpt, Option.of(content));
-  }
-
-  public HoodieAvroHFileReader(Path path, FileSystem fs, Configuration 
hadoopConf, CacheConfig config, Option<Schema> schemaOpt) throws IOException {
-    this(path, fs, hadoopConf, config, schemaOpt, Option.empty());
+  public HoodieAvroHFileReader(Configuration conf, Path path, Option<Schema> 
schemaOption) {
+    this.conf = conf;
+    this.path = Option.of(path);
+    this.bytesContent = Option.empty();
+    this.sharedHFileReader = Option.empty();
+    this.schema = schemaOption.map(Lazy::eagerly)
+        .orElseGet(() -> Lazy.lazily(() -> 
fetchSchema(getSharedHFileReader())));
   }
 
-  public HoodieAvroHFileReader(Path path, FileSystem fs, Configuration 
hadoopConf, CacheConfig config, Option<Schema> schemaOpt, Option<byte[]> 
content) throws IOException {
-    this.path = path;
-    this.fs = fs;
-    this.hadoopConf = hadoopConf;
-    this.config = config;
-    this.content = content;
-
-    // Shared reader is instantiated lazily.
-    this.sharedReader = Option.empty();
-    this.sharedScanner = Option.empty();
-    this.schema = schemaOpt.map(Lazy::eagerly)
+  public HoodieAvroHFileReader(Configuration conf, byte[] content, 
Option<Schema> schemaOption) {

Review Comment:
   Makes sense.  Renamed.



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