[HUDI-1180] Upgrade HBase to 2.4.9 (#5004)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
@@ -33,10 +33,12 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHFileUtils;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
import org.apache.hadoop.hbase.CellUtil;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
@@ -178,9 +180,7 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
private static HFile.Reader createReader(String hFilePath, Configuration conf, FileSystem fileSystem) {
|
||||
try {
|
||||
LOG.info("Opening HFile for reading :" + hFilePath);
|
||||
HFile.Reader reader = HFile.createReader(fileSystem, new HFilePathForReader(hFilePath),
|
||||
new CacheConfig(conf), conf);
|
||||
return reader;
|
||||
return HoodieHFileUtils.createHFileReader(fileSystem, new HFilePathForReader(hFilePath), new CacheConfig(conf), conf);
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
@@ -259,7 +259,7 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
|
||||
private HoodieBootstrapIndexInfo fetchBootstrapIndexInfo() throws IOException {
|
||||
return TimelineMetadataUtils.deserializeAvroMetadata(
|
||||
partitionIndexReader().loadFileInfo().get(INDEX_INFO_KEY),
|
||||
partitionIndexReader().getHFileInfo().get(INDEX_INFO_KEY),
|
||||
HoodieBootstrapIndexInfo.class);
|
||||
}
|
||||
|
||||
@@ -306,7 +306,7 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
try {
|
||||
boolean available = scanner.seekTo();
|
||||
while (available) {
|
||||
keys.add(converter.apply(getUserKeyFromCellKey(CellUtil.getCellKeyAsString(scanner.getKeyValue()))));
|
||||
keys.add(converter.apply(getUserKeyFromCellKey(CellUtil.getCellKeyAsString(scanner.getCell()))));
|
||||
available = scanner.next();
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
@@ -528,13 +528,13 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
@Override
|
||||
public void begin() {
|
||||
try {
|
||||
HFileContext meta = new HFileContextBuilder().build();
|
||||
HFileContext meta = new HFileContextBuilder().withCellComparator(new HoodieKVComparator()).build();
|
||||
this.indexByPartitionWriter = HFile.getWriterFactory(metaClient.getHadoopConf(),
|
||||
new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByPartitionPath)
|
||||
.withFileContext(meta).withComparator(new HoodieKVComparator()).create();
|
||||
.withFileContext(meta).create();
|
||||
this.indexByFileIdWriter = HFile.getWriterFactory(metaClient.getHadoopConf(),
|
||||
new CacheConfig(metaClient.getHadoopConf())).withPath(metaClient.getFs(), indexByFileIdPath)
|
||||
.withFileContext(meta).withComparator(new HoodieKVComparator()).create();
|
||||
.withFileContext(meta).create();
|
||||
} catch (IOException ioe) {
|
||||
throw new HoodieIOException(ioe.getMessage(), ioe);
|
||||
}
|
||||
@@ -581,6 +581,6 @@ public class HFileBootstrapIndex extends BootstrapIndex {
|
||||
* This class is explicitly used as Key Comparator to workaround hard coded
|
||||
* legacy format class names inside HBase. Otherwise we will face issues with shading.
|
||||
*/
|
||||
public static class HoodieKVComparator extends KeyValue.KVComparator {
|
||||
public static class HoodieKVComparator extends CellComparatorImpl {
|
||||
}
|
||||
}
|
||||
|
||||
@@ -424,6 +424,9 @@ public abstract class AbstractHoodieLogRecordReader {
|
||||
processDataBlock((HoodieAvroDataBlock) lastBlock, keys);
|
||||
break;
|
||||
case HFILE_DATA_BLOCK:
|
||||
if (!keys.isPresent()) {
|
||||
keys = Option.of(Collections.emptyList());
|
||||
}
|
||||
processDataBlock((HoodieHFileDataBlock) lastBlock, keys);
|
||||
break;
|
||||
case PARQUET_DATA_BLOCK:
|
||||
|
||||
@@ -208,7 +208,7 @@ public class HoodieLogFileReader implements HoodieLogFormat.Reader {
|
||||
String.format("HFile block could not be of version (%d)", HoodieLogFormatVersion.DEFAULT_VERSION));
|
||||
|
||||
return new HoodieHFileDataBlock(inputStream, content, readBlockLazily, logBlockContentLoc,
|
||||
Option.ofNullable(readerSchema), header, footer, enableRecordLookups);
|
||||
Option.ofNullable(readerSchema), header, footer, enableRecordLookups, logFile.getPath());
|
||||
|
||||
case PARQUET_DATA_BLOCK:
|
||||
checkState(nextBlockVersion.getVersion() != HoodieLogFormatVersion.DEFAULT_VERSION,
|
||||
|
||||
@@ -18,6 +18,18 @@
|
||||
|
||||
package org.apache.hudi.common.table.log.block;
|
||||
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFileSystem;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -30,17 +42,6 @@ import org.apache.hadoop.hbase.io.hfile.CacheConfig;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFile;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileContextBuilder;
|
||||
import org.apache.hudi.avro.HoodieAvroUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFSUtils;
|
||||
import org.apache.hudi.common.fs.inline.InLineFileSystem;
|
||||
import org.apache.hudi.common.util.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.io.storage.HoodieHBaseKVComparator;
|
||||
import org.apache.hudi.io.storage.HoodieHFileReader;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -65,6 +66,9 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
private static final int DEFAULT_BLOCK_SIZE = 1024 * 1024;
|
||||
|
||||
private final Option<Compression.Algorithm> compressionAlgorithm;
|
||||
// This path is used for constructing HFile reader context, which should not be
|
||||
// interpreted as the actual file path for the HFile data blocks
|
||||
private final Path pathForReader;
|
||||
|
||||
public HoodieHFileDataBlock(FSDataInputStream inputStream,
|
||||
Option<byte[]> content,
|
||||
@@ -73,16 +77,20 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
Option<Schema> readerSchema,
|
||||
Map<HeaderMetadataType, String> header,
|
||||
Map<HeaderMetadataType, String> footer,
|
||||
boolean enablePointLookups) {
|
||||
boolean enablePointLookups,
|
||||
Path pathForReader) {
|
||||
super(content, inputStream, readBlockLazily, Option.of(logBlockContentLocation), readerSchema, header, footer, HoodieHFileReader.KEY_FIELD_NAME, enablePointLookups);
|
||||
this.compressionAlgorithm = Option.empty();
|
||||
this.pathForReader = pathForReader;
|
||||
}
|
||||
|
||||
public HoodieHFileDataBlock(List<IndexedRecord> records,
|
||||
Map<HeaderMetadataType, String> header,
|
||||
Compression.Algorithm compressionAlgorithm) {
|
||||
Compression.Algorithm compressionAlgorithm,
|
||||
Path pathForReader) {
|
||||
super(records, header, new HashMap<>(), HoodieHFileReader.KEY_FIELD_NAME);
|
||||
this.compressionAlgorithm = Option.of(compressionAlgorithm);
|
||||
this.pathForReader = pathForReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -95,6 +103,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
HFileContext context = new HFileContextBuilder()
|
||||
.withBlockSize(DEFAULT_BLOCK_SIZE)
|
||||
.withCompression(compressionAlgorithm.get())
|
||||
.withCellComparator(new HoodieHBaseKVComparator())
|
||||
.build();
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
@@ -128,7 +137,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
}
|
||||
|
||||
HFile.Writer writer = HFile.getWriterFactory(conf, cacheConfig)
|
||||
.withOutputStream(ostream).withFileContext(context).withComparator(new HoodieHBaseKVComparator()).create();
|
||||
.withOutputStream(ostream).withFileContext(context).create();
|
||||
|
||||
// Write the records
|
||||
sortedRecordsMap.forEach((recordKey, recordBytes) -> {
|
||||
@@ -155,7 +164,8 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
|
||||
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
|
||||
|
||||
// Read the content
|
||||
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(content);
|
||||
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(
|
||||
FSUtils.getFs(pathForReader.toString(), new Configuration()), pathForReader, content);
|
||||
// Sets up the writer schema
|
||||
reader.withSchema(writerSchema);
|
||||
Iterator<IndexedRecord> recordIterator = reader.getRecordIterator(readerSchema);
|
||||
|
||||
@@ -19,11 +19,11 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import org.apache.hadoop.hbase.KeyValue;
|
||||
import org.apache.hadoop.hbase.CellComparatorImpl;
|
||||
|
||||
/**
|
||||
* This class is explicitly used as Key Comparator to work around the hard coded
|
||||
* legacy format class names inside HBase. Otherwise, we will face issues with shading.
|
||||
*/
|
||||
public class HoodieHBaseKVComparator extends KeyValue.KVComparator {
|
||||
public class HoodieHBaseKVComparator extends CellComparatorImpl {
|
||||
}
|
||||
|
||||
@@ -18,18 +18,16 @@
|
||||
|
||||
package org.apache.hudi.io.storage;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
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.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
@@ -41,26 +39,37 @@ 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.HFileInfo;
|
||||
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
|
||||
import org.apache.hadoop.hbase.nio.ByteBuff;
|
||||
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.ClosableIterator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.io.ByteBufferBackedInputStream;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Iterator;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader<R> {
|
||||
public static final String KEY_FIELD_NAME = "key";
|
||||
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";
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
|
||||
|
||||
private Path path;
|
||||
private Configuration conf;
|
||||
private HFile.Reader reader;
|
||||
@@ -70,55 +79,35 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// key retrieval.
|
||||
private HFileScanner keyScanner;
|
||||
|
||||
public static final String KEY_FIELD_NAME = "key";
|
||||
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);
|
||||
this.reader = HoodieHFileUtils.createHFileReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, 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 = HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, 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);
|
||||
public HoodieHFileReader(FileSystem fs, Path dummyPath, byte[] content) throws IOException {
|
||||
this.reader = HoodieHFileUtils.createHFileReader(fs, dummyPath, content);
|
||||
}
|
||||
|
||||
@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;
|
||||
@@ -133,10 +122,10 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
|
||||
@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();
|
||||
ByteBuff serializedFilter = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false).getBufferWithoutHeader();
|
||||
byte[] filterBytes = new byte[serializedFilter.remaining()];
|
||||
serializedFilter.get(filterBytes); // read the bytes that were written
|
||||
return BloomFilterFactory.fromString(new String(filterBytes),
|
||||
@@ -206,7 +195,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
final HFileScanner scanner = reader.getScanner(false, false);
|
||||
if (scanner.seekTo()) {
|
||||
do {
|
||||
Cell c = scanner.getKeyValue();
|
||||
Cell c = scanner.getCell();
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(c, writerSchema, readerSchema, keyFieldSchema);
|
||||
recordList.add(keyAndRecordPair);
|
||||
} while (scanner.next());
|
||||
@@ -250,7 +239,6 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
*/
|
||||
public List<Pair<String, R>> readRecords(List<String> keys, Schema schema) throws IOException {
|
||||
this.schema = schema;
|
||||
reader.loadFileInfo();
|
||||
List<Pair<String, R>> records = new ArrayList<>();
|
||||
for (String key: keys) {
|
||||
Option<R> value = getRecordByKey(key, schema);
|
||||
@@ -263,7 +251,6 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
|
||||
public ClosableIterator<R> getRecordIterator(List<String> keys, Schema schema) throws IOException {
|
||||
this.schema = schema;
|
||||
reader.loadFileInfo();
|
||||
Iterator<String> iterator = keys.iterator();
|
||||
return new ClosableIterator<R>() {
|
||||
private R next;
|
||||
@@ -310,7 +297,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
// To handle when hasNext() is called multiple times for idempotency and/or the first time
|
||||
if (this.next == null && !this.eof) {
|
||||
if (!scanner.isSeeked() && scanner.seekTo()) {
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema);
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema);
|
||||
this.next = keyAndRecordPair.getSecond();
|
||||
}
|
||||
}
|
||||
@@ -331,7 +318,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
R retVal = this.next;
|
||||
if (scanner.next()) {
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getKeyValue(), getSchema(), readerSchema, keyFieldSchema);
|
||||
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema);
|
||||
this.next = keyAndRecordPair.getSecond();
|
||||
} else {
|
||||
this.next = null;
|
||||
@@ -371,7 +358,7 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
|
||||
}
|
||||
|
||||
if (keyScanner.seekTo(kv) == 0) {
|
||||
Cell c = keyScanner.getKeyValue();
|
||||
Cell c = keyScanner.getCell();
|
||||
// Extract the byte value before releasing the lock since we cannot hold on to the returned cell afterwards
|
||||
value = Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
|
||||
}
|
||||
|
||||
@@ -0,0 +1,87 @@
|
||||
/*
|
||||
* 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 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.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.HFileInfo;
|
||||
import org.apache.hadoop.hbase.io.hfile.ReaderContext;
|
||||
import org.apache.hadoop.hbase.io.hfile.ReaderContextBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Util class for HFile reading and writing in Hudi
|
||||
*/
|
||||
public class HoodieHFileUtils {
|
||||
// Based on HBase 2.4.9, the primaryReplicaReader is mainly used for constructing
|
||||
// block cache key, so if we do not use block cache then it is OK to set it as any
|
||||
// value. We use true here.
|
||||
private static final boolean USE_PRIMARY_REPLICA_READER = true;
|
||||
|
||||
/**
|
||||
* Creates HFile reader for a file with default `primaryReplicaReader` as true.
|
||||
*
|
||||
* @param fs File system.
|
||||
* @param path Path to file to read.
|
||||
* @param cacheConfig Cache configuration.
|
||||
* @param configuration Configuration
|
||||
* @return HFile reader
|
||||
* @throws IOException Upon error.
|
||||
*/
|
||||
public static HFile.Reader createHFileReader(
|
||||
FileSystem fs, Path path, CacheConfig cacheConfig, Configuration configuration) throws IOException {
|
||||
return HFile.createReader(fs, path, cacheConfig, USE_PRIMARY_REPLICA_READER, configuration);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates HFile reader for byte array with default `primaryReplicaReader` as true.
|
||||
*
|
||||
* @param fs File system.
|
||||
* @param dummyPath Dummy path to file to read.
|
||||
* @param content Content in byte array.
|
||||
* @return HFile reader
|
||||
* @throws IOException Upon error.
|
||||
*/
|
||||
public static HFile.Reader createHFileReader(
|
||||
FileSystem fs, Path dummyPath, byte[] content) throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
HoodieHFileReader.SeekableByteArrayInputStream bis = new HoodieHFileReader.SeekableByteArrayInputStream(content);
|
||||
FSDataInputStream fsdis = new FSDataInputStream(bis);
|
||||
FSDataInputStreamWrapper stream = new FSDataInputStreamWrapper(fsdis);
|
||||
ReaderContext context = new ReaderContextBuilder()
|
||||
.withFilePath(dummyPath)
|
||||
.withInputStreamWrapper(stream)
|
||||
.withFileSize(content.length)
|
||||
.withFileSystem(fs)
|
||||
.withPrimaryReplicaReader(USE_PRIMARY_REPLICA_READER)
|
||||
.withReaderType(ReaderContext.ReaderType.STREAM)
|
||||
.build();
|
||||
HFileInfo fileInfo = new HFileInfo(context, conf);
|
||||
HFile.Reader reader = HFile.createReader(context, fileInfo, new CacheConfig(conf), conf);
|
||||
fileInfo.initMetaAndIndex(reader);
|
||||
return reader;
|
||||
}
|
||||
}
|
||||
2185
hudi-common/src/main/resources/hbase-site.xml
Normal file
2185
hudi-common/src/main/resources/hbase-site.xml
Normal file
File diff suppressed because it is too large
Load Diff
Reference in New Issue
Block a user