xiarixiaoyao commented on a change in pull request #4020:
URL: https://github.com/apache/hudi/pull/4020#discussion_r751863440
##########
File path:
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieHFileReader.java
##########
@@ -72,56 +75,58 @@
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(getFs(path.toString(), configuration),
path, cacheConfig, true, conf);
}
public HoodieHFileReader(Configuration configuration, Path path, CacheConfig
cacheConfig, FileSystem inlineFs) throws IOException {
this.conf = configuration;
this.path = path;
this.fsDataInputStream = inlineFs.open(path);
- this.reader = HFile.createReader(inlineFs, path, cacheConfig,
configuration);
+ this.reader = HFile.createReader(inlineFs, 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);
+ ReaderContext context = new ReaderContextBuilder()
+ .withFilePath(path)
+ .withInputStreamWrapper(stream)
+ .withFileSize(getFs("hoodie", conf).getFileStatus(path).getLen())
+ .withFileSystem(stream.getHfs())
+ .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();
+ return new String[] { new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
+ new String(fileInfo.get(KEY_MAX_RECORD.getBytes()))};
}
@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);
- }
+ HFileInfo fileInfo = reader.getHFileInfo();
+ schema = new Schema.Parser().parse(new
String(fileInfo.get(KEY_SCHEMA.getBytes())));
}
return schema;
}
@Override
public BloomFilter readBloomFilter() {
- Map<byte[], byte[]> fileInfo;
+ HFileInfo fileInfo;
try {
- fileInfo = reader.loadFileInfo();
- ByteBuffer serializedFilter =
reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false);
+ fileInfo = reader.getHFileInfo();
Review comment:
A little worried,This method does not exist even in hbase2.2.3
--
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]