1
0

[HUDI-3760] Adding capability to fetch Metadata Records by prefix (#5208)

- Adding capability to fetch Metadata Records by key prefix so that Data Skipping could fetch only Column Stats 
- Index records pertaining to the columns being queried by, instead of reading out whole Index.
- Fixed usages of HFileScanner in HFileReader. few code paths uses cached scanner if available. Other code paths uses its own HFileScanner w/ positional read. 

Brief change log
- Rebasing ColumnStatsIndexSupport to rely on HoodieBackedTableMetadata in lieu of reading t/h Spark DS
- Adding methods enabling key-prefix lookups to HoodiFileReader, HoodieHFileReader
- Wiring key-prefix lookup t/h LogRecordScanner impls
- Cleaning up HoodieHFileReader impl

Co-authored-by: sivabalan <n.siva.b@gmail.com>
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com>
This commit is contained in:
Alexey Kudinkin
2022-04-06 09:11:08 -07:00
committed by GitHub
parent 7612549bcc
commit 9e87d164b3
46 changed files with 1387 additions and 698 deletions

View File

@@ -69,7 +69,7 @@ public abstract class BaseHoodieTableFileIndex {
private final String[] partitionColumns;
private final FileSystemViewStorageConfig fileSystemStorageConfig;
private final HoodieMetadataConfig metadataConfig;
protected final HoodieMetadataConfig metadataConfig;
private final HoodieTableQueryType queryType;
private final Option<String> specifiedQueryInstant;

View File

@@ -289,8 +289,8 @@ public final class HoodieMetadataConfig extends HoodieConfig {
return getString(DIR_FILTER_REGEX);
}
public boolean enableFullScan() {
return getBoolean(ENABLE_FULL_SCAN_LOG_FILES);
public boolean allowFullScan() {
return getBooleanOrDefault(ENABLE_FULL_SCAN_LOG_FILES);
}
public boolean populateMetaFields() {

View File

@@ -57,7 +57,6 @@ import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayDeque;
import java.util.Arrays;
import java.util.Collections;
import java.util.Deque;
import java.util.HashSet;
import java.util.List;
@@ -129,7 +128,7 @@ public abstract class AbstractHoodieLogRecordReader {
// Store the last instant log blocks (needed to implement rollback)
private Deque<HoodieLogBlock> currentInstantLogBlocks = new ArrayDeque<>();
// Enables full scan of log records
protected final boolean enableFullScan;
protected final boolean forceFullScan;
private int totalScannedLogFiles;
// Progress
private float progress = 0.0f;
@@ -150,7 +149,7 @@ public abstract class AbstractHoodieLogRecordReader {
protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
boolean withOperationField, boolean enableFullScan,
boolean withOperationField, boolean forceFullScan,
Option<String> partitionName, InternalSchema internalSchema) {
this.readerSchema = readerSchema;
this.latestInstantTime = latestInstantTime;
@@ -167,7 +166,7 @@ public abstract class AbstractHoodieLogRecordReader {
this.bufferSize = bufferSize;
this.instantRange = instantRange;
this.withOperationField = withOperationField;
this.enableFullScan = enableFullScan;
this.forceFullScan = forceFullScan;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
this.path = basePath;
@@ -189,10 +188,14 @@ public abstract class AbstractHoodieLogRecordReader {
}
public synchronized void scan() {
scan(Option.empty());
scanInternal(Option.empty());
}
public synchronized void scan(Option<List<String>> keys) {
public synchronized void scan(List<String> keys) {
scanInternal(Option.of(new KeySpec(keys, true)));
}
protected synchronized void scanInternal(Option<KeySpec> keySpecOpt) {
currentInstantLogBlocks = new ArrayDeque<>();
progress = 0.0f;
totalLogFiles = new AtomicLong(0);
@@ -205,15 +208,16 @@ public abstract class AbstractHoodieLogRecordReader {
HoodieTimeline completedInstantsTimeline = commitsTimeline.filterCompletedInstants();
HoodieTimeline inflightInstantsTimeline = commitsTimeline.filterInflights();
try {
// Get the key field based on populate meta fields config
// and the table type
final String keyField = getKeyField();
// Iterate over the paths
boolean enableRecordLookups = !forceFullScan;
logFormatReaderWrapper = new HoodieLogFormatReader(fs,
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile))).collect(Collectors.toList()),
readerSchema, readBlocksLazily, reverseReader, bufferSize, !enableFullScan, keyField, internalSchema);
readerSchema, readBlocksLazily, reverseReader, bufferSize, enableRecordLookups, keyField, internalSchema);
Set<HoodieLogFile> scannedLogFiles = new HashSet<>();
while (logFormatReaderWrapper.hasNext()) {
HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
@@ -250,7 +254,7 @@ public abstract class AbstractHoodieLogRecordReader {
if (isNewInstantBlock(logBlock) && !readBlocksLazily) {
// If this is an avro data block belonging to a different commit/instant,
// then merge the last blocks and records into the main result
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keys);
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keySpecOpt);
}
// store the current block
currentInstantLogBlocks.push(logBlock);
@@ -260,7 +264,7 @@ public abstract class AbstractHoodieLogRecordReader {
if (isNewInstantBlock(logBlock) && !readBlocksLazily) {
// If this is a delete data block belonging to a different commit/instant,
// then merge the last blocks and records into the main result
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keys);
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keySpecOpt);
}
// store deletes so can be rolled back
currentInstantLogBlocks.push(logBlock);
@@ -335,7 +339,7 @@ public abstract class AbstractHoodieLogRecordReader {
// merge the last read block when all the blocks are done reading
if (!currentInstantLogBlocks.isEmpty()) {
LOG.info("Merging the final data blocks");
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keys);
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size(), keySpecOpt);
}
// Done
progress = 1.0f;
@@ -370,11 +374,11 @@ public abstract class AbstractHoodieLogRecordReader {
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and call subclass processors to
* handle it.
*/
private void processDataBlock(HoodieDataBlock dataBlock, Option<List<String>> keys) throws Exception {
try (ClosableIterator<IndexedRecord> recordItr = dataBlock.getRecordItr(keys.orElse(Collections.emptyList()))) {
private void processDataBlock(HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt) throws Exception {
try (ClosableIterator<IndexedRecord> recordIterator = getRecordsIterator(dataBlock, keySpecOpt)) {
Option<Schema> schemaOption = getMergedSchema(dataBlock);
while (recordItr.hasNext()) {
IndexedRecord currentRecord = recordItr.next();
while (recordIterator.hasNext()) {
IndexedRecord currentRecord = recordIterator.next();
IndexedRecord record = schemaOption.isPresent() ? HoodieAvroUtils.rewriteRecordWithNewSchema(currentRecord, schemaOption.get()) : currentRecord;
processNextRecord(createHoodieRecord(record, this.hoodieTableMetaClient.getTableConfig(), this.payloadClassFQN,
this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName));
@@ -449,23 +453,20 @@ public abstract class AbstractHoodieLogRecordReader {
* Process the set of log blocks belonging to the last instant which is read fully.
*/
private void processQueuedBlocksForInstant(Deque<HoodieLogBlock> logBlocks, int numLogFilesSeen,
Option<List<String>> keys) throws Exception {
Option<KeySpec> keySpecOpt) throws Exception {
while (!logBlocks.isEmpty()) {
LOG.info("Number of remaining logblocks to merge " + logBlocks.size());
// poll the element at the bottom of the stack since that's the order it was inserted
HoodieLogBlock lastBlock = logBlocks.pollLast();
switch (lastBlock.getBlockType()) {
case AVRO_DATA_BLOCK:
processDataBlock((HoodieAvroDataBlock) lastBlock, keys);
processDataBlock((HoodieAvroDataBlock) lastBlock, keySpecOpt);
break;
case HFILE_DATA_BLOCK:
if (!keys.isPresent()) {
keys = Option.of(Collections.emptyList());
}
processDataBlock((HoodieHFileDataBlock) lastBlock, keys);
processDataBlock((HoodieHFileDataBlock) lastBlock, keySpecOpt);
break;
case PARQUET_DATA_BLOCK:
processDataBlock((HoodieParquetDataBlock) lastBlock, keys);
processDataBlock((HoodieParquetDataBlock) lastBlock, keySpecOpt);
break;
case DELETE_BLOCK:
Arrays.stream(((HoodieDeleteBlock) lastBlock).getRecordsToDelete()).forEach(this::processNextDeletedRecord);
@@ -481,6 +482,15 @@ public abstract class AbstractHoodieLogRecordReader {
progress = numLogFilesSeen - 1 / logFilePaths.size();
}
private ClosableIterator<IndexedRecord> getRecordsIterator(HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt) throws IOException {
if (keySpecOpt.isPresent()) {
KeySpec keySpec = keySpecOpt.get();
return dataBlock.getRecordIterator(keySpec.keys, keySpec.fullKey);
}
return dataBlock.getRecordIterator();
}
/**
* Return progress of scanning as a float between 0.0 to 1.0.
*/
@@ -504,7 +514,7 @@ public abstract class AbstractHoodieLogRecordReader {
return payloadClassFQN;
}
protected Option<String> getPartitionName() {
public Option<String> getPartitionName() {
return partitionName;
}
@@ -520,6 +530,16 @@ public abstract class AbstractHoodieLogRecordReader {
return withOperationField;
}
protected static class KeySpec {
private final List<String> keys;
private final boolean fullKey;
public KeySpec(List<String> keys, boolean fullKey) {
this.keys = keys;
this.fullKey = fullKey;
}
}
/**
* Builder used to build {@code AbstractHoodieLogRecordScanner}.
*/

View File

@@ -53,13 +53,7 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
private static final Logger LOG = LogManager.getLogger(HoodieLogFormatReader.class);
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles, Schema readerSchema, boolean readBlocksLazily,
boolean reverseLogReader, int bufferSize, boolean enableInlineReading,
String recordKeyField) throws IOException {
this(fs, logFiles, readerSchema, readBlocksLazily, reverseLogReader, bufferSize, enableInlineReading, recordKeyField, InternalSchema.getEmptyInternalSchema());
}
HoodieLogFormatReader(FileSystem fs, List<HoodieLogFile> logFiles, Schema readerSchema, boolean readBlocksLazily,
boolean reverseLogReader, int bufferSize, boolean enableInlineReading,
boolean reverseLogReader, int bufferSize, boolean enableRecordLookups,
String recordKeyField, InternalSchema internalSchema) throws IOException {
this.logFiles = logFiles;
this.fs = fs;
@@ -69,12 +63,12 @@ public class HoodieLogFormatReader implements HoodieLogFormat.Reader {
this.bufferSize = bufferSize;
this.prevReadersInOpenState = new ArrayList<>();
this.recordKeyField = recordKeyField;
this.enableInlineReading = enableInlineReading;
this.enableInlineReading = enableRecordLookups;
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
if (logFiles.size() > 0) {
HoodieLogFile nextLogFile = logFiles.remove(0);
this.currentReader = new HoodieLogFileReader(fs, nextLogFile, readerSchema, bufferSize, readBlocksLazily, false,
enableInlineReading, recordKeyField, internalSchema);
enableRecordLookups, recordKeyField, internalSchema);
}
}

View File

@@ -45,6 +45,8 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/**
* Scans through all the blocks in a list of HoodieLogFile and builds up a compacted/merged list of records which will
* be used as a lookup table when merging the base columnar file with the redo log file.
@@ -76,14 +78,14 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
protected HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths, Schema readerSchema,
String latestInstantTime, Long maxMemorySizeInBytes, boolean readBlocksLazily,
boolean reverseReader, int bufferSize, String spillableMapBasePath,
Option<InstantRange> instantRange, boolean autoScan,
Option<InstantRange> instantRange,
ExternalSpillableMap.DiskMapType diskMapType,
boolean isBitCaskDiskMapCompressionEnabled,
boolean withOperationField, boolean enableFullScan,
boolean withOperationField, boolean forceFullScan,
Option<String> partitionName, InternalSchema internalSchema) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
instantRange, withOperationField,
enableFullScan, partitionName, internalSchema);
forceFullScan, partitionName, internalSchema);
try {
// Store merged records for all versions for this log file, set the in-memory footprint to maxInMemoryMapSize
this.records = new ExternalSpillableMap<>(maxMemorySizeInBytes, spillableMapBasePath, new DefaultSizeEstimator(),
@@ -93,7 +95,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
throw new HoodieIOException("IOException when creating ExternalSpillableMap at " + spillableMapBasePath, e);
}
if (autoScan) {
if (forceFullScan) {
performScan();
}
}
@@ -115,10 +117,12 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
@Override
public Iterator<HoodieRecord<? extends HoodieRecordPayload>> iterator() {
checkState(forceFullScan, "Record reader has to be in full-scan mode to use this API");
return records.iterator();
}
public Map<String, HoodieRecord<? extends HoodieRecordPayload>> getRecords() {
checkState(forceFullScan, "Record reader has to be in full-scan mode to use this API");
return records;
}
@@ -211,8 +215,6 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
// incremental filtering
protected Option<InstantRange> instantRange = Option.empty();
protected String partitionName;
// auto scan default true
private boolean autoScan = true;
// operation field default false
private boolean withOperationField = false;
@@ -290,11 +292,6 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
return this;
}
public Builder withAutoScan(boolean autoScan) {
this.autoScan = autoScan;
return this;
}
public Builder withInternalSchema(InternalSchema internalSchema) {
this.internalSchema = internalSchema == null ? InternalSchema.getEmptyInternalSchema() : internalSchema;
return this;
@@ -315,7 +312,7 @@ public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordReader
public HoodieMergedLogRecordScanner build() {
return new HoodieMergedLogRecordScanner(fs, basePath, logFilePaths, readerSchema,
latestInstantTime, maxMemorySizeInBytes, readBlocksLazily, reverseReader,
bufferSize, spillableMapBasePath, instantRange, autoScan,
bufferSize, spillableMapBasePath, instantRange,
diskMapType, isBitCaskDiskMapCompressionEnabled, withOperationField, true,
Option.ofNullable(partitionName), internalSchema);
}

View File

@@ -314,7 +314,7 @@ public class HoodieAvroDataBlock extends HoodieDataBlock {
output.write(schemaContent);
List<IndexedRecord> records = new ArrayList<>();
try (ClosableIterator<IndexedRecord> recordItr = getRecordItr()) {
try (ClosableIterator<IndexedRecord> recordItr = getRecordIterator()) {
recordItr.forEachRemaining(records::add);
}

View File

@@ -138,7 +138,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
/**
* Returns all the records iterator contained w/in this block.
*/
public final ClosableIterator<IndexedRecord> getRecordItr() {
public final ClosableIterator<IndexedRecord> getRecordIterator() {
if (records.isPresent()) {
return list2Iterator(records.get());
}
@@ -162,21 +162,21 @@ 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 ClosableIterator<IndexedRecord> getRecordItr(List<String> keys) throws IOException {
public final ClosableIterator<IndexedRecord> getRecordIterator(List<String> keys, boolean fullKey) throws IOException {
boolean fullScan = keys.isEmpty();
if (enablePointLookups && !fullScan) {
return lookupRecords(keys);
return lookupRecords(keys, fullKey);
}
// Otherwise, we fetch all the records and filter out all the records, but the
// ones requested
ClosableIterator<IndexedRecord> allRecords = getRecordItr();
ClosableIterator<IndexedRecord> allRecords = getRecordIterator();
if (fullScan) {
return allRecords;
}
HashSet<String> keySet = new HashSet<>(keys);
return FilteringIterator.getInstance(allRecords, keySet, this::getRecordKey);
return FilteringIterator.getInstance(allRecords, keySet, fullKey, this::getRecordKey);
}
protected ClosableIterator<IndexedRecord> readRecordsFromBlockPayload() throws IOException {
@@ -193,7 +193,7 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
}
}
protected ClosableIterator<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
protected ClosableIterator<IndexedRecord> lookupRecords(List<String> keys, boolean fullKey) throws IOException {
throw new UnsupportedOperationException(
String.format("Point lookups are not supported by this Data block type (%s)", getBlockType())
);
@@ -252,21 +252,25 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
private final ClosableIterator<T> nested; // nested iterator
private final Set<String> keys; // the filtering keys
private final boolean fullKey;
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) {
private FilteringIterator(ClosableIterator<T> nested, Set<String> keys, boolean fullKey, Function<T, Option<String>> keyExtract) {
this.nested = nested;
this.keys = keys;
this.fullKey = fullKey;
this.keyExtract = keyExtract;
}
public static <T extends IndexedRecord> FilteringIterator<T> getInstance(
ClosableIterator<T> nested,
Set<String> keys,
boolean fullKey,
Function<T, Option<String>> keyExtract) {
return new FilteringIterator<>(nested, keys, keyExtract);
return new FilteringIterator<>(nested, keys, fullKey, keyExtract);
}
@Override
@@ -278,7 +282,13 @@ public abstract class HoodieDataBlock extends HoodieLogBlock {
public boolean hasNext() {
while (this.nested.hasNext()) {
this.next = this.nested.next();
if (keys.contains(keyExtract.apply(this.next).orElse(null))) {
String key = keyExtract.apply(this.next)
.orElseGet(() -> {
throw new IllegalStateException(String.format("Record without a key (%s)", this.next));
});
if (fullKey && keys.contains(key)
|| !fullKey && keys.stream().anyMatch(key::startsWith)) {
return true;
}
}

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.common.table.log.block;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.fs.inline.InLineFSUtils;
@@ -47,6 +48,7 @@ import org.apache.log4j.Logger;
import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
@@ -149,6 +151,8 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
}
});
writer.appendFileInfo(HoodieHFileReader.SCHEMA_KEY.getBytes(), getSchema().toString().getBytes());
writer.close();
ostream.flush();
ostream.close();
@@ -163,11 +167,9 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
// Get schema from the header
Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
FileSystem fs = FSUtils.getFs(pathForReader.toString(), new Configuration());
// Read the content
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(
FSUtils.getFs(pathForReader.toString(), new Configuration()), pathForReader, content);
// Sets up the writer schema
reader.withSchema(writerSchema);
HoodieHFileReader<IndexedRecord> reader = new HoodieHFileReader<>(fs, pathForReader, content, Option.of(writerSchema));
Iterator<IndexedRecord> recordIterator = reader.getRecordIterator(readerSchema);
return new ClosableIterator<IndexedRecord>() {
@Override
@@ -189,7 +191,7 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
// TODO abstract this w/in HoodieDataBlock
@Override
protected ClosableIterator<IndexedRecord> lookupRecords(List<String> keys) throws IOException {
protected ClosableIterator<IndexedRecord> lookupRecords(List<String> keys, boolean fullKey) throws IOException {
HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get();
// NOTE: It's important to extend Hadoop configuration here to make sure configuration
@@ -204,13 +206,18 @@ public class HoodieHFileDataBlock extends HoodieDataBlock {
blockContentLoc.getContentPositionInLogFile(),
blockContentLoc.getBlockSize());
// 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);
// HFile read will be efficient if keys are sorted, since on storage records are sorted by key.
// This will avoid unnecessary seeks.
List<String> sortedKeys = new ArrayList<>(keys);
Collections.sort(sortedKeys);
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);
final ClosableIterator<IndexedRecord> recordIterator =
fullKey ? reader.getRecordsByKeysIterator(sortedKeys, readerSchema) : reader.getRecordsByKeyPrefixIterator(sortedKeys, readerSchema);
return new ClosableIterator<IndexedRecord>() {
@Override
public boolean hasNext() {

View File

@@ -257,7 +257,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline {
HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block;
// TODO If we can store additional metadata in datablock, we can skip parsing records
// (such as startTime, endTime of records in the block)
try (ClosableIterator<IndexedRecord> itr = avroBlock.getRecordItr()) {
try (ClosableIterator<IndexedRecord> itr = avroBlock.getRecordIterator()) {
StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true)
// Filter blocks in desired time window
.filter(r -> commitsFilter.apply((GenericRecord) r))

View File

@@ -32,9 +32,12 @@ import java.util.Map;
import java.util.Objects;
import java.util.Properties;
import java.util.Set;
import java.util.Spliterator;
import java.util.Spliterators;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
public class CollectionUtils {
@@ -48,6 +51,16 @@ public class CollectionUtils {
return !isNullOrEmpty(c);
}
/**
* Collects provided {@link Iterator} to a {@link Stream}
*/
public static <T> Stream<T> toStream(Iterator<T> iterator) {
return StreamSupport.stream(
Spliterators.spliteratorUnknownSize(iterator, Spliterator.ORDERED),
false
);
}
/**
* Combines provided arrays into one
*/

View File

@@ -18,32 +18,28 @@
package org.apache.hudi.io.storage;
import java.io.IOException;
import java.util.Iterator;
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.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.Option;
import java.io.IOException;
import java.io.UnsupportedEncodingException;
import java.util.List;
import java.util.Set;
public interface HoodieFileReader<R extends IndexedRecord> extends AutoCloseable {
public String[] readMinMaxRecordKeys();
String[] readMinMaxRecordKeys();
public BloomFilter readBloomFilter();
BloomFilter readBloomFilter();
public Set<String> filterRowKeys(Set<String> candidateRowKeys);
Set<String> filterRowKeys(Set<String> candidateRowKeys);
default Map<String, R> getRecordsByKeys(List<String> rowKeys) throws IOException {
throw new UnsupportedOperationException();
}
ClosableIterator<R> getRecordIterator(Schema readerSchema) throws IOException;
public Iterator<R> getRecordIterator(Schema readerSchema) throws IOException;
default Iterator<R> getRecordIterator() throws IOException {
default ClosableIterator<R> getRecordIterator() throws IOException {
return getRecordIterator(getSchema());
}
@@ -55,6 +51,22 @@ public interface HoodieFileReader<R extends IndexedRecord> extends AutoCloseable
return getRecordByKey(key, getSchema());
}
default ClosableIterator<R> getRecordsByKeysIterator(List<String> keys, Schema schema) throws IOException {
throw new UnsupportedOperationException();
}
default ClosableIterator<R> getRecordsByKeysIterator(List<String> keys) throws IOException {
return getRecordsByKeysIterator(keys, getSchema());
}
default ClosableIterator<R> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema schema) throws IOException {
throw new UnsupportedEncodingException();
}
default ClosableIterator<R> getRecordsByKeyPrefixIterator(List<String> keyPrefixes) throws IOException {
return getRecordsByKeyPrefixIterator(keyPrefixes, getSchema());
}
Schema getSchema();
void close();

View File

@@ -18,21 +18,10 @@
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.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.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;
@@ -44,97 +33,117 @@ 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.io.ByteBufferBackedInputStream;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.util.LazyRef;
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.Collections;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
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.ValidationUtils.checkState;
/**
* NOTE: PLEASE READ DOCS & COMMENTS CAREFULLY BEFORE MAKING CHANGES
* <p>
* {@link HoodieFileReader} implementation allowing to read from {@link HFile}.
*/
public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileReader<R> {
public static final String KEY_FIELD_NAME = "key";
public static final String KEY_SCHEMA = "schema";
// 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";
private static final Logger LOG = LogManager.getLogger(HoodieHFileReader.class);
private Path path;
private Configuration conf;
private HFile.Reader reader;
private FSDataInputStream fsDataInputStream;
private Schema schema;
// Scanner used to read individual keys. This is cached to prevent the overhead of opening the scanner for each
// key retrieval.
private HFileScanner keyScanner;
private final Path path;
public HoodieHFileReader(Configuration configuration, Path path, CacheConfig cacheConfig) throws IOException {
this.conf = configuration;
this.path = path;
this.reader = HoodieHFileUtils.createHFileReader(FSUtils.getFs(path.toString(), configuration), path, cacheConfig, conf);
private final LazyRef<Schema> schema;
// NOTE: Reader is ONLY THREAD-SAFE for {@code Scanner} operating in Positional Read ("pread")
// mode (ie created w/ "pread = true")
private final HFile.Reader reader;
// NOTE: Scanner caches read blocks, therefore it's important to re-use scanner
// wherever possible
private final HFileScanner sharedScanner;
private final Object sharedScannerLock = new Object();
public HoodieHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig) throws IOException {
this(path,
HoodieHFileUtils.createHFileReader(FSUtils.getFs(path.toString(), hadoopConf), path, cacheConfig, hadoopConf),
Option.empty());
}
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 = HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, configuration);
public HoodieHFileReader(Configuration hadoopConf, Path path, CacheConfig cacheConfig, FileSystem fs) throws IOException {
this(path, HoodieHFileUtils.createHFileReader(fs, path, cacheConfig, hadoopConf), Option.empty());
}
public HoodieHFileReader(FileSystem fs, Path dummyPath, byte[] content) throws IOException {
this.reader = HoodieHFileUtils.createHFileReader(fs, dummyPath, content);
public HoodieHFileReader(FileSystem fs, Path dummyPath, byte[] content, Option<Schema> schemaOpt) throws IOException {
this(null, HoodieHFileUtils.createHFileReader(fs, dummyPath, content), schemaOpt);
}
public HoodieHFileReader(Path path, HFile.Reader reader, Option<Schema> schemaOpt) throws IOException {
this.path = path;
this.reader = reader;
// For shared scanner, which is primarily used for point-lookups, we're caching blocks
// by default, to minimize amount of traffic to the underlying storage
this.sharedScanner = getHFileScanner(reader, true);
this.schema = schemaOpt.map(LazyRef::eager)
.orElseGet(() -> LazyRef.lazy(() -> fetchSchema(reader)));
}
@Override
public String[] readMinMaxRecordKeys() {
// NOTE: This access to reader is thread-safe
HFileInfo fileInfo = reader.getHFileInfo();
return new String[] {new String(fileInfo.get(KEY_MIN_RECORD.getBytes())),
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) {
HFileInfo fileInfo = reader.getHFileInfo();
schema = new Schema.Parser().parse(new String(fileInfo.get(KEY_SCHEMA.getBytes())));
}
return schema;
}
/**
* Sets up the writer schema explicitly.
*/
public void withSchema(Schema schema) {
this.schema = schema;
}
@Override
public BloomFilter readBloomFilter() {
HFileInfo fileInfo;
try {
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),
// NOTE: This access to reader is thread-safe
HFileInfo fileInfo = reader.getHFileInfo();
ByteBuff buf = reader.getMetaBlock(KEY_BLOOM_FILTER_META_BLOCK, false).getBufferWithoutHeader();
// We have to copy bytes here, since we can't reuse buffer's underlying
// array as is, since it contains additional metadata (header)
byte[] bytes = new byte[buf.remaining()];
buf.get(bytes);
return BloomFilterFactory.fromString(new String(bytes),
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 Schema getSchema() {
return schema.get();
}
/**
* Filter keys by availability.
* <p>
@@ -145,292 +154,423 @@ public class HoodieHFileReader<R extends IndexedRecord> implements HoodieFileRea
*/
@Override
public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
return candidateRowKeys.stream().filter(k -> {
try {
return isKeyAvailable(k);
} catch (IOException e) {
LOG.error("Failed to check key availability: " + k);
return false;
}
}).collect(Collectors.toSet());
}
checkState(candidateRowKeys instanceof TreeSet,
String.format("HFile reader expects a TreeSet as iterating over ordered keys is more performant, got (%s)", candidateRowKeys.getClass().getSimpleName()));
@Override
public Map<String, R> getRecordsByKeys(List<String> rowKeys) throws IOException {
return filterRecordsImpl(new TreeSet<>(rowKeys));
}
/**
* Filter records by sorted keys.
* <p>
* TODO: Implement single seek and sequential scan till the last candidate key
* instead of repeated seeks.
*
* @param sortedCandidateRowKeys - Sorted set of keys to fetch records for
* @return Map of keys to fetched records
* @throws IOException When the deserialization of records fail
*/
private synchronized Map<String, R> filterRecordsImpl(TreeSet<String> sortedCandidateRowKeys) throws IOException {
HashMap<String, R> filteredRecords = new HashMap<>();
for (String key : sortedCandidateRowKeys) {
Option<R> record = getRecordByKey(key);
if (record.isPresent()) {
filteredRecords.put(key, record.get());
}
}
return filteredRecords;
}
/**
* 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 {
final HFileScanner scanner = reader.getScanner(false, false);
if (scanner.seekTo()) {
do {
Cell c = scanner.getCell();
final Pair<String, R> keyAndRecordPair = getRecordFromCell(c, writerSchema, readerSchema, keyFieldSchema);
recordList.add(keyAndRecordPair);
} while (scanner.next());
}
return recordList;
} catch (IOException e) {
throw new HoodieException("Error reading hfile " + path + " as a dataframe", e);
}
}
/**
* 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 {
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;
List<Pair<String, R>> records = new ArrayList<>();
for (String key: keys) {
Option<R> value = getRecordByKey(key, schema);
if (value.isPresent()) {
records.add(new Pair(key, value.get()));
}
}
return records;
}
public ClosableIterator<R> getRecordIterator(List<String> keys, Schema schema) throws IOException {
this.schema = schema;
Iterator<String> iterator = keys.iterator();
return new ClosableIterator<R>() {
private R next;
@Override
public void close() {
}
@Override
public boolean hasNext() {
synchronized (sharedScannerLock) {
return candidateRowKeys.stream().filter(k -> {
try {
while (iterator.hasNext()) {
Option<R> value = getRecordByKey(iterator.next(), schema);
if (value.isPresent()) {
next = value.get();
return true;
}
}
return false;
return isKeyAvailable(k, sharedScanner);
} catch (IOException e) {
throw new HoodieIOException("unable to read next record from hfile ", e);
LOG.error("Failed to check key availability: " + k);
return false;
}
}
@Override
public R next() {
return next;
}
};
}).collect(Collectors.toSet());
}
}
@SuppressWarnings("unchecked")
@Override
public Iterator getRecordIterator(Schema readerSchema) throws IOException {
final HFileScanner scanner = reader.getScanner(false, false);
final Option<Schema.Field> keyFieldSchema = Option.ofNullable(readerSchema.getField(KEY_FIELD_NAME));
ValidationUtils.checkState(keyFieldSchema != null,
"Missing key field '" + KEY_FIELD_NAME + "' in the schema!");
return new Iterator<R>() {
private R next = null;
private boolean eof = false;
@Override
public boolean hasNext() {
try {
// 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.getCell(), getSchema(), readerSchema, keyFieldSchema);
this.next = keyAndRecordPair.getSecond();
}
}
return this.next != null;
} catch (IOException io) {
throw new HoodieIOException("unable to read next record from hfile ", io);
}
}
@Override
public R next() {
try {
// To handle case when next() is called before hasNext()
if (this.next == null) {
if (!hasNext()) {
throw new HoodieIOException("No more records left to read from hfile");
}
}
R retVal = this.next;
if (scanner.next()) {
final Pair<String, R> keyAndRecordPair = getRecordFromCell(scanner.getCell(), getSchema(), readerSchema, keyFieldSchema);
this.next = keyAndRecordPair.getSecond();
} else {
this.next = null;
this.eof = true;
}
return retVal;
} catch (IOException io) {
throw new HoodieIOException("unable to read next record from parquet file ", io);
}
}
};
}
private boolean isKeyAvailable(String key) throws IOException {
final KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
synchronized (this) {
if (keyScanner == null) {
keyScanner = reader.getScanner(false, false);
}
if (keyScanner.seekTo(kv) == 0) {
return true;
}
public Option<R> getRecordByKey(String key, Schema readerSchema) throws IOException {
synchronized (sharedScannerLock) {
return (Option<R>) fetchRecordByKeyInternal(sharedScanner, key, getSchema(), readerSchema);
}
return false;
}
@SuppressWarnings("unchecked")
@Override
public Option getRecordByKey(String key, Schema readerSchema) throws IOException {
byte[] value = null;
final Option<Schema.Field> keyFieldSchema = Option.ofNullable(readerSchema.getField(KEY_FIELD_NAME));
ValidationUtils.checkState(keyFieldSchema != null);
KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
synchronized (this) {
if (keyScanner == null) {
keyScanner = reader.getScanner(false, false);
}
if (keyScanner.seekTo(kv) == 0) {
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());
}
}
if (value != null) {
R record = deserialize(key.getBytes(), value, getSchema(), readerSchema, keyFieldSchema);
return Option.of(record);
}
return Option.empty();
public ClosableIterator<R> getRecordIterator(Schema readerSchema) throws IOException {
// TODO eval whether seeking scanner would be faster than pread
HFileScanner scanner = getHFileScanner(reader, false);
return (ClosableIterator<R>) new RecordIterator(scanner, getSchema(), readerSchema);
}
private Pair<String, R> getRecordFromCell(Cell cell, Schema writerSchema, Schema readerSchema, Option<Schema.Field> keyFieldSchema) throws IOException {
final byte[] keyBytes = Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), cell.getRowOffset() + cell.getRowLength());
final byte[] valueBytes = Arrays.copyOfRange(cell.getValueArray(), cell.getValueOffset(), cell.getValueOffset() + cell.getValueLength());
R record = deserialize(keyBytes, valueBytes, writerSchema, readerSchema, keyFieldSchema);
return new Pair<>(new String(keyBytes), record);
@SuppressWarnings("unchecked")
@Override
public ClosableIterator<R> getRecordsByKeysIterator(List<String> keys, Schema readerSchema) throws IOException {
// We're caching blocks for this scanner to minimize amount of traffic
// to the underlying storage as we fetched (potentially) sparsely distributed
// keys
HFileScanner scanner = getHFileScanner(reader, true);
return (ClosableIterator<R>) new RecordByKeyIterator(scanner, keys, getSchema(), readerSchema);
}
/**
* Deserialize the record byte array contents to record object.
*
* @param keyBytes - Record key as byte array
* @param valueBytes - Record content as byte array
* @param writerSchema - Writer schema
* @param readerSchema - Reader schema
* @param keyFieldSchema - Key field id in the schema
* @return Deserialized record object
*/
private R deserialize(final byte[] keyBytes, final byte[] valueBytes, Schema writerSchema, Schema readerSchema,
Option<Schema.Field> keyFieldSchema) throws IOException {
R record = (R) HoodieAvroUtils.bytesToAvro(valueBytes, writerSchema, readerSchema);
materializeRecordIfNeeded(keyBytes, record, keyFieldSchema);
return record;
}
/**
* Materialize the record for any missing fields, if needed.
*
* @param keyBytes - Key byte array
* @param record - Record object to materialize
* @param keyFieldSchema - Key field id in the schema
*/
private void materializeRecordIfNeeded(final byte[] keyBytes, R record, Option<Schema.Field> keyFieldSchema) {
if (keyFieldSchema.isPresent()) {
final Object keyObject = record.get(keyFieldSchema.get().pos());
if (keyObject != null && keyObject.toString().isEmpty()) {
record.put(keyFieldSchema.get().pos(), new String(keyBytes));
}
}
@SuppressWarnings("unchecked")
@Override
public ClosableIterator<R> getRecordsByKeyPrefixIterator(List<String> keyPrefixes, Schema readerSchema) throws IOException {
// We're caching blocks for this scanner to minimize amount of traffic
// to the underlying storage as we fetched (potentially) sparsely distributed
// keys
HFileScanner scanner = getHFileScanner(reader, true);
return (ClosableIterator<R>) new RecordByKeyPrefixIterator(scanner, keyPrefixes, getSchema(), readerSchema);
}
@Override
public long getTotalRecords() {
// NOTE: This access to reader is thread-safe
return reader.getEntries();
}
@Override
public synchronized void close() {
public void close() {
try {
reader.close();
reader = null;
if (fsDataInputStream != null) {
fsDataInputStream.close();
synchronized (this) {
reader.close();
}
keyScanner = null;
} catch (IOException e) {
throw new HoodieIOException("Error closing the hfile reader", e);
}
}
private boolean isKeyAvailable(String key, HFileScanner keyScanner) throws IOException {
final KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
return keyScanner.seekTo(kv) == 0;
}
private static Iterator<GenericRecord> getRecordByKeyPrefixIteratorInternal(HFileScanner scanner,
String keyPrefix,
Schema writerSchema,
Schema readerSchema) throws IOException {
KeyValue kv = new KeyValue(keyPrefix.getBytes(), null, null, null);
// NOTE: HFile persists both keys/values as bytes, therefore lexicographical sorted is
// essentially employed
//
// For the HFile containing list of cells c[0], c[1], ..., c[N], `seekTo(cell)` would return
// following:
// a) -1, if cell < c[0], no position;
// b) 0, such that c[i] = cell and scanner is left in position i;
// c) and 1, such that c[i] < cell, and scanner is left in position i.
//
// Consider entries w/ the following keys in HFile: [key01, key02, key03, key04,..., key20];
// In case looked up key-prefix is
// - "key", `seekTo()` will return -1 and place the cursor just before "key01",
// `getCell()` will return "key01" entry
// - "key03", `seekTo()` will return 0 (exact match) and place the cursor just before "key03",
// `getCell()` will return "key03" entry
// - "key1", `seekTo()` will return 1 (first not lower than) and place the cursor just before
// "key10" (i.e. on "key09");
//
int val = scanner.seekTo(kv);
if (val == 1) {
// Try moving to next entry, matching the prefix key; if we're at the EOF,
// `next()` will return false
if (!scanner.next()) {
return Collections.emptyIterator();
}
}
class KeyPrefixIterator implements Iterator<GenericRecord> {
private GenericRecord next = null;
private boolean eof = false;
@Override
public boolean hasNext() {
if (next != null) {
return true;
} else if (eof) {
return false;
}
Cell c = Objects.requireNonNull(scanner.getCell());
byte[] keyBytes = copyKeyFromCell(c);
String key = new String(keyBytes);
// Check whether we're still reading records corresponding to the key-prefix
if (!key.startsWith(keyPrefix)) {
return false;
}
// Extract the byte value before releasing the lock since we cannot hold on to the returned cell afterwards
byte[] valueBytes = copyValueFromCell(c);
try {
next = deserialize(keyBytes, valueBytes, writerSchema, readerSchema);
// In case scanner is not able to advance, it means we reached EOF
eof = !scanner.next();
} catch (IOException e) {
throw new HoodieIOException("Failed to deserialize payload", e);
}
return true;
}
@Override
public GenericRecord next() {
GenericRecord next = this.next;
this.next = null;
return next;
}
}
return new KeyPrefixIterator();
}
private static Option<GenericRecord> fetchRecordByKeyInternal(HFileScanner scanner, String key, Schema writerSchema, Schema readerSchema) throws IOException {
KeyValue kv = new KeyValue(key.getBytes(), null, null, null);
if (scanner.seekTo(kv) != 0) {
return Option.empty();
}
Cell c = scanner.getCell();
byte[] valueBytes = copyValueFromCell(c);
GenericRecord record = deserialize(key.getBytes(), valueBytes, writerSchema, readerSchema);
return Option.of(record);
}
private static GenericRecord getRecordFromCell(Cell cell, Schema writerSchema, Schema readerSchema) throws IOException {
final byte[] keyBytes = copyKeyFromCell(cell);
final byte[] valueBytes = copyValueFromCell(cell);
return deserialize(keyBytes, valueBytes, writerSchema, readerSchema);
}
private static GenericRecord deserializeUnchecked(final byte[] keyBytes,
final byte[] valueBytes,
Schema writerSchema,
Schema readerSchema) {
try {
return deserialize(keyBytes, valueBytes, writerSchema, readerSchema);
} catch (IOException e) {
throw new HoodieIOException("Failed to deserialize payload", e);
}
}
private static GenericRecord deserialize(final byte[] keyBytes,
final byte[] valueBytes,
Schema writerSchema,
Schema readerSchema) throws IOException {
GenericRecord record = HoodieAvroUtils.bytesToAvro(valueBytes, writerSchema, readerSchema);
getKeySchema(readerSchema).ifPresent(keyFieldSchema -> {
final Object keyObject = record.get(keyFieldSchema.pos());
if (keyObject != null && keyObject.toString().isEmpty()) {
record.put(keyFieldSchema.pos(), new String(keyBytes));
}
});
return record;
}
private static Schema fetchSchema(HFile.Reader reader) {
HFileInfo fileInfo = reader.getHFileInfo();
return new Schema.Parser().parse(new String(fileInfo.get(SCHEMA_KEY.getBytes())));
}
private static byte[] copyKeyFromCell(Cell cell) {
return Arrays.copyOfRange(cell.getRowArray(), cell.getRowOffset(), cell.getRowOffset() + cell.getRowLength());
}
private static byte[] copyValueFromCell(Cell c) {
return Arrays.copyOfRange(c.getValueArray(), c.getValueOffset(), c.getValueOffset() + c.getValueLength());
}
/**
* NOTE: THIS SHOULD ONLY BE USED FOR TESTING, RECORDS ARE MATERIALIZED EAGERLY
* <p>
* Reads all the records with given schema
*/
public static <R extends IndexedRecord> List<R> readAllRecords(HoodieHFileReader<R> reader) throws IOException {
Schema schema = reader.getSchema();
return toStream(reader.getRecordIterator(schema))
.collect(Collectors.toList());
}
/**
* NOTE: THIS SHOULD ONLY BE USED FOR TESTING, RECORDS ARE MATERIALIZED EAGERLY
* <p>
* Reads all the records with given schema and filtering keys.
*/
public static <R extends IndexedRecord> List<R> readRecords(HoodieHFileReader<R> reader,
List<String> keys) throws IOException {
return readRecords(reader, keys, reader.getSchema());
}
/**
* NOTE: THIS SHOULD ONLY BE USED FOR TESTING, RECORDS ARE MATERIALIZED EAGERLY
* <p>
* Reads all the records with given schema and filtering keys.
*/
public static <R extends IndexedRecord> List<R> readRecords(HoodieHFileReader<R> reader,
List<String> keys,
Schema schema) throws IOException {
Collections.sort(keys);
return toStream(reader.getRecordsByKeysIterator(keys, schema))
.collect(Collectors.toList());
}
private static HFileScanner getHFileScanner(HFile.Reader reader, boolean cacheBlocks) {
// NOTE: Only scanners created in Positional Read ("pread") mode could share the same reader,
// since scanners in default mode will be seeking w/in the underlying stream
return reader.getScanner(cacheBlocks, true);
}
private static Option<Schema.Field> getKeySchema(Schema schema) {
return Option.ofNullable(schema.getField(KEY_FIELD_NAME));
}
private static class RecordByKeyPrefixIterator implements ClosableIterator<GenericRecord> {
private final Iterator<String> keyPrefixesIterator;
private Iterator<GenericRecord> recordsIterator;
private final HFileScanner scanner;
private final Schema writerSchema;
private final Schema readerSchema;
private GenericRecord next = null;
RecordByKeyPrefixIterator(HFileScanner scanner, List<String> keyPrefixes, Schema writerSchema, Schema readerSchema) throws IOException {
this.keyPrefixesIterator = keyPrefixes.iterator();
this.scanner = scanner;
this.scanner.seekTo(); // position at the beginning of the file
this.writerSchema = writerSchema;
this.readerSchema = readerSchema;
}
@Override
public boolean hasNext() {
try {
while (true) {
// NOTE: This is required for idempotency
if (next != null) {
return true;
} else if (recordsIterator != null && recordsIterator.hasNext()) {
next = recordsIterator.next();
return true;
} else if (keyPrefixesIterator.hasNext()) {
String currentKeyPrefix = keyPrefixesIterator.next();
recordsIterator =
getRecordByKeyPrefixIteratorInternal(scanner, currentKeyPrefix, writerSchema, readerSchema);
} else {
return false;
}
}
} catch (IOException e) {
throw new HoodieIOException("Unable to read next record from HFile", e);
}
}
@Override
public GenericRecord next() {
GenericRecord next = this.next;
this.next = null;
return next;
}
@Override
public void close() {
scanner.close();
}
}
private static class RecordByKeyIterator implements ClosableIterator<GenericRecord> {
private final Iterator<String> keyIterator;
private final HFileScanner scanner;
private final Schema readerSchema;
private final Schema writerSchema;
private GenericRecord next = null;
RecordByKeyIterator(HFileScanner scanner, List<String> keys, Schema writerSchema, Schema readerSchema) throws IOException {
this.keyIterator = keys.iterator();
this.scanner = scanner;
this.scanner.seekTo(); // position at the beginning of the file
this.writerSchema = writerSchema;
this.readerSchema = readerSchema;
}
@Override
public boolean hasNext() {
try {
// NOTE: This is required for idempotency
if (next != null) {
return true;
}
while (keyIterator.hasNext()) {
Option<GenericRecord> value = fetchRecordByKeyInternal(scanner, keyIterator.next(), writerSchema, readerSchema);
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 GenericRecord next() {
GenericRecord next = this.next;
this.next = null;
return next;
}
@Override
public void close() {
scanner.close();
}
}
private static class RecordIterator implements ClosableIterator<GenericRecord> {
private final HFileScanner scanner;
private final Schema writerSchema;
private final Schema readerSchema;
private GenericRecord next = null;
RecordIterator(HFileScanner scanner, Schema writerSchema, Schema readerSchema) {
this.scanner = scanner;
this.writerSchema = writerSchema;
this.readerSchema = readerSchema;
}
@Override
public boolean hasNext() {
try {
// NOTE: This is required for idempotency
if (next != null) {
return true;
}
boolean hasRecords;
if (!scanner.isSeeked()) {
hasRecords = scanner.seekTo();
} else {
hasRecords = scanner.next();
}
if (!hasRecords) {
return false;
}
this.next = getRecordFromCell(scanner.getCell(), writerSchema, readerSchema);
return true;
} catch (IOException io) {
throw new HoodieIOException("unable to read next record from hfile ", io);
}
}
@Override
public GenericRecord next() {
GenericRecord next = this.next;
this.next = null;
return next;
}
@Override
public void close() {
scanner.close();
}
}
static class SeekableByteArrayInputStream extends ByteBufferBackedInputStream implements Seekable, PositionedReadable {
public SeekableByteArrayInputStream(byte[] buf) {
super(buf);

View File

@@ -18,9 +18,6 @@
package org.apache.hudi.io.storage;
import java.io.IOException;
import java.util.Iterator;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
@@ -29,6 +26,7 @@ import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.util.AvroOrcUtils;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.OrcReaderIterator;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.orc.OrcFile;
@@ -37,6 +35,9 @@ import org.apache.orc.Reader.Options;
import org.apache.orc.RecordReader;
import org.apache.orc.TypeDescription;
import java.io.IOException;
import java.util.Set;
public class HoodieOrcReader<R extends IndexedRecord> implements HoodieFileReader {
private Path path;
private Configuration conf;
@@ -64,12 +65,12 @@ public class HoodieOrcReader<R extends IndexedRecord> implements HoodieFileReade
}
@Override
public Iterator<R> getRecordIterator(Schema schema) throws IOException {
public ClosableIterator<R> getRecordIterator(Schema schema) throws IOException {
try {
Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf));
TypeDescription orcSchema = AvroOrcUtils.createOrcSchema(schema);
RecordReader recordReader = reader.rows(new Options(conf).schema(orcSchema));
return new OrcReaderIterator(recordReader, schema, orcSchema);
return new OrcReaderIterator<>(recordReader, schema, orcSchema);
} catch (IOException io) {
throw new HoodieIOException("Unable to create an ORC reader.", io);
}

View File

@@ -18,12 +18,6 @@
package org.apache.hudi.io.storage;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.conf.Configuration;
@@ -31,12 +25,17 @@ import org.apache.hadoop.fs.Path;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Set;
public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileReader<R> {
private final Path path;
@@ -66,10 +65,10 @@ public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileR
}
@Override
public Iterator<R> getRecordIterator(Schema schema) throws IOException {
public ClosableIterator<R> getRecordIterator(Schema schema) throws IOException {
AvroReadSupport.setAvroReadSchema(conf, schema);
ParquetReader<R> reader = AvroParquetReader.<R>builder(path).withConf(conf).build();
ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader);
ParquetReaderIterator<R> parquetReaderIterator = new ParquetReaderIterator<>(reader);
readerIterators.add(parquetReaderIterator);
return parquetReaderIterator;
}
@@ -81,7 +80,7 @@ public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileR
@Override
public void close() {
readerIterators.forEach(entry -> entry.close());
readerIterators.forEach(ParquetReaderIterator::close);
}
@Override

View File

@@ -378,7 +378,7 @@ public abstract class BaseTableMetadata implements HoodieTableMetadata {
protected abstract Option<HoodieRecord<HoodieMetadataPayload>> getRecordByKey(String key, String partitionName);
protected abstract List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> key, String partitionName);
public abstract List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> key, String partitionName);
protected HoodieEngineContext getEngineContext() {
return engineContext != null ? engineContext : new HoodieLocalEngineContext(hadoopConf.get());

View File

@@ -21,9 +21,11 @@ package org.apache.hudi.metadata;
import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
@@ -159,4 +161,9 @@ public class FileSystemBackedTableMetadata implements HoodieTableMetadata {
throws HoodieMetadataException {
throw new HoodieMetadataException("Unsupported operation: getColumnsStats!");
}
@Override
public HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes, String partitionName) {
throw new HoodieMetadataException("Unsupported operation: getRecordsByKeyPrefixes!");
}
}

View File

@@ -18,6 +18,9 @@
package org.apache.hudi.metadata;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.model.HoodieMetadataRecord;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
@@ -25,23 +28,24 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.common.config.HoodieCommonConfig;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.function.SerializableFunction;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.util.ClosableIterator;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.SpillableMapUtils;
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;
@@ -49,10 +53,6 @@ import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.io.storage.HoodieFileReader;
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -60,15 +60,22 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import static org.apache.hudi.common.util.CollectionUtils.isNullOrEmpty;
import static org.apache.hudi.common.util.CollectionUtils.toStream;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_BLOOM_FILTERS;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS;
import static org.apache.hudi.metadata.HoodieTableMetadataUtil.PARTITION_NAME_FILES;
/**
* Table metadata provided by an internal DFS backed Hudi metadata table.
@@ -77,6 +84,8 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
private static final Logger LOG = LogManager.getLogger(HoodieBackedTableMetadata.class);
private static final Schema METADATA_RECORD_SCHEMA = HoodieMetadataRecord.getClassSchema();
private String metadataBasePath;
// Metadata table's timeline and metaclient
private HoodieTableMetaClient metadataMetaClient;
@@ -133,28 +142,79 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
}
@Override
protected List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> keys,
String partitionName) {
public HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes,
String partitionName) {
// NOTE: Since we partition records to a particular file-group by full key, we will have
// to scan all file-groups for all key-prefixes as each of these might contain some
// records matching the key-prefix
List<FileSlice> partitionFileSlices =
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(metadataMetaClient, partitionName);
return engineContext.parallelize(partitionFileSlices)
.flatMap(
(SerializableFunction<FileSlice, Iterator<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>>>) fileSlice -> {
// NOTE: Since this will be executed by executors, we can't access previously cached
// readers, and therefore have to always open new ones
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
openReaders(partitionName, fileSlice);
try {
List<Long> timings = new ArrayList<>();
HoodieFileReader baseFileReader = readers.getKey();
HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
if (baseFileReader == null && logRecordScanner == null) {
// TODO: what do we do if both does not exist? should we throw an exception and let caller do the fallback ?
return Collections.emptyIterator();
}
boolean fullKeys = false;
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords =
readLogRecords(logRecordScanner, keyPrefixes, fullKeys, timings);
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> mergedRecords =
readFromBaseAndMergeWithLogRecords(baseFileReader, keyPrefixes, fullKeys, logRecords, timings, partitionName);
LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms",
keyPrefixes.size(), timings));
return mergedRecords.iterator();
} catch (IOException ioe) {
throw new HoodieIOException("Error merging records from metadata table for " + keyPrefixes.size() + " key : ", ioe);
} finally {
closeReader(readers);
}
}
)
.map(keyRecordPair -> keyRecordPair.getValue().orElse(null))
.filter(Objects::nonNull);
}
@Override
public List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> keys,
String partitionName) {
Map<Pair<String, FileSlice>, List<String>> partitionFileSliceToKeysMap = getPartitionFileSliceToKeysMapping(partitionName, keys);
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> result = new ArrayList<>();
AtomicInteger fileSlicesKeysCount = new AtomicInteger();
partitionFileSliceToKeysMap.forEach((partitionFileSlicePair, fileSliceKeys) -> {
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers = openReadersIfNeeded(partitionName,
partitionFileSlicePair.getRight());
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
getOrCreateReaders(partitionName, partitionFileSlicePair.getRight());
try {
List<Long> timings = new ArrayList<>();
HoodieFileReader baseFileReader = readers.getKey();
HoodieMetadataMergedLogRecordReader logRecordScanner = readers.getRight();
if (baseFileReader == null && logRecordScanner == null) {
return;
}
// local map to assist in merging with base file records
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords = readLogRecords(logRecordScanner,
fileSliceKeys, timings);
result.addAll(readFromBaseAndMergeWithLogRecords(baseFileReader, fileSliceKeys, logRecords,
boolean fullKeys = true;
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords =
readLogRecords(logRecordScanner, fileSliceKeys, fullKeys, timings);
result.addAll(readFromBaseAndMergeWithLogRecords(baseFileReader, fileSliceKeys, fullKeys, logRecords,
timings, partitionName));
LOG.debug(String.format("Metadata read for %s keys took [baseFileRead, logMerge] %s ms",
fileSliceKeys.size(), timings));
fileSlicesKeysCount.addAndGet(fileSliceKeys.size());
@@ -171,81 +231,127 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
}
private Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> readLogRecords(HoodieMetadataMergedLogRecordReader logRecordScanner,
List<String> keys, List<Long> timings) {
List<String> keys,
boolean fullKey,
List<Long> timings) {
HoodieTimer timer = new HoodieTimer().startTimer();
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords = new HashMap<>();
// Retrieve records from log file
timer.startTimer();
if (logRecordScanner != null) {
if (metadataConfig.enableFullScan()) {
// path which does full scan of log files
for (String key : keys) {
logRecords.put(key, logRecordScanner.getRecordByKey(key).get(0).getValue());
}
} else {
// this path will do seeks pertaining to the keys passed in
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> logRecordsList = logRecordScanner.getRecordsByKeys(keys);
for (Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : logRecordsList) {
logRecords.put(entry.getKey(), entry.getValue());
}
if (logRecordScanner == null) {
timings.add(timer.endTimer());
return Collections.emptyMap();
}
String partitionName = logRecordScanner.getPartitionName().get();
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords = new HashMap<>();
if (isFullScanAllowedForPartition(partitionName)) {
checkArgument(fullKey, "If full-scan is required, only full keys could be used!");
// Path which does full scan of log files
for (String key : keys) {
logRecords.put(key, logRecordScanner.getRecordByKey(key).get(0).getValue());
}
} else {
for (String key : keys) {
logRecords.put(key, Option.empty());
// This path will do seeks pertaining to the keys passed in
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> logRecordsList =
fullKey ? logRecordScanner.getRecordsByKeys(keys)
: logRecordScanner.getRecordsByKeyPrefixes(keys)
.stream()
.map(record -> Pair.of(record.getRecordKey(), Option.of(record)))
.collect(Collectors.toList());
for (Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : logRecordsList) {
logRecords.put(entry.getKey(), entry.getValue());
}
}
timings.add(timer.endTimer());
return logRecords;
}
private List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> readFromBaseAndMergeWithLogRecords(HoodieFileReader baseFileReader,
List<String> keys, Map<String,
Option<HoodieRecord<HoodieMetadataPayload>>> logRecords, List<Long> timings, String partitionName) throws IOException {
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> result = new ArrayList<>();
// merge with base records
List<String> keys,
boolean fullKeys,
Map<String, Option<HoodieRecord<HoodieMetadataPayload>>> logRecords,
List<Long> timings,
String partitionName) throws IOException {
HoodieTimer timer = new HoodieTimer().startTimer();
timer.startTimer();
HoodieRecord<HoodieMetadataPayload> hoodieRecord = null;
// Retrieve record from base file
if (baseFileReader != null) {
HoodieTimer readTimer = new HoodieTimer();
Map<String, GenericRecord> baseFileRecords = baseFileReader.getRecordsByKeys(keys);
for (String key : keys) {
readTimer.startTimer();
if (baseFileRecords.containsKey(key)) {
hoodieRecord = getRecord(Option.of(baseFileRecords.get(key)), partitionName);
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
// merge base file record w/ log record if present
if (logRecords.containsKey(key) && logRecords.get(key).isPresent()) {
HoodieRecordPayload mergedPayload = logRecords.get(key).get().getData().preCombine(hoodieRecord.getData());
result.add(Pair.of(key, Option.of(new HoodieAvroRecord(hoodieRecord.getKey(), mergedPayload))));
} else {
// only base record
result.add(Pair.of(key, Option.of(hoodieRecord)));
}
} else {
// only log record
result.add(Pair.of(key, logRecords.get(key)));
}
}
if (baseFileReader == null) {
// No base file at all
timings.add(timer.endTimer());
} else {
// no base file at all
timings.add(timer.endTimer());
for (Map.Entry<String, Option<HoodieRecord<HoodieMetadataPayload>>> entry : logRecords.entrySet()) {
result.add(Pair.of(entry.getKey(), entry.getValue()));
if (fullKeys) {
// In case full-keys (not key-prefixes) were provided, it's expected that the list of
// records will contain an (optional) entry for each corresponding key
return keys.stream()
.map(key -> Pair.of(key, logRecords.getOrDefault(key, Option.empty())))
.collect(Collectors.toList());
} else {
return logRecords.entrySet().stream()
.map(entry -> Pair.of(entry.getKey(), entry.getValue()))
.collect(Collectors.toList());
}
}
return result;
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> result = new ArrayList<>();
HoodieTimer readTimer = new HoodieTimer();
readTimer.startTimer();
Map<String, HoodieRecord<HoodieMetadataPayload>> records =
fetchBaseFileRecordsByKeys(baseFileReader, keys, fullKeys, partitionName);
metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.BASEFILE_READ_STR, readTimer.endTimer()));
// Iterate over all provided log-records, merging them into existing records
for (Option<HoodieRecord<HoodieMetadataPayload>> logRecordOpt : logRecords.values()) {
if (logRecordOpt.isPresent()) {
HoodieRecord<HoodieMetadataPayload> logRecord = logRecordOpt.get();
records.merge(
logRecord.getRecordKey(),
logRecord,
(oldRecord, newRecord) ->
new HoodieAvroRecord<>(oldRecord.getKey(), newRecord.getData().preCombine(oldRecord.getData()))
);
}
}
timings.add(timer.endTimer());
if (fullKeys) {
// In case full-keys (not key-prefixes) were provided, it's expected that the list of
// records will contain an (optional) entry for each corresponding key
return keys.stream()
.map(key -> Pair.of(key, Option.ofNullable(records.get(key))))
.collect(Collectors.toList());
} else {
return records.values().stream()
.map(record -> Pair.of(record.getRecordKey(), Option.of(record)))
.collect(Collectors.toList());
}
}
private HoodieRecord<HoodieMetadataPayload> getRecord(Option<GenericRecord> baseRecord, String partitionName) {
ValidationUtils.checkState(baseRecord.isPresent());
private Map<String, HoodieRecord<HoodieMetadataPayload>> fetchBaseFileRecordsByKeys(HoodieFileReader<GenericRecord> baseFileReader,
List<String> keys,
boolean fullKeys,
String partitionName) throws IOException {
ClosableIterator<GenericRecord> records = fullKeys ? baseFileReader.getRecordsByKeysIterator(keys)
: baseFileReader.getRecordsByKeyPrefixIterator(keys);
return toStream(records)
.map(record -> Pair.of(
(String) record.get(HoodieMetadataPayload.KEY_FIELD_NAME),
composeRecord(record, partitionName)))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
}
private HoodieRecord<HoodieMetadataPayload> composeRecord(GenericRecord avroRecord, String partitionName) {
if (metadataTableConfig.populateMetaFields()) {
return SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
return SpillableMapUtils.convertToHoodieRecordPayload(avroRecord,
metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(), false);
}
return SpillableMapUtils.convertToHoodieRecordPayload(baseRecord.get(),
return SpillableMapUtils.convertToHoodieRecordPayload(avroRecord,
metadataTableConfig.getPayloadClass(), metadataTableConfig.getPreCombineField(),
Pair.of(metadataTableConfig.getRecordKeyFieldProp(), metadataTableConfig.getPartitionFieldProp()),
false, Option.of(partitionName));
@@ -279,34 +385,35 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
* Create a file reader and the record scanner for a given partition and file slice
* if readers are not already available.
*
* @param partitionName - Partition name
* @param slice - The file slice to open readers for
* @param partitionName - Partition name
* @param slice - The file slice to open readers for
* @return File reader and the record scanner pair for the requested file slice
*/
private Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> openReadersIfNeeded(String partitionName, FileSlice slice) {
return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> {
try {
HoodieTimer timer = new HoodieTimer().startTimer();
private Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> getOrCreateReaders(String partitionName, FileSlice slice) {
return partitionReaders.computeIfAbsent(Pair.of(partitionName, slice.getFileId()), k -> openReaders(partitionName, slice));
}
// Open base file reader
Pair<HoodieFileReader, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey();
final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
private Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> openReaders(String partitionName, FileSlice slice) {
try {
HoodieTimer timer = new HoodieTimer().startTimer();
// Open base file reader
Pair<HoodieFileReader, Long> baseFileReaderOpenTimePair = getBaseFileReader(slice, timer);
HoodieFileReader baseFileReader = baseFileReaderOpenTimePair.getKey();
final long baseFileOpenMs = baseFileReaderOpenTimePair.getValue();
// Open the log record scanner using the log files from the latest file slice
List<HoodieLogFile> logFiles = slice.getLogFiles().collect(Collectors.toList());
Pair<HoodieMetadataMergedLogRecordReader, Long> logRecordScannerOpenTimePair =
getLogRecordScanner(logFiles, partitionName);
HoodieMetadataMergedLogRecordReader logRecordScanner = logRecordScannerOpenTimePair.getKey();
final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue();
// Open the log record scanner using the log files from the latest file slice
List<HoodieLogFile> logFiles = slice.getLogFiles().collect(Collectors.toList());
Pair<HoodieMetadataMergedLogRecordReader, Long> logRecordScannerOpenTimePair =
getLogRecordScanner(logFiles, partitionName);
HoodieMetadataMergedLogRecordReader logRecordScanner = logRecordScannerOpenTimePair.getKey();
final long logScannerOpenMs = logRecordScannerOpenTimePair.getValue();
metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR,
+baseFileOpenMs + logScannerOpenMs));
return Pair.of(baseFileReader, logRecordScanner);
} catch (IOException e) {
throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e);
}
});
metrics.ifPresent(metrics -> metrics.updateMetrics(HoodieMetadataMetrics.SCAN_STR,
+baseFileOpenMs + logScannerOpenMs));
return Pair.of(baseFileReader, logRecordScanner);
} catch (IOException e) {
throw new HoodieIOException("Error opening readers for metadata table partition " + partitionName, e);
}
}
private Pair<HoodieFileReader, Long> getBaseFileReader(FileSlice slice, HoodieTimer timer) throws IOException {
@@ -349,7 +456,14 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
return validInstantTimestamps;
}
public Pair<HoodieMetadataMergedLogRecordReader, Long> getLogRecordScanner(List<HoodieLogFile> logFiles, String partitionName) {
public Pair<HoodieMetadataMergedLogRecordReader, Long> getLogRecordScanner(List<HoodieLogFile> logFiles,
String partitionName) {
return getLogRecordScanner(logFiles, partitionName, Option.empty());
}
public Pair<HoodieMetadataMergedLogRecordReader, Long> getLogRecordScanner(List<HoodieLogFile> logFiles,
String partitionName,
Option<Boolean> allowFullScanOverride) {
HoodieTimer timer = new HoodieTimer().startTimer();
List<String> sortedLogFilePaths = logFiles.stream()
.sorted(HoodieLogFile.getLogFileComparator())
@@ -363,6 +477,8 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
Option<HoodieInstant> latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant();
String latestMetadataInstantTime = latestMetadataInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP);
boolean allowFullScan = allowFullScanOverride.orElseGet(() -> isFullScanAllowedForPartition(partitionName));
// Load the schema
Schema schema = HoodieAvroUtils.addMetadataFields(HoodieMetadataRecord.getClassSchema());
HoodieCommonConfig commonConfig = HoodieCommonConfig.newBuilder().fromProperties(metadataConfig.getProps()).build();
@@ -378,7 +494,7 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
.withDiskMapType(commonConfig.getSpillableDiskMapType())
.withBitCaskDiskMapCompressionEnabled(commonConfig.isBitCaskDiskMapCompressionEnabled())
.withLogBlockTimestamps(validInstantTimestamps)
.enableFullScan(metadataConfig.enableFullScan())
.allowFullScan(allowFullScan)
.withPartition(partitionName)
.build();
@@ -388,6 +504,21 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
return Pair.of(logRecordScanner, logScannerOpenMs);
}
// NOTE: We're allowing eager full-scan of the log-files only for "files" partition.
// Other partitions (like "column_stats", "bloom_filters") will have to be fetched
// t/h point-lookups
private boolean isFullScanAllowedForPartition(String partitionName) {
switch (partitionName) {
case PARTITION_NAME_FILES:
return metadataConfig.allowFullScan();
case PARTITION_NAME_COLUMN_STATS:
case PARTITION_NAME_BLOOM_FILTERS:
default:
return false;
}
}
/**
* Returns a list of commits which were rolled back as part of a Rollback or Restore operation.
*
@@ -433,6 +564,10 @@ public class HoodieBackedTableMetadata extends BaseTableMetadata {
private synchronized void close(Pair<String, String> partitionFileSlicePair) {
Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers =
partitionReaders.remove(partitionFileSlicePair);
closeReader(readers);
}
private void closeReader(Pair<HoodieFileReader, HoodieMetadataMergedLogRecordReader> readers) {
if (readers != null) {
try {
if (readers.getKey() != null) {

View File

@@ -18,11 +18,13 @@
package org.apache.hudi.metadata;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.DeleteRecord;
import org.apache.hudi.common.model.HoodieAvroRecord;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.log.InstantRange;
@@ -31,19 +33,16 @@ import org.apache.hudi.common.util.SpillableMapUtils;
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.internal.schema.InternalSchema;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/**
* A {@code HoodieMergedLogRecordScanner} implementation which only merged records matching providing keys. This is
@@ -53,38 +52,16 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
private static final Logger LOG = LogManager.getLogger(HoodieMetadataMergedLogRecordReader.class);
// Set of all record keys that are to be read in memory
private Set<String> mergeKeyFilter;
private HoodieMetadataMergedLogRecordReader(FileSystem fs, String basePath, String partitionName,
List<String> logFilePaths,
Schema readerSchema, String latestInstantTime,
Long maxMemorySizeInBytes, int bufferSize,
String spillableMapBasePath, Set<String> mergeKeyFilter,
String spillableMapBasePath,
ExternalSpillableMap.DiskMapType diskMapType,
boolean isBitCaskDiskMapCompressionEnabled,
Option<InstantRange> instantRange, boolean enableFullScan) {
Option<InstantRange> instantRange, boolean allowFullScan) {
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, maxMemorySizeInBytes, false, false, bufferSize,
spillableMapBasePath, instantRange, false, diskMapType, isBitCaskDiskMapCompressionEnabled, false,
enableFullScan, Option.of(partitionName), InternalSchema.getEmptyInternalSchema());
this.mergeKeyFilter = mergeKeyFilter;
if (enableFullScan) {
performScan();
}
}
@Override
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws IOException {
if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(hoodieRecord.getRecordKey())) {
super.processNextRecord(hoodieRecord);
}
}
@Override
protected void processNextDeletedRecord(DeleteRecord deleteRecord) {
if (mergeKeyFilter.isEmpty() || mergeKeyFilter.contains(deleteRecord.getRecordKey())) {
super.processNextDeletedRecord(deleteRecord);
}
spillableMapBasePath, instantRange, diskMapType, isBitCaskDiskMapCompressionEnabled, false, allowFullScan, Option.of(partitionName), InternalSchema.getEmptyInternalSchema());
}
@Override
@@ -118,24 +95,37 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
* @return {@code HoodieRecord} if key was found else {@code Option.empty()}
*/
public synchronized List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordByKey(String key) {
checkState(forceFullScan, "Record reader has to be in full-scan mode to use this API");
return Collections.singletonList(Pair.of(key, Option.ofNullable((HoodieRecord) records.get(key))));
}
@SuppressWarnings("unchecked")
public List<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes) {
// Following operations have to be atomic, otherwise concurrent
// readers would race with each other and could crash when
// processing log block records as part of scan.
synchronized (this) {
records.clear();
scanInternal(Option.of(new KeySpec(keyPrefixes, false)));
return records.values().stream()
.filter(Objects::nonNull)
.map(record -> (HoodieRecord<HoodieMetadataPayload>) record)
.collect(Collectors.toList());
}
}
@SuppressWarnings("unchecked")
public synchronized List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> getRecordsByKeys(List<String> keys) {
// Following operations have to be atomic, otherwise concurrent
// readers would race with each other and could crash when
// processing log block records as part of scan.
records.clear();
scan(Option.of(keys));
List<Pair<String, Option<HoodieRecord<HoodieMetadataPayload>>>> metadataRecords = new ArrayList<>();
keys.forEach(entry -> {
if (records.containsKey(entry)) {
metadataRecords.add(Pair.of(entry, Option.ofNullable((HoodieRecord) records.get(entry))));
} else {
metadataRecords.add(Pair.of(entry, Option.empty()));
}
});
return metadataRecords;
synchronized (this) {
records.clear();
scan(keys);
return keys.stream()
.map(key -> Pair.of(key, Option.ofNullable((HoodieRecord<HoodieMetadataPayload>) records.get(key))))
.collect(Collectors.toList());
}
}
@Override
@@ -147,9 +137,7 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
* Builder used to build {@code HoodieMetadataMergedLogRecordScanner}.
*/
public static class Builder extends HoodieMergedLogRecordScanner.Builder {
private Set<String> mergeKeyFilter = Collections.emptySet();
private boolean enableFullScan = HoodieMetadataConfig.ENABLE_FULL_SCAN_LOG_FILES.defaultValue();
private boolean enableInlineReading;
private boolean allowFullScan = HoodieMetadataConfig.ENABLE_FULL_SCAN_LOG_FILES.defaultValue();
@Override
public Builder withFileSystem(FileSystem fs) {
@@ -227,26 +215,21 @@ public class HoodieMetadataMergedLogRecordReader extends HoodieMergedLogRecordSc
return this;
}
public Builder withMergeKeyFilter(Set<String> mergeKeyFilter) {
this.mergeKeyFilter = mergeKeyFilter;
return this;
}
public Builder withLogBlockTimestamps(Set<String> validLogBlockTimestamps) {
withInstantRange(Option.of(new ExplicitMatchRange(validLogBlockTimestamps)));
return this;
}
public Builder enableFullScan(boolean enableFullScan) {
this.enableFullScan = enableFullScan;
public Builder allowFullScan(boolean enableFullScan) {
this.allowFullScan = enableFullScan;
return this;
}
@Override
public HoodieMetadataMergedLogRecordReader build() {
return new HoodieMetadataMergedLogRecordReader(fs, basePath, partitionName, logFilePaths, readerSchema,
latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath, mergeKeyFilter,
diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange, enableFullScan);
latestInstantTime, maxMemorySizeInBytes, bufferSize, spillableMapBasePath,
diskMapType, isBitCaskDiskMapCompressionEnabled, instantRange, allowFullScan);
}
}

View File

@@ -75,8 +75,8 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.TypeUtils.unsafeCast;
import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant;
import static org.apache.hudi.common.util.DateTimeUtils.instantToMicros;
import static org.apache.hudi.common.util.DateTimeUtils.microsToInstant;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
import static org.apache.hudi.metadata.HoodieTableMetadata.RECORDKEY_PARTITION_LIST;
@@ -391,7 +391,7 @@ public class HoodieMetadataPayload implements HoodieRecordPayload<HoodieMetadata
}
@Override
public Option<IndexedRecord> getInsertValue(Schema schema, Properties properties) throws IOException {
public Option<IndexedRecord> getInsertValue(Schema schemaIgnored, Properties propertiesIgnored) throws IOException {
if (key == null) {
return Option.empty();
}

View File

@@ -24,7 +24,9 @@ import org.apache.hudi.avro.model.HoodieMetadataColumnStats;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.data.HoodieData;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
@@ -159,6 +161,17 @@ public interface HoodieTableMetadata extends Serializable, AutoCloseable {
Map<Pair<String, String>, HoodieMetadataColumnStats> getColumnStats(final List<Pair<String, String>> partitionNameFileNameList, final String columnName)
throws HoodieMetadataException;
/**
* Fetch records by key prefixes. Key prefix passed is expected to match the same prefix as stored in Metadata table partitions. For eg, in case of col stats partition,
* actual keys in metadata partition is encoded values of column name, partition name and file name. So, key prefixes passed to this method is expected to be encoded already.
*
* @param keyPrefixes list of key prefixes for which interested records are looked up for.
* @param partitionName partition name in metadata table where the records are looked up for.
* @return {@link HoodieData} of {@link HoodieRecord}s with records matching the passed in key prefixes.
*/
HoodieData<HoodieRecord<HoodieMetadataPayload>> getRecordsByKeyPrefixes(List<String> keyPrefixes,
String partitionName);
/**
* Get the instant time to which the metadata is synced w.r.t data timeline.
*/

View File

@@ -105,9 +105,9 @@ public class HoodieTableMetadataUtil {
private static final Logger LOG = LogManager.getLogger(HoodieTableMetadataUtil.class);
protected static final String PARTITION_NAME_FILES = "files";
protected static final String PARTITION_NAME_COLUMN_STATS = "column_stats";
protected static final String PARTITION_NAME_BLOOM_FILTERS = "bloom_filters";
public static final String PARTITION_NAME_FILES = "files";
public static final String PARTITION_NAME_COLUMN_STATS = "column_stats";
public static final String PARTITION_NAME_BLOOM_FILTERS = "bloom_filters";
/**
* Collects {@link HoodieColumnRangeMetadata} for the provided collection of records, pretending
@@ -815,7 +815,7 @@ public class HoodieTableMetadataUtil {
* @param path
* @return
*/
static String getPartition(@Nonnull String path) {
public static String getPartition(@Nonnull String path) {
return EMPTY_PARTITION_NAME.equals(path) ? NON_PARTITIONED_NAME : path;
}

View File

@@ -0,0 +1,64 @@
/*
* 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.util;
import java.util.function.Supplier;
// TODO java-doc
public class LazyRef<T> {
private volatile boolean initialized;
private Supplier<T> initializer;
private T ref;
private LazyRef(Supplier<T> initializer) {
this.initializer = initializer;
this.ref = null;
this.initialized = false;
}
private LazyRef(T ref) {
this.initializer = null;
this.ref = ref;
this.initialized = true;
}
public T get() {
if (!initialized) {
synchronized (this) {
if (!initialized) {
this.ref = initializer.get();
this.initializer = null;
initialized = true;
}
}
}
return ref;
}
public static <T> LazyRef<T> lazy(Supplier<T> initializer) {
return new LazyRef<>(initializer);
}
public static <T> LazyRef<T> eager(T ref) {
return new LazyRef<>(ref);
}
}