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

@@ -29,13 +29,13 @@ import com.uber.hoodie.common.table.log.block.HoodieAvroDataBlock;
import com.uber.hoodie.common.table.log.block.HoodieDeleteBlock;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
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 org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -45,10 +45,8 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import java.util.stream.Collectors;
/**
* IO Operation to append data onto an existing file.
@@ -74,7 +72,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
String fileId,
Iterator<HoodieRecord<T>> recordItr) {
super(config, commitTime, hoodieTable);
WriteStatus writeStatus = new WriteStatus();
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieDeltaWriteStat());
this.writeStatus = writeStatus;
this.fileId = fileId;
@@ -128,6 +126,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
}
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
Optional<IndexedRecord> avroRecord = hoodieRecord.getData().getInsertValue(schema);
@@ -145,11 +144,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
}
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord);
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return avroRecord;
} catch (Exception e) {
logger.error("Error writing record " + hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
}
return Optional.empty();
}