1
0

capture record metadata before deflating for record counting

This commit is contained in:
Kaushik Devarajaiah
2017-08-22 14:48:04 -07:00
committed by vinoth chandar
parent f2980052cd
commit c98ee057fc
9 changed files with 192 additions and 23 deletions

View File

@@ -23,6 +23,7 @@ import com.uber.hoodie.common.model.HoodieRecordLocation;
import com.uber.hoodie.common.model.HoodieRecordPayload;
import com.uber.hoodie.common.model.HoodieWriteStat;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieInsertException;
import com.uber.hoodie.io.storage.HoodieStorageWriter;
@@ -50,7 +51,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
public HoodieCreateHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable, String partitionPath) {
super(config, commitTime, hoodieTable);
this.status = new WriteStatus();
this.status = ReflectionUtils.loadClass(config.getWriteStatusClassName());
status.setFileId(UUID.randomUUID().toString());
status.setPartitionPath(partitionPath);
@@ -89,6 +90,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
* @param record
*/
public void write(HoodieRecord record) {
Optional recordMetadata = record.getData().getMetadata();
try {
Optional<IndexedRecord> avroRecord = record.getData().getInsertValue(schema);
@@ -100,13 +102,12 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieIOH
} else {
recordsDeleted++;
}
record.deflate();
status.markSuccess(record);
status.markSuccess(record, recordMetadata);
} catch (Throwable t) {
// Not throwing exception from here, since we don't want to fail the entire job
// for a single record
status.markFailure(record, t);
status.markFailure(record, t, recordMetadata);
logger.error("Error writing record " + record, t);
}
}