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

@@ -24,6 +24,7 @@ import com.uber.hoodie.common.util.HoodieAvroUtils;
import com.uber.hoodie.config.HoodieWriteConfig;
import com.uber.hoodie.exception.HoodieIOException;
import com.uber.hoodie.table.HoodieTable;
import java.io.IOException;
import org.apache.avro.Schema;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -31,68 +32,67 @@ import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import java.io.IOException;
public abstract class HoodieIOHandle<T extends HoodieRecordPayload> {
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
protected final String commitTime;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTable<T> hoodieTable;
protected HoodieTimeline hoodieTimeline;
protected TableFileSystemView.ReadOptimizedView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView();
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
private static Logger logger = LogManager.getLogger(HoodieIOHandle.class);
protected final String commitTime;
protected final HoodieWriteConfig config;
protected final FileSystem fs;
protected final HoodieTable<T> hoodieTable;
protected HoodieTimeline hoodieTimeline;
protected TableFileSystemView.ReadOptimizedView fileSystemView;
protected final Schema schema;
public HoodieIOHandle(HoodieWriteConfig config, String commitTime,
HoodieTable<T> hoodieTable) {
this.commitTime = commitTime;
this.config = config;
this.fs = FSUtils.getFs();
this.hoodieTable = hoodieTable;
this.hoodieTimeline = hoodieTable.getCompletedCommitTimeline();
this.fileSystemView = hoodieTable.getROFileSystemView();
this.schema =
HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
}
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
Path path = new Path(config.getBasePath(), partitionPath);
try {
fs.mkdirs(path); // create a new partition as needed.
} catch (IOException e) {
throw new HoodieIOException("Failed to make dir " + path, e);
}
public Path makeNewPath(String partitionPath, int taskPartitionId, String fileName) {
Path path = new Path(config.getBasePath(), partitionPath);
try {
fs.mkdirs(path); // create a new partition as needed.
} catch (IOException e) {
throw new HoodieIOException("Failed to make dir " + path, e);
return new Path(path.toString(),
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
}
/**
* Deletes any new tmp files written during the current commit, into the partition
*/
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
String commitTime,
String partitionPath,
int taskPartitionId) {
FileSystem fs = FSUtils.getFs();
try {
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
if (prevFailedFiles != null) {
logger.info("Deleting " + prevFailedFiles.length
+ " files generated by previous failed attempts.");
for (FileStatus status : prevFailedFiles) {
fs.delete(status.getPath(), false);
}
return new Path(path.toString(),
FSUtils.makeDataFileName(commitTime, taskPartitionId, fileName));
}
} catch (IOException e) {
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
e);
}
}
/**
* Deletes any new tmp files written during the current commit, into the partition
*/
public static void cleanupTmpFilesFromCurrentCommit(HoodieWriteConfig config,
String commitTime,
String partitionPath,
int taskPartitionId) {
FileSystem fs = FSUtils.getFs();
try {
FileStatus[] prevFailedFiles = fs.globStatus(new Path(String
.format("%s/%s/%s", config.getBasePath(), partitionPath,
FSUtils.maskWithoutFileId(commitTime, taskPartitionId))));
if (prevFailedFiles != null) {
logger.info("Deleting " + prevFailedFiles.length
+ " files generated by previous failed attempts.");
for (FileStatus status : prevFailedFiles) {
fs.delete(status.getPath(), false);
}
}
} catch (IOException e) {
throw new HoodieIOException("Failed to cleanup Temp files from commit " + commitTime,
e);
}
}
public Schema getSchema() {
return schema;
}
public Schema getSchema() {
return schema;
}
}