1
0

[HUDI-1434] fix incorrect log file path in HoodieWriteStat (#2300)

* [HUDI-1434] fix incorrect log file path in HoodieWriteStat

* HoodieWriteHandle#close() returns a list of WriteStatus objs

* Handle rolled-over log files and return a WriteStatus per log file written

 - Combined data and delete block logging into a single call
 - Lazily initialize and manage write status based on returned AppendResult
 - Use FSUtils.getFileSize() to set final file size, consistent with other handles
 - Added tests around returned values in AppendResult
 - Added validation of the file sizes returned in write stat

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
Gary Li
2020-12-31 06:22:15 +08:00
committed by GitHub
parent ef28763f08
commit 605b617cfa
29 changed files with 591 additions and 298 deletions

View File

@@ -71,7 +71,7 @@ public class CopyOnWriteInsertHandler<T extends HoodieRecordPayload>
public void consumeOneRecord(HoodieInsertValueGenResult<HoodieRecord> payload) {
final HoodieRecord insertPayload = payload.record;
String partitionPath = insertPayload.getPartitionPath();
HoodieWriteHandle handle = handles.get(partitionPath);
HoodieWriteHandle<?,?,?,?> handle = handles.get(partitionPath);
if (handle == null) {
// If the records are sorted, this means that we encounter a new partition path
// and the records for the previous partition path are all written,
@@ -87,7 +87,7 @@ public class CopyOnWriteInsertHandler<T extends HoodieRecordPayload>
if (!handle.canWrite(payload.record)) {
// Handle is full. Close the handle and add the WriteStatus
statuses.add(handle.close());
statuses.addAll(handle.close());
// Open new handle
handle = writeHandleFactory.create(config, instantTime, hoodieTable,
insertPayload.getPartitionPath(), idPrefix, taskContextSupplier);
@@ -108,8 +108,8 @@ public class CopyOnWriteInsertHandler<T extends HoodieRecordPayload>
}
private void closeOpenHandles() {
for (HoodieWriteHandle handle : handles.values()) {
statuses.add(handle.close());
for (HoodieWriteHandle<?,?,?,?> handle : handles.values()) {
statuses.addAll(handle.close());
}
handles.clear();
}

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.TaskContextSupplier;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BaseFile;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieDeltaWriteStat;
import org.apache.hudi.common.model.HoodieKey;
@@ -30,9 +31,9 @@ import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
import org.apache.hudi.common.model.IOType;
import org.apache.hudi.common.table.log.AppendResult;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.log.HoodieLogFormat.Writer;
import org.apache.hudi.common.table.log.block.HoodieDataBlock;
@@ -42,6 +43,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType;
import org.apache.hudi.common.table.view.TableFileSystemView.SliceView;
import org.apache.hudi.common.util.DefaultSizeEstimator;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.SizeEstimator;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieAppendException;
@@ -61,6 +63,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
/**
* IO Operation to append data onto an existing file.
@@ -69,48 +72,49 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
private static final Logger LOG = LogManager.getLogger(HoodieAppendHandle.class);
// This acts as the sequenceID for records written
private static AtomicLong recordIndex = new AtomicLong(1);
private static final AtomicLong RECORD_COUNTER = new AtomicLong(1);
private final String fileId;
// Buffer for holding records in memory before they are flushed to disk
private List<IndexedRecord> recordList = new ArrayList<>();
private final List<IndexedRecord> recordList = new ArrayList<>();
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
private List<HoodieKey> keysToDelete = new ArrayList<>();
private final List<HoodieKey> keysToDelete = new ArrayList<>();
// Incoming records to be written to logs.
private final Iterator<HoodieRecord<T>> recordItr;
// Writer to log into the file group's latest slice.
private Writer writer;
private Iterator<HoodieRecord<T>> recordItr;
private final List<WriteStatus> statuses;
// Total number of records written during an append
private long recordsWritten = 0;
// Total number of records deleted during an append
private long recordsDeleted = 0;
// Total number of records updated during an append
private long updatedRecordsWritten = 0;
// Total number of new records inserted into the delta file
private long insertRecordsWritten = 0;
// Average record size for a HoodieRecord. This size is updated at the end of every log block flushed to disk
private long averageRecordSize = 0;
private HoodieLogFile currentLogFile;
private Writer writer;
// Flag used to initialize some metadata
private boolean doInit = true;
// Total number of bytes written during this append phase (an estimation)
private long estimatedNumberOfBytesWritten;
// Total number of bytes written to file
private long sizeInBytes = 0;
// Number of records that must be written to meet the max block size for a log block
private int numberOfRecords = 0;
// Max block size to limit to for a log block
private int maxBlockSize = config.getLogFileDataBlockMaxSize();
private final int maxBlockSize = config.getLogFileDataBlockMaxSize();
// Header metadata for a log block
private Map<HeaderMetadataType, String> header = new HashMap<>();
// Total number of new records inserted into the delta file
private long insertRecordsWritten = 0;
private final Map<HeaderMetadataType, String> header = new HashMap<>();
private SizeEstimator<HoodieRecord> sizeEstimator;
public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr, TaskContextSupplier taskContextSupplier) {
super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier);
writeStatus.setStat(new HoodieDeltaWriteStat());
this.fileId = fileId;
this.recordItr = recordItr;
sizeEstimator = new DefaultSizeEstimator();
this.statuses = new ArrayList<>();
}
public HoodieAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
@@ -125,20 +129,32 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
// Set the base commit time as the current instantTime for new inserts into log files
String baseInstantTime;
String baseFile = "";
List<String> logFiles = new ArrayList<>();
if (fileSlice.isPresent()) {
baseInstantTime = fileSlice.get().getBaseInstantTime();
baseFile = fileSlice.get().getBaseFile().map(BaseFile::getFileName).orElse("");
logFiles = fileSlice.get().getLogFiles().map(HoodieLogFile::getFileName).collect(Collectors.toList());
} else {
baseInstantTime = instantTime;
// This means there is no base data file, start appending to a new log file
fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
LOG.info("New InsertHandle for partition :" + partitionPath);
LOG.info("New AppendHandle for partition :" + partitionPath);
}
writeStatus.getStat().setPrevCommit(baseInstantTime);
// Prepare the first write status
writeStatus.setStat(new HoodieDeltaWriteStat());
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(partitionPath);
writeStatus.getStat().setPartitionPath(partitionPath);
writeStatus.getStat().setFileId(fileId);
averageRecordSize = sizeEstimator.sizeEstimate(record);
HoodieDeltaWriteStat deltaWriteStat = (HoodieDeltaWriteStat) writeStatus.getStat();
deltaWriteStat.setPrevCommit(baseInstantTime);
deltaWriteStat.setPartitionPath(partitionPath);
deltaWriteStat.setFileId(fileId);
deltaWriteStat.setBaseFile(baseFile);
deltaWriteStat.setLogFiles(logFiles);
try {
//save hoodie partition meta in the partition path
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, baseInstantTime,
@@ -151,31 +167,25 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
createMarkerFile(partitionPath, FSUtils.makeDataFileName(baseInstantTime, writeToken, fileId, hoodieTable.getBaseFileExtension()));
this.writer = createLogWriter(fileSlice, baseInstantTime);
this.currentLogFile = writer.getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
} catch (Exception e) {
LOG.error("Error in update task at commit " + instantTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException("Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit "
+ instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + partitionPath, e);
}
Path path = partitionPath.length() == 0 ? new Path(writer.getLogFile().getFileName())
: new Path(partitionPath, writer.getLogFile().getFileName());
writeStatus.getStat().setPath(path.toString());
doInit = false;
}
}
private Option<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
Option recordMetadata = hoodieRecord.getData().getMetadata();
Option<Map<String, String>> recordMetadata = hoodieRecord.getData().getMetadata();
try {
Option<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(writerSchema);
if (avroRecord.isPresent()) {
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
String seqId =
HoodieRecord.generateSequenceId(instantTime, getPartitionId(), recordIndex.getAndIncrement());
HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement());
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
hoodieRecord.getPartitionPath(), fileId);
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
@@ -203,6 +213,105 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
return Option.empty();
}
private void initNewStatus() {
HoodieDeltaWriteStat prevStat = (HoodieDeltaWriteStat) this.writeStatus.getStat();
// Make a new write status and copy basic fields over.
HoodieDeltaWriteStat stat = new HoodieDeltaWriteStat();
stat.setFileId(fileId);
stat.setPartitionPath(partitionPath);
stat.setPrevCommit(prevStat.getPrevCommit());
stat.setBaseFile(prevStat.getBaseFile());
stat.setLogFiles(new ArrayList<>(prevStat.getLogFiles()));
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
this.writeStatus.setFileId(fileId);
this.writeStatus.setPartitionPath(partitionPath);
this.writeStatus.setStat(stat);
}
private String makeFilePath(HoodieLogFile logFile) {
return partitionPath.length() == 0
? new Path(logFile.getFileName()).toString()
: new Path(partitionPath, logFile.getFileName()).toString();
}
private void resetWriteCounts() {
recordsWritten = 0;
updatedRecordsWritten = 0;
insertRecordsWritten = 0;
recordsDeleted = 0;
}
private void updateWriteCounts(HoodieDeltaWriteStat stat, AppendResult result) {
stat.setNumWrites(recordsWritten);
stat.setNumUpdateWrites(updatedRecordsWritten);
stat.setNumInserts(insertRecordsWritten);
stat.setNumDeletes(recordsDeleted);
stat.setTotalWriteBytes(result.size());
}
private void accumulateWriteCounts(HoodieDeltaWriteStat stat, AppendResult result) {
stat.setNumWrites(stat.getNumWrites() + recordsWritten);
stat.setNumUpdateWrites(stat.getNumUpdateWrites() + updatedRecordsWritten);
stat.setNumInserts(stat.getNumInserts() + insertRecordsWritten);
stat.setNumDeletes(stat.getNumDeletes() + recordsDeleted);
stat.setTotalWriteBytes(stat.getTotalWriteBytes() + result.size());
}
private void updateWriteStat(HoodieDeltaWriteStat stat, AppendResult result) {
stat.setPath(makeFilePath(result.logFile()));
stat.setLogOffset(result.offset());
stat.setLogVersion(result.logFile().getLogVersion());
if (!stat.getLogFiles().contains(result.logFile().getFileName())) {
stat.addLogFiles(result.logFile().getFileName());
}
}
private void updateRuntimeStats(HoodieDeltaWriteStat stat) {
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer());
stat.setRuntimeStats(runtimeStats);
}
private void accumulateRuntimeStats(HoodieDeltaWriteStat stat) {
RuntimeStats runtimeStats = stat.getRuntimeStats();
assert runtimeStats != null;
runtimeStats.setTotalUpsertTime(runtimeStats.getTotalUpsertTime() + timer.endTimer());
}
private void updateWriteStatus(HoodieDeltaWriteStat stat, AppendResult result) {
updateWriteStat(stat, result);
updateWriteCounts(stat, result);
updateRuntimeStats(stat);
statuses.add(this.writeStatus);
}
private void processAppendResult(AppendResult result) {
HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat();
if (stat.getPath() == null) {
// first time writing to this log block.
updateWriteStatus(stat, result);
} else if (stat.getPath().endsWith(result.logFile().getFileName())) {
// append/continued writing to the same log file
stat.setLogOffset(Math.min(stat.getLogOffset(), result.offset()));
accumulateWriteCounts(stat, result);
accumulateRuntimeStats(stat);
} else {
// written to a newer log file, due to rollover/otherwise.
initNewStatus();
stat = (HoodieDeltaWriteStat) this.writeStatus.getStat();
updateWriteStatus(stat, result);
}
resetWriteCounts();
assert stat.getRuntimeStats() != null;
LOG.info(String.format("AppendHandle for partitionPath %s filePath %s, took %d ms.", partitionPath,
stat.getPath(), stat.getRuntimeStats().getTotalUpsertTime()));
timer.startTimer();
}
public void doAppend() {
while (recordItr.hasNext()) {
HoodieRecord record = recordItr.next();
@@ -210,24 +319,30 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
flushToDiskIfRequired(record);
writeToBuffer(record);
}
doAppend(header);
appendDataAndDeleteBlocks(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
}
private void doAppend(Map<HeaderMetadataType, String> header) {
private void appendDataAndDeleteBlocks(Map<HeaderMetadataType, String> header) {
try {
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, writerSchemaWithMetafields.toString());
List<HoodieLogBlock> blocks = new ArrayList<>(2);
if (recordList.size() > 0) {
writer = writer.appendBlock(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
recordList.clear();
blocks.add(HoodieDataBlock.getBlock(hoodieTable.getLogDataBlockFormat(), recordList, header));
}
if (keysToDelete.size() > 0) {
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header));
blocks.add(new HoodieDeleteBlock(keysToDelete.toArray(new HoodieKey[keysToDelete.size()]), header));
}
if (blocks.size() > 0) {
AppendResult appendResult = writer.appendBlocks(blocks);
processAppendResult(appendResult);
recordList.clear();
keysToDelete.clear();
}
} catch (Exception e) {
throw new HoodieAppendException("Failed while appending records to " + currentLogFile.getPath(), e);
throw new HoodieAppendException("Failed while appending records to " + writer.getLogFile().getPath(), e);
}
}
@@ -239,7 +354,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
@Override
public void write(HoodieRecord record, Option<IndexedRecord> insertValue) {
Option recordMetadata = record.getData().getMetadata();
Option<Map<String, String>> recordMetadata = record.getData().getMetadata();
try {
init(record);
flushToDiskIfRequired(record);
@@ -253,48 +368,34 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
}
@Override
public WriteStatus close() {
public List<WriteStatus> close() {
try {
// flush any remaining records to disk
doAppend(header);
appendDataAndDeleteBlocks(header);
if (writer != null) {
sizeInBytes = writer.getCurrentSize();
writer.close();
// update final size, once for all log files
for (WriteStatus status: statuses) {
long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath()));
status.getStat().setFileSizeInBytes(logFileSize);
}
}
HoodieWriteStat stat = writeStatus.getStat();
stat.setFileId(this.fileId);
stat.setNumWrites(recordsWritten);
stat.setNumUpdateWrites(updatedRecordsWritten);
stat.setNumInserts(insertRecordsWritten);
stat.setNumDeletes(recordsDeleted);
stat.setTotalWriteBytes(estimatedNumberOfBytesWritten);
stat.setFileSizeInBytes(sizeInBytes);
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalUpsertTime(timer.endTimer());
stat.setRuntimeStats(runtimeStats);
LOG.info(String.format("AppendHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(),
stat.getFileId(), runtimeStats.getTotalUpsertTime()));
return writeStatus;
return statuses;
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
}
@Override
public WriteStatus getWriteStatus() {
return writeStatus;
}
@Override
public IOType getIOType() {
return IOType.APPEND;
}
public List<WriteStatus> writeStatuses() {
return statuses;
}
private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTime)
throws IOException, InterruptedException {
Option<HoodieLogFile> latestLogFile = fileSlice.get().getLatestLogFile();
@@ -303,9 +404,10 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
.onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime)
.withLogVersion(latestLogFile.map(HoodieLogFile::getLogVersion).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withFileSize(latestLogFile.map(HoodieLogFile::getFileSize).orElse(0L))
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
.withLogWriteToken(latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken))
.withRolloverLogWriteToken(writeToken)
.withLogWriteToken(latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken))
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
}
@@ -340,7 +442,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
// avg of new and old
LOG.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + sizeEstimator.sizeEstimate(record)) / 2;
doAppend(header);
appendDataAndDeleteBlocks(header);
estimatedNumberOfBytesWritten += averageRecordSize * numberOfRecords;
numberOfRecords = 0;
}

View File

@@ -44,7 +44,9 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends HoodieWriteHandle<T, I, K, O> {
@@ -162,11 +164,6 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
}
}
@Override
public WriteStatus getWriteStatus() {
return writeStatus;
}
@Override
public IOType getIOType() {
return IOType.CREATE;
@@ -176,9 +173,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
* Performs actions to durably, persist the current changes and returns a WriteStatus object.
*/
@Override
public WriteStatus close() {
LOG
.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten);
public List<WriteStatus> close() {
LOG.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten);
try {
fileWriter.close();
@@ -203,7 +199,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
LOG.info(String.format("CreateHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(),
stat.getFileId(), runtimeStats.getTotalCreateTime()));
return writeStatus;
return Collections.singletonList(writeStatus);
} catch (IOException e) {
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
}

View File

@@ -50,8 +50,10 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -258,7 +260,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
}
@Override
public WriteStatus close() {
public List<WriteStatus> close() {
try {
// write out any pending records (this can happen when inserts are turned into updates)
Iterator<HoodieRecord<T>> newRecordsItr = (keyToNewRecords instanceof ExternalSpillableMap)
@@ -301,7 +303,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
LOG.info(String.format("MergeHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(),
stat.getFileId(), runtimeStats.getTotalUpsertTime()));
return writeStatus;
return Collections.singletonList(writeStatus);
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
@@ -333,11 +335,6 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
return oldFilePath;
}
@Override
public WriteStatus getWriteStatus() {
return writeStatus;
}
@Override
public IOType getIOType() {
return IOType.MERGE;

View File

@@ -31,6 +31,7 @@ import org.apache.avro.generic.GenericRecord;
import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.PriorityQueue;
import java.util.Queue;
@@ -101,7 +102,7 @@ public class HoodieSortedMergeHandle<T extends HoodieRecordPayload, I, K, O> ext
}
@Override
public WriteStatus close() {
public List<WriteStatus> close() {
// write out any pending records (this can happen when inserts are turned into updates)
newRecordKeysSorted.stream().forEach(key -> {
try {

View File

@@ -45,6 +45,8 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
/**
* Base class for all write operations logically performed at the file group level.
@@ -56,7 +58,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
protected final Schema writerSchema;
protected final Schema writerSchemaWithMetafields;
protected HoodieTimer timer;
protected final WriteStatus writeStatus;
protected WriteStatus writeStatus;
protected final String partitionPath;
protected final String fileId;
protected final String writeToken;
@@ -167,9 +169,15 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload, I, K, O>
return HoodieAvroUtils.rewriteRecord(record, writerSchemaWithMetafields);
}
public abstract WriteStatus close();
public abstract List<WriteStatus> close();
public abstract WriteStatus getWriteStatus();
public List<WriteStatus> writeStatuses() {
return Collections.singletonList(writeStatus);
}
public String getPartitionPath() {
return partitionPath;
}
public abstract IOType getIOType();

View File

@@ -106,7 +106,7 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
} else {
return this.writer;
}
} catch (InterruptedException | IOException e) {
} catch (IOException e) {
throw new HoodieException("Unable to initialize HoodieLogFormat writer", e);
}
}
@@ -335,7 +335,7 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
Map<HeaderMetadataType, String> header = new HashMap<>();
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString());
HoodieAvroDataBlock block = new HoodieAvroDataBlock(records, header);
this.writer = writer.appendBlock(block);
writer.appendBlock(block);
records.clear();
}
}

View File

@@ -108,7 +108,7 @@ public abstract class AbstractMarkerBasedRollbackStrategy<T extends HoodieRecord
// generate metadata
Map<HoodieLogBlock.HeaderMetadataType, String> header = RollbackUtils.generateHeader(instantToRollback.getTimestamp(), instantTime);
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
writer.appendBlock(new HoodieCommandBlock(header));
} finally {
try {
if (writer != null) {