1
0

[HUDI-3516] Implement record iterator for HoodieDataBlock (#4909)

*  Use iterator to void eager materialization to be memory friendly
This commit is contained in:
Bo Cui
2022-03-02 10:19:36 +08:00
committed by GitHub
parent a81a6326d5
commit 3fdc9332e5
14 changed files with 452 additions and 217 deletions

View File

@@ -33,6 +33,7 @@ import org.apache.hudi.common.table.log.block.HoodieHFileDataBlock;
import org.apache.hudi.common.table.log.block.HoodieLogBlock;
import org.apache.hudi.common.table.log.block.HoodieParquetDataBlock;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.ValidationUtils;
@@ -50,8 +51,8 @@ import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Deque;
import java.util.HashSet;
import java.util.List;
@@ -359,17 +360,13 @@ public abstract class AbstractHoodieLogRecordReader {
* handle it.
*/
private void processDataBlock(HoodieDataBlock dataBlock, Option<List<String>> keys) throws Exception {
// TODO (NA) - Implement getRecordItr() in HoodieAvroDataBlock and use that here
List<IndexedRecord> recs = new ArrayList<>();
if (!keys.isPresent()) {
recs = dataBlock.getRecords();
} else {
recs = dataBlock.getRecords(keys.get());
}
totalLogRecords.addAndGet(recs.size());
for (IndexedRecord rec : recs) {
processNextRecord(createHoodieRecord(rec, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
try (ClosableIterator<IndexedRecord> recordItr = dataBlock.getRecordItr(keys.orElse(Collections.emptyList()))) {
while (recordItr.hasNext()) {
IndexedRecord record = recordItr.next();
processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
totalLogRecords.incrementAndGet();
}
}
}

View File

@@ -31,6 +31,7 @@ import org.apache.avro.io.EncoderFactory;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.fs.SizeAwareDataInputStream;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
@@ -60,7 +61,6 @@ import static org.apache.hudi.common.util.ValidationUtils.checkState;
public class HoodieAvroDataBlock extends HoodieDataBlock {
private final ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
private final ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
public HoodieAvroDataBlock(FSDataInputStream inputStream,
Option<byte[]> content,
@@ -75,8 +75,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
public HoodieAvroDataBlock(@Nonnull List<IndexedRecord> records,
@Nonnull Map<HeaderMetadataType, String> header,
@Nonnull String keyField
) {
@Nonnull String keyField) {
super(records, header, new HashMap<>(), keyField);
}
@@ -99,9 +98,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
output.writeInt(records.size());
// 3. Write the records
Iterator<IndexedRecord> itr = records.iterator();
while (itr.hasNext()) {
IndexedRecord s = itr.next();
for (IndexedRecord s : records) {
ByteArrayOutputStream temp = new ByteArrayOutputStream();
BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(temp, encoderCache.get());
encoderCache.set(encoder);
@@ -120,49 +117,80 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
throw new HoodieIOException("IOException converting HoodieAvroDataBlock to bytes", e);
}
}
encoderCache.remove();
output.close();
return baos.toByteArray();
}
// TODO (na) - Break down content into smaller chunks of byte [] to be GC as they are used
// TODO (na) - Implement a recordItr instead of recordList
@Override
protected List<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
checkState(readerSchema != null, "Reader's schema has to be non-null");
protected ClosableIterator<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
checkState(this.readerSchema != null, "Reader's schema has to be non-null");
return RecordIterator.getInstance(this, content);
}
SizeAwareDataInputStream dis =
new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(content)));
private static class RecordIterator implements ClosableIterator<IndexedRecord> {
private byte[] content;
private final SizeAwareDataInputStream dis;
private final GenericDatumReader<IndexedRecord> reader;
private final ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
// 1. Read version for this data block
int version = dis.readInt();
HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version);
private int totalRecords = 0;
private int readRecords = 0;
// Get schema from the header
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
private RecordIterator(Schema readerSchema, Schema writerSchema, byte[] content) throws IOException {
this.content = content;
GenericDatumReader<IndexedRecord> reader = new GenericDatumReader<>(writerSchema, readerSchema);
this.dis = new SizeAwareDataInputStream(new DataInputStream(new ByteArrayInputStream(this.content)));
// 2. Get the total records
int totalRecords = 0;
if (logBlockVersion.hasRecordCount()) {
totalRecords = dis.readInt();
}
List<IndexedRecord> records = new ArrayList<>(totalRecords);
// 1. Read version for this data block
int version = this.dis.readInt();
HoodieAvroDataBlockVersion logBlockVersion = new HoodieAvroDataBlockVersion(version);
// 3. Read the content
for (int i = 0; i < totalRecords; i++) {
int recordLength = dis.readInt();
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(content, dis.getNumberOfBytesRead(),
recordLength, decoderCache.get());
decoderCache.set(decoder);
IndexedRecord record = reader.read(null, decoder);
records.add(record);
dis.skipBytes(recordLength);
this.reader = new GenericDatumReader<>(writerSchema, readerSchema);
if (logBlockVersion.hasRecordCount()) {
this.totalRecords = this.dis.readInt();
}
}
dis.close();
public static RecordIterator getInstance(HoodieAvroDataBlock dataBlock, byte[] content) throws IOException {
// Get schema from the header
Schema writerSchema = new Schema.Parser().parse(dataBlock.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
return new RecordIterator(dataBlock.readerSchema, writerSchema, content);
}
return records;
@Override
public void close() {
try {
this.dis.close();
this.decoderCache.remove();
this.content = null;
} catch (IOException e) {
// ignore
}
}
@Override
public boolean hasNext() {
return readRecords < totalRecords;
}
@Override
public IndexedRecord next() {
try {
int recordLength = this.dis.readInt();
BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(this.content, this.dis.getNumberOfBytesRead(),
recordLength, this.decoderCache.get());
this.decoderCache.set(decoder);
IndexedRecord record = this.reader.read(null, decoder);
this.dis.skipBytes(recordLength);
this.readRecords++;
return record;
} catch (IOException e) {
throw new HoodieIOException("Unable to convert bytes to record.", e);
}
}
}
//----------------------------------------------------------------------------------------
@@ -256,7 +284,10 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
output.writeInt(schemaContent.length);
output.write(schemaContent);
List<IndexedRecord> records = getRecords();
List<IndexedRecord> records = new ArrayList<>();
try (ClosableIterator<IndexedRecord> recordItr = getRecordItr()) {
recordItr.forEachRemaining(records::add);
}
// 3. Write total number of records
output.writeInt(records.size());

View File

@@ -18,17 +18,21 @@
package org.apache.hudi.common.table.log.block;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FSDataInputStream;
import java.io.IOException;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import java.util.Set;
import java.util.function.Function;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
@@ -45,7 +49,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
// TODO rebase records/content to leverage Either to warrant
// that they are mutex (used by read/write flows respectively)
private Option<List<IndexedRecord>> records;
private final Option<List<IndexedRecord>> records;
/**
* Key field's name w/in the record's schema
@@ -110,18 +114,18 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
}
/**
* Returns all the records contained w/in this block
* Returns all the records iterator contained w/in this block.
*/
public final List<IndexedRecord> getRecords() {
if (!records.isPresent()) {
try {
// in case records are absent, read content lazily and then convert to IndexedRecords
records = Option.of(readRecordsFromBlockPayload());
} catch (IOException io) {
throw new HoodieIOException("Unable to convert content bytes to records", io);
}
public final ClosableIterator<IndexedRecord> getRecordItr() {
if (records.isPresent()) {
return list2Iterator(records.get());
}
try {
// in case records are absent, read content lazily and then convert to IndexedRecords
return readRecordsFromBlockPayload();
} catch (IOException io) {
throw new HoodieIOException("Unable to convert content bytes to records", io);
}
return records.get();
}
public Schema getSchema() {
@@ -136,7 +140,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
* @return List of IndexedRecords for the keys of interest.
* @throws IOException in case of failures encountered when reading/parsing records
*/
public final List<IndexedRecord> getRecords(List<String> keys) throws IOException {
public final ClosableIterator<IndexedRecord> getRecordItr(List<String> keys) throws IOException {
boolean fullScan = keys.isEmpty();
if (enablePointLookups && !fullScan) {
return lookupRecords(keys);
@@ -144,18 +148,16 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
// Otherwise, we fetch all the records and filter out all the records, but the
// ones requested
List<IndexedRecord> allRecords = getRecords();
ClosableIterator<IndexedRecord> allRecords = getRecordItr();
if (fullScan) {
return allRecords;
}
HashSet<String> keySet = new HashSet<>(keys);
return allRecords.stream()
.filter(record -> keySet.contains(getRecordKey(record).orElse(null)))
.collect(Collectors.toList());
return FilteringIterator.getInstance(allRecords, keySet, this::getRecordKey);
}
protected List<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
protected ClosableIterator<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
if (readBlockLazily && !getContent().isPresent()) {
// read log block contents from disk
inflate();
@@ -169,7 +171,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
}
}
protected List<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
protected ClosableIterator<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
throw new UnsupportedOperationException(
String.format("Point lookups are not supported by this Data block type (%s)", getBlockType())
);
@@ -177,7 +179,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
protected abstract byte[] serializeRecords(List<IndexedRecord> records) throws IOException;
protected abstract List<IndexedRecord> deserializeRecords(byte[] content) throws IOException;
protected abstract ClosableIterator<IndexedRecord> deserializeRecords(byte[] content) throws IOException;
public abstract HoodieLogBlockType getBlockType();
@@ -190,4 +192,80 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
.map(keyField -> record.get(keyField.pos()))
.map(Object::toString);
}
/**
* Converts the given list to closable iterator.
*/
static <T> ClosableIterator<T> list2Iterator(List<T> list) {
Iterator<T> iterator = list.iterator();
return new ClosableIterator<T>() {
@Override
public void close() {
// ignored
}
@Override
public boolean hasNext() {
return iterator.hasNext();
}
@Override
public T next() {
return iterator.next();
}
};
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------
/**
* A {@link ClosableIterator} that supports filtering strategy with given keys.
* User should supply the key extraction function for fetching string format keys.
*
* @param <T> the element type
*/
private static class FilteringIterator<T extends IndexedRecord> implements ClosableIterator<T> {
private final ClosableIterator<T> nested; // nested iterator
private final Set<String> keys; // the filtering keys
private final Function<T, Option<String>> keyExtract; // function to extract the key
private T next;
private FilteringIterator(ClosableIterator<T> nested, Set<String> keys, Function<T, Option<String>> keyExtract) {
this.nested = nested;
this.keys = keys;
this.keyExtract = keyExtract;
}
public static <T extends IndexedRecord> FilteringIterator<T> getInstance(
ClosableIterator<T> nested,
Set<String> keys,
Function<T, Option<String>> keyExtract) {
return new FilteringIterator<>(nested, keys, keyExtract);
}
@Override
public void close() {
this.nested.close();
}
@Override
public boolean hasNext() {
while (this.nested.hasNext()) {
this.next = this.nested.next();
if (keys.contains(keyExtract.apply(this.next).orElse(null))) {
return true;
}
}
return false;
}
@Override
public T next() {
return this.next;
}
}
}

View File

@@ -30,16 +30,17 @@ 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.hadoop.hbase.util.Pair;
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;
@@ -51,7 +52,6 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.TreeMap;
import java.util.stream.Collectors;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
@@ -148,7 +148,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
}
@Override
protected List<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
protected ClosableIterator<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
checkState(readerSchema != null, "Reader's schema has to be non-null");
// Get schema from the header
@@ -156,14 +156,30 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
// Read the content
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(content);
List<Pair<String, IndexedRecord>> records = reader.readAllRecords(writerSchema, readerSchema);
// Sets up the writer schema
reader.withSchema(writerSchema);
Iterator<IndexedRecord> recordIterator = reader.getRecordIterator(readerSchema);
return new ClosableIterator<IndexedRecord>() {
@Override
public void close() {
reader.close();
}
return records.stream().map(Pair::getSecond).collect(Collectors.toList());
@Override
public boolean hasNext() {
return recordIterator.hasNext();
}
@Override
public IndexedRecord next() {
return recordIterator.next();
}
};
}
// TODO abstract this w/in HoodieDataBlock
@Override
protected List<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
protected ClosableIterator<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
// NOTE: It's important to extend Hadoop configuration here to make sure configuration
@@ -180,12 +196,27 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
// HFile read will be efficient if keys are sorted, since on storage, records are sorted by key. This will avoid unnecessary seeks.
Collections.sort(keys);
try (HoodieHFileReader<IndexedRecord> reader =
new HoodieHFileReader<>(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf))) {
// Get writer's schema from the header
List<Pair<String, IndexedRecord>> logRecords = reader.readRecords(keys, readerSchema);
return logRecords.stream().map(Pair::getSecond).collect(Collectors.toList());
}
final HoodieHFileReader<IndexedRecord> reader =
new HoodieHFileReader<>(inlineConf, inlinePath, new CacheConfig(inlineConf), inlinePath.getFileSystem(inlineConf));
// Get writer's schema from the header
final ClosableIterator<IndexedRecord> recordIterator = reader.getRecordIterator(keys, readerSchema);
return new ClosableIterator<IndexedRecord>() {
@Override
public boolean hasNext() {
return recordIterator.hasNext();
}
@Override
public IndexedRecord next() {
return recordIterator.next();
}
@Override
public void close() {
recordIterator.close();
reader.close();
}
};
}
private byte[] serializeRecord(IndexedRecord record) {

View File

@@ -18,19 +18,21 @@
package org.apache.hudi.common.table.log.block;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
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.ParquetReaderIterator;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetStreamWriter;
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.FSDataOutputStream;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroWriteSupport;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
import org.apache.hudi.common.fs.inline.InLineFileSystem;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.io.storage.HoodieAvroParquetConfig;
import org.apache.hudi.io.storage.HoodieParquetStreamWriter;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroSchemaConverter;
@@ -41,11 +43,10 @@ import org.apache.parquet.hadoop.util.HadoopInputFile;
import org.apache.parquet.io.InputFile;
import javax.annotation.Nonnull;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@@ -121,9 +122,9 @@ public class HoodieParquetDataBlock extends HoodieDataBlock {
return baos.toByteArray();
}
public static Iterator<IndexedRecord> getProjectedParquetRecordsIterator(Configuration conf,
Schema readerSchema,
InputFile inputFile) throws IOException {
public static ClosableIterator<IndexedRecord> getProjectedParquetRecordsIterator(Configuration conf,
Schema readerSchema,
InputFile inputFile) throws IOException {
AvroReadSupport.setAvroReadSchema(conf, readerSchema);
AvroReadSupport.setRequestedProjection(conf, readerSchema);
@@ -138,7 +139,7 @@ public class HoodieParquetDataBlock extends HoodieDataBlock {
* requested by the caller (providing projected Reader's schema)
*/
@Override
protected List<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
protected ClosableIterator<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
// NOTE: It's important to extend Hadoop configuration here to make sure configuration
@@ -152,20 +153,14 @@ public class HoodieParquetDataBlock extends HoodieDataBlock {
blockContentLoc.getContentPositionInLogFile(),
blockContentLoc.getBlockSize());
ArrayList<IndexedRecord> records = new ArrayList<>();
getProjectedParquetRecordsIterator(
return getProjectedParquetRecordsIterator(
inlineConf,
readerSchema,
HadoopInputFile.fromPath(inlineLogFilePath, inlineConf)
)
.forEachRemaining(records::add);
return records;
HadoopInputFile.fromPath(inlineLogFilePath, inlineConf));
}
@Override
protected List<IndexedRecord> deserializeRecords(byte[] content) throws IOException {
protected ClosableIterator<IndexedRecord> deserializeRecords(byte[] content) {
throw new UnsupportedOperationException("Should not be invoked");
}
}

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.common.util.Option;
@@ -54,10 +55,12 @@ import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
/**
* Represents the Archived Timeline for the Hoodie table. Instants for the last 12 hours (configurable) is in the
@@ -248,15 +251,14 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next();
// TODO If we can store additional metadata in datablock, we can skip parsing records
// (such as startTime, endTime of records in the block)
List<IndexedRecord> records = blk.getRecords();
// Filter blocks in desired time window
instantsInRange.addAll(
records.stream()
.filter(r -> commitsFilter.apply((GenericRecord) r))
.map(r -> readCommit((GenericRecord) r, loadInstantDetails))
.filter(c -> filter == null || filter.isInRange(c))
.collect(Collectors.toList())
);
try (ClosableIterator<IndexedRecord> itr = blk.getRecordItr()) {
StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true)
// Filter blocks in desired time window
.filter(r -> commitsFilter.apply((GenericRecord) r))
.map(r -> readCommit((GenericRecord) r, loadInstantDetails))
.filter(c -> filter == null || filter.isInRange(c))
.forEach(instantsInRange::add);
}
}
if (filter != null) {

View File

@@ -50,6 +50,7 @@ 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;
@@ -123,6 +124,13 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
return schema;
}
/**
* Sets up the writer schema explicitly.
*/
public void withSchema(Schema schema) {
this.schema = schema;
}
@Override
public BloomFilter readBloomFilter() {
Map<byte[], byte[]> fileInfo;
@@ -184,7 +192,14 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
return filteredRecords;
}
public List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) throws IOException {
/**
* Reads all the records with given schema.
*
* <p>NOTE: This should only be used for testing,
* the records are materialized eagerly into a list and returned,
* use {@code getRecordIterator} where possible.
*/
private List<Pair<String, R>> readAllRecords(Schema writerSchema, Schema readerSchema) {
final Option<Schema.Field> keyFieldSchema = Option.ofNullable(readerSchema.getField(KEY_FIELD_NAME));
List<Pair<String, R>> recordList = new LinkedList<>();
try {
@@ -203,17 +218,36 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
}
}
public List<Pair<String, R>> readAllRecords() throws IOException {
Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get(KEY_SCHEMA.getBytes())));
/**
* Reads all the records with current schema.
*
* <p>NOTE: This should only be used for testing,
* the records are materialized eagerly into a list and returned,
* use {@code getRecordIterator} where possible.
*/
public List<Pair<String, R>> readAllRecords() {
Schema schema = getSchema();
return readAllRecords(schema, schema);
}
/**
* Reads all the records with current schema and filtering keys.
*
* <p>NOTE: This should only be used for testing,
* the records are materialized eagerly into a list and returned,
* use {@code getRecordIterator} where possible.
*/
public List<Pair<String, R>> readRecords(List<String> keys) throws IOException {
reader.loadFileInfo();
Schema schema = new Schema.Parser().parse(new String(reader.loadFileInfo().get(KEY_SCHEMA.getBytes())));
return readRecords(keys, schema);
return readRecords(keys, getSchema());
}
/**
* Reads all the records with given schema and filtering keys.
*
* <p>NOTE: This should only be used for testing,
* the records are materialized eagerly into a list and returned,
* use {@code getRecordIterator} where possible.
*/
public List<Pair<String, R>> readRecords(List<String> keys, Schema schema) throws IOException {
this.schema = schema;
reader.loadFileInfo();
@@ -227,6 +261,39 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
return records;
}
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;
@Override
public void close() {
}
@Override
public boolean hasNext() {
try {
while (iterator.hasNext()) {
Option<R> value = getRecordByKey(iterator.next(), schema);
if (value.isPresent()) {
next = value.get();
return true;
}
}
return false;
} catch (IOException e) {
throw new HoodieIOException("unable to read next record from hfile ", e);
}
}
@Override
public R next() {
return next;
}
};
}
@Override
public Iterator getRecordIterator(Schema readerSchema) throws IOException {
final HFileScanner scanner = reader.getScanner(false, false);