Timeline Service with Incremental View Syncing support
This commit is contained in:
committed by
vinoth chandar
parent
446f99aa0f
commit
64fec64097
@@ -26,13 +26,14 @@ import com.uber.hoodie.common.model.HoodieRecord;
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import com.uber.hoodie.common.model.HoodieRecordPayload;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat.RuntimeStats;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView;
|
||||
import com.uber.hoodie.common.table.TableFileSystemView.RealtimeView;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFormat.Writer;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.common.util.Option;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieAppendException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
@@ -67,7 +68,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private 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 TableFileSystemView.RealtimeView fileSystemView;
|
||||
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
// Total number of records written during an append
|
||||
@@ -98,7 +99,6 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
super(config, commitTime, hoodieTable);
|
||||
writeStatus.setStat(new HoodieDeltaWriteStat());
|
||||
this.fileId = fileId;
|
||||
this.fileSystemView = hoodieTable.getRTFileSystemView();
|
||||
this.recordItr = recordItr;
|
||||
}
|
||||
|
||||
@@ -110,15 +110,15 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
if (doInit) {
|
||||
this.partitionPath = record.getPartitionPath();
|
||||
// extract some information from the first record
|
||||
Optional<FileSlice> fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
|
||||
.filter(fileSlice1 -> fileSlice1.getFileId().equals(fileId)).findFirst();
|
||||
RealtimeView rtView = hoodieTable.getRTFileSystemView();
|
||||
Option<FileSlice> fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
|
||||
// Set the base commit time as the current commitTime for new inserts into log files
|
||||
String baseInstantTime = commitTime;
|
||||
if (fileSlice.isPresent()) {
|
||||
baseInstantTime = fileSlice.get().getBaseInstantTime();
|
||||
} else {
|
||||
// This means there is no base data file, start appending to a new log file
|
||||
fileSlice = Optional.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
|
||||
fileSlice = Option.of(new FileSlice(partitionPath, baseInstantTime, this.fileId));
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
writeStatus.getStat().setPrevCommit(baseInstantTime);
|
||||
@@ -242,6 +242,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
try {
|
||||
// flush any remaining records to disk
|
||||
doAppend(header);
|
||||
long sizeInBytes = writer.getCurrentSize();
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
@@ -251,6 +252,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
writeStatus.getStat().setNumInserts(insertRecordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
writeStatus.getStat().setTotalWriteBytes(estimatedNumberOfBytesWritten);
|
||||
writeStatus.getStat().setFileSizeInBytes(sizeInBytes);
|
||||
writeStatus.getStat().setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalUpsertTime(timer.endTimer());
|
||||
@@ -266,7 +268,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
private Writer createLogWriter(Optional<FileSlice> fileSlice, String baseCommitTime)
|
||||
private Writer createLogWriter(Option<FileSlice> fileSlice, String baseCommitTime)
|
||||
throws IOException, InterruptedException {
|
||||
return HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
@@ -305,4 +307,4 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user