Implement HoodieLogFormat replacing Avro as the default log format
This commit is contained in:
committed by
vinoth chandar
parent
3c984447da
commit
240c91241b
@@ -22,15 +22,17 @@ import com.uber.hoodie.common.model.HoodieDeltaWriteStat;
|
||||
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.table.log.HoodieLogAppendConfig;
|
||||
import com.uber.hoodie.common.table.log.HoodieLogFile;
|
||||
import com.uber.hoodie.common.table.log.avro.RollingAvroLogAppender;
|
||||
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.util.FSUtils;
|
||||
import com.uber.hoodie.common.util.HoodieAvroUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieAppendException;
|
||||
import com.uber.hoodie.exception.HoodieUpsertException;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -51,11 +53,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
private String partitionPath;
|
||||
private RollingAvroLogAppender logAppender;
|
||||
private List<HoodieRecord<T>> records;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private HoodieLogFile currentLogFile;
|
||||
private Writer writer;
|
||||
|
||||
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime,
|
||||
HoodieTable<T> hoodieTable, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
@@ -84,17 +86,15 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
writeStatus.getStat().setFileId(fileId);
|
||||
|
||||
try {
|
||||
HoodieLogAppendConfig logConfig = HoodieLogAppendConfig.newBuilder()
|
||||
.onPartitionPath(
|
||||
new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).withBaseCommitTime(baseCommitTime).withSchema(schema)
|
||||
.withFs(fs).withLogFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.logAppender = new RollingAvroLogAppender(logConfig);
|
||||
this.currentLogFile = logAppender.getConfig().getLogFile();
|
||||
this.writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(fileId).overBaseCommit(baseCommitTime)
|
||||
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
this.currentLogFile = writer.getLogFile();
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogVersion(currentLogFile.getLogVersion());
|
||||
((HoodieDeltaWriteStat) writeStatus.getStat())
|
||||
.setLogOffset(logAppender.getCurrentSize());
|
||||
.setLogOffset(writer.getCurrentSize());
|
||||
} catch (Exception e) {
|
||||
logger.error("Error in update task at commit " + commitTime, e);
|
||||
writeStatus.setGlobalError(e);
|
||||
@@ -139,11 +139,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
|
||||
public void doAppend() {
|
||||
Iterator<IndexedRecord> recordItr =
|
||||
List<IndexedRecord> recordItr =
|
||||
records.stream().map(this::getIndexedRecord).filter(Optional::isPresent)
|
||||
.map(Optional::get).iterator();
|
||||
.map(Optional::get).collect(Collectors.toList());
|
||||
try {
|
||||
logAppender.append(recordItr);
|
||||
writer = writer.appendBlock(new HoodieAvroDataBlock(recordItr, schema));
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
||||
@@ -152,8 +152,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
|
||||
public void close() {
|
||||
try {
|
||||
if (logAppender != null) {
|
||||
logAppender.close();
|
||||
if (writer != null) {
|
||||
writer.close();
|
||||
}
|
||||
writeStatus.getStat().setNumWrites(recordsWritten);
|
||||
writeStatus.getStat().setNumDeletes(recordsDeleted);
|
||||
|
||||
Reference in New Issue
Block a user