1
0

Reformatting code per Google Code Style all over

This commit is contained in:
Vinoth Chandar
2017-11-12 22:54:56 -08:00
committed by vinoth chandar
parent 5a62480a92
commit e45679f5e2
254 changed files with 21580 additions and 21108 deletions

View File

@@ -16,19 +16,23 @@
package com.uber.hoodie.io;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.WriteStatus;
import com.uber.hoodie.common.model.HoodiePartitionMetadata;
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;
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.HoodieUpsertException;
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.HashMap;
import java.util.Iterator;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
@@ -36,197 +40,197 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Optional;
@SuppressWarnings("Duplicates")
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
private WriteStatus writeStatus;
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
private HoodieStorageWriter<IndexedRecord> storageWriter;
private Path newFilePath;
private Path oldFilePath;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
private static Logger logger = LogManager.getLogger(HoodieMergeHandle.class);
public HoodieMergeHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, hoodieTable);
init(fileId, recordItr);
}
private WriteStatus writeStatus;
private HashMap<String, HoodieRecord<T>> keyToNewRecords;
private HoodieStorageWriter<IndexedRecord> storageWriter;
private Path newFilePath;
private Path oldFilePath;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
/**
* Load the new incoming records in a map, and extract the old file path.
*/
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
this.keyToNewRecords = new HashMap<>();
public HoodieMergeHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
super(config, commitTime, hoodieTable);
init(fileId, recordItr);
}
try {
// Load the new records in a map
while (newRecordsItr.hasNext()) {
HoodieRecord<T> record = newRecordsItr.next();
// If the first record, we need to extract some info out
if (oldFilePath == null) {
String latestValidFilePath = fileSystemView
.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
/**
* Load the new incoming records in a map, and extract the old file path.
*/
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
this.keyToNewRecords = new HashMap<>();
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), record.getPartitionPath()));
partitionMetadata.trySave(TaskContext.getPartitionId());
try {
// Load the new records in a map
while (newRecordsItr.hasNext()) {
HoodieRecord<T> record = newRecordsItr.next();
// If the first record, we need to extract some info out
if (oldFilePath == null) {
String latestValidFilePath = fileSystemView
.getLatestDataFiles(record.getPartitionPath())
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst()
.get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
+ latestValidFilePath);
String relativePath = new Path( record.getPartitionPath() + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
newFilePath = new Path(config.getBasePath(), relativePath);
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), record.getPartitionPath()));
partitionMetadata.trySave(TaskContext.getPartitionId());
// handle cases of partial failures, for update task
if (fs.exists(newFilePath)) {
fs.delete(newFilePath, false);
}
oldFilePath = new Path(
config.getBasePath() + "/" + record.getPartitionPath() + "/"
+ latestValidFilePath);
String relativePath = new Path(record.getPartitionPath() + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
newFilePath = new Path(config.getBasePath(), relativePath);
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
oldFilePath.toString(), newFilePath.toString()));
// file name is same for all records, in this bunch
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
writeStatus.getStat().setFileId(fileId);
writeStatus.getStat().setPath(relativePath);
}
keyToNewRecords.put(record.getRecordKey(), record);
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
}
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
// handle cases of partial failures, for update task
if (fs.exists(newFilePath)) {
fs.delete(newFilePath, false);
}
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
oldFilePath.toString(), newFilePath.toString()));
// file name is same for all records, in this bunch
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(record.getPartitionPath());
writeStatus.getStat().setFileId(fileId);
writeStatus.getStat().setPath(relativePath);
}
keyToNewRecords.put(record.getRecordKey(), record);
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
}
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, newFilePath, hoodieTable, config, schema);
} catch (Exception e) {
logger.error("Error in update task at commit " + commitTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ commitTime + " on path " + hoodieTable.getMetaClient().getBasePath(), e);
}
}
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord, Optional<IndexedRecord> indexedRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
if(indexedRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
recordsWritten++;
updatedRecordsWritten++;
} else {
recordsDeleted++;
}
private boolean writeUpdateRecord(HoodieRecord<T> hoodieRecord,
Optional<IndexedRecord> indexedRecord) {
Optional recordMetadata = hoodieRecord.getData().getMetadata();
try {
if (indexedRecord.isPresent()) {
storageWriter.writeAvroWithMetadata(indexedRecord.get(), hoodieRecord);
recordsWritten++;
updatedRecordsWritten++;
} else {
recordsDeleted++;
}
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return true;
} catch (Exception e) {
logger.error("Error writing record "+ hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
}
return false;
hoodieRecord.deflate();
writeStatus.markSuccess(hoodieRecord, recordMetadata);
return true;
} catch (Exception e) {
logger.error("Error writing record " + hoodieRecord, e);
writeStatus.markFailure(hoodieRecord, e, recordMetadata);
}
return false;
}
/**
* Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
*/
public void write(GenericRecord oldRecord) {
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
boolean copyOldRecord = true;
if (keyToNewRecords.containsKey(key)) {
try {
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, schema);
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
/**
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
* the file.
*/
public void write(GenericRecord oldRecord) {
String key = oldRecord.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
boolean copyOldRecord = true;
if (keyToNewRecords.containsKey(key)) {
try {
Optional<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
.combineAndGetUpdateValue(oldRecord, schema);
if (writeUpdateRecord(hoodieRecord, combinedAvroRecord)) {
/* ONLY WHEN
* 1) we have an update for this key AND
* 2) We are able to successfully write the the combined new value
*
* We no longer need to copy the old record over.
*/
copyOldRecord = false;
}
keyToNewRecords.remove(key);
} catch (Exception e) {
throw new HoodieUpsertException("Failed to combine/merge new record with old value in storage, for new record {"
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
}
}
if (copyOldRecord) {
// this should work as it is, since this is an existing record
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
+ getOldFilePath() + " to new file " + newFilePath;
try {
storageWriter.writeAvro(key, oldRecord);
} catch (ClassCastException e) {
logger.error(
"Schema mismatch when rewriting old record " + oldRecord + " from file "
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
.toString(true));
throw new HoodieUpsertException(errMsg, e);
} catch (IOException e) {
logger.error("Failed to merge old record into new file for key " + key + " from old file "
+ getOldFilePath() + " to new file " + newFilePath, e);
throw new HoodieUpsertException(errMsg, e);
}
recordsWritten ++;
copyOldRecord = false;
}
keyToNewRecords.remove(key);
} catch (Exception e) {
throw new HoodieUpsertException(
"Failed to combine/merge new record with old value in storage, for new record {"
+ keyToNewRecords.get(key) + "}, old value {" + oldRecord + "}", e);
}
}
public void close() {
try {
// write out any pending records (this can happen when inserts are turned into updates)
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
while (pendingRecordsItr.hasNext()) {
String key = pendingRecordsItr.next();
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
}
keyToNewRecords.clear();
if (storageWriter != null) {
storageWriter.close();
}
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
if (copyOldRecord) {
// this should work as it is, since this is an existing record
String errMsg = "Failed to merge old record into new file for key " + key + " from old file "
+ getOldFilePath() + " to new file " + newFilePath;
try {
storageWriter.writeAvro(key, oldRecord);
} catch (ClassCastException e) {
logger.error(
"Schema mismatch when rewriting old record " + oldRecord + " from file "
+ getOldFilePath() + " to file " + newFilePath + " with schema " + schema
.toString(true));
throw new HoodieUpsertException(errMsg, e);
} catch (IOException e) {
logger.error("Failed to merge old record into new file for key " + key + " from old file "
+ getOldFilePath() + " to new file " + newFilePath, e);
throw new HoodieUpsertException(errMsg, e);
}
recordsWritten++;
}
}
public Path getOldFilePath() {
return oldFilePath;
}
public void close() {
try {
// write out any pending records (this can happen when inserts are turned into updates)
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
while (pendingRecordsItr.hasNext()) {
String key = pendingRecordsItr.next();
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
writeUpdateRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
}
keyToNewRecords.clear();
public WriteStatus getWriteStatus() {
return writeStatus;
if (storageWriter != null) {
storageWriter.close();
}
writeStatus.getStat().setTotalWriteBytes(FSUtils.getFileSize(fs, newFilePath));
writeStatus.getStat().setNumWrites(recordsWritten);
writeStatus.getStat().setNumDeletes(recordsDeleted);
writeStatus.getStat().setNumUpdateWrites(updatedRecordsWritten);
writeStatus.getStat().setTotalWriteErrors(writeStatus.getFailedRecords().size());
} catch (IOException e) {
throw new HoodieUpsertException("Failed to close UpdateHandle", e);
}
}
public Path getOldFilePath() {
return oldFilePath;
}
public WriteStatus getWriteStatus() {
return writeStatus;
}
}