Adding ability for inserts to be written to log files
This commit is contained in:
committed by
vinoth chandar
parent
34827d50e1
commit
3da063f83b
@@ -31,8 +31,8 @@ import com.uber.hoodie.io.storage.HoodieStorageWriter;
|
||||
import com.uber.hoodie.io.storage.HoodieStorageWriterFactory;
|
||||
import com.uber.hoodie.table.HoodieTable;
|
||||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.Optional;
|
||||
import java.util.UUID;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
@@ -49,12 +49,13 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private Path tempPath = null;
|
||||
private long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private Iterator<HoodieRecord<T>> recordIterator;
|
||||
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath) {
|
||||
String partitionPath, String fileId) {
|
||||
super(config, commitTime, hoodieTable);
|
||||
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
|
||||
status.setFileId(UUID.randomUUID().toString());
|
||||
status.setFileId(fileId);
|
||||
status.setPartitionPath(partitionPath);
|
||||
|
||||
final int sparkPartitionId = TaskContext.getPartitionId();
|
||||
@@ -77,12 +78,13 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
logger.info("New InsertHandle for partition :" + partitionPath);
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether we can accept the incoming records, into the current file, depending on
|
||||
* <p>
|
||||
* - Whether it belongs to the same partitionPath as existing records - Whether the current file
|
||||
* written bytes lt max file size
|
||||
*/
|
||||
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
|
||||
String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordIterator) {
|
||||
this(config, commitTime, hoodieTable, partitionPath, fileId);
|
||||
this.recordIterator = recordIterator;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean canWrite(HoodieRecord record) {
|
||||
return storageWriter.canWrite() && record.getPartitionPath().equals(status.getPartitionPath());
|
||||
}
|
||||
@@ -114,9 +116,30 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Writes all records passed
|
||||
*/
|
||||
public void write() {
|
||||
try {
|
||||
while (recordIterator.hasNext()) {
|
||||
HoodieRecord<T> record = recordIterator.next();
|
||||
write(record, record.getData().getInsertValue(schema));
|
||||
}
|
||||
} catch (IOException io) {
|
||||
throw new HoodieInsertException(
|
||||
"Failed to insert records for path " + getStorageWriterPath(), io);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public WriteStatus getWriteStatus() {
|
||||
return status;
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object
|
||||
*/
|
||||
@Override
|
||||
public WriteStatus close() {
|
||||
logger.info("Closing the file " + status.getFileId() + " as we are done with all the records "
|
||||
+ recordsWritten);
|
||||
|
||||
Reference in New Issue
Block a user