Support union mode in HoodieRealtimeRecordReader for pure insert workloads
Also Replace BufferedIteratorPayload abstraction with function passing
This commit is contained in:
committed by
vinoth chandar
parent
93f345a032
commit
dfc0c61eb7
@@ -19,7 +19,6 @@ package com.uber.hoodie.common.table.log;
|
||||
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HeaderMetadataType.INSTANT_TIME;
|
||||
import static com.uber.hoodie.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
@@ -29,19 +28,14 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.util.HoodieTimer;
|
||||
import com.uber.hoodie.common.util.SpillableMapUtils;
|
||||
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Arrays;
|
||||
import java.util.Deque;
|
||||
import java.util.Iterator;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema;
|
||||
@@ -53,24 +47,38 @@ import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* 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. NOTE: If readBlockLazily is
|
||||
* Implements logic to scan log blocks and expose valid and deleted log records to subclass implementation.
|
||||
* Subclass is free to either apply merging or expose raw data back to the caller.
|
||||
*
|
||||
* NOTE: If readBlockLazily is
|
||||
* turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to
|
||||
* avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and
|
||||
* forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2
|
||||
* Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | |
|
||||
* | Read Block N Metadata | | Read Block N Data | <p> This results in two I/O passes over the log file.
|
||||
*/
|
||||
public abstract class AbstractHoodieLogRecordScanner {
|
||||
|
||||
public class HoodieCompactedLogRecordScanner implements
|
||||
Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
|
||||
private static final Logger log = LogManager.getLogger(AbstractHoodieLogRecordScanner.class);
|
||||
|
||||
private static final Logger log = LogManager.getLogger(HoodieCompactedLogRecordScanner.class);
|
||||
|
||||
// Final map of compacted/merged records
|
||||
private final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
|
||||
// Reader schema for the records
|
||||
private final Schema readerSchema;
|
||||
// Latest valid instant time
|
||||
private final String latestInstantTime;
|
||||
private final HoodieTableMetaClient hoodieTableMetaClient;
|
||||
// Merge strategy to use when combining records from log
|
||||
private final String payloadClassFQN;
|
||||
// Log File Paths
|
||||
private final List<String> logFilePaths;
|
||||
// Read Lazily flag
|
||||
private final boolean readBlocksLazily;
|
||||
// Reverse reader - Not implemented yet (NA -> Why do we need ?)
|
||||
// but present here for plumbing for future implementation
|
||||
private final boolean reverseReader;
|
||||
// Buffer Size for log file reader
|
||||
private final int bufferSize;
|
||||
// FileSystem
|
||||
private final FileSystem fs;
|
||||
// Total log files read - for metrics
|
||||
private AtomicLong totalLogFiles = new AtomicLong(0);
|
||||
// Total log blocks read - for metrics
|
||||
@@ -81,46 +89,47 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
private AtomicLong totalRollbacks = new AtomicLong(0);
|
||||
// Total number of corrupt blocks written across all log files
|
||||
private AtomicLong totalCorruptBlocks = new AtomicLong(0);
|
||||
// Total final list of compacted/merged records
|
||||
private long totalRecordsToUpdate;
|
||||
// Latest valid instant time
|
||||
private String latestInstantTime;
|
||||
private HoodieTableMetaClient hoodieTableMetaClient;
|
||||
// Merge strategy to use when combining records from log
|
||||
private String payloadClassFQN;
|
||||
// Store the last instant log blocks (needed to implement rollback)
|
||||
private Deque<HoodieLogBlock> currentInstantLogBlocks = new ArrayDeque<>();
|
||||
// Stores the total time taken to perform reading and merging of log blocks
|
||||
private long totalTimeTakenToReadAndMergeBlocks = 0L;
|
||||
// A timer for calculating elapsed time in millis
|
||||
public HoodieTimer timer = new HoodieTimer();
|
||||
|
||||
public HoodieCompactedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes,
|
||||
boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) {
|
||||
// Progress
|
||||
private float progress = 0.0f;
|
||||
|
||||
public AbstractHoodieLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime,
|
||||
boolean readBlocksLazily, boolean reverseReader, int bufferSize) {
|
||||
this.readerSchema = readerSchema;
|
||||
this.latestInstantTime = latestInstantTime;
|
||||
this.hoodieTableMetaClient = new HoodieTableMetaClient(fs.getConf(), basePath);
|
||||
// load class from the payload fully qualified class name
|
||||
this.payloadClassFQN = this.hoodieTableMetaClient.getTableConfig().getPayloadClass();
|
||||
this.totalLogFiles.addAndGet(logFilePaths.size());
|
||||
timer.startTimer();
|
||||
this.logFilePaths = logFilePaths;
|
||||
this.readBlocksLazily = readBlocksLazily;
|
||||
this.reverseReader = reverseReader;
|
||||
this.fs = fs;
|
||||
this.bufferSize = bufferSize;
|
||||
}
|
||||
|
||||
/**
|
||||
* Scan Log files
|
||||
*/
|
||||
public void scan() {
|
||||
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 StringConverter(), new HoodieRecordConverter(readerSchema, payloadClassFQN));
|
||||
// iterate over the paths
|
||||
HoodieLogFormatReader logFormatReaderWrapper =
|
||||
new HoodieLogFormatReader(fs,
|
||||
logFilePaths.stream().map(logFile -> new HoodieLogFile(new Path(logFile)))
|
||||
.collect(Collectors.toList()), readerSchema, readBlocksLazily, reverseReader, bufferSize);
|
||||
HoodieLogFile logFile;
|
||||
Set<HoodieLogFile> scannedLogFiles = new HashSet<>();
|
||||
while (logFormatReaderWrapper.hasNext()) {
|
||||
logFile = logFormatReaderWrapper.getLogFile();
|
||||
HoodieLogFile logFile = logFormatReaderWrapper.getLogFile();
|
||||
log.info("Scanning log file " + logFile);
|
||||
scannedLogFiles.add(logFile);
|
||||
totalLogFiles.set(scannedLogFiles.size());
|
||||
// Use the HoodieLogFileReader to iterate through the blocks in the log file
|
||||
HoodieLogBlock r = logFormatReaderWrapper.next();
|
||||
totalLogBlocks.incrementAndGet();
|
||||
if (r.getBlockType() != CORRUPT_BLOCK
|
||||
&& !HoodieTimeline.compareTimestamps(r.getLogBlockHeader().get(INSTANT_TIME),
|
||||
this.latestInstantTime,
|
||||
@@ -134,7 +143,7 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
if (isNewInstantBlock(r) && !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
|
||||
merge(records, currentInstantLogBlocks);
|
||||
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size());
|
||||
}
|
||||
// store the current block
|
||||
currentInstantLogBlocks.push(r);
|
||||
@@ -144,7 +153,7 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
if (isNewInstantBlock(r) && !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
|
||||
merge(records, currentInstantLogBlocks);
|
||||
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size());
|
||||
}
|
||||
// store deletes so can be rolled back
|
||||
currentInstantLogBlocks.push(r);
|
||||
@@ -208,7 +217,6 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
break;
|
||||
default:
|
||||
throw new UnsupportedOperationException("Command type not yet supported.");
|
||||
|
||||
}
|
||||
break;
|
||||
case CORRUPT_BLOCK:
|
||||
@@ -224,19 +232,14 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
// merge the last read block when all the blocks are done reading
|
||||
if (!currentInstantLogBlocks.isEmpty()) {
|
||||
log.info("Merging the final data blocks");
|
||||
merge(records, currentInstantLogBlocks);
|
||||
processQueuedBlocksForInstant(currentInstantLogBlocks, scannedLogFiles.size());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// Done
|
||||
progress = 1.0f;
|
||||
} catch (Exception e) {
|
||||
log.error("Got exception when reading log file", e);
|
||||
throw new HoodieIOException("IOException when reading log file ");
|
||||
}
|
||||
this.totalRecordsToUpdate = records.size();
|
||||
this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer();
|
||||
log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes);
|
||||
log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records.getInMemoryMapNumEntries());
|
||||
log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records.getCurrentInMemoryMapSize());
|
||||
log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records.getDiskBasedMapNumEntries());
|
||||
log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes());
|
||||
log.debug("Total time taken for scanning and compacting log files => " + totalTimeTakenToReadAndMergeBlocks);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -250,66 +253,69 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and merge with the application
|
||||
* specific payload if the same key was found before. Sufficient to just merge the log records since the base data is
|
||||
* merged on previous compaction. Finally, merge this log block with the accumulated records
|
||||
* Iterate over the GenericRecord in the block, read the hoodie key and partition path and
|
||||
* call subclass processors to handle it.
|
||||
*/
|
||||
private Map<String, HoodieRecord<? extends HoodieRecordPayload>> merge(
|
||||
HoodieAvroDataBlock dataBlock) throws IOException {
|
||||
// TODO (NA) - Implemnt getRecordItr() in HoodieAvroDataBlock and use that here
|
||||
private void processAvroDataBlock(HoodieAvroDataBlock dataBlock) throws Exception {
|
||||
// TODO (NA) - Implement getRecordItr() in HoodieAvroDataBlock and use that here
|
||||
List<IndexedRecord> recs = dataBlock.getRecords();
|
||||
totalLogRecords.addAndGet(recs.size());
|
||||
recs.forEach(rec -> {
|
||||
String key = ((GenericRecord) rec).get(HoodieRecord.RECORD_KEY_METADATA_FIELD)
|
||||
.toString();
|
||||
for (IndexedRecord rec : recs) {
|
||||
HoodieRecord<? extends HoodieRecordPayload> hoodieRecord =
|
||||
SpillableMapUtils.convertToHoodieRecordPayload((GenericRecord) rec, this.payloadClassFQN);
|
||||
if (records.containsKey(key)) {
|
||||
// Merge and store the merged record
|
||||
HoodieRecordPayload combinedValue = records.get(key).getData()
|
||||
.preCombine(hoodieRecord.getData());
|
||||
records
|
||||
.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()),
|
||||
combinedValue));
|
||||
} else {
|
||||
// Put the record as is
|
||||
records.put(key, hoodieRecord);
|
||||
}
|
||||
});
|
||||
return records;
|
||||
processNextRecord(hoodieRecord);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Merge the last seen log blocks with the accumulated records
|
||||
* Process next record
|
||||
*
|
||||
* @param hoodieRecord Hoodie Record to process
|
||||
*/
|
||||
private void merge(Map<String, HoodieRecord<? extends HoodieRecordPayload>> records,
|
||||
Deque<HoodieLogBlock> lastBlocks) throws IOException {
|
||||
protected abstract void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord)
|
||||
throws Exception;
|
||||
|
||||
/**
|
||||
* Process next deleted key
|
||||
*
|
||||
* @param key Deleted record key
|
||||
*/
|
||||
protected abstract void processNextDeletedKey(String key);
|
||||
|
||||
/**
|
||||
* Process the set of log blocks belonging to the last instant which is read fully.
|
||||
*/
|
||||
private void processQueuedBlocksForInstant(Deque<HoodieLogBlock> lastBlocks, int numLogFilesSeen)
|
||||
throws Exception {
|
||||
while (!lastBlocks.isEmpty()) {
|
||||
log.info("Number of remaining logblocks to merge " + lastBlocks.size());
|
||||
// poll the element at the bottom of the stack since that's the order it was inserted
|
||||
HoodieLogBlock lastBlock = lastBlocks.pollLast();
|
||||
switch (lastBlock.getBlockType()) {
|
||||
case AVRO_DATA_BLOCK:
|
||||
merge((HoodieAvroDataBlock) lastBlock);
|
||||
processAvroDataBlock((HoodieAvroDataBlock) lastBlock);
|
||||
break;
|
||||
case DELETE_BLOCK:
|
||||
// TODO : If delete is the only block written and/or records are present in parquet file
|
||||
// TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry
|
||||
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(records::remove);
|
||||
Arrays.stream(((HoodieDeleteBlock) lastBlock).getKeysToDelete()).forEach(this::processNextDeletedKey);
|
||||
break;
|
||||
case CORRUPT_BLOCK:
|
||||
log.warn("Found a corrupt block which was not rolled back");
|
||||
break;
|
||||
default:
|
||||
//TODO <vb> : Need to understand if COMMAND_BLOCK has to be handled?
|
||||
break;
|
||||
}
|
||||
}
|
||||
// At this step the lastBlocks are consumed. We track approximate progress by number of log-files seen
|
||||
progress = numLogFilesSeen - 1 / logFilePaths.size();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<? extends HoodieRecordPayload>> iterator() {
|
||||
return records.iterator();
|
||||
/**
|
||||
* Return progress of scanning as a float between 0.0 to 1.0
|
||||
*/
|
||||
public float getProgress() {
|
||||
return progress;
|
||||
}
|
||||
|
||||
public long getTotalLogFiles() {
|
||||
@@ -324,12 +330,8 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
return totalLogBlocks.get();
|
||||
}
|
||||
|
||||
public Map<String, HoodieRecord<? extends HoodieRecordPayload>> getRecords() {
|
||||
return records;
|
||||
}
|
||||
|
||||
public long getTotalRecordsToUpdate() {
|
||||
return totalRecordsToUpdate;
|
||||
protected String getPayloadClassFQN() {
|
||||
return payloadClassFQN;
|
||||
}
|
||||
|
||||
public long getTotalRollbacks() {
|
||||
@@ -339,9 +341,4 @@ public class HoodieCompactedLogRecordScanner implements
|
||||
public long getTotalCorruptBlocks() {
|
||||
return totalCorruptBlocks.get();
|
||||
}
|
||||
|
||||
public long getTotalTimeTakenToReadAndMergeBlocks() {
|
||||
return totalTimeTakenToReadAndMergeBlocks;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,131 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.log;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.DefaultSizeEstimator;
|
||||
import com.uber.hoodie.common.util.HoodieRecordSizeEstimator;
|
||||
import com.uber.hoodie.common.util.HoodieTimer;
|
||||
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
|
||||
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
|
||||
import com.uber.hoodie.common.util.collection.converter.StringConverter;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*
|
||||
* NOTE: If readBlockLazily is
|
||||
* turned on, does not merge, instead keeps reading log blocks and merges everything at once This is an optimization to
|
||||
* avoid seek() back and forth to read new block (forward seek()) and lazily read content of seen block (reverse and
|
||||
* forward seek()) during merge | | Read Block 1 Metadata | | Read Block 1 Data | | | Read Block 2
|
||||
* Metadata | | Read Block 2 Data | | I/O Pass 1 | ..................... | I/O Pass 2 | ................. | |
|
||||
* | Read Block N Metadata | | Read Block N Data | <p> This results in two I/O passes over the log file.
|
||||
*/
|
||||
|
||||
public class HoodieMergedLogRecordScanner extends AbstractHoodieLogRecordScanner
|
||||
implements Iterable<HoodieRecord<? extends HoodieRecordPayload>> {
|
||||
|
||||
private static final Logger log = LogManager.getLogger(HoodieMergedLogRecordScanner.class);
|
||||
|
||||
// Final map of compacted/merged records
|
||||
private final ExternalSpillableMap<String, HoodieRecord<? extends HoodieRecordPayload>> records;
|
||||
|
||||
// count of merged records in log
|
||||
private long numMergedRecordsInLog;
|
||||
|
||||
// Stores the total time taken to perform reading and merging of log blocks
|
||||
private final long totalTimeTakenToReadAndMergeBlocks;
|
||||
// A timer for calculating elapsed time in millis
|
||||
public final HoodieTimer timer = new HoodieTimer();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
public HoodieMergedLogRecordScanner(FileSystem fs, String basePath, List<String> logFilePaths,
|
||||
Schema readerSchema, String latestInstantTime, Long maxMemorySizeInBytes,
|
||||
boolean readBlocksLazily, boolean reverseReader, int bufferSize, String spillableMapBasePath) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize);
|
||||
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 StringConverter(), new HoodieRecordConverter(readerSchema, getPayloadClassFQN()),
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(readerSchema));
|
||||
// Do the scan and merge
|
||||
timer.startTimer();
|
||||
scan();
|
||||
this.totalTimeTakenToReadAndMergeBlocks = timer.endTimer();
|
||||
this.numMergedRecordsInLog = records.size();
|
||||
log.info("MaxMemoryInBytes allowed for compaction => " + maxMemorySizeInBytes);
|
||||
log.info("Number of entries in MemoryBasedMap in ExternalSpillableMap => " + records
|
||||
.getInMemoryMapNumEntries());
|
||||
log.info("Total size in bytes of MemoryBasedMap in ExternalSpillableMap => " + records
|
||||
.getCurrentInMemoryMapSize());
|
||||
log.info("Number of entries in DiskBasedMap in ExternalSpillableMap => " + records
|
||||
.getDiskBasedMapNumEntries());
|
||||
log.info("Size of file spilled to disk => " + records.getSizeOfFileOnDiskInBytes());
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("IOException when reading log file ");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<HoodieRecord<? extends HoodieRecordPayload>> iterator() {
|
||||
return records.iterator();
|
||||
}
|
||||
|
||||
public Map<String, HoodieRecord<? extends HoodieRecordPayload>> getRecords() {
|
||||
return records;
|
||||
}
|
||||
|
||||
public long getNumMergedRecordsInLog() {
|
||||
return numMergedRecordsInLog;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) {
|
||||
String key = hoodieRecord.getRecordKey();
|
||||
if (records.containsKey(key)) {
|
||||
// Merge and store the merged record
|
||||
HoodieRecordPayload combinedValue = records.get(key).getData().preCombine(hoodieRecord.getData());
|
||||
records.put(key, new HoodieRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue));
|
||||
} else {
|
||||
// Put the record as is
|
||||
records.put(key, hoodieRecord);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processNextDeletedKey(String key) {
|
||||
// TODO : If delete is the only block written and/or records are present in parquet file
|
||||
// TODO : Mark as tombstone (optional.empty()) for data instead of deleting the entry
|
||||
records.remove(key);
|
||||
}
|
||||
|
||||
public long getTotalTimeTakenToReadAndMergeBlocks() {
|
||||
return totalTimeTakenToReadAndMergeBlocks;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,55 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.table.log;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import java.util.List;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
public class HoodieUnMergedLogRecordScanner extends AbstractHoodieLogRecordScanner {
|
||||
|
||||
private final LogRecordScannerCallback callback;
|
||||
|
||||
public HoodieUnMergedLogRecordScanner(FileSystem fs, String basePath,
|
||||
List<String> logFilePaths, Schema readerSchema, String latestInstantTime,
|
||||
boolean readBlocksLazily, boolean reverseReader, int bufferSize,
|
||||
LogRecordScannerCallback callback) {
|
||||
super(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize);
|
||||
this.callback = callback;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processNextRecord(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) throws Exception {
|
||||
// Just call callback without merging
|
||||
callback.apply(hoodieRecord);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void processNextDeletedKey(String key) {
|
||||
throw new IllegalStateException("Not expected to see delete records in this log-scan mode. Check Job Config");
|
||||
}
|
||||
|
||||
@FunctionalInterface
|
||||
public static interface LogRecordScannerCallback {
|
||||
|
||||
public void apply(HoodieRecord<? extends HoodieRecordPayload> record) throws Exception;
|
||||
}
|
||||
}
|
||||
@@ -18,6 +18,7 @@ package com.uber.hoodie.common.table.log.block;
|
||||
|
||||
import com.google.common.collect.Maps;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.HoodieMergedLogRecordScanner;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import com.uber.hoodie.exception.HoodieIOException;
|
||||
import java.io.ByteArrayOutputStream;
|
||||
@@ -219,7 +220,7 @@ public abstract class HoodieLogBlock {
|
||||
|
||||
/**
|
||||
* Read or Skip block content of a log block in the log file. Depends on lazy reading enabled in
|
||||
* {@link com.uber.hoodie.common.table.log.HoodieCompactedLogRecordScanner}
|
||||
* {@link HoodieMergedLogRecordScanner}
|
||||
*/
|
||||
public static byte[] readOrSkipContent(FSDataInputStream inputStream,
|
||||
Integer contentLength, boolean readBlockLazily) throws IOException {
|
||||
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util;
|
||||
|
||||
import com.twitter.common.objectsize.ObjectSizeCalculator;
|
||||
|
||||
/**
|
||||
* Default implementation of size-estimator that uses Twitter's ObjectSizeCalculator
|
||||
* @param <T>
|
||||
*/
|
||||
public class DefaultSizeEstimator<T> implements SizeEstimator<T> {
|
||||
|
||||
@Override
|
||||
public long sizeEstimate(T t) {
|
||||
return ObjectSizeCalculator.getObjectSize(t);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,52 @@
|
||||
/*
|
||||
* Copyright (c) 2016 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util;
|
||||
|
||||
import com.twitter.common.objectsize.ObjectSizeCalculator;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Size Estimator for Hoodie record payload
|
||||
* @param <T>
|
||||
*/
|
||||
public class HoodieRecordSizeEstimator<T extends HoodieRecordPayload> implements SizeEstimator<HoodieRecord<T>> {
|
||||
|
||||
private static Logger log = LogManager.getLogger(HoodieRecordSizeEstimator.class);
|
||||
|
||||
// Schema used to get GenericRecord from HoodieRecordPayload then convert to bytes and vice-versa
|
||||
private final Schema schema;
|
||||
|
||||
public HoodieRecordSizeEstimator(Schema schema) {
|
||||
this.schema = schema;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeEstimate(HoodieRecord<T> hoodieRecord) {
|
||||
// Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst
|
||||
// all records in the JVM. Calculate and print the size of the Schema and of the Record to
|
||||
// note the sizes and differences. A correct estimation in such cases is handled in
|
||||
/** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/
|
||||
long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord);
|
||||
long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema);
|
||||
log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema);
|
||||
return sizeOfRecord;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util;
|
||||
|
||||
/**
|
||||
* An interface to estimate the size of payload in memory
|
||||
* @param <T>
|
||||
*/
|
||||
public interface SizeEstimator<T> {
|
||||
|
||||
/**
|
||||
* This method is used to estimate the size of a payload in memory.
|
||||
* The default implementation returns the total allocated size, in bytes, of the object
|
||||
* and all other objects reachable from it
|
||||
*/
|
||||
long sizeEstimate(T t);
|
||||
}
|
||||
@@ -20,7 +20,6 @@ import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.util.collection.DiskBasedMap;
|
||||
import com.uber.hoodie.common.util.collection.converter.Converter;
|
||||
import com.uber.hoodie.common.util.collection.io.storage.SizeAwareDataOutputStream;
|
||||
import com.uber.hoodie.exception.HoodieCorruptedDataException;
|
||||
import java.io.IOException;
|
||||
@@ -99,8 +98,8 @@ public class SpillableMapUtils {
|
||||
* Compute a bytes representation of the payload by serializing the contents This is used to estimate the size of the
|
||||
* payload (either in memory or when written to disk)
|
||||
*/
|
||||
public static <R> long computePayloadSize(R value, Converter<R> valueConverter) throws IOException {
|
||||
return valueConverter.sizeEstimate(value);
|
||||
public static <R> long computePayloadSize(R value, SizeEstimator<R> valueSizeEstimator) throws IOException {
|
||||
return valueSizeEstimator.sizeEstimate(value);
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -17,6 +17,7 @@
|
||||
package com.uber.hoodie.common.util.collection;
|
||||
|
||||
import com.twitter.common.objectsize.ObjectSizeCalculator;
|
||||
import com.uber.hoodie.common.util.SizeEstimator;
|
||||
import com.uber.hoodie.common.util.collection.converter.Converter;
|
||||
import com.uber.hoodie.exception.HoodieNotSupportedException;
|
||||
import java.io.IOException;
|
||||
@@ -56,6 +57,10 @@ public class ExternalSpillableMap<T, R> implements Map<T, R> {
|
||||
private final Converter<T> keyConverter;
|
||||
// Value converter to convert value type to bytes
|
||||
private final Converter<R> valueConverter;
|
||||
// Size Estimator for key type
|
||||
private final SizeEstimator<T> keySizeEstimator;
|
||||
// Size Estimator for key types
|
||||
private final SizeEstimator<R> valueSizeEstimator;
|
||||
// current space occupied by this map in-memory
|
||||
private Long currentInMemoryMapSize;
|
||||
// An estimate of the size of each payload written to this map
|
||||
@@ -64,7 +69,8 @@ public class ExternalSpillableMap<T, R> implements Map<T, R> {
|
||||
private boolean shouldEstimatePayloadSize = true;
|
||||
|
||||
public ExternalSpillableMap(Long maxInMemorySizeInBytes, String baseFilePath,
|
||||
Converter<T> keyConverter, Converter<R> valueConverter) throws IOException {
|
||||
Converter<T> keyConverter, Converter<R> valueConverter,
|
||||
SizeEstimator<T> keySizeEstimator, SizeEstimator<R> valueSizeEstimator) throws IOException {
|
||||
this.inMemoryMap = new HashMap<>();
|
||||
this.diskBasedMap = new DiskBasedMap<>(baseFilePath, keyConverter, valueConverter);
|
||||
this.maxInMemorySizeInBytes = (long) Math
|
||||
@@ -72,6 +78,8 @@ public class ExternalSpillableMap<T, R> implements Map<T, R> {
|
||||
this.currentInMemoryMapSize = 0L;
|
||||
this.keyConverter = keyConverter;
|
||||
this.valueConverter = valueConverter;
|
||||
this.keySizeEstimator = keySizeEstimator;
|
||||
this.valueSizeEstimator = valueSizeEstimator;
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -146,7 +154,7 @@ public class ExternalSpillableMap<T, R> implements Map<T, R> {
|
||||
// At first, use the sizeEstimate of a record being inserted into the spillable map.
|
||||
// Note, the converter may over estimate the size of a record in the JVM
|
||||
this.estimatedPayloadSize =
|
||||
keyConverter.sizeEstimate(key) + valueConverter.sizeEstimate(value);
|
||||
keySizeEstimator.sizeEstimate(key) + valueSizeEstimator.sizeEstimate(value);
|
||||
log.info("Estimated Payload size => " + estimatedPayloadSize);
|
||||
} else if (shouldEstimatePayloadSize
|
||||
&& inMemoryMap.size() % NUMBER_OF_RECORDS_TO_ESTIMATE_PAYLOAD_SIZE == 0) {
|
||||
|
||||
@@ -31,9 +31,4 @@ public interface Converter<T> {
|
||||
* This method is used to convert the serialized payload (in bytes) to the actual payload instance
|
||||
*/
|
||||
T getData(byte[] bytes);
|
||||
|
||||
/**
|
||||
* This method is used to estimate the size of a payload in memory
|
||||
*/
|
||||
long sizeEstimate(T t);
|
||||
}
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.common.util.collection.converter;
|
||||
|
||||
import com.twitter.common.objectsize.ObjectSizeCalculator;
|
||||
import com.uber.hoodie.common.model.HoodieKey;
|
||||
import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
@@ -87,16 +86,4 @@ public class HoodieRecordConverter<V> implements
|
||||
throw new HoodieNotSerializableException("Cannot de-serialize value from bytes", io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeEstimate(HoodieRecord<? extends HoodieRecordPayload> hoodieRecord) {
|
||||
// Most HoodieRecords are bound to have data + schema. Although, the same schema object is shared amongst
|
||||
// all records in the JVM. Calculate and print the size of the Schema and of the Record to
|
||||
// note the sizes and differences. A correct estimation in such cases is handled in
|
||||
/** {@link com.uber.hoodie.common.util.collection.ExternalSpillableMap} **/
|
||||
long sizeOfRecord = ObjectSizeCalculator.getObjectSize(hoodieRecord);
|
||||
long sizeOfSchema = ObjectSizeCalculator.getObjectSize(schema);
|
||||
log.info("SizeOfRecord => " + sizeOfRecord + " SizeOfSchema => " + sizeOfSchema);
|
||||
return sizeOfRecord;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,7 +16,6 @@
|
||||
|
||||
package com.uber.hoodie.common.util.collection.converter;
|
||||
|
||||
import com.twitter.common.objectsize.ObjectSizeCalculator;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
|
||||
/**
|
||||
@@ -33,9 +32,4 @@ public class StringConverter implements Converter<String> {
|
||||
public String getData(byte[] bytes) {
|
||||
return new String(bytes);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long sizeEstimate(String s) {
|
||||
return ObjectSizeCalculator.getObjectSize(s);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,162 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util.queue;
|
||||
|
||||
import com.uber.hoodie.common.util.DefaultSizeEstimator;
|
||||
import com.uber.hoodie.common.util.SizeEstimator;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.ExecutorCompletionService;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.commons.lang3.concurrent.ConcurrentUtils;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Executor which orchestrates concurrent producers and consumers communicating through a bounded in-memory queue.
|
||||
* This class takes as input the size limit, queue producer(s), consumer and transformer
|
||||
* and exposes API to orchestrate concurrent execution of these actors communicating through a central bounded queue
|
||||
*/
|
||||
public class BoundedInMemoryExecutor<I, O, E> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(BoundedInMemoryExecutor.class);
|
||||
|
||||
// Executor service used for launching writer thread.
|
||||
private final ExecutorService executorService;
|
||||
// Used for buffering records which is controlled by HoodieWriteConfig#WRITE_BUFFER_LIMIT_BYTES.
|
||||
private final BoundedInMemoryQueue<I, O> queue;
|
||||
// Producers
|
||||
private final List<BoundedInMemoryQueueProducer<I>> producers;
|
||||
// Consumer
|
||||
private final Optional<BoundedInMemoryQueueConsumer<O, E>> consumer;
|
||||
|
||||
public BoundedInMemoryExecutor(final long bufferLimitInBytes,
|
||||
BoundedInMemoryQueueProducer<I> producer,
|
||||
Optional<BoundedInMemoryQueueConsumer<O, E>> consumer,
|
||||
final Function<I, O> transformFunction) {
|
||||
this(bufferLimitInBytes, Arrays.asList(producer), consumer, transformFunction, new DefaultSizeEstimator<>());
|
||||
}
|
||||
|
||||
public BoundedInMemoryExecutor(final long bufferLimitInBytes,
|
||||
List<BoundedInMemoryQueueProducer<I>> producers,
|
||||
Optional<BoundedInMemoryQueueConsumer<O, E>> consumer,
|
||||
final Function<I, O> transformFunction,
|
||||
final SizeEstimator<O> sizeEstimator) {
|
||||
this.producers = producers;
|
||||
this.consumer = consumer;
|
||||
// Ensure single thread for each producer thread and one for consumer
|
||||
this.executorService = Executors.newFixedThreadPool(producers.size() + 1);
|
||||
this.queue = new BoundedInMemoryQueue<>(bufferLimitInBytes, transformFunction, sizeEstimator);
|
||||
}
|
||||
|
||||
/**
|
||||
* Callback to implement environment specific behavior before executors (producers/consumer)
|
||||
* run.
|
||||
*/
|
||||
public void preExecute() {
|
||||
// Do Nothing in general context
|
||||
}
|
||||
|
||||
/**
|
||||
* Start all Producers
|
||||
*/
|
||||
public ExecutorCompletionService<Boolean> startProducers() {
|
||||
// Latch to control when and which producer thread will close the queue
|
||||
final CountDownLatch latch = new CountDownLatch(producers.size());
|
||||
final ExecutorCompletionService<Boolean> completionService =
|
||||
new ExecutorCompletionService<Boolean>(executorService);
|
||||
producers.stream().map(producer -> {
|
||||
return completionService.submit(() -> {
|
||||
try {
|
||||
preExecute();
|
||||
producer.produce(queue);
|
||||
} catch (Exception e) {
|
||||
logger.error("error consuming records", e);
|
||||
queue.markAsFailed(e);
|
||||
throw e;
|
||||
} finally {
|
||||
synchronized (latch) {
|
||||
latch.countDown();
|
||||
if (latch.getCount() == 0) {
|
||||
// Mark production as done so that consumer will be able to exit
|
||||
queue.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
return true;
|
||||
});
|
||||
}).collect(Collectors.toList());
|
||||
return completionService;
|
||||
}
|
||||
|
||||
/**
|
||||
* Start only consumer
|
||||
*/
|
||||
private Future<E> startConsumer() {
|
||||
return consumer.map(consumer -> {
|
||||
return executorService.submit(
|
||||
() -> {
|
||||
logger.info("starting consumer thread");
|
||||
preExecute();
|
||||
try {
|
||||
E result = consumer.consume(queue);
|
||||
logger.info("Queue Consumption is done; notifying producer threads");
|
||||
return result;
|
||||
} catch (Exception e) {
|
||||
logger.error("error consuming records", e);
|
||||
queue.markAsFailed(e);
|
||||
throw e;
|
||||
}
|
||||
});
|
||||
}).orElse(ConcurrentUtils.constantFuture(null));
|
||||
}
|
||||
|
||||
/**
|
||||
* Main API to run both production and consumption
|
||||
*/
|
||||
public E execute() {
|
||||
try {
|
||||
ExecutorCompletionService<Boolean> producerService = startProducers();
|
||||
Future<E> future = startConsumer();
|
||||
// Wait for consumer to be done
|
||||
return future.get();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
public boolean isRemaining() {
|
||||
return queue.iterator().hasNext();
|
||||
}
|
||||
|
||||
public void shutdownNow() {
|
||||
executorService.shutdownNow();
|
||||
}
|
||||
|
||||
public BoundedInMemoryQueue<I, O> getQueue() {
|
||||
return queue;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,273 @@
|
||||
/*
|
||||
* Copyright (c) 2018 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util.queue;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.uber.hoodie.common.util.DefaultSizeEstimator;
|
||||
import com.uber.hoodie.common.util.SizeEstimator;
|
||||
import com.uber.hoodie.exception.HoodieException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.Function;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Used for enqueueing input records. Queue limit is controlled by {@link #memoryLimit}.
|
||||
* Unlike standard bounded queue implementations, this queue bounds the size by memory bytes occupied by its
|
||||
* tenants. The standard implementation bounds by the number of entries in the queue.
|
||||
*
|
||||
* It internally samples every {@link #RECORD_SAMPLING_RATE}th record and adjusts number of records in
|
||||
* queue accordingly. This is done to ensure that we don't OOM.
|
||||
*
|
||||
* This queue supports multiple producer single consumer pattern.
|
||||
*
|
||||
* @param <I> input payload data type
|
||||
* @param <O> output payload data type
|
||||
*/
|
||||
public class BoundedInMemoryQueue<I, O> implements Iterable<O> {
|
||||
|
||||
// interval used for polling records in the queue.
|
||||
public static final int RECORD_POLL_INTERVAL_SEC = 1;
|
||||
// rate used for sampling records to determine avg record size in bytes.
|
||||
public static final int RECORD_SAMPLING_RATE = 64;
|
||||
// maximum records that will be cached
|
||||
private static final int RECORD_CACHING_LIMIT = 128 * 1024;
|
||||
private static Logger logger = LogManager.getLogger(BoundedInMemoryQueue.class);
|
||||
// It indicates number of records to cache. We will be using sampled record's average size to
|
||||
// determine how many
|
||||
// records we should cache and will change (increase/decrease) permits accordingly.
|
||||
@VisibleForTesting
|
||||
public final Semaphore rateLimiter = new Semaphore(1);
|
||||
// used for sampling records with "RECORD_SAMPLING_RATE" frequency.
|
||||
public final AtomicLong samplingRecordCounter = new AtomicLong(-1);
|
||||
// internal queue for records.
|
||||
private final LinkedBlockingQueue<Optional<O>> queue = new
|
||||
LinkedBlockingQueue<>();
|
||||
// maximum amount of memory to be used for queueing records.
|
||||
private final long memoryLimit;
|
||||
// it holds the root cause of the exception in case either queueing records (consuming from
|
||||
// inputIterator) fails or
|
||||
// thread reading records from queue fails.
|
||||
private final AtomicReference<Exception> hasFailed = new AtomicReference(null);
|
||||
// used for indicating that all the records from queue are read successfully.
|
||||
private final AtomicBoolean isReadDone = new AtomicBoolean(false);
|
||||
// used for indicating that all records have been enqueued
|
||||
private final AtomicBoolean isWriteDone = new AtomicBoolean(false);
|
||||
// Function to transform the input payload to the expected output payload
|
||||
private final Function<I, O> transformFunction;
|
||||
// Payload Size Estimator
|
||||
private final SizeEstimator<O> payloadSizeEstimator;
|
||||
// Singleton (w.r.t this instance) Iterator for this queue
|
||||
private final QueueIterator iterator;
|
||||
// indicates rate limit (number of records to cache). it is updated whenever there is a change
|
||||
// in avg record size.
|
||||
@VisibleForTesting
|
||||
public int currentRateLimit = 1;
|
||||
// indicates avg record size in bytes. It is updated whenever a new record is sampled.
|
||||
@VisibleForTesting
|
||||
public long avgRecordSizeInBytes = 0;
|
||||
// indicates number of samples collected so far.
|
||||
private long numSamples = 0;
|
||||
|
||||
/**
|
||||
* Construct BoundedInMemoryQueue with default SizeEstimator
|
||||
*
|
||||
* @param memoryLimit MemoryLimit in bytes
|
||||
* @param transformFunction Transformer Function to convert input payload type to stored payload type
|
||||
*/
|
||||
public BoundedInMemoryQueue(final long memoryLimit, final Function<I, O> transformFunction) {
|
||||
this(memoryLimit, transformFunction, new DefaultSizeEstimator() {
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Construct BoundedInMemoryQueue with passed in size estimator
|
||||
*
|
||||
* @param memoryLimit MemoryLimit in bytes
|
||||
* @param transformFunction Transformer Function to convert input payload type to stored payload type
|
||||
* @param payloadSizeEstimator Payload Size Estimator
|
||||
*/
|
||||
public BoundedInMemoryQueue(
|
||||
final long memoryLimit,
|
||||
final Function<I, O> transformFunction,
|
||||
final SizeEstimator<O> payloadSizeEstimator) {
|
||||
this.memoryLimit = memoryLimit;
|
||||
this.transformFunction = transformFunction;
|
||||
this.payloadSizeEstimator = payloadSizeEstimator;
|
||||
this.iterator = new QueueIterator();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public int size() {
|
||||
return this.queue.size();
|
||||
}
|
||||
|
||||
/**
|
||||
* Samples records with "RECORD_SAMPLING_RATE" frequency and computes average record size in bytes. It is used
|
||||
* for determining how many maximum records to queue. Based on change in avg size it ma increase or decrease
|
||||
* available permits.
|
||||
*
|
||||
* @param payload Payload to size
|
||||
*/
|
||||
private void adjustBufferSizeIfNeeded(final O payload) throws InterruptedException {
|
||||
if (this.samplingRecordCounter.incrementAndGet() % RECORD_SAMPLING_RATE != 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
final long recordSizeInBytes = payloadSizeEstimator.sizeEstimate(payload);
|
||||
final long newAvgRecordSizeInBytes = Math
|
||||
.max(1, (avgRecordSizeInBytes * numSamples + recordSizeInBytes) / (numSamples + 1));
|
||||
final int newRateLimit = (int) Math
|
||||
.min(RECORD_CACHING_LIMIT, Math.max(1, this.memoryLimit / newAvgRecordSizeInBytes));
|
||||
|
||||
// If there is any change in number of records to cache then we will either release (if it increased) or acquire
|
||||
// (if it decreased) to adjust rate limiting to newly computed value.
|
||||
if (newRateLimit > currentRateLimit) {
|
||||
rateLimiter.release(newRateLimit - currentRateLimit);
|
||||
} else if (newRateLimit < currentRateLimit) {
|
||||
rateLimiter.acquire(currentRateLimit - newRateLimit);
|
||||
}
|
||||
currentRateLimit = newRateLimit;
|
||||
avgRecordSizeInBytes = newAvgRecordSizeInBytes;
|
||||
numSamples++;
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts record into queue after applying transformation
|
||||
*
|
||||
* @param t Item to be queueed
|
||||
*/
|
||||
public void insertRecord(I t) throws Exception {
|
||||
// If already closed, throw exception
|
||||
if (isWriteDone.get()) {
|
||||
throw new IllegalStateException("Queue closed for enqueueing new entries");
|
||||
}
|
||||
|
||||
// We need to stop queueing if queue-reader has failed and exited.
|
||||
throwExceptionIfFailed();
|
||||
|
||||
rateLimiter.acquire();
|
||||
// We are retrieving insert value in the record queueing thread to offload computation
|
||||
// around schema validation
|
||||
// and record creation to it.
|
||||
final O payload = transformFunction.apply(t);
|
||||
adjustBufferSizeIfNeeded(payload);
|
||||
queue.put(Optional.of(payload));
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if records are either available in the queue or expected to be written in future
|
||||
*/
|
||||
private boolean expectMoreRecords() {
|
||||
return !isWriteDone.get() || (isWriteDone.get() && !queue.isEmpty());
|
||||
}
|
||||
|
||||
/**
|
||||
* Reader interface but never exposed to outside world as this is a single consumer queue.
|
||||
* Reading is done through a singleton iterator for this queue.
|
||||
*/
|
||||
private Optional<O> readNextRecord() {
|
||||
if (this.isReadDone.get()) {
|
||||
return Optional.empty();
|
||||
}
|
||||
|
||||
rateLimiter.release();
|
||||
Optional<O> newRecord = Optional.empty();
|
||||
while (expectMoreRecords()) {
|
||||
try {
|
||||
throwExceptionIfFailed();
|
||||
newRecord = queue.poll(RECORD_POLL_INTERVAL_SEC, TimeUnit.SECONDS);
|
||||
if (newRecord != null) {
|
||||
break;
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
logger.error("error reading records from queue", e);
|
||||
throw new HoodieException(e);
|
||||
}
|
||||
}
|
||||
if (newRecord != null && newRecord.isPresent()) {
|
||||
return newRecord;
|
||||
} else {
|
||||
// We are done reading all the records from internal iterator.
|
||||
this.isReadDone.set(true);
|
||||
return Optional.empty();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Puts an empty entry to queue to denote termination
|
||||
*/
|
||||
public void close() throws InterruptedException {
|
||||
// done queueing records notifying queue-reader.
|
||||
isWriteDone.set(true);
|
||||
}
|
||||
|
||||
private void throwExceptionIfFailed() {
|
||||
if (this.hasFailed.get() != null) {
|
||||
throw new HoodieException("operation has failed", this.hasFailed.get());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* API to allow producers and consumer to communicate termination due to failure
|
||||
*/
|
||||
public void markAsFailed(Exception e) {
|
||||
this.hasFailed.set(e);
|
||||
// release the permits so that if the queueing thread is waiting for permits then it will
|
||||
// get it.
|
||||
this.rateLimiter.release(RECORD_CACHING_LIMIT + 1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<O> iterator() {
|
||||
return iterator;
|
||||
}
|
||||
|
||||
/**
|
||||
* Iterator for the memory bounded queue
|
||||
*/
|
||||
private final class QueueIterator implements Iterator<O> {
|
||||
|
||||
// next record to be read from queue.
|
||||
private O nextRecord;
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
if (this.nextRecord == null) {
|
||||
Optional<O> res = readNextRecord();
|
||||
this.nextRecord = res.orElse(null);
|
||||
}
|
||||
return this.nextRecord != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public O next() {
|
||||
Preconditions.checkState(hasNext() && this.nextRecord != null);
|
||||
final O ret = this.nextRecord;
|
||||
this.nextRecord = null;
|
||||
return ret;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,63 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util.queue;
|
||||
|
||||
import java.util.Iterator;
|
||||
|
||||
|
||||
/**
|
||||
* Consume entries from queue and execute callback function
|
||||
*/
|
||||
public abstract class BoundedInMemoryQueueConsumer<I, O> {
|
||||
|
||||
/**
|
||||
* API to de-queue entries to memory bounded queue
|
||||
*
|
||||
* @param queue In Memory bounded queue
|
||||
*/
|
||||
public O consume(BoundedInMemoryQueue<?, I> queue) throws Exception {
|
||||
Iterator<I> iterator = queue.iterator();
|
||||
|
||||
while (iterator.hasNext()) {
|
||||
consumeOneRecord(iterator.next());
|
||||
}
|
||||
|
||||
// Notifies done
|
||||
finish();
|
||||
|
||||
return getResult();
|
||||
}
|
||||
|
||||
/**
|
||||
* Consumer One record
|
||||
*/
|
||||
protected abstract void consumeOneRecord(I record);
|
||||
|
||||
/**
|
||||
* Notifies implementation that we have exhausted consuming records from queue
|
||||
*/
|
||||
protected abstract void finish();
|
||||
|
||||
/**
|
||||
* Return result of consuming records so far
|
||||
*/
|
||||
protected abstract O getResult();
|
||||
|
||||
|
||||
}
|
||||
@@ -0,0 +1,35 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util.queue;
|
||||
|
||||
/**
|
||||
* Producer for BoundedInMemoryQueue. Memory Bounded Buffer supports
|
||||
* multiple producers single consumer pattern.
|
||||
*
|
||||
* @param <I> Input type for buffer items produced
|
||||
*/
|
||||
public interface BoundedInMemoryQueueProducer<I> {
|
||||
|
||||
/**
|
||||
* API to enqueue entries to memory bounded queue
|
||||
*
|
||||
* @param queue In Memory bounded queue
|
||||
*/
|
||||
void produce(BoundedInMemoryQueue<I, ?> queue) throws Exception;
|
||||
}
|
||||
@@ -0,0 +1,46 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util.queue;
|
||||
|
||||
import java.util.function.Function;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Buffer producer which allows custom functions to insert entries to queue.
|
||||
*
|
||||
* @param <I> Type of entry produced for queue
|
||||
*/
|
||||
public class FunctionBasedQueueProducer<I> implements BoundedInMemoryQueueProducer<I> {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(FunctionBasedQueueProducer.class);
|
||||
|
||||
private final Function<BoundedInMemoryQueue<I, ?>, Boolean> producerFunction;
|
||||
|
||||
public FunctionBasedQueueProducer(Function<BoundedInMemoryQueue<I, ?>, Boolean> producerFunction) {
|
||||
this.producerFunction = producerFunction;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void produce(BoundedInMemoryQueue<I, ?> queue) {
|
||||
logger.info("starting function which will enqueue records");
|
||||
producerFunction.apply(queue);
|
||||
logger.info("finished function which will enqueue records");
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,49 @@
|
||||
/*
|
||||
* Copyright (c) 2017 Uber Technologies, Inc. (hoodie-dev-group@uber.com)
|
||||
*
|
||||
* Licensed 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 com.uber.hoodie.common.util.queue;
|
||||
|
||||
import java.util.Iterator;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
/**
|
||||
* Iterator based producer which pulls entry from iterator and produces items for the queue
|
||||
*
|
||||
* @param <I> Item type produced for the buffer.
|
||||
*/
|
||||
public class IteratorBasedQueueProducer<I> implements BoundedInMemoryQueueProducer<I> {
|
||||
|
||||
private static final Logger logger = LogManager.getLogger(IteratorBasedQueueProducer.class);
|
||||
|
||||
// input iterator for producing items in the buffer.
|
||||
private final Iterator<I> inputIterator;
|
||||
|
||||
public IteratorBasedQueueProducer(Iterator<I> inputIterator) {
|
||||
this.inputIterator = inputIterator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void produce(BoundedInMemoryQueue<I, ?> queue) throws Exception {
|
||||
logger.info("starting to buffer records");
|
||||
while (inputIterator.hasNext()) {
|
||||
queue.insertRecord(inputIterator.next());
|
||||
}
|
||||
logger.info("finished buffering records");
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user