1
0

CodeStyle formatting to conform to basic Checkstyle rules.

The code-style rules follow google style with some changes:

1. Increase line length from 100 to 120
2. Disable JavaDoc related checkstyles as this needs more manual work.

Both source and test code are checked for code-style
This commit is contained in:
Balaji Varadarajan
2018-03-20 16:29:20 -07:00
committed by vinoth chandar
parent 987f5d6b96
commit 788e4f2d2e
200 changed files with 6209 additions and 5975 deletions

View File

@@ -26,14 +26,18 @@ import com.uber.hoodie.common.table.TableFileSystemView;
import com.uber.hoodie.common.util.FSUtils;
import com.uber.hoodie.common.util.ReflectionUtils;
import com.uber.hoodie.common.util.collection.ExternalSpillableMap;
import com.uber.hoodie.common.util.collection.converter.StringConverter;
import com.uber.hoodie.common.util.collection.converter.HoodieRecordConverter;
import com.uber.hoodie.common.util.collection.converter.StringConverter;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
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.Iterator;
import java.util.Map;
import java.util.Optional;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
@@ -41,11 +45,6 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import java.io.IOException;
import java.util.Iterator;
import java.util.Map;
import java.util.Optional;
@SuppressWarnings("Duplicates")
public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHandle<T> {
@@ -62,59 +61,46 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
private long recordsDeleted = 0;
private long updatedRecordsWritten = 0;
public HoodieMergeHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr,
String fileId) {
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
Iterator<HoodieRecord<T>> recordItr, String fileId) {
super(config, commitTime, hoodieTable);
this.fileSystemView = hoodieTable.getROFileSystemView();
init(fileId, init(fileId, recordItr));
}
public HoodieMergeHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
Map<String, HoodieRecord<T>> keyToNewRecords,
String fileId) {
public HoodieMergeHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
Map<String, HoodieRecord<T>> keyToNewRecords, String fileId) {
super(config, commitTime, hoodieTable);
this.fileSystemView = hoodieTable.getROFileSystemView();
this.keyToNewRecords = keyToNewRecords;
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath());
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
.getPartitionPath());
}
/**
* Extract old file path, initialize StorageWriter and WriteStatus
* @param fileId
* @param partitionPath
*/
private void init(String fileId, String partitionPath) {
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieWriteStat());
this.writeStatus = writeStatus;
try {
String latestValidFilePath = fileSystemView
.getLatestDataFiles(partitionPath)
.filter(dataFile -> dataFile.getFileId().equals(fileId))
.findFirst()
.get().getFileName();
String latestValidFilePath = fileSystemView.getLatestDataFiles(partitionPath)
.filter(dataFile -> dataFile.getFileId().equals(fileId)).findFirst().get().getFileName();
writeStatus.getStat().setPrevCommit(FSUtils.getCommitTime(latestValidFilePath));
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs,
commitTime,
new Path(config.getBasePath()),
new Path(config.getBasePath(), partitionPath));
HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(fs, commitTime,
new Path(config.getBasePath()), new Path(config.getBasePath(), partitionPath));
partitionMetadata.trySave(TaskContext.getPartitionId());
oldFilePath = new Path(
config.getBasePath() + "/" + partitionPath + "/"
+ latestValidFilePath);
config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
String relativePath = new Path(partitionPath + "/" + FSUtils
.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId)).toString();
newFilePath = new Path(config.getBasePath(), relativePath);
if (config.shouldUseTempFolderForCopyOnWriteForMerge()) {
this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(),
fileId, TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
this.tempPath = makeTempPath(partitionPath, TaskContext.getPartitionId(), fileId,
TaskContext.get().stageId(), TaskContext.get().taskAttemptId());
}
// handle cases of partial failures, for update task
@@ -122,8 +108,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
fs.delete(newFilePath, false);
}
logger.info(String.format("Merging new data into oldPath %s, as newPath %s",
oldFilePath.toString(), getStorageWriterPath().toString()));
logger.info(String
.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),
getStorageWriterPath().toString()));
// file name is same for all records, in this bunch
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(partitionPath);
@@ -143,9 +130,6 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
/**
* Load the new incoming records in a map and return partitionPath
* @param fileId
* @param newRecordsItr
* @return
*/
private String init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
try {
@@ -153,7 +137,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
logger.info("MaxMemoryPerPartitionMerge => " + config.getMaxMemoryPerPartitionMerge());
this.keyToNewRecords = new ExternalSpillableMap<>(config.getMaxMemoryPerPartitionMerge(),
Optional.empty(), new StringConverter(), new HoodieRecordConverter(schema, config.getPayloadClass()));
} catch(IOException io) {
} catch (IOException io) {
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
}
String partitionPath = null;
@@ -164,14 +148,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
}
logger.debug("Number of entries in MemoryBasedMap => " +
((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
+ "Total size in bytes of MemoryBasedMap => " +
((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
+ "Number of entries in DiskBasedMap => " +
((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
+ "Size of file spilled to disk => " +
((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
logger.debug("Number of entries in MemoryBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries()
+ "Total size in bytes of MemoryBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize()
+ "Number of entries in DiskBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries()
+ "Size of file spilled to disk => "
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
return partitionPath;
}
@@ -189,7 +173,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
}
writeStatus.markSuccess(hoodieRecord, recordMetadata);
// deflate record payload after recording success. This will help users access payload as a part of marking
// deflate record payload after recording success. This will help users access payload as a
// part of marking
// record successful.
hoodieRecord.deflate();
return true;
@@ -201,8 +186,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
}
/**
* Go through an old record. Here if we detect a newer version shows up, we write the new one to
* the file.
* 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();
@@ -213,12 +197,12 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
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.
*/
/* 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);
@@ -236,10 +220,9 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
try {
storageWriter.writeAvro(key, oldRecord);
} catch (ClassCastException e) {
logger.error(
"Schema mismatch when rewriting old record " + oldRecord + " from file "
+ getOldFilePath() + " to file " + getStorageWriterPath() + " with schema " + schema
.toString(true));
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file "
+ getOldFilePath() + " to file " + getStorageWriterPath() + " 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 "