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

@@ -37,14 +37,6 @@ 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.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import org.apache.spark.util.SizeEstimator;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Comparator;
@@ -53,6 +45,13 @@ import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.avro.generic.GenericRecord;
import org.apache.avro.generic.IndexedRecord;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.TaskContext;
import org.apache.spark.util.SizeEstimator;
/**
* IO Operation to append data onto an existing file.
@@ -61,14 +60,13 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
private static Logger logger = LogManager.getLogger(HoodieAppendHandle.class);
private static AtomicLong recordIndex = new AtomicLong(1);
private TableFileSystemView.RealtimeView fileSystemView;
private final WriteStatus writeStatus;
private final String fileId;
private String partitionPath;
private Iterator<HoodieRecord<T>> recordItr;
List<IndexedRecord> recordList = new ArrayList<>();
List<String> keysToDelete = new ArrayList<>();
private TableFileSystemView.RealtimeView fileSystemView;
private String partitionPath;
private Iterator<HoodieRecord<T>> recordItr;
private long recordsWritten = 0;
private long recordsDeleted = 0;
private long averageRecordSize = 0;
@@ -76,11 +74,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
private Writer writer;
private boolean doInit = true;
public HoodieAppendHandle(HoodieWriteConfig config,
String commitTime,
HoodieTable<T> hoodieTable,
String fileId,
Iterator<HoodieRecord<T>> recordItr) {
public HoodieAppendHandle(HoodieWriteConfig config, String commitTime, HoodieTable<T> hoodieTable,
String fileId, Iterator<HoodieRecord<T>> recordItr) {
super(config, commitTime, hoodieTable);
WriteStatus writeStatus = ReflectionUtils.loadClass(config.getWriteStatusClassName());
writeStatus.setStat(new HoodieDeltaWriteStat());
@@ -93,42 +88,40 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
private void init(String partitionPath) {
// extract some information from the first record
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId))
.findFirst().get();
// HACK(vc) This also assumes a base file. It will break, if appending without one.
String latestValidFilePath = fileSlice.getDataFile().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(partitionPath);
writeStatus.getStat().setFileId(fileId);
this.partitionPath = partitionPath;
FileSlice fileSlice = fileSystemView.getLatestFileSlices(partitionPath)
.filter(fileSlice1 -> fileSlice1.getDataFile().get().getFileId().equals(fileId)).findFirst()
.get();
// HACK(vc) This also assumes a base file. It will break, if appending without one.
String latestValidFilePath = fileSlice.getDataFile().get().getFileName();
String baseCommitTime = FSUtils.getCommitTime(latestValidFilePath);
writeStatus.getStat().setPrevCommit(baseCommitTime);
writeStatus.setFileId(fileId);
writeStatus.setPartitionPath(partitionPath);
writeStatus.getStat().setFileId(fileId);
this.partitionPath = partitionPath;
try {
this.writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(fileSlice.getLogFiles()
.map(logFile -> logFile.getLogVersion())
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withSizeThreshold(config.getLogFileMaxSize())
.withFs(fs).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
this.currentLogFile = writer.getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat())
.setLogOffset(writer.getCurrentSize());
} 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 HDFS path " + hoodieTable
.getMetaClient().getBasePath() + partitionPath, e);
}
Path path = new Path(partitionPath,
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPath(path.toString());
try {
this.writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(new Path(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
fileSlice.getLogFiles().map(logFile -> logFile.getLogVersion())
.max(Comparator.naturalOrder()).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
this.currentLogFile = writer.getLogFile();
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogVersion(currentLogFile.getLogVersion());
((HoodieDeltaWriteStat) writeStatus.getStat()).setLogOffset(writer.getCurrentSize());
} 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 HDFS path " + hoodieTable.getMetaClient().getBasePath()
+ partitionPath, e);
}
Path path = new Path(partitionPath,
FSUtils.makeDataFileName(commitTime, TaskContext.getPartitionId(), fileId));
writeStatus.getStat().setPath(path.toString());
}
private Optional<IndexedRecord> getIndexedRecord(HoodieRecord<T> hoodieRecord) {
@@ -150,7 +143,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
}
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 avroRecord;
@@ -165,7 +159,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
// to make sure we don't append records with older (shorter) schema than already appended
public void doAppend() {
int maxBlockSize = config.getLogFileDataBlockMaxSize(); int numberOfRecords = 0;
int maxBlockSize = config.getLogFileDataBlockMaxSize();
int numberOfRecords = 0;
Map<HoodieLogBlock.HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, commitTime);
header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, schema.toString());
@@ -173,16 +168,17 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
HoodieRecord record = recordItr.next();
// update the new location of the record, so we know where to find it next
record.setNewLocation(new HoodieRecordLocation(commitTime, fileId));
if(doInit) {
if (doInit) {
init(record.getPartitionPath());
averageRecordSize = SizeEstimator.estimate(record);
doInit = false;
}
// Append if max number of records reached to achieve block size
if(numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
// Recompute averageRecordSize before writing a new block and update existing value with avg of new and old
if (numberOfRecords >= (int) (maxBlockSize / averageRecordSize)) {
// Recompute averageRecordSize before writing a new block and update existing value with
// avg of new and old
logger.info("AvgRecordSize => " + averageRecordSize);
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record))/2;
averageRecordSize = (averageRecordSize + SizeEstimator.estimate(record)) / 2;
doAppend(header);
numberOfRecords = 0;
}