1
0

[HUDI-296] Explore use of spotless to auto fix formatting errors (#945)

- Add spotless format fixing to project
- One time reformatting for conformity
- Build fails for formatting changes and mvn spotless:apply autofixes them
This commit is contained in:
leesf
2019-10-10 20:19:40 +08:00
committed by vinoth chandar
parent 834c591955
commit b19bed442d
381 changed files with 7350 additions and 9064 deletions

View File

@@ -32,8 +32,8 @@ import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
/**
* Abstract class taking care of holding common member variables (FileSystem, SparkContext, HoodieConfigs)
* Also, manages embedded timeline-server if enabled.
* Abstract class taking care of holding common member variables (FileSystem, SparkContext, HoodieConfigs) Also, manages
* embedded timeline-server if enabled.
*/
public abstract class AbstractHoodieClient implements Serializable, AutoCloseable {
@@ -45,10 +45,9 @@ public abstract class AbstractHoodieClient implements Serializable, AutoCloseabl
protected final String basePath;
/**
* Timeline Server has the same lifetime as that of Client.
* Any operations done on the same timeline service will be able to take advantage
* of the cached file-system view. New completed actions will be synced automatically
* in an incremental fashion.
* Timeline Server has the same lifetime as that of Client. Any operations done on the same timeline service will be
* able to take advantage of the cached file-system view. New completed actions will be synced automatically in an
* incremental fashion.
*/
private transient Option<EmbeddedTimelineService> timelineServer;
private final boolean shouldStopTimelineServer;

View File

@@ -69,8 +69,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build());
}
public CompactionAdminClient(JavaSparkContext jsc, String basePath,
Option<EmbeddedTimelineService> timelineServer) {
public CompactionAdminClient(JavaSparkContext jsc, String basePath, Option<EmbeddedTimelineService> timelineServer) {
super(jsc, HoodieWriteConfig.newBuilder().withPath(basePath).build(), timelineServer);
}
@@ -78,11 +77,11 @@ public class CompactionAdminClient extends AbstractHoodieClient {
* Validate all compaction operations in a compaction plan. Verifies the file-slices are consistent with corresponding
* compaction operations.
*
* @param metaClient Hoodie Table Meta Client
* @param metaClient Hoodie Table Meta Client
* @param compactionInstant Compaction Instant
*/
public List<ValidationOpResult> validateCompactionPlan(HoodieTableMetaClient metaClient,
String compactionInstant, int parallelism) throws IOException {
public List<ValidationOpResult> validateCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant,
int parallelism) throws IOException {
HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant);
HoodieTableFileSystemView fsView =
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
@@ -108,19 +107,17 @@ public class CompactionAdminClient extends AbstractHoodieClient {
* This operation MUST be executed with compactions and writer turned OFF.
*
* @param compactionInstant Compaction Instant
* @param skipValidation Skip validation step
* @param parallelism Parallelism
* @param dryRun Dry Run
* @param skipValidation Skip validation step
* @param parallelism Parallelism
* @param dryRun Dry Run
*/
public List<RenameOpResult> unscheduleCompactionPlan(
String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception {
public List<RenameOpResult> unscheduleCompactionPlan(String compactionInstant, boolean skipValidation,
int parallelism, boolean dryRun) throws Exception {
HoodieTableMetaClient metaClient = createMetaClient(false);
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
getRenamingActionsForUnschedulingCompactionPlan(metaClient, compactionInstant, parallelism,
Option.empty(), skipValidation);
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions = getRenamingActionsForUnschedulingCompactionPlan(metaClient,
compactionInstant, parallelism, Option.empty(), skipValidation);
List<RenameOpResult> res =
runRenamingOps(metaClient, renameActions, parallelism, dryRun);
List<RenameOpResult> res = runRenamingOps(metaClient, renameActions, parallelism, dryRun);
Option<Boolean> success =
Option.fromJavaOptional(res.stream().map(r -> (r.isExecuted() && r.isSuccess())).reduce(Boolean::logicalAnd));
@@ -145,21 +142,20 @@ public class CompactionAdminClient extends AbstractHoodieClient {
}
/**
* Remove a fileId from pending compaction. Removes the associated compaction operation and rename delta-files
* that were generated for that file-id after the compaction operation was scheduled.
* Remove a fileId from pending compaction. Removes the associated compaction operation and rename delta-files that
* were generated for that file-id after the compaction operation was scheduled.
*
* This operation MUST be executed with compactions and writer turned OFF.
*
* @param fgId FileGroupId to be unscheduled
* @param fgId FileGroupId to be unscheduled
* @param skipValidation Skip validation
* @param dryRun Dry Run Mode
* @param dryRun Dry Run Mode
*/
public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId,
boolean skipValidation, boolean dryRun) throws Exception {
public List<RenameOpResult> unscheduleCompactionFileId(HoodieFileGroupId fgId, boolean skipValidation, boolean dryRun)
throws Exception {
HoodieTableMetaClient metaClient = createMetaClient(false);
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId,
Option.empty(), skipValidation);
getRenamingActionsForUnschedulingCompactionForFileId(metaClient, fgId, Option.empty(), skipValidation);
List<RenameOpResult> res = runRenamingOps(metaClient, renameActions, 1, dryRun);
@@ -167,15 +163,15 @@ public class CompactionAdminClient extends AbstractHoodieClient {
// Ready to remove this file-Id from compaction request
Pair<String, HoodieCompactionOperation> compactionOperationWithInstant =
CompactionUtils.getAllPendingCompactionOperations(metaClient).get(fgId);
HoodieCompactionPlan plan = CompactionUtils
.getCompactionPlan(metaClient, compactionOperationWithInstant.getKey());
List<HoodieCompactionOperation> newOps = plan.getOperations().stream()
.filter(op -> (!op.getFileId().equals(fgId.getFileId()))
&& (!op.getPartitionPath().equals(fgId.getPartitionPath()))).collect(Collectors.toList());
HoodieCompactionPlan plan =
CompactionUtils.getCompactionPlan(metaClient, compactionOperationWithInstant.getKey());
List<HoodieCompactionOperation> newOps = plan.getOperations().stream().filter(
op -> (!op.getFileId().equals(fgId.getFileId())) && (!op.getPartitionPath().equals(fgId.getPartitionPath())))
.collect(Collectors.toList());
HoodieCompactionPlan newPlan =
HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build();
HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION,
compactionOperationWithInstant.getLeft());
HoodieInstant inflight =
new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionOperationWithInstant.getLeft());
Path inflightPath = new Path(metaClient.getMetaPath(), inflight.getFileName());
if (metaClient.getFs().exists(inflightPath)) {
// revert if in inflight state
@@ -189,28 +185,28 @@ public class CompactionAdminClient extends AbstractHoodieClient {
}
/**
* Renames delta files to make file-slices consistent with the timeline as dictated by Hoodie metadata.
* Use when compaction unschedule fails partially.
* Renames delta files to make file-slices consistent with the timeline as dictated by Hoodie metadata. Use when
* compaction unschedule fails partially.
*
* This operation MUST be executed with compactions and writer turned OFF.
*
* @param compactionInstant Compaction Instant to be repaired
* @param dryRun Dry Run Mode
* @param dryRun Dry Run Mode
*/
public List<RenameOpResult> repairCompaction(String compactionInstant,
int parallelism, boolean dryRun) throws Exception {
public List<RenameOpResult> repairCompaction(String compactionInstant, int parallelism, boolean dryRun)
throws Exception {
HoodieTableMetaClient metaClient = createMetaClient(false);
List<ValidationOpResult> validationResults =
validateCompactionPlan(metaClient, compactionInstant, parallelism);
List<ValidationOpResult> failed = validationResults.stream()
.filter(v -> !v.isSuccess()).collect(Collectors.toList());
List<ValidationOpResult> validationResults = validateCompactionPlan(metaClient, compactionInstant, parallelism);
List<ValidationOpResult> failed =
validationResults.stream().filter(v -> !v.isSuccess()).collect(Collectors.toList());
if (failed.isEmpty()) {
return new ArrayList<>();
}
final HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getCommitsAndCompactionTimeline());
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions = failed.stream().flatMap(v ->
getRenamingActionsToAlignWithCompactionOperation(metaClient, compactionInstant,
final HoodieTableFileSystemView fsView =
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
List<Pair<HoodieLogFile, HoodieLogFile>> renameActions =
failed.stream().flatMap(v -> getRenamingActionsToAlignWithCompactionOperation(metaClient, compactionInstant,
v.getOperation(), Option.of(fsView)).stream()).collect(Collectors.toList());
return runRenamingOps(metaClient, renameActions, parallelism, dryRun);
}
@@ -218,11 +214,10 @@ public class CompactionAdminClient extends AbstractHoodieClient {
/**
* Construction Compaction Plan from compaction instant
*/
private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient,
String compactionInstant) throws IOException {
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(
metaClient.getActiveTimeline().getInstantAuxiliaryDetails(
HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant)
throws IOException {
HoodieCompactionPlan compactionPlan = AvroUtils.deserializeCompactionPlan(metaClient.getActiveTimeline()
.getInstantAuxiliaryDetails(HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get());
return compactionPlan;
}
@@ -230,28 +225,26 @@ public class CompactionAdminClient extends AbstractHoodieClient {
* Get Renaming actions to ensure the log-files of merged file-slices is aligned with compaction operation. This
* method is used to recover from failures during unschedule compaction operations.
*
* @param metaClient Hoodie Table Meta Client
* @param metaClient Hoodie Table Meta Client
* @param compactionInstant Compaction Instant
* @param op Compaction Operation
* @param fsViewOpt File System View
* @param op Compaction Operation
* @param fsViewOpt File System View
*/
protected static List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsToAlignWithCompactionOperation(
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation op,
Option<HoodieTableFileSystemView> fsViewOpt) {
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get()
: new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
HoodieInstant lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant().get();
FileSlice merged =
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(op.getPartitionPath(), lastInstant.getTimestamp())
.filter(fs -> fs.getFileId().equals(op.getFileId())).findFirst().get();
final int maxVersion =
op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
.reduce((x, y) -> x > y ? x : y).orElse(0);
final int maxVersion = op.getDeltaFilePaths().stream().map(lf -> FSUtils.getFileVersionFromLog(new Path(lf)))
.reduce((x, y) -> x > y ? x : y).orElse(0);
List<HoodieLogFile> logFilesToBeMoved =
merged.getLogFiles().filter(lf -> lf.getLogVersion() > maxVersion).collect(Collectors.toList());
return logFilesToBeMoved.stream().map(lf -> {
Preconditions.checkArgument(lf.getLogVersion() - maxVersion > 0,
"Expect new log version to be sane");
Preconditions.checkArgument(lf.getLogVersion() - maxVersion > 0, "Expect new log version to be sane");
HoodieLogFile newLogFile = new HoodieLogFile(new Path(lf.getPath().getParent(),
FSUtils.makeLogFileName(lf.getFileId(), "." + FSUtils.getFileExtensionFromLog(lf.getPath()),
compactionInstant, lf.getLogVersion() - maxVersion, HoodieLogFormat.UNKNOWN_WRITE_TOKEN)));
@@ -280,16 +273,15 @@ public class CompactionAdminClient extends AbstractHoodieClient {
/**
* Check if a compaction operation is valid
*
* @param metaClient Hoodie Table Meta client
* @param metaClient Hoodie Table Meta client
* @param compactionInstant Compaction Instant
* @param operation Compaction Operation
* @param fsViewOpt File System View
* @param operation Compaction Operation
* @param fsViewOpt File System View
*/
private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient metaClient,
String compactionInstant, CompactionOperation operation, Option<HoodieTableFileSystemView> fsViewOpt)
throws IOException {
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient metaClient, String compactionInstant,
CompactionOperation operation, Option<HoodieTableFileSystemView> fsViewOpt) throws IOException {
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get()
: new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
Option<HoodieInstant> lastInstant = metaClient.getCommitsAndCompactionTimeline().lastInstant();
try {
if (lastInstant.isPresent()) {
@@ -300,45 +292,42 @@ public class CompactionAdminClient extends AbstractHoodieClient {
FileSlice fs = fileSliceOptional.get();
Option<HoodieDataFile> df = fs.getDataFile();
if (operation.getDataFilePath().isPresent()) {
String expPath = metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath()
.toString();
Preconditions.checkArgument(df.isPresent(), "Data File must be present. File Slice was : "
+ fs + ", operation :" + operation);
String expPath =
metaClient.getFs().getFileStatus(new Path(operation.getDataFilePath().get())).getPath().toString();
Preconditions.checkArgument(df.isPresent(),
"Data File must be present. File Slice was : " + fs + ", operation :" + operation);
Preconditions.checkArgument(df.get().getPath().equals(expPath),
"Base Path in operation is specified as " + expPath + " but got path " + df.get().getPath());
}
Set<HoodieLogFile> logFilesInFileSlice = fs.getLogFiles().collect(Collectors.toSet());
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFilePaths().stream()
.map(dp -> {
try {
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(dp));
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
return new HoodieLogFile(fileStatuses[0]);
} catch (FileNotFoundException fe) {
throw new CompactionValidationException(fe.getMessage());
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}).collect(Collectors.toSet());
Set<HoodieLogFile> missing =
logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
.collect(Collectors.toSet());
Set<HoodieLogFile> logFilesInCompactionOp = operation.getDeltaFilePaths().stream().map(dp -> {
try {
FileStatus[] fileStatuses = metaClient.getFs().listStatus(new Path(dp));
Preconditions.checkArgument(fileStatuses.length == 1, "Expect only 1 file-status");
return new HoodieLogFile(fileStatuses[0]);
} catch (FileNotFoundException fe) {
throw new CompactionValidationException(fe.getMessage());
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
}
}).collect(Collectors.toSet());
Set<HoodieLogFile> missing = logFilesInCompactionOp.stream().filter(lf -> !logFilesInFileSlice.contains(lf))
.collect(Collectors.toSet());
Preconditions.checkArgument(missing.isEmpty(),
"All log files specified in compaction operation is not present. Missing :" + missing
+ ", Exp :" + logFilesInCompactionOp + ", Got :" + logFilesInFileSlice);
Set<HoodieLogFile> diff =
logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
.collect(Collectors.toSet());
"All log files specified in compaction operation is not present. Missing :" + missing + ", Exp :"
+ logFilesInCompactionOp + ", Got :" + logFilesInFileSlice);
Set<HoodieLogFile> diff = logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
.collect(Collectors.toSet());
Preconditions.checkArgument(diff.stream().allMatch(lf -> lf.getBaseCommitTime().equals(compactionInstant)),
"There are some log-files which are neither specified in compaction plan "
+ "nor present after compaction request instant. Some of these :" + diff);
} else {
throw new CompactionValidationException("Unable to find file-slice for file-id (" + operation.getFileId()
+ " Compaction operation is invalid.");
throw new CompactionValidationException(
"Unable to find file-slice for file-id (" + operation.getFileId() + " Compaction operation is invalid.");
}
} else {
throw new CompactionValidationException("Unable to find any committed instant. Compaction Operation may "
+ "be pointing to stale file-slices");
throw new CompactionValidationException(
"Unable to find any committed instant. Compaction Operation may " + "be pointing to stale file-slices");
}
} catch (CompactionValidationException | IllegalArgumentException e) {
return new ValidationOpResult(operation, false, Option.of(e));
@@ -349,7 +338,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
/**
* Execute Renaming operation
*
* @param metaClient HoodieTable MetaClient
* @param metaClient HoodieTable MetaClient
* @param renameActions List of rename operations
*/
private List<RenameOpResult> runRenamingOps(HoodieTableMetaClient metaClient,
@@ -374,8 +363,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
}).collect();
} else {
log.info("Dry-Run Mode activated for rename operations");
return renameActions.parallelStream()
.map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty()))
return renameActions.parallelStream().map(lfPair -> new RenameOpResult(lfPair, false, false, Option.empty()))
.collect(Collectors.toList());
}
}
@@ -385,28 +373,28 @@ public class CompactionAdminClient extends AbstractHoodieClient {
* Generate renaming actions for unscheduling a pending compaction plan. NOTE: Can only be used safely when no writer
* (ingestion/compaction) is running.
*
* @param metaClient Hoodie Table MetaClient
* @param metaClient Hoodie Table MetaClient
* @param compactionInstant Compaction Instant to be unscheduled
* @param fsViewOpt Cached File System View
* @param skipValidation Skip Validation
* @param fsViewOpt Cached File System View
* @param skipValidation Skip Validation
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
* compaction.
* compaction.
*/
protected List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionPlan(
HoodieTableMetaClient metaClient, String compactionInstant, int parallelism,
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
HoodieTableFileSystemView fsView = fsViewOpt.isPresent() ? fsViewOpt.get() :
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
HoodieTableFileSystemView fsView = fsViewOpt.isPresent() ? fsViewOpt.get()
: new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
HoodieCompactionPlan plan = getCompactionPlan(metaClient, compactionInstant);
if (plan.getOperations() != null) {
log.info("Number of Compaction Operations :" + plan.getOperations().size()
+ " for instant :" + compactionInstant);
log.info(
"Number of Compaction Operations :" + plan.getOperations().size() + " for instant :" + compactionInstant);
List<CompactionOperation> ops = plan.getOperations().stream()
.map(CompactionOperation::convertFromAvroRecordInstance).collect(Collectors.toList());
return jsc.parallelize(ops, parallelism).flatMap(op -> {
try {
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant,
op, Option.of(fsView), skipValidation).iterator();
return getRenamingActionsForUnschedulingCompactionOperation(metaClient, compactionInstant, op,
Option.of(fsView), skipValidation).iterator();
} catch (IOException ioe) {
throw new HoodieIOException(ioe.getMessage(), ioe);
} catch (CompactionValidationException ve) {
@@ -422,20 +410,20 @@ public class CompactionAdminClient extends AbstractHoodieClient {
* Generate renaming actions for unscheduling a compaction operation NOTE: Can only be used safely when no writer
* (ingestion/compaction) is running.
*
* @param metaClient Hoodie Table MetaClient
* @param metaClient Hoodie Table MetaClient
* @param compactionInstant Compaction Instant
* @param operation Compaction Operation
* @param fsViewOpt Cached File System View
* @param skipValidation Skip Validation
* @param operation Compaction Operation
* @param fsViewOpt Cached File System View
* @param skipValidation Skip Validation
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
* compaction.
* compaction.
*/
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionOperation(
HoodieTableMetaClient metaClient, String compactionInstant, CompactionOperation operation,
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
List<Pair<HoodieLogFile, HoodieLogFile>> result = new ArrayList<>();
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get() :
new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
HoodieTableFileSystemView fileSystemView = fsViewOpt.isPresent() ? fsViewOpt.get()
: new HoodieTableFileSystemView(metaClient, metaClient.getCommitsAndCompactionTimeline());
if (!skipValidation) {
validateCompactionOperation(metaClient, compactionInstant, operation, Option.of(fileSystemView));
}
@@ -444,15 +432,13 @@ public class CompactionAdminClient extends AbstractHoodieClient {
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(operation.getPartitionPath(), lastInstant.getTimestamp())
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
List<HoodieLogFile> logFilesToRepair =
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
.sorted(HoodieLogFile.getLogFileComparator())
.collect(Collectors.toList());
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
FileSlice fileSliceForCompaction =
fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime(), true)
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
int maxUsedVersion =
fileSliceForCompaction.getLogFiles().findFirst().map(HoodieLogFile::getLogVersion)
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
int maxUsedVersion = fileSliceForCompaction.getLogFiles().findFirst().map(HoodieLogFile::getLogVersion)
.orElse(HoodieLogFile.LOGFILE_BASE_VERSION - 1);
String logExtn = fileSliceForCompaction.getLogFiles().findFirst().map(lf -> "." + lf.getFileExtension())
.orElse(HoodieLogFile.DELTA_EXTENSION);
String parentPath = fileSliceForCompaction.getDataFile().map(df -> new Path(df.getPath()).getParent().toString())
@@ -471,16 +457,16 @@ public class CompactionAdminClient extends AbstractHoodieClient {
* Generate renaming actions for unscheduling a fileId from pending compaction. NOTE: Can only be used safely when no
* writer (ingestion/compaction) is running.
*
* @param metaClient Hoodie Table MetaClient
* @param fgId FileGroupId to remove compaction
* @param fsViewOpt Cached File System View
* @param metaClient Hoodie Table MetaClient
* @param fgId FileGroupId to remove compaction
* @param fsViewOpt Cached File System View
* @param skipValidation Skip Validation
* @return list of pairs of log-files (old, new) and for each pair, rename must be done to successfully unschedule
* compaction.
* compaction.
*/
public List<Pair<HoodieLogFile, HoodieLogFile>> getRenamingActionsForUnschedulingCompactionForFileId(
HoodieTableMetaClient metaClient, HoodieFileGroupId fgId,
Option<HoodieTableFileSystemView> fsViewOpt, boolean skipValidation) throws IOException {
HoodieTableMetaClient metaClient, HoodieFileGroupId fgId, Option<HoodieTableFileSystemView> fsViewOpt,
boolean skipValidation) throws IOException {
Map<HoodieFileGroupId, Pair<String, HoodieCompactionOperation>> allPendingCompactions =
CompactionUtils.getAllPendingCompactionOperations(metaClient);
if (allPendingCompactions.containsKey(fgId)) {
@@ -496,20 +482,19 @@ public class CompactionAdminClient extends AbstractHoodieClient {
*/
public static class RenameOpResult extends OperationResult<RenameInfo> {
public RenameOpResult() {
public RenameOpResult() {}
public RenameOpResult(Pair<HoodieLogFile, HoodieLogFile> op, boolean success, Option<Exception> exception) {
super(
new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(), op.getRight().getPath().toString()),
success, exception);
}
public RenameOpResult(Pair<HoodieLogFile, HoodieLogFile> op, boolean success,
public RenameOpResult(Pair<HoodieLogFile, HoodieLogFile> op, boolean executed, boolean success,
Option<Exception> exception) {
super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(),
op.getRight().getPath().toString()), success, exception);
}
public RenameOpResult(
Pair<HoodieLogFile, HoodieLogFile> op, boolean executed, boolean success,
Option<Exception> exception) {
super(new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(),
op.getRight().getPath().toString()), executed, success, exception);
super(
new RenameInfo(op.getKey().getFileId(), op.getKey().getPath().toString(), op.getRight().getPath().toString()),
executed, success, exception);
}
}
@@ -518,11 +503,9 @@ public class CompactionAdminClient extends AbstractHoodieClient {
*/
public static class ValidationOpResult extends OperationResult<CompactionOperation> {
public ValidationOpResult() {
}
public ValidationOpResult() {}
public ValidationOpResult(
CompactionOperation operation, boolean success, Option<Exception> exception) {
public ValidationOpResult(CompactionOperation operation, boolean success, Option<Exception> exception) {
super(operation, success, exception);
}
}
@@ -533,8 +516,7 @@ public class CompactionAdminClient extends AbstractHoodieClient {
public String srcPath;
public String destPath;
public RenameInfo() {
}
public RenameInfo() {}
public RenameInfo(String fileId, String srcPath, String destPath) {
this.fileId = fileId;

View File

@@ -58,9 +58,8 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
private static final Logger logger = LogManager.getLogger(HoodieReadClient.class);
/**
* TODO: We need to persist the index type into hoodie.properties and be able to access the index
* just with a simple basepath pointing to the dataset. Until, then just always assume a
* BloomIndex
* TODO: We need to persist the index type into hoodie.properties and be able to access the index just with a simple
* basepath pointing to the dataset. Until, then just always assume a BloomIndex
*/
private final transient HoodieIndex<T> index;
private final HoodieTimeline commitTimeline;
@@ -70,13 +69,11 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
/**
* @param basePath path to Hoodie dataset
*/
public HoodieReadClient(JavaSparkContext jsc, String basePath,
Option<EmbeddedTimelineService> timelineService) {
public HoodieReadClient(JavaSparkContext jsc, String basePath, Option<EmbeddedTimelineService> timelineService) {
this(jsc, HoodieWriteConfig.newBuilder().withPath(basePath)
// by default we use HoodieBloomIndex
.withIndexConfig(
HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.build(), timelineService);
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build(),
timelineService);
}
/**
@@ -130,8 +127,7 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
private void assertSqlContext() {
if (!sqlContextOpt.isPresent()) {
throw new IllegalStateException(
"SQLContext must be set, when performing dataframe operations");
throw new IllegalStateException("SQLContext must be set, when performing dataframe operations");
}
}
@@ -152,17 +148,16 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
*/
public Dataset<Row> readROView(JavaRDD<HoodieKey> hoodieKeys, int parallelism) {
assertSqlContext();
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD = index
.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
JavaPairRDD<HoodieKey, Option<String>> keyToFileRDD = lookupResultRDD
.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2)));
JavaPairRDD<HoodieKey, Option<Pair<String, String>>> lookupResultRDD =
index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
JavaPairRDD<HoodieKey, Option<String>> keyToFileRDD =
lookupResultRDD.mapToPair(r -> new Tuple2<>(r._1, convertToDataFilePath(r._2)));
List<String> paths = keyToFileRDD.filter(keyFileTuple -> keyFileTuple._2().isPresent())
.map(keyFileTuple -> keyFileTuple._2().get()).collect();
// record locations might be same for multiple keys, so need a unique list
Set<String> uniquePaths = new HashSet<>(paths);
Dataset<Row> originalDF = sqlContextOpt.get().read()
.parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
Dataset<Row> originalDF = sqlContextOpt.get().read().parquet(uniquePaths.toArray(new String[uniquePaths.size()]));
StructType schema = originalDF.schema();
JavaPairRDD<HoodieKey, Row> keyRowRDD = originalDF.javaRDD().mapToPair(row -> {
HoodieKey key = new HoodieKey(row.getAs(HoodieRecord.RECORD_KEY_METADATA_FIELD),
@@ -176,18 +171,16 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[FullFilePath]]
* If the optional FullFilePath value is not present, then the key is not found. If the
* FullFilePath value is present, it is the path component (without scheme) of the URI underlying
* file
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[FullFilePath]] If the optional
* FullFilePath value is not present, then the key is not found. If the FullFilePath value is present, it is the path
* component (without scheme) of the URI underlying file
*/
public JavaPairRDD<HoodieKey, Option<String>> checkExists(JavaRDD<HoodieKey> hoodieKeys) {
return index.fetchRecordLocation(hoodieKeys, jsc, hoodieTable);
}
/**
* Filter out HoodieRecords that already exists in the output folder. This is useful in
* deduplication.
* Filter out HoodieRecords that already exists in the output folder. This is useful in deduplication.
*
* @param hoodieRecords Input RDD of Hoodie records.
* @return A subset of hoodieRecords RDD, with existing records filtered out.
@@ -198,27 +191,27 @@ public class HoodieReadClient<T extends HoodieRecordPayload> extends AbstractHoo
}
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present). Input RDD should contain no duplicates if needed.
* Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually
* present). Input RDD should contain no duplicates if needed.
*
* @param hoodieRecords Input RDD of Hoodie records
* @return Tagged RDD of Hoodie records
*/
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords)
throws HoodieIndexException {
public JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> hoodieRecords) throws HoodieIndexException {
return index.tagLocation(hoodieRecords, jsc, hoodieTable);
}
/**
* Return all pending compactions with instant time for clients to decide what to compact next.
*
* @return
*/
public List<Pair<String, HoodieCompactionPlan>> getPendingCompactions() {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(jsc.hadoopConfiguration(),
hoodieTable.getMetaClient().getBasePath(), true);
HoodieTableMetaClient metaClient =
new HoodieTableMetaClient(jsc.hadoopConfiguration(), hoodieTable.getMetaClient().getBasePath(), true);
return CompactionUtils.getAllPendingCompactionPlans(metaClient).stream()
.map(instantWorkloadPair ->
Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue()))
.map(
instantWorkloadPair -> Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue()))
.collect(Collectors.toList());
}
}

View File

@@ -64,14 +64,11 @@ public class WriteStatus implements Serializable {
}
/**
* Mark write as success, optionally using given parameters for the purpose of calculating some
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver.
* Mark write as success, optionally using given parameters for the purpose of calculating some aggregate metrics.
* This method is not meant to cache passed arguments, since WriteStatus objects are collected in Spark Driver.
*
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
* it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link
* HoodieRecord} before deflation.
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link HoodieRecord} before deflation.
*/
public void markSuccess(HoodieRecord record, Option<Map<String, String>> optionalRecordMetadata) {
if (trackSuccessRecords) {
@@ -81,14 +78,11 @@ public class WriteStatus implements Serializable {
}
/**
* Mark write as failed, optionally using given parameters for the purpose of calculating some
* aggregate metrics. This method is not meant to cache passed arguments, since WriteStatus
* objects are collected in Spark Driver.
* Mark write as failed, optionally using given parameters for the purpose of calculating some aggregate metrics. This
* method is not meant to cache passed arguments, since WriteStatus objects are collected in Spark Driver.
*
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies
* it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link
* HoodieRecord} before deflation.
* @param record deflated {@code HoodieRecord} containing information that uniquely identifies it.
* @param optionalRecordMetadata optional metadata related to data contained in {@link HoodieRecord} before deflation.
*/
public void markFailure(HoodieRecord record, Throwable t, Option<Map<String, String>> optionalRecordMetadata) {
if (failedRecords.isEmpty() || (random.nextDouble() <= failureFraction)) {

View File

@@ -40,10 +40,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
// Turn on inline compaction - after fw delta commits a inline compaction will be run
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
// Run a compaction every N delta commits
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP =
"hoodie.compact.inline.max" + ".delta.commits";
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP =
"hoodie.cleaner.fileversions" + ".retained";
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max" + ".delta.commits";
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = "hoodie.cleaner.fileversions" + ".retained";
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
public static final String MAX_COMMITS_TO_KEEP_PROP = "hoodie.keep.max.commits";
public static final String MIN_COMMITS_TO_KEEP_PROP = "hoodie.keep.min.commits";
@@ -56,25 +54,21 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
* Configs related to specific table types
**/
// Number of inserts, that will be put each partition/bucket for writing
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE =
"hoodie.copyonwrite.insert" + ".split.size";
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert" + ".split.size";
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
// Config to control whether we control insert split sizes automatically based on average
// record sizes
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS =
"hoodie.copyonwrite.insert" + ".auto.split";
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert" + ".auto.split";
// its off by default
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(true);
// This value is used as a guessimate for the record size, if we can't determine this from
// previous commits
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE =
"hoodie.copyonwrite" + ".record.size.estimate";
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite" + ".record.size.estimate";
// Used to determine how much more can be packed into a small file, before it exceeds the size
// limit.
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String
.valueOf(1024);
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String.valueOf(1024);
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
@@ -82,8 +76,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
// 200GB of target IO per compaction
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class
.getName();
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class.getName();
// used to merge records written to log file
public static final String DEFAULT_PAYLOAD_CLASS = HoodieAvroPayload.class.getName();
public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
@@ -91,15 +84,12 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
// used to choose a trade off between IO vs Memory when performing compaction process
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file
// size + small memory
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP =
"hoodie.compaction.lazy" + ".block.read";
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy" + ".block.read";
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
// used to choose whether to enable reverse log reading (reverse log traversal)
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP =
"hoodie.compaction" + ".reverse.log.read";
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction" + ".reverse.log.read";
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS
.name();
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
private static final String DEFAULT_AUTO_CLEAN = "true";
private static final String DEFAULT_INLINE_COMPACT = "false";
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "1";
@@ -108,8 +98,8 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = "30";
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = "20";
private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = String.valueOf(10);
public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = "hoodie.compaction.daybased.target"
+ ".partitions";
public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP =
"hoodie.compaction.daybased.target" + ".partitions";
// 500GB of target IO per compaction (both read and write)
public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10);
@@ -188,14 +178,12 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
}
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS,
String.valueOf(autoTuneInsertSplits));
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
return this;
}
public Builder approxRecordSize(int recordSizeEstimate) {
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
String.valueOf(recordSizeEstimate));
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
return this;
}
@@ -215,32 +203,27 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
}
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP,
String.valueOf(targetIOPerCompactionInMB));
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
return this;
}
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP,
String.valueOf(maxNumDeltaCommitsBeforeCompaction));
props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
return this;
}
public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP,
String.valueOf(compactionLazyBlockReadEnabled));
props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, String.valueOf(compactionLazyBlockReadEnabled));
return this;
}
public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP,
String.valueOf(compactionReverseLogReadEnabled));
props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, String.valueOf(compactionReverseLogReadEnabled));
return this;
}
public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPerCompaction) {
props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP,
String.valueOf(targetPartitionsPerCompaction));
props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, String.valueOf(targetPartitionsPerCompaction));
return this;
}
@@ -251,8 +234,7 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
public HoodieCompactionConfig build() {
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP,
DEFAULT_AUTO_CLEAN);
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
DEFAULT_INLINE_COMPACT);
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
@@ -261,27 +243,25 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
DEFAULT_CLEANER_POLICY);
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP),
CLEANER_COMMITS_RETAINED_PROP, DEFAULT_CLEANER_COMMITS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP,
DEFAULT_CLEANER_COMMITS_RETAINED);
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
DEFAULT_MAX_COMMITS_TO_KEEP);
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
DEFAULT_MIN_COMMITS_TO_KEEP);
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES),
PARQUET_SMALL_FILE_LIMIT_BYTES, DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES), PARQUET_SMALL_FILE_LIMIT_BYTES,
DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE,
DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM,
DEFAULT_CLEANER_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP),
COMPACTION_STRATEGY_PROP, DEFAULT_COMPACTION_STRATEGY);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP),
PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP), COMPACTION_STRATEGY_PROP,
DEFAULT_COMPACTION_STRATEGY);
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP), PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS);
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
@@ -299,13 +279,15 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
// commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull
int minInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
int maxInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
int cleanerCommitsRetained = Integer
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
int cleanerCommitsRetained =
Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
Preconditions.checkArgument(maxInstantsToKeep > minInstantsToKeep);
Preconditions.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
String.format("Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
+ "missing data from few instants.", HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP,
minInstantsToKeep, HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, cleanerCommitsRetained));
String.format(
"Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
+ "missing data from few instants.",
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, minInstantsToKeep,
HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, cleanerCommitsRetained));
return config;
}
}

View File

@@ -32,8 +32,8 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
/**
* Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not
* be honored for HBase Puts
* Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not be honored for HBase
* Puts
*/
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
@@ -48,18 +48,16 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = "hoodie.index.hbase.put.batch.size.autocompute";
public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = "false";
/**
* Property to set the fraction of the global share of QPS that should be allocated to this job.
* Let's say there are 3 jobs which have input size in terms of number of rows required for
* HbaseIndexing as x, 2x, 3x respectively. Then this fraction for the jobs would be (0.17) 1/6,
* 0.33 (2/6) and 0.5 (3/6) respectively.
* Property to set the fraction of the global share of QPS that should be allocated to this job. Let's say there are 3
* jobs which have input size in terms of number of rows required for HbaseIndexing as x, 2x, 3x respectively. Then
* this fraction for the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
*/
public static final String HBASE_QPS_FRACTION_PROP = "hoodie.index.hbase.qps.fraction";
/**
* Property to set maximum QPS allowed per Region Server. This should be same across various
* jobs. This is intended to limit the aggregate QPS generated across various jobs to an Hbase
* Region Server. It is recommended to set this value based on global indexing throughput needs
* and most importantly, how much the HBase installation in use is able to tolerate without
* Region Servers going down.
* Property to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to
* limit the aggregate QPS generated across various jobs to an Hbase Region Server. It is recommended to set this
* value based on global indexing throughput needs and most importantly, how much the HBase installation in use is
* able to tolerate without Region Servers going down.
*/
public static String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = "hoodie.index.hbase.max.qps.per.region.server";
/**
@@ -71,18 +69,17 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
*/
public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = 1000;
/**
* Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming
* Region Servers
* Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming Region Servers
*/
public static final float DEFAULT_HBASE_QPS_FRACTION = 0.5f;
/**
* Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume
* Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume
*/
public static final String HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = "hoodie.index.hbase.dynamic_qps";
public static final boolean DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = false;
/**
* Min and Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads
* Min and Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads
*/
public static final String HBASE_MIN_QPS_FRACTION_PROP = "hoodie.index.hbase.min.qps.fraction";
public static final String DEFAULT_HBASE_MIN_QPS_FRACTION_PROP = "0.002";
@@ -90,7 +87,7 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
public static final String HBASE_MAX_QPS_FRACTION_PROP = "hoodie.index.hbase.max.qps.fraction";
public static final String DEFAULT_HBASE_MAX_QPS_FRACTION_PROP = "0.06";
/**
* Hoodie index desired puts operation time in seconds
* Hoodie index desired puts operation time in seconds
*/
public static final String HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = "hoodie.index.hbase.desired_puts_time_in_secs";
public static final int DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = 600;
@@ -105,7 +102,7 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
public static final String DEFAULT_HBASE_ZK_PATH_QPS_ROOT = "/QPS_ROOT";
public HoodieHBaseIndexConfig(final Properties props) {
super(props);
super(props);
}
public static HoodieHBaseIndexConfig.Builder newBuilder() {
@@ -218,18 +215,15 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
/**
* <p>
* Method to set maximum QPS allowed per Region Server. This should be same across various
* jobs. This is intended to limit the aggregate QPS generated across various jobs to an
* Hbase Region Server.
* Method to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to
* limit the aggregate QPS generated across various jobs to an Hbase Region Server.
* </p>
* <p>
* It is recommended to set this value based on your global indexing throughput needs and
* most importantly, how much your HBase installation is able to tolerate without Region
* Servers going down.
* It is recommended to set this value based on your global indexing throughput needs and most importantly, how much
* your HBase installation is able to tolerate without Region Servers going down.
* </p>
*/
public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(
int maxQPSPerRegionServer) {
public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(int maxQPSPerRegionServer) {
// This should be same across various jobs
props.setProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
String.valueOf(maxQPSPerRegionServer));
@@ -238,30 +232,30 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
public HoodieHBaseIndexConfig build() {
HoodieHBaseIndexConfig config = new HoodieHBaseIndexConfig(props);
setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP),
HBASE_GET_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP),
HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP), HBASE_GET_BATCH_SIZE_PROP,
String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), HBASE_PUT_BATCH_SIZE_PROP,
String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP),
HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE));
setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP),
HBASE_QPS_FRACTION_PROP, String.valueOf(DEFAULT_HBASE_QPS_FRACTION));
setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP), HBASE_QPS_FRACTION_PROP,
String.valueOf(DEFAULT_HBASE_QPS_FRACTION));
setDefaultOnCondition(props, !props.containsKey(HBASE_MAX_QPS_PER_REGION_SERVER_PROP),
HBASE_MAX_QPS_PER_REGION_SERVER_PROP, String.valueOf(DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER));
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY),
HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS),
HBASE_INDEX_QPS_ALLOCATOR_CLASS, String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS), HBASE_INDEX_QPS_ALLOCATOR_CLASS,
String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS),
HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
setDefaultOnCondition(props, !props.containsKey(HBASE_ZK_PATH_QPS_ROOT),
HBASE_ZK_PATH_QPS_ROOT, String.valueOf(DEFAULT_HBASE_ZK_PATH_QPS_ROOT));
setDefaultOnCondition(props, !props.containsKey(HBASE_ZK_PATH_QPS_ROOT), HBASE_ZK_PATH_QPS_ROOT,
String.valueOf(DEFAULT_HBASE_ZK_PATH_QPS_ROOT));
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS),
HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS),
HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS),
HBASE_INDEX_QPS_ALLOCATOR_CLASS, String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS), HBASE_INDEX_QPS_ALLOCATOR_CLASS,
String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
return config;
}

View File

@@ -34,7 +34,7 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
public static final String DEFAULT_INDEX_TYPE = HoodieIndex.IndexType.BLOOM.name();
// ***** Bloom Index configs *****
// ***** Bloom Index configs *****
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
@@ -42,8 +42,7 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP =
"hoodie.bloom.index.prune.by" + ".ranges";
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by" + ".ranges";
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
@@ -67,8 +66,7 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL =
"hoodie.bloom.index.input.storage" + ".level";
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage" + ".level";
public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
private HoodieIndexConfig(Properties props) {
@@ -175,20 +173,18 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
public HoodieIndexConfig build() {
HoodieIndexConfig config = new HoodieIndexConfig(props);
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP,
DEFAULT_INDEX_TYPE);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES),
BLOOM_FILTER_NUM_ENTRIES, DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP,
DEFAULT_BLOOM_FILTER_FPP);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP),
BLOOM_INDEX_PARALLELISM_PROP, DEFAULT_BLOOM_INDEX_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, DEFAULT_INDEX_TYPE);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES), BLOOM_FILTER_NUM_ENTRIES,
DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP), BLOOM_INDEX_PARALLELISM_PROP,
DEFAULT_BLOOM_INDEX_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP),
BLOOM_INDEX_USE_CACHING_PROP, DEFAULT_BLOOM_INDEX_USE_CACHING);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL),
BLOOM_INDEX_INPUT_STORAGE_LEVEL, DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP), BLOOM_INDEX_USE_CACHING_PROP,
DEFAULT_BLOOM_INDEX_USE_CACHING);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL), BLOOM_INDEX_INPUT_STORAGE_LEVEL,
DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_TREE_BASED_FILTER_PROP),
BLOOM_INDEX_TREE_BASED_FILTER_PROP, DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER);
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP),

View File

@@ -41,8 +41,7 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
// Default max memory fraction during compaction, excess spills to disk
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6);
// Default memory size per compaction (used if SparkEnv is absent), excess spills to disk
public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES =
1024 * 1024 * 1024L; // 1GB
public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = 1024 * 1024 * 1024L; // 1GB
// Property to set the max memory for merge
public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size";
// Property to set the max memory for compaction
@@ -88,20 +87,17 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
}
public Builder withMaxMemoryFractionPerPartitionMerge(double maxMemoryFractionPerPartitionMerge) {
props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP,
String.valueOf(maxMemoryFractionPerPartitionMerge));
props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, String.valueOf(maxMemoryFractionPerPartitionMerge));
return this;
}
public Builder withMaxMemoryFractionPerCompaction(double maxMemoryFractionPerCompaction) {
props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
String.valueOf(maxMemoryFractionPerCompaction));
props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, String.valueOf(maxMemoryFractionPerCompaction));
return this;
}
public Builder withMaxDFSStreamBufferSize(int maxStreamBufferSize) {
props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP,
String.valueOf(maxStreamBufferSize));
props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(maxStreamBufferSize));
return this;
}
@@ -130,19 +126,16 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
if (SparkEnv.get() != null) {
// 1 GB is the default conf used by Spark, look at SparkContext.scala
long executorMemoryInBytes = Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
* 1024L;
long executorMemoryInBytes = Utils.memoryStringToMb(
SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024 * 1024L;
// 0.6 is the default value used by Spark,
// look at {@link
// https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkConf.scala#L507}
double memoryFraction = Double
.valueOf(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP,
DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
double memoryFraction = Double.valueOf(
SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_FRACTION_PROP, DEFAULT_SPARK_EXECUTOR_MEMORY_FRACTION));
double maxMemoryFractionForMerge = Double.valueOf(maxMemoryFraction);
double userAvailableMemory = executorMemoryInBytes * (1 - memoryFraction);
long maxMemoryForMerge = (long) Math
.floor(userAvailableMemory * maxMemoryFractionForMerge);
long maxMemoryForMerge = (long) Math.floor(userAvailableMemory * maxMemoryFractionForMerge);
return maxMemoryForMerge;
} else {
return DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
@@ -151,29 +144,19 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
public HoodieMemoryConfig build() {
HoodieMemoryConfig config = new HoodieMemoryConfig(props);
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP),
MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP,
DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FRACTION_FOR_MERGE_PROP),
setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP),
MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FRACTION_FOR_MERGE_PROP),
MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FOR_MERGE_PROP),
MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP))));
setDefaultOnCondition(props,
!props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP),
MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(
getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP))));
setDefaultOnCondition(props,
!props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP),
MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
setDefaultOnCondition(props,
!props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP),
SPILLABLE_MAP_BASE_PATH_PROP, DEFAULT_SPILLABLE_MAP_BASE_PATH);
setDefaultOnCondition(props,
!props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FOR_MERGE_PROP), MAX_MEMORY_FOR_MERGE_PROP,
String.valueOf(getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP))));
setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP), MAX_MEMORY_FOR_COMPACTION_PROP,
String.valueOf(getMaxMemoryAllowedForMerge(props.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP))));
setDefaultOnCondition(props, !props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP), MAX_DFS_STREAM_BUFFER_SIZE_PROP,
String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
setDefaultOnCondition(props, !props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP), SPILLABLE_MAP_BASE_PATH_PROP,
DEFAULT_SPILLABLE_MAP_BASE_PATH);
setDefaultOnCondition(props, !props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
return config;
}

View File

@@ -35,8 +35,7 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig {
public static final String METRICS_ON = METRIC_PREFIX + ".on";
public static final boolean DEFAULT_METRICS_ON = false;
public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType
.GRAPHITE;
public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType.GRAPHITE;
// Graphite
public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
@@ -103,8 +102,7 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig {
public HoodieMetricsConfig build() {
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON,
String.valueOf(DEFAULT_METRICS_ON));
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON));
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE,
DEFAULT_METRICS_REPORTER_TYPE.name());
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST,

View File

@@ -38,8 +38,7 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
// used to size log files
public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String
.valueOf(1024 * 1024 * 1024); // 1 GB
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024 * 1024 * 1024); // 1 GB
// used to size data blocks in log file
public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size";
public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB
@@ -122,20 +121,20 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
public HoodieStorageConfig build() {
HoodieStorageConfig config = new HoodieStorageConfig(props);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES),
PARQUET_FILE_MAX_BYTES, DEFAULT_PARQUET_FILE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES),
PARQUET_BLOCK_SIZE_BYTES, DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES),
PARQUET_PAGE_SIZE_BYTES, DEFAULT_PARQUET_PAGE_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), PARQUET_FILE_MAX_BYTES,
DEFAULT_PARQUET_FILE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES), PARQUET_BLOCK_SIZE_BYTES,
DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES), PARQUET_PAGE_SIZE_BYTES,
DEFAULT_PARQUET_PAGE_SIZE_BYTES);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES),
LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES),
LOGFILE_SIZE_MAX_BYTES, DEFAULT_LOGFILE_SIZE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO),
PARQUET_COMPRESSION_RATIO, DEFAULT_STREAM_COMPRESSION_RATIO);
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_CODEC),
PARQUET_COMPRESSION_CODEC, DEFAULT_PARQUET_COMPRESSION_CODEC);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES), LOGFILE_SIZE_MAX_BYTES,
DEFAULT_LOGFILE_SIZE_MAX_BYTES);
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO), PARQUET_COMPRESSION_RATIO,
DEFAULT_STREAM_COMPRESSION_RATIO);
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_CODEC), PARQUET_COMPRESSION_CODEC,
DEFAULT_PARQUET_COMPRESSION_CODEC);
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
return config;

View File

@@ -61,8 +61,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
private static final String DEFAULT_WRITE_STATUS_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
private static final String HOODIE_AUTO_COMMIT_PROP = "hoodie.auto.commit";
private static final String DEFAULT_HOODIE_AUTO_COMMIT = "true";
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP =
"hoodie.assume.date" + ".partitioning";
private static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date" + ".partitioning";
private static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
private static final String HOODIE_WRITE_STATUS_CLASS_PROP = "hoodie.writestatus.class";
private static final String DEFAULT_HOODIE_WRITE_STATUS_CLASS = WriteStatus.class.getName();
@@ -143,8 +142,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getWriteBufferLimitBytes() {
return Integer
.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
return Integer.parseInt(props.getProperty(WRITE_BUFFER_LIMIT_BYTES, DEFAULT_WRITE_BUFFER_LIMIT_BYTES));
}
public boolean shouldCombineBeforeInsert() {
@@ -191,18 +189,15 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
* compaction properties
**/
public HoodieCleaningPolicy getCleanerPolicy() {
return HoodieCleaningPolicy
.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
return HoodieCleaningPolicy.valueOf(props.getProperty(HoodieCompactionConfig.CLEANER_POLICY_PROP));
}
public int getCleanerFileVersionsRetained() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_FILE_VERSIONS_RETAINED_PROP));
}
public int getCleanerCommitsRetained() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
}
public int getMaxCommitsToKeep() {
@@ -214,23 +209,19 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getParquetSmallFileLimit() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.PARQUET_SMALL_FILE_LIMIT_BYTES));
}
public int getCopyOnWriteInsertSplitSize() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE));
}
public int getCopyOnWriteRecordSizeEstimate() {
return Integer.parseInt(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE));
}
public boolean shouldAutoTuneInsertSplits() {
return Boolean.parseBoolean(
props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
return Boolean.parseBoolean(props.getProperty(HoodieCompactionConfig.COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS));
}
public int getCleanerParallelism() {
@@ -246,28 +237,23 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getInlineCompactDeltaCommitMax() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP));
}
public CompactionStrategy getCompactionStrategy() {
return ReflectionUtils
.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
return ReflectionUtils.loadClass(props.getProperty(HoodieCompactionConfig.COMPACTION_STRATEGY_PROP));
}
public Long getTargetIOPerCompactionInMB() {
return Long
.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
return Long.parseLong(props.getProperty(HoodieCompactionConfig.TARGET_IO_PER_COMPACTION_IN_MB_PROP));
}
public Boolean getCompactionLazyBlockReadEnabled() {
return Boolean
.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP));
}
public Boolean getCompactionReverseLogReadEnabled() {
return Boolean.valueOf(
props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
return Boolean.valueOf(props.getProperty(HoodieCompactionConfig.COMPACTION_REVERSE_LOG_READ_ENABLED_PROP));
}
public String getPayloadClass() {
@@ -275,13 +261,11 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getTargetPartitionsPerDayBasedCompaction() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP));
}
public int getCommitArchivalBatchSize() {
return Integer
.parseInt(props.getProperty(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE_PROP));
return Integer.parseInt(props.getProperty(HoodieCompactionConfig.COMMITS_ARCHIVAL_BATCH_SIZE_PROP));
}
/**
@@ -352,9 +336,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
/**
* Fraction of the global share of QPS that should be allocated to this job.
* Let's say there are 3 jobs which have input size in terms of number of rows
* required for HbaseIndexing as x, 2x, 3x respectively. Then this fraction for
* Fraction of the global share of QPS that should be allocated to this job. Let's say there are 3 jobs which have
* input size in terms of number of rows required for HbaseIndexing as x, 2x, 3x respectively. Then this fraction for
* the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
*/
public float getHbaseIndexQPSFraction() {
@@ -370,8 +353,8 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
/**
* This should be same across various jobs. This is intended to limit the aggregate
* QPS generated across various Hoodie jobs to an Hbase Region Server
* This should be same across various jobs. This is intended to limit the aggregate QPS generated across various
* Hoodie jobs to an Hbase Region Server
*/
public int getHbaseIndexMaxQPSPerRegionServer() {
return Integer.parseInt(props.getProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP));
@@ -382,8 +365,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public boolean getBloomIndexPruneByRanges() {
return Boolean
.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
return Boolean.parseBoolean(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_PRUNE_BY_RANGES_PROP));
}
public boolean getBloomIndexUseCaching() {
@@ -403,8 +385,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public StorageLevel getBloomIndexInputStorageLevel() {
return StorageLevel
.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
return StorageLevel.fromString(props.getProperty(HoodieIndexConfig.BLOOM_INDEX_INPUT_STORAGE_LEVEL));
}
/**
@@ -423,8 +404,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public int getLogFileDataBlockMaxSize() {
return Integer
.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
return Integer.parseInt(props.getProperty(HoodieStorageConfig.LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES));
}
public int getLogFileMaxSize() {
@@ -451,8 +431,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public MetricsReporterType getMetricsReporterType() {
return MetricsReporterType
.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
return MetricsReporterType.valueOf(props.getProperty(HoodieMetricsConfig.METRICS_REPORTER_TYPE));
}
public String getGraphiteServerHost() {
@@ -475,9 +454,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public Double getMaxMemoryFractionPerCompaction() {
return Double
.valueOf(
props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP));
return Double.valueOf(props.getProperty(HoodieMemoryConfig.MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP));
}
public Long getMaxMemoryPerPartitionMerge() {
@@ -637,8 +614,7 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
}
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP,
String.valueOf(assumeDatePartitioning));
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
return this;
}
@@ -671,48 +647,42 @@ public class HoodieWriteConfig extends DefaultHoodieConfig {
public HoodieWriteConfig build() {
// Check for mandatory properties
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM,
setDefaultOnCondition(props, !props.containsKey(INSERT_PARALLELISM), INSERT_PARALLELISM, DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM), BULKINSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(BULKINSERT_PARALLELISM),
BULKINSERT_PARALLELISM, DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM,
DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP),
COMBINE_BEFORE_INSERT_PROP, DEFAULT_COMBINE_BEFORE_INSERT);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP),
COMBINE_BEFORE_UPSERT_PROP, DEFAULT_COMBINE_BEFORE_UPSERT);
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL),
WRITE_STATUS_STORAGE_LEVEL, DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP),
HOODIE_AUTO_COMMIT_PROP, DEFAULT_HOODIE_AUTO_COMMIT);
setDefaultOnCondition(props, !props.containsKey(UPSERT_PARALLELISM), UPSERT_PARALLELISM, DEFAULT_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_INSERT_PROP), COMBINE_BEFORE_INSERT_PROP,
DEFAULT_COMBINE_BEFORE_INSERT);
setDefaultOnCondition(props, !props.containsKey(COMBINE_BEFORE_UPSERT_PROP), COMBINE_BEFORE_UPSERT_PROP,
DEFAULT_COMBINE_BEFORE_UPSERT);
setDefaultOnCondition(props, !props.containsKey(WRITE_STATUS_STORAGE_LEVEL), WRITE_STATUS_STORAGE_LEVEL,
DEFAULT_WRITE_STATUS_STORAGE_LEVEL);
setDefaultOnCondition(props, !props.containsKey(HOODIE_AUTO_COMMIT_PROP), HOODIE_AUTO_COMMIT_PROP,
DEFAULT_HOODIE_AUTO_COMMIT);
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP),
HOODIE_WRITE_STATUS_CLASS_PROP, DEFAULT_HOODIE_WRITE_STATUS_CLASS);
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM),
FINALIZE_WRITE_PARALLELISM, DEFAULT_FINALIZE_WRITE_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(HOODIE_WRITE_STATUS_CLASS_PROP), HOODIE_WRITE_STATUS_CLASS_PROP,
DEFAULT_HOODIE_WRITE_STATUS_CLASS);
setDefaultOnCondition(props, !props.containsKey(FINALIZE_WRITE_PARALLELISM), FINALIZE_WRITE_PARALLELISM,
DEFAULT_FINALIZE_WRITE_PARALLELISM);
setDefaultOnCondition(props, !props.containsKey(EMBEDDED_TIMELINE_SERVER_ENABLED),
EMBEDDED_TIMELINE_SERVER_ENABLED, DEFAULT_EMBEDDED_TIMELINE_SERVER_ENABLED);
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP),
MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP), MAX_CONSISTENCY_CHECKS_PROP,
String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
setDefaultOnCondition(props, !props.containsKey(FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP),
FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP, DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
FAIL_ON_TIMELINE_ARCHIVING_ENABLED_PROP, DEFAULT_FAIL_ON_TIMELINE_ARCHIVING_ENABLED);
// Make sure the props is propagated
setDefaultOnCondition(props, !isIndexConfigSet,
HoodieIndexConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isStorageConfigSet,
HoodieStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isIndexConfigSet, HoodieIndexConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isStorageConfigSet, HoodieStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isCompactionConfigSet,
HoodieCompactionConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMetricsConfigSet,
HoodieMetricsConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMemoryConfigSet,
HoodieMemoryConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMetricsConfigSet, HoodieMetricsConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isMemoryConfigSet, HoodieMemoryConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isViewConfigSet,
FileSystemViewStorageConfig.newBuilder().fromProperties(props).build());
setDefaultOnCondition(props, !isConsistencyGuardSet,

View File

@@ -19,8 +19,9 @@
package org.apache.hudi.exception;
/**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta
* commit </p>
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a delta commit
* </p>
*/
public class HoodieAppendException extends HoodieException {

View File

@@ -19,7 +19,8 @@
package org.apache.hudi.exception;
/**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a Commit
* </p>
*/
public class HoodieCommitException extends HoodieException {

View File

@@ -20,7 +20,9 @@ package org.apache.hudi.exception;
/**
* <p> Exception thrown when dependent system is not available </p>
* <p>
* Exception thrown when dependent system is not available
* </p>
*/
public class HoodieDependentSystemUnavailableException extends HoodieException {

View File

@@ -19,8 +19,9 @@
package org.apache.hudi.exception;
/**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk
* insert </p>
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a bulk insert
* </p>
*/
public class HoodieInsertException extends HoodieException {

View File

@@ -19,8 +19,9 @@
package org.apache.hudi.exception;
/**
* <p> Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a
* incremental upsert </p>
* <p>
* Exception thrown for any higher level errors when <code>HoodieClient</code> is doing a incremental upsert
* </p>
*/
public class HoodieUpsertException extends HoodieException {

View File

@@ -31,16 +31,16 @@ import org.apache.spark.api.java.function.Function2;
/**
* Map function that handles a sorted stream of HoodieRecords
*/
public class BulkInsertMapFunction<T extends HoodieRecordPayload> implements
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
public class BulkInsertMapFunction<T extends HoodieRecordPayload>
implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<List<WriteStatus>>> {
private String commitTime;
private HoodieWriteConfig config;
private HoodieTable<T> hoodieTable;
private List<String> fileIDPrefixes;
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config,
HoodieTable<T> hoodieTable, List<String> fileIDPrefixes) {
public BulkInsertMapFunction(String commitTime, HoodieWriteConfig config, HoodieTable<T> hoodieTable,
List<String> fileIDPrefixes) {
this.commitTime = commitTime;
this.config = config;
this.hoodieTable = hoodieTable;

View File

@@ -37,11 +37,10 @@ import org.apache.hudi.io.HoodieWriteHandle;
import org.apache.hudi.table.HoodieTable;
/**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* files.
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new files.
*/
public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extends
LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload>
extends LazyIterableIterator<HoodieRecord<T>, List<WriteStatus>> {
protected final HoodieWriteConfig hoodieConfig;
protected final String commitTime;
@@ -80,25 +79,23 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
* Transformer function to help transform a HoodieRecord. This transformer is used by BufferedIterator to offload some
* expensive operations of transformation to the reader thread.
*/
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>,
HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(Schema schema) {
static <T extends HoodieRecordPayload> Function<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>> getTransformFunction(
Schema schema) {
return hoodieRecord -> new HoodieInsertValueGenResult(hoodieRecord, schema);
}
@Override
protected void start() {
}
protected void start() {}
@Override
protected List<WriteStatus> computeNext() {
// Executor service used for launching writer thread.
BoundedInMemoryExecutor<HoodieRecord<T>,
HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor = null;
BoundedInMemoryExecutor<HoodieRecord<T>, HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> bufferedIteratorExecutor =
null;
try {
final Schema schema = new Schema.Parser().parse(hoodieConfig.getSchema());
bufferedIteratorExecutor =
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr,
getInsertHandler(), getTransformFunction(schema));
new SparkBoundedInMemoryExecutor<>(hoodieConfig, inputItr, getInsertHandler(), getTransformFunction(schema));
final List<WriteStatus> result = bufferedIteratorExecutor.execute();
assert result != null && !result.isEmpty() && !bufferedIteratorExecutor.isRemaining();
return result;
@@ -112,8 +109,7 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
}
@Override
protected void end() {
}
protected void end() {}
protected String getNextFileId(String idPfx) {
return String.format("%s-%d", idPfx, numFilesWritten++);
@@ -124,11 +120,10 @@ public class CopyOnWriteLazyInsertIterable<T extends HoodieRecordPayload> extend
}
/**
* Consumes stream of hoodie records from in-memory queue and
* writes to one or more create-handles
* Consumes stream of hoodie records from in-memory queue and writes to one or more create-handles
*/
protected class CopyOnWriteInsertHandler extends
BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
protected class CopyOnWriteInsertHandler
extends BoundedInMemoryQueueConsumer<HoodieInsertValueGenResult<HoodieRecord>, List<WriteStatus>> {
protected final List<WriteStatus> statuses = new ArrayList<>();
protected HoodieWriteHandle handle;

View File

@@ -21,16 +21,15 @@ package org.apache.hudi.func;
import java.util.Iterator;
/**
* (NOTE: Adapted from Apache SystemML) This class is a generic base class for lazy, single pass
* inputItr classes in order to simplify the implementation of lazy iterators for mapPartitions use
* cases. Note [SPARK-3369], which gives the reasons for backwards compatibility with regard to the
* iterable API despite Spark's single pass nature.
* (NOTE: Adapted from Apache SystemML) This class is a generic base class for lazy, single pass inputItr classes in
* order to simplify the implementation of lazy iterators for mapPartitions use cases. Note [SPARK-3369], which gives
* the reasons for backwards compatibility with regard to the iterable API despite Spark's single pass nature.
* <p>
* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input)
* Provide a way to obtain a inputItr of type O (output), out of an inputItr of type I (input)
* <p>
* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next()
* to obtain them - Assumes hasNext() gets called atleast once. - Concrete Implementation is
* responsible for calling inputIterator.next() and doing the processing in computeNext()
* Things to remember: - Assumes Spark calls hasNext() to check for elements, before calling next() to obtain them -
* Assumes hasNext() gets called atleast once. - Concrete Implementation is responsible for calling inputIterator.next()
* and doing the processing in computeNext()
*/
public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterator<O> {
@@ -88,13 +87,13 @@ public abstract class LazyIterableIterator<I, O> implements Iterable<O>, Iterato
@Override
public Iterator<O> iterator() {
//check for consumed inputItr
// check for consumed inputItr
if (consumed) {
throw new RuntimeException("Invalid repeated inputItr consumption.");
}
//hand out self as inputItr exactly once (note: do not hand out the input
//inputItr since it is consumed by the self inputItr implementation)
// hand out self as inputItr exactly once (note: do not hand out the input
// inputItr since it is consumed by the self inputItr implementation)
consumed = true;
return this;
}

View File

@@ -29,11 +29,9 @@ import org.apache.hudi.io.HoodieAppendHandle;
import org.apache.hudi.table.HoodieTable;
/**
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new
* log files.
* Lazy Iterable, that writes a stream of HoodieRecords sorted by the partitionPath, into new log files.
*/
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends
CopyOnWriteLazyInsertIterable<T> {
public class MergeOnReadLazyInsertIterable<T extends HoodieRecordPayload> extends CopyOnWriteLazyInsertIterable<T> {
public MergeOnReadLazyInsertIterable(Iterator<HoodieRecord<T>> sortedRecordItr, HoodieWriteConfig config,
String commitTime, HoodieTable<T> hoodieTable, String idPfx) {

View File

@@ -32,8 +32,7 @@ public class OperationResult<T> implements Serializable {
private boolean success;
private Option<Exception> exception;
public OperationResult() {
}
public OperationResult() {}
public OperationResult(T operation, boolean success, Option<Exception> exception) {
this.operation = operation;
@@ -67,11 +66,7 @@ public class OperationResult<T> implements Serializable {
@Override
public String toString() {
return "OperationResult{"
+ "operation=" + operation
+ ", executed=" + executed
+ ", success=" + success
+ ", exception=" + exception
+ '}';
return "OperationResult{" + "operation=" + operation + ", executed=" + executed + ", success=" + success
+ ", exception=" + exception + '}';
}
}

View File

@@ -25,8 +25,8 @@ import org.apache.hudi.exception.HoodieIOException;
import org.apache.parquet.hadoop.ParquetReader;
/**
* This class wraps a parquet reader and provides an iterator based api to
* read from a parquet file. This is used in {@link BoundedInMemoryQueue}
* This class wraps a parquet reader and provides an iterator based api to read from a parquet file. This is used in
* {@link BoundedInMemoryQueue}
*/
public class ParquetReaderIterator<T> implements Iterator<T> {

View File

@@ -36,17 +36,13 @@ public class SparkBoundedInMemoryExecutor<I, O, E> extends BoundedInMemoryExecut
final TaskContext sparkThreadTaskContext;
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, final Iterator<I> inputItr,
BoundedInMemoryQueueConsumer<O, E> consumer,
Function<I, O> bufferedIteratorTransform) {
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> bufferedIteratorTransform) {
this(hoodieConfig, new IteratorBasedQueueProducer<>(inputItr), consumer, bufferedIteratorTransform);
}
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig,
BoundedInMemoryQueueProducer<I> producer,
BoundedInMemoryQueueConsumer<O, E> consumer,
Function<I, O> bufferedIteratorTransform) {
super(hoodieConfig.getWriteBufferLimitBytes(), producer,
Option.of(consumer), bufferedIteratorTransform);
public SparkBoundedInMemoryExecutor(final HoodieWriteConfig hoodieConfig, BoundedInMemoryQueueProducer<I> producer,
BoundedInMemoryQueueConsumer<O, E> consumer, Function<I, O> bufferedIteratorTransform) {
super(hoodieConfig.getWriteBufferLimitBytes(), producer, Option.of(consumer), bufferedIteratorTransform);
this.sparkThreadTaskContext = TaskContext.get();
}

View File

@@ -65,18 +65,18 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
}
/**
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[partitionPath, fileID]]
* If the optional is empty, then the key is not found.
* Checks if the given [Keys] exists in the hoodie table and returns [Key, Option[partitionPath, fileID]] If the
* optional is empty, then the key is not found.
*/
public abstract JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(
JavaRDD<HoodieKey> hoodieKeys, final JavaSparkContext jsc, HoodieTable<T> hoodieTable);
/**
* Looks up the index and tags each incoming record with a location of a file that contains the
* row (if it is actually present)
* Looks up the index and tags each incoming record with a location of a file that contains the row (if it is actually
* present)
*/
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD,
JavaSparkContext jsc, HoodieTable<T> hoodieTable) throws HoodieIndexException;
public abstract JavaRDD<HoodieRecord<T>> tagLocation(JavaRDD<HoodieRecord<T>> recordRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Extracts the location of written records, and updates the index.
@@ -84,8 +84,7 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
* TODO(vc): We may need to propagate the record as well in a WriteStatus class
*/
public abstract JavaRDD<WriteStatus> updateLocation(JavaRDD<WriteStatus> writeStatusRDD, JavaSparkContext jsc,
HoodieTable<T> hoodieTable)
throws HoodieIndexException;
HoodieTable<T> hoodieTable) throws HoodieIndexException;
/**
* Rollback the efffects of the commit made at commitTime.
@@ -93,17 +92,17 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
public abstract boolean rollbackCommit(String commitTime);
/**
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the
* `partitionPath`. Such an implementation is able to obtain the same mapping, for two hoodie keys
* with same `recordKey` but different `partitionPath`
* An index is `global` if {@link HoodieKey} to fileID mapping, does not depend on the `partitionPath`. Such an
* implementation is able to obtain the same mapping, for two hoodie keys with same `recordKey` but different
* `partitionPath`
*
* @return whether or not, the index implementation is global in nature
*/
public abstract boolean isGlobal();
/**
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e
* having a {@link FileSlice}, with no data file.
* This is used by storage to determine, if its safe to send inserts, straight to the log, i.e having a
* {@link FileSlice}, with no data file.
*
* @return Returns true/false depending on whether the impl has this capability
*/
@@ -111,8 +110,8 @@ public abstract class HoodieIndex<T extends HoodieRecordPayload> implements Seri
/**
* An index is "implicit" with respect to storage, if just writing new data to a file slice,
* updates the index as well. This is used by storage, to save memory footprint in certain cases.
* An index is "implicit" with respect to storage, if just writing new data to a file slice, updates the index as
* well. This is used by storage, to save memory footprint in certain cases.
*/
public abstract boolean isImplicitWithStorage();

View File

@@ -40,7 +40,9 @@ import org.apache.spark.api.java.function.Function2;
/**
* Hoodie Index implementation backed by an in-memory Hash map. <p> ONLY USE FOR LOCAL TESTING
* Hoodie Index implementation backed by an in-memory Hash map.
* <p>
* ONLY USE FOR LOCAL TESTING
*/
public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
@@ -80,7 +82,7 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
if (newLocation.isPresent()) {
recordLocationMap.put(key, newLocation.get());
} else {
//Delete existing index for a deleted record
// Delete existing index for a deleted record
recordLocationMap.remove(key);
}
}
@@ -122,12 +124,10 @@ public class InMemoryHashIndex<T extends HoodieRecordPayload> extends HoodieInde
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
class LocationTagFunction implements
Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
class LocationTagFunction implements Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> {
@Override
public Iterator<HoodieRecord<T>> call(Integer partitionNum,
Iterator<HoodieRecord<T>> hoodieRecordIterator) {
public Iterator<HoodieRecord<T>> call(Integer partitionNum, Iterator<HoodieRecord<T>> hoodieRecordIterator) {
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
while (hoodieRecordIterator.hasNext()) {
HoodieRecord<T> rec = hoodieRecordIterator.next();

View File

@@ -35,6 +35,7 @@ import org.apache.spark.Partitioner;
* Partitions bloom filter checks by spreading out comparisons across buckets of work.
*
* Each bucket incurs the following cost
*
* <pre>
* 1) Read bloom filter from file footer
* 2) Check keys against bloom filter
@@ -47,6 +48,7 @@ import org.apache.spark.Partitioner;
* could bound the amount of skew to std_dev(numberOfBucketsPerPartition) * cost of (3), lower than sort partitioning.
*
* Approach has two goals :
*
* <pre>
* 1) Pack as many buckets from same file group into same partition, to amortize cost of (1) and (2) further
* 2) Spread buckets across partitions evenly to achieve skew reduction
@@ -76,8 +78,7 @@ public class BucketizedBloomCheckPartitioner extends Partitioner {
Map<String, Integer> bucketsPerFileGroup = new HashMap<>();
// Compute the buckets needed per file group, using simple uniform distribution
fileGroupToComparisons.forEach((f, c) ->
bucketsPerFileGroup.put(f, (int) Math.ceil((c * 1.0) / keysPerBucket)));
fileGroupToComparisons.forEach((f, c) -> bucketsPerFileGroup.put(f, (int) Math.ceil((c * 1.0) / keysPerBucket)));
int totalBuckets = bucketsPerFileGroup.values().stream().mapToInt(i -> i).sum();
// If totalBuckets > targetPartitions, no need to have extra partitions
this.partitions = Math.min(targetPartitions, totalBuckets);

View File

@@ -78,12 +78,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
// Step 1: Extract out thinner JavaPairRDD of (partitionPath, recordKey)
JavaPairRDD<String, String> partitionRecordKeyPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
recordRDD.mapToPair(record -> new Tuple2<>(record.getPartitionPath(), record.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD = lookupIndex(partitionRecordKeyPairRDD, jsc,
hoodieTable);
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD =
lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
// Cache the result, for subsequent stages.
if (config.getBloomIndexUseCaching()) {
@@ -96,8 +96,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
// Step 4: Tag the incoming records, as inserts or updates, by joining with existing record keys
// Cost: 4 sec.
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(keyFilenamePairRDD,
recordRDD);
JavaRDD<HoodieRecord<T>> taggedRecordRDD = tagLocationBacktoRecords(keyFilenamePairRDD, recordRDD);
if (config.getBloomIndexUseCaching()) {
recordRDD.unpersist(); // unpersist the input Record RDD
@@ -108,8 +107,8 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
/**
* Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is
* not found.
* Returns an RDD mapping each HoodieKey with a partitionPath/fileID which contains it. Option.Empty if the key is not
* found.
*
* @param hoodieKeys keys to lookup
* @param jsc spark context
@@ -118,12 +117,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
@Override
public JavaPairRDD<HoodieKey, Option<Pair<String, String>>> fetchRecordLocation(JavaRDD<HoodieKey> hoodieKeys,
JavaSparkContext jsc, HoodieTable<T> hoodieTable) {
JavaPairRDD<String, String> partitionRecordKeyPairRDD = hoodieKeys
.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
JavaPairRDD<String, String> partitionRecordKeyPairRDD =
hoodieKeys.mapToPair(key -> new Tuple2<>(key.getPartitionPath(), key.getRecordKey()));
// Lookup indexes for all the partition/recordkey pair
JavaPairRDD<HoodieKey, HoodieRecordLocation> recordKeyLocationRDD = lookupIndex(partitionRecordKeyPairRDD, jsc,
hoodieTable);
JavaPairRDD<HoodieKey, HoodieRecordLocation> recordKeyLocationRDD =
lookupIndex(partitionRecordKeyPairRDD, jsc, hoodieTable);
JavaPairRDD<HoodieKey, String> keyHoodieKeyPairRDD = hoodieKeys.mapToPair(key -> new Tuple2<>(key, null));
return keyHoodieKeyPairRDD.leftOuterJoin(recordKeyLocationRDD).mapToPair(keyLoc -> {
@@ -149,19 +148,19 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
List<String> affectedPartitionPathList = new ArrayList<>(recordsPerPartition.keySet());
// Step 2: Load all involved files as <Partition, filename> pairs
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList = loadInvolvedFiles(affectedPartitionPathList, jsc,
hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo = fileInfoList.stream()
.collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
List<Tuple2<String, BloomIndexFileInfo>> fileInfoList =
loadInvolvedFiles(affectedPartitionPathList, jsc, hoodieTable);
final Map<String, List<BloomIndexFileInfo>> partitionToFileInfo =
fileInfoList.stream().collect(groupingBy(Tuple2::_1, mapping(Tuple2::_2, toList())));
// Step 3: Obtain a RDD, for each incoming record, that already exists, with the file id,
// that contains it.
Map<String, Long> comparisonsPerFileGroup = computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo,
partitionRecordKeyPairRDD);
Map<String, Long> comparisonsPerFileGroup =
computeComparisonsPerFileGroup(recordsPerPartition, partitionToFileInfo, partitionRecordKeyPairRDD);
int safeParallelism = computeSafeParallelism(recordsPerPartition, comparisonsPerFileGroup);
int joinParallelism = determineParallelism(partitionRecordKeyPairRDD.partitions().size(), safeParallelism);
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, joinParallelism,
hoodieTable, comparisonsPerFileGroup);
return findMatchingFilesForRecordKeys(partitionToFileInfo, partitionRecordKeyPairRDD, joinParallelism, hoodieTable,
comparisonsPerFileGroup);
}
/**
@@ -175,13 +174,13 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
if (config.getBloomIndexPruneByRanges()) {
// we will just try exploding the input and then count to determine comparisons
// FIX(vc): Only do sampling here and extrapolate?
fileToComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo,
partitionRecordKeyPairRDD).mapToPair(t -> t).countByKey();
fileToComparisons = explodeRecordRDDWithFileComparisons(partitionToFileInfo, partitionRecordKeyPairRDD)
.mapToPair(t -> t).countByKey();
} else {
fileToComparisons = new HashMap<>();
partitionToFileInfo.entrySet().stream().forEach(e -> {
for (BloomIndexFileInfo fileInfo : e.getValue()) {
//each file needs to be compared against all the records coming into the partition
// each file needs to be compared against all the records coming into the partition
fileToComparisons.put(fileInfo.getFileId(), recordsPerPartition.get(e.getKey()));
}
});
@@ -191,34 +190,41 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
/**
* Compute the minimum parallelism needed to play well with the spark 2GB limitation.. The index lookup can be skewed
* in three dimensions : #files, #partitions, #records <p> To be able to smoothly handle skews, we need to compute how
* to split each partitions into subpartitions. We do it here, in a way that keeps the amount of each Spark join
* partition to < 2GB. <p> If {@link HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is
* specified as a NON-zero number, then that is used explicitly.
* in three dimensions : #files, #partitions, #records
* <p>
* To be able to smoothly handle skews, we need to compute how to split each partitions into subpartitions. We do it
* here, in a way that keeps the amount of each Spark join partition to < 2GB.
* <p>
* If {@link HoodieIndexConfig#BLOOM_INDEX_PARALLELISM_PROP} is specified as a NON-zero number, then that is used
* explicitly.
*/
int computeSafeParallelism(Map<String, Long> recordsPerPartition, Map<String, Long> comparisonsPerFileGroup) {
long totalComparisons = comparisonsPerFileGroup.values().stream().mapToLong(Long::longValue).sum();
long totalFiles = comparisonsPerFileGroup.size();
long totalRecords = recordsPerPartition.values().stream().mapToLong(Long::longValue).sum();
int parallelism = (int) (totalComparisons / MAX_ITEMS_PER_SHUFFLE_PARTITION + 1);
logger.info(String.format("TotalRecords %d, TotalFiles %d, TotalAffectedPartitions %d, TotalComparisons %d, "
+ "SafeParallelism %d", totalRecords, totalFiles, recordsPerPartition.size(), totalComparisons, parallelism));
logger.info(String.format(
"TotalRecords %d, TotalFiles %d, TotalAffectedPartitions %d, TotalComparisons %d, " + "SafeParallelism %d",
totalRecords, totalFiles, recordsPerPartition.size(), totalComparisons, parallelism));
return parallelism;
}
/**
* Its crucial to pick the right parallelism. <p> totalSubPartitions : this is deemed safe limit, to be nice with
* Spark. inputParallelism : typically number of input file splits <p> We pick the max such that, we are always safe,
* but go higher if say a there are a lot of input files. (otherwise, we will fallback to number of partitions in
* input and end up with slow performance)
* Its crucial to pick the right parallelism.
* <p>
* totalSubPartitions : this is deemed safe limit, to be nice with Spark. inputParallelism : typically number of input
* file splits
* <p>
* We pick the max such that, we are always safe, but go higher if say a there are a lot of input files. (otherwise,
* we will fallback to number of partitions in input and end up with slow performance)
*/
private int determineParallelism(int inputParallelism, int totalSubPartitions) {
// If bloom index parallelism is set, use it to to check against the input parallelism and
// take the max
int indexParallelism = Math.max(inputParallelism, config.getBloomIndexParallelism());
int joinParallelism = Math.max(totalSubPartitions, indexParallelism);
logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${" + config
.getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, "
logger.info("InputParallelism: ${" + inputParallelism + "}, " + "IndexParallelism: ${"
+ config.getBloomIndexParallelism() + "}, " + "TotalSubParts: ${" + totalSubPartitions + "}, "
+ "Join Parallelism set to : " + joinParallelism);
return joinParallelism;
}
@@ -231,11 +237,10 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
final HoodieTable hoodieTable) {
// Obtain the latest data files from all the partitions.
List<Pair<String, String>> partitionPathFileIDList = jsc
.parallelize(partitions, Math.max(partitions.size(), 1))
.flatMap(partitionPath -> {
Option<HoodieInstant> latestCommitTime = hoodieTable.getMetaClient().getCommitsTimeline()
.filterCompletedInstants().lastInstant();
List<Pair<String, String>> partitionPathFileIDList =
jsc.parallelize(partitions, Math.max(partitions.size(), 1)).flatMap(partitionPath -> {
Option<HoodieInstant> latestCommitTime =
hoodieTable.getMetaClient().getCommitsTimeline().filterCompletedInstants().lastInstant();
List<Pair<String, String>> filteredFiles = new ArrayList<>();
if (latestCommitTime.isPresent()) {
filteredFiles = hoodieTable.getROFileSystemView()
@@ -259,8 +264,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}).collect();
} else {
return partitionPathFileIDList.stream()
.map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue())))
.collect(toList());
.map(pf -> new Tuple2<>(pf.getKey(), new BloomIndexFileInfo(pf.getValue()))).collect(toList());
}
}
@@ -307,9 +311,9 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
JavaRDD<Tuple2<String, HoodieKey>> explodeRecordRDDWithFileComparisons(
final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo,
JavaPairRDD<String, String> partitionRecordKeyPairRDD) {
IndexFileFilter indexFileFilter = config.useBloomIndexTreebasedFilter()
? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo)
: new ListBasedIndexFileFilter(partitionToFileIndexInfo);
IndexFileFilter indexFileFilter =
config.useBloomIndexTreebasedFilter() ? new IntervalTreeBasedIndexFileFilter(partitionToFileIndexInfo)
: new ListBasedIndexFileFilter(partitionToFileIndexInfo);
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
String recordKey = partitionRecordKeyPair._2();
@@ -322,10 +326,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
}
/**
* Find out <RowKey, filename> pair. All workload grouped by file-level. <p> Join PairRDD(PartitionPath, RecordKey)
* and PairRDD(PartitionPath, File) & then repartition such that each RDD partition is a file, then for each file, we
* do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey <p> Make sure the parallelism is atleast the groupby
* parallelism for tagging location
* Find out <RowKey, filename> pair. All workload grouped by file-level.
* <p>
* Join PairRDD(PartitionPath, RecordKey) and PairRDD(PartitionPath, File) & then repartition such that each RDD
* partition is a file, then for each file, we do (1) load bloom filter, (2) load rowKeys, (3) Tag rowKey
* <p>
* Make sure the parallelism is atleast the groupby parallelism for tagging location
*/
@VisibleForTesting
JavaPairRDD<HoodieKey, HoodieRecordLocation> findMatchingFilesForRecordKeys(
@@ -336,33 +342,24 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
explodeRecordRDDWithFileComparisons(partitionToFileIndexInfo, partitionRecordKeyPairRDD);
if (config.useBloomIndexBucketizedChecking()) {
Partitioner partitioner = new BucketizedBloomCheckPartitioner(
shuffleParallelism,
fileGroupToComparisons,
config.getBloomIndexKeysPerBucket()
);
Partitioner partitioner = new BucketizedBloomCheckPartitioner(shuffleParallelism, fileGroupToComparisons,
config.getBloomIndexKeysPerBucket());
fileComparisonsRDD = fileComparisonsRDD
.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
.repartitionAndSortWithinPartitions(partitioner)
.map(Tuple2::_2);
fileComparisonsRDD = fileComparisonsRDD.mapToPair(t -> new Tuple2<>(Pair.of(t._1, t._2.getRecordKey()), t))
.repartitionAndSortWithinPartitions(partitioner).map(Tuple2::_2);
} else {
fileComparisonsRDD = fileComparisonsRDD.sortBy(Tuple2::_1, true, shuffleParallelism);
}
return fileComparisonsRDD
.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
.flatMap(List::iterator)
.filter(lr -> lr.getMatchingRecordKeys().size() > 0)
return fileComparisonsRDD.mapPartitionsWithIndex(new HoodieBloomIndexCheckFunction(hoodieTable, config), true)
.flatMap(List::iterator).filter(lr -> lr.getMatchingRecordKeys().size() > 0)
.flatMapToPair(lookupResult -> lookupResult.getMatchingRecordKeys().stream()
.map(recordKey -> new Tuple2<>(new HoodieKey(recordKey, lookupResult.getPartitionPath()),
new HoodieRecordLocation(lookupResult.getBaseInstantTime(), lookupResult.getFileId())))
.collect(Collectors.toList())
.iterator());
.collect(Collectors.toList()).iterator());
}
HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord,
Option<HoodieRecordLocation> location) {
HoodieRecord<T> getTaggedRecord(HoodieRecord<T> inputRecord, Option<HoodieRecordLocation> location) {
HoodieRecord<T> record = inputRecord;
if (location.isPresent()) {
// When you have a record in multiple files in the same partition, then rowKeyRecordPairRDD
@@ -383,12 +380,12 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
*/
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyRecordPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getKey(), record));
JavaPairRDD<HoodieKey, HoodieRecord<T>> keyRecordPairRDD =
recordRDD.mapToPair(record -> new Tuple2<>(record.getKey(), record));
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
// so we do left outer join.
return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values().map(
v1 -> getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull())));
return keyRecordPairRDD.leftOuterJoin(keyFilenamePairRDD).values()
.map(v1 -> getTaggedRecord(v1._1, Option.ofNullable(v1._2.orNull())));
}
@Override

View File

@@ -34,11 +34,10 @@ import org.apache.spark.api.java.function.Function2;
import scala.Tuple2;
/**
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the
* actual files
* Function performing actual checking of RDD partition containing (fileId, hoodieKeys) against the actual files
*/
public class HoodieBloomIndexCheckFunction implements
Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
public class HoodieBloomIndexCheckFunction
implements Function2<Integer, Iterator<Tuple2<String, HoodieKey>>, Iterator<List<KeyLookupResult>>> {
private final HoodieTable hoodieTable;
@@ -59,14 +58,12 @@ public class HoodieBloomIndexCheckFunction implements
private HoodieKeyLookupHandle keyLookupHandle;
LazyKeyCheckIterator(
Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
LazyKeyCheckIterator(Iterator<Tuple2<String, HoodieKey>> filePartitionRecordKeyTripletItr) {
super(filePartitionRecordKeyTripletItr);
}
@Override
protected void start() {
}
protected void start() {}
@Override
protected List<HoodieKeyLookupHandle.KeyLookupResult> computeNext() {
@@ -113,7 +110,6 @@ public class HoodieBloomIndexCheckFunction implements
}
@Override
protected void end() {
}
protected void end() {}
}
}

View File

@@ -59,9 +59,8 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
final HoodieTable hoodieTable) {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
try {
List<String> allPartitionPaths = FSUtils
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
return super.loadInvolvedFiles(allPartitionPaths, jsc, hoodieTable);
} catch (IOException e) {
throw new HoodieIOException("Failed to load all partitions", e);
@@ -88,9 +87,9 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
entry.getValue().forEach(indexFile -> indexToPartitionMap.put(indexFile.getFileId(), entry.getKey()));
}
IndexFileFilter indexFileFilter = config.getBloomIndexPruneByRanges()
? new IntervalTreeBasedGlobalIndexFileFilter(partitionToFileIndexInfo)
: new ListBasedGlobalIndexFileFilter(partitionToFileIndexInfo);
IndexFileFilter indexFileFilter =
config.getBloomIndexPruneByRanges() ? new IntervalTreeBasedGlobalIndexFileFilter(partitionToFileIndexInfo)
: new ListBasedGlobalIndexFileFilter(partitionToFileIndexInfo);
return partitionRecordKeyPairRDD.map(partitionRecordKeyPair -> {
String recordKey = partitionRecordKeyPair._2();
@@ -109,8 +108,8 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
@Override
protected JavaRDD<HoodieRecord<T>> tagLocationBacktoRecords(
JavaPairRDD<HoodieKey, HoodieRecordLocation> keyFilenamePairRDD, JavaRDD<HoodieRecord<T>> recordRDD) {
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD = recordRDD
.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
JavaPairRDD<String, HoodieRecord<T>> rowKeyRecordPairRDD =
recordRDD.mapToPair(record -> new Tuple2<>(record.getRecordKey(), record));
// Here as the recordRDD might have more data than rowKeyRDD (some rowKeys' fileId is null),
// so we do left outer join.

View File

@@ -28,7 +28,7 @@ import java.util.stream.Collectors;
/**
* Interval Tree based index look up for Global Index. Builds an {@link KeyRangeLookupTree} for all index files (across
* all partitions) and uses it to search for matching index files for any given recordKey that needs to be looked up.
* all partitions) and uses it to search for matching index files for any given recordKey that needs to be looked up.
*/
class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter {
@@ -41,16 +41,16 @@ class IntervalTreeBasedGlobalIndexFileFilter implements IndexFileFilter {
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}s
*/
IntervalTreeBasedGlobalIndexFileFilter(final Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
List<BloomIndexFileInfo> allIndexFiles = partitionToFileIndexInfo.values().stream().flatMap(Collection::stream)
.collect(Collectors.toList());
List<BloomIndexFileInfo> allIndexFiles =
partitionToFileIndexInfo.values().stream().flatMap(Collection::stream).collect(Collectors.toList());
// Note that the interval tree implementation doesn't have auto-balancing to ensure logN search time.
// So, we are shuffling the input here hoping the tree will not have any skewness. If not, the tree could be skewed
// which could result in N search time instead of NlogN.
Collections.shuffle(allIndexFiles);
allIndexFiles.forEach(indexFile -> {
if (indexFile.hasKeyRanges()) {
indexLookUpTree.insert(new KeyRangeNode(indexFile.getMinRecordKey(),
indexFile.getMaxRecordKey(), indexFile.getFileId()));
indexLookUpTree
.insert(new KeyRangeNode(indexFile.getMinRecordKey(), indexFile.getMaxRecordKey(), indexFile.getFileId()));
} else {
filesWithNoRanges.add(indexFile.getFileId());
}

View File

@@ -48,8 +48,8 @@ class IntervalTreeBasedIndexFileFilter implements IndexFileFilter {
KeyRangeLookupTree lookUpTree = new KeyRangeLookupTree();
bloomIndexFiles.forEach(indexFileInfo -> {
if (indexFileInfo.hasKeyRanges()) {
lookUpTree.insert(new KeyRangeNode(indexFileInfo.getMinRecordKey(),
indexFileInfo.getMaxRecordKey(), indexFileInfo.getFileId()));
lookUpTree.insert(new KeyRangeNode(indexFileInfo.getMinRecordKey(), indexFileInfo.getMaxRecordKey(),
indexFileInfo.getFileId()));
} else {
if (!partitionToFilesWithNoRanges.containsKey(partition)) {
partitionToFilesWithNoRanges.put(partition, new HashSet<>());

View File

@@ -50,25 +50,16 @@ class KeyRangeLookupTree implements Serializable {
*
* If no root exists, make {@code newNode} as the root and return the new root.
*
* If current root and newNode matches with min record key and max record key,
* merge two nodes. In other words, add files from {@code newNode} to current root.
* Return current root.
* If current root and newNode matches with min record key and max record key, merge two nodes. In other words, add
* files from {@code newNode} to current root. Return current root.
*
* If current root is < newNode
* if current root has no right sub tree
* update current root's right sub tree max and min
* set newNode as right sub tree
* else
* update root's right sub tree min and max with newNode's min and max record key as applicable
* recursively call insert() with root's right subtree as new root
* If current root is < newNode if current root has no right sub tree update current root's right sub tree max and min
* set newNode as right sub tree else update root's right sub tree min and max with newNode's min and max record key
* as applicable recursively call insert() with root's right subtree as new root
*
* else // current root is >= newNode
* if current root has no left sub tree
* update current root's left sub tree max and min
* set newNode as left sub tree
* else
* update root's left sub tree min and max with newNode's min and max record key as applicable
* recursively call insert() with root's left subtree as new root
* else // current root is >= newNode if current root has no left sub tree update current root's left sub tree max and
* min set newNode as left sub tree else update root's left sub tree min and max with newNode's min and max record key
* as applicable recursively call insert() with root's left subtree as new root
*
* @param root refers to the current root of the look up tree
* @param newNode newNode the new {@link KeyRangeNode} to be inserted

View File

@@ -62,15 +62,10 @@ class KeyRangeNode implements Comparable<KeyRangeNode>, Serializable {
@Override
public String toString() {
return "KeyRangeNode{"
+ "minRecordKey='" + minRecordKey + '\''
+ ", maxRecordKey='" + maxRecordKey + '\''
+ ", fileNameList=" + fileNameList
+ ", rightSubTreeMax='" + rightSubTreeMax + '\''
+ ", leftSubTreeMax='" + leftSubTreeMax + '\''
+ ", rightSubTreeMin='" + rightSubTreeMin + '\''
+ ", leftSubTreeMin='" + leftSubTreeMin + '\''
+ '}';
return "KeyRangeNode{" + "minRecordKey='" + minRecordKey + '\'' + ", maxRecordKey='" + maxRecordKey + '\''
+ ", fileNameList=" + fileNameList + ", rightSubTreeMax='" + rightSubTreeMax + '\'' + ", leftSubTreeMax='"
+ leftSubTreeMax + '\'' + ", rightSubTreeMin='" + rightSubTreeMin + '\'' + ", leftSubTreeMin='" + leftSubTreeMin
+ '\'' + '}';
}
/**
@@ -78,8 +73,8 @@ class KeyRangeNode implements Comparable<KeyRangeNode>, Serializable {
*
* @param that the {@link KeyRangeNode} to be compared with
* @return the result of comparison. 0 if both min and max are equal in both. 1 if this {@link KeyRangeNode} is
* greater than the {@code that} keyRangeNode. -1 if {@code that} keyRangeNode is greater than this {@link
* KeyRangeNode}
* greater than the {@code that} keyRangeNode. -1 if {@code that} keyRangeNode is greater than this
* {@link KeyRangeNode}
*/
@Override
public int compareTo(KeyRangeNode that) {

View File

@@ -30,8 +30,7 @@ class ListBasedGlobalIndexFileFilter extends ListBasedIndexFileFilter {
*
* @param partitionToFileIndexInfo Map of partition to List of {@link BloomIndexFileInfo}
*/
ListBasedGlobalIndexFileFilter(
Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
ListBasedGlobalIndexFileFilter(Map<String, List<BloomIndexFileInfo>> partitionToFileIndexInfo) {
super(partitionToFileIndexInfo);
}

View File

@@ -68,10 +68,8 @@ import scala.Tuple2;
*/
public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME =
"spark.executor.instances";
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME =
"spark.dynamicAllocation.enabled";
public static final String DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME = "spark.executor.instances";
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME = "spark.dynamicAllocation.enabled";
public static final String DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME =
"spark.dynamicAllocation.maxExecutors";
@@ -114,9 +112,8 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
public HBaseIndexQPSResourceAllocator createQPSResourceAllocator(HoodieWriteConfig config) {
try {
logger.info("createQPSResourceAllocator :" + config.getHBaseQPSResourceAllocatorClass());
final HBaseIndexQPSResourceAllocator resourceAllocator =
(HBaseIndexQPSResourceAllocator) ReflectionUtils.loadClass(
config.getHBaseQPSResourceAllocatorClass(), config);
final HBaseIndexQPSResourceAllocator resourceAllocator = (HBaseIndexQPSResourceAllocator) ReflectionUtils
.loadClass(config.getHBaseQPSResourceAllocatorClass(), config);
return resourceAllocator;
} catch (Exception e) {
logger.warn("error while instantiating HBaseIndexQPSResourceAllocator", e);
@@ -143,14 +140,14 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
try {
return ConnectionFactory.createConnection(hbaseConfig);
} catch (IOException e) {
throw new HoodieDependentSystemUnavailableException(
HoodieDependentSystemUnavailableException.HBASE, quorum + ":" + port);
throw new HoodieDependentSystemUnavailableException(HoodieDependentSystemUnavailableException.HBASE,
quorum + ":" + port);
}
}
/**
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is
* closed when JVM exits
* Since we are sharing the HbaseConnection across tasks in a JVM, make sure the HbaseConnectio is closed when JVM
* exits
*/
private void addShutDownHook() {
Runtime.getRuntime().addShutdownHook(new Thread() {
@@ -172,103 +169,95 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
private Get generateStatement(String key) throws IOException {
return new Get(Bytes.toBytes(key)).setMaxVersions(1)
.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
return new Get(Bytes.toBytes(key)).setMaxVersions(1).addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN)
.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN).addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN);
}
private boolean checkIfValidCommit(HoodieTableMetaClient metaClient, String commitTs) {
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().filterCompletedInstants();
// Check if the last commit ts for this row is 1) present in the timeline or
// 2) is less than the first commit ts in the timeline
return !commitTimeline.empty() && (commitTimeline
.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|| HoodieTimeline
.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
HoodieTimeline.GREATER));
return !commitTimeline.empty()
&& (commitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commitTs))
|| HoodieTimeline.compareTimestamps(commitTimeline.firstInstant().get().getTimestamp(), commitTs,
HoodieTimeline.GREATER));
}
/**
* Function that tags each HoodieRecord with an existing location, if known.
*/
private Function2<Integer, Iterator<HoodieRecord<T>>,
Iterator<HoodieRecord<T>>> locationTagFunction(HoodieTableMetaClient metaClient) {
private Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>> locationTagFunction(
HoodieTableMetaClient metaClient) {
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>)
(partitionNum, hoodieRecordIterator) -> {
return (Function2<Integer, Iterator<HoodieRecord<T>>, Iterator<HoodieRecord<T>>>) (partitionNum,
hoodieRecordIterator) -> {
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
Integer multiGetBatchSize = config.getHbaseIndexGetBatchSize();
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null || hbaseConnection.isClosed()) {
hbaseConnection = getHBaseConnection();
}
}
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
List<Get> statements = new ArrayList<>();
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
// Do the tagging.
while (hoodieRecordIterator.hasNext()) {
HoodieRecord rec = hoodieRecordIterator.next();
statements.add(generateStatement(rec.getRecordKey()));
currentBatchOfRecords.add(rec);
// iterator till we reach batch size
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
// get results for batch from Hbase
Result[] results = doGet(hTable, statements);
// clear statements to be GC'd
statements.clear();
for (Result result : results) {
// first, attempt to grab location from HBase
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
if (result.getRow() != null) {
String keyFromResult = Bytes.toString(result.getRow());
String commitTs = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
String fileId = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
String partitionPath = Bytes
.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
// Grab the global HBase connection
synchronized (HBaseIndex.class) {
if (hbaseConnection == null || hbaseConnection.isClosed()) {
hbaseConnection = getHBaseConnection();
}
}
List<HoodieRecord<T>> taggedRecords = new ArrayList<>();
HTable hTable = null;
try {
hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName));
List<Get> statements = new ArrayList<>();
List<HoodieRecord> currentBatchOfRecords = new LinkedList<>();
// Do the tagging.
while (hoodieRecordIterator.hasNext()) {
HoodieRecord rec = hoodieRecordIterator.next();
statements.add(generateStatement(rec.getRecordKey()));
currentBatchOfRecords.add(rec);
// iterator till we reach batch size
if (statements.size() >= multiGetBatchSize || !hoodieRecordIterator.hasNext()) {
// get results for batch from Hbase
Result[] results = doGet(hTable, statements);
// clear statements to be GC'd
statements.clear();
for (Result result : results) {
// first, attempt to grab location from HBase
HoodieRecord currentRecord = currentBatchOfRecords.remove(0);
if (result.getRow() != null) {
String keyFromResult = Bytes.toString(result.getRow());
String commitTs = Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN));
String fileId = Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN));
String partitionPath = Bytes.toString(result.getValue(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN));
if (checkIfValidCommit(metaClient, commitTs)) {
currentRecord = new HoodieRecord(
new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData());
currentRecord.unseal();
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
currentRecord.seal();
taggedRecords.add(currentRecord);
// the key from Result and the key being processed should be same
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
} else { //if commit is invalid, treat this as a new taggedRecord
taggedRecords.add(currentRecord);
}
} else {
taggedRecords.add(currentRecord);
}
if (checkIfValidCommit(metaClient, commitTs)) {
currentRecord = new HoodieRecord(new HoodieKey(currentRecord.getRecordKey(), partitionPath),
currentRecord.getData());
currentRecord.unseal();
currentRecord.setCurrentLocation(new HoodieRecordLocation(commitTs, fileId));
currentRecord.seal();
taggedRecords.add(currentRecord);
// the key from Result and the key being processed should be same
assert (currentRecord.getRecordKey().contentEquals(keyFromResult));
} else { // if commit is invalid, treat this as a new taggedRecord
taggedRecords.add(currentRecord);
}
} else {
taggedRecords.add(currentRecord);
}
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Tag indexed locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
}
return taggedRecords.iterator();
};
}
} catch (IOException e) {
throw new HoodieIndexException("Failed to Tag indexed locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
hTable.close();
} catch (IOException e) {
// Ignore
}
}
}
return taggedRecords.iterator();
};
}
private Result[] doGet(HTable hTable, List<Get> keys) throws IOException {
@@ -310,15 +299,12 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
continue;
}
Put put = new Put(Bytes.toBytes(rec.getRecordKey()));
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN,
Bytes.toBytes(loc.get().getInstantTime()));
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN,
Bytes.toBytes(loc.get().getFileId()));
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN,
Bytes.toBytes(rec.getPartitionPath()));
put.addColumn(SYSTEM_COLUMN_FAMILY, COMMIT_TS_COLUMN, Bytes.toBytes(loc.get().getInstantTime()));
put.addColumn(SYSTEM_COLUMN_FAMILY, FILE_NAME_COLUMN, Bytes.toBytes(loc.get().getFileId()));
put.addColumn(SYSTEM_COLUMN_FAMILY, PARTITION_PATH_COLUMN, Bytes.toBytes(rec.getPartitionPath()));
puts.add(put);
} else {
//Delete existing index for a deleted record
// Delete existing index for a deleted record
Delete delete = new Delete(Bytes.toBytes(rec.getRecordKey()));
deletes.add(delete);
}
@@ -328,7 +314,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
}
doPutsAndDeletes(hTable, puts, deletes);
}
//process remaining puts and deletes, if any
// process remaining puts and deletes, if any
doPutsAndDeletes(hTable, puts, deletes);
} catch (Exception e) {
Exception we = new Exception("Error updating index for " + writeStatus, e);
@@ -338,8 +324,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
writeStatusList.add(writeStatus);
}
} catch (IOException e) {
throw new HoodieIndexException(
"Failed to Update Index locations because of exception with HBase Client", e);
throw new HoodieIndexException("Failed to Update Index locations because of exception with HBase Client", e);
} finally {
if (hTable != null) {
try {
@@ -356,8 +341,7 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
/**
* Helper method to facilitate performing puts and deletes in Hbase
*/
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes)
throws IOException {
private void doPutsAndDeletes(HTable hTable, List<Put> puts, List<Delete> deletes) throws IOException {
if (puts.size() > 0) {
hTable.put(puts);
}
@@ -385,58 +369,49 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
final HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator = createQPSResourceAllocator(this.config);
setPutBatchSize(writeStatusRDD, hBaseIndexQPSResourceAllocator, jsc);
logger.info("multiPutBatchSize: before hbase puts" + multiPutBatchSize);
JavaRDD<WriteStatus> writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(
updateLocationFunction(), true);
JavaRDD<WriteStatus> writeStatusJavaRDD = writeStatusRDD.mapPartitionsWithIndex(updateLocationFunction(), true);
// caching the index updated status RDD
writeStatusJavaRDD = writeStatusJavaRDD.persist(config.getWriteStatusStorageLevel());
return writeStatusJavaRDD;
}
private void setPutBatchSize(JavaRDD<WriteStatus> writeStatusRDD,
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator,
final JavaSparkContext jsc) {
HBaseIndexQPSResourceAllocator hBaseIndexQPSResourceAllocator, final JavaSparkContext jsc) {
if (config.getHbaseIndexPutBatchSizeAutoCompute()) {
SparkConf conf = jsc.getConf();
int maxExecutors = conf.getInt(DEFAULT_SPARK_EXECUTOR_INSTANCES_CONFIG_NAME, 1);
if (conf.getBoolean(DEFAULT_SPARK_DYNAMIC_ALLOCATION_ENABLED_CONFIG_NAME, false)) {
maxExecutors = Math.max(maxExecutors, conf.getInt(
DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME, 1));
maxExecutors =
Math.max(maxExecutors, conf.getInt(DEFAULT_SPARK_DYNAMIC_ALLOCATION_MAX_EXECUTORS_CONFIG_NAME, 1));
}
/*
Each writeStatus represents status information from a write done in one of the IOHandles.
If a writeStatus has any insert, it implies that the corresponding task contacts HBase for
doing puts, since we only do puts for inserts from HBaseIndex.
* Each writeStatus represents status information from a write done in one of the IOHandles. If a writeStatus has
* any insert, it implies that the corresponding task contacts HBase for doing puts, since we only do puts for
* inserts from HBaseIndex.
*/
final Tuple2<Long, Integer> numPutsParallelismTuple = getHBasePutAccessParallelism(writeStatusRDD);
final Tuple2<Long, Integer> numPutsParallelismTuple = getHBasePutAccessParallelism(writeStatusRDD);
final long numPuts = numPutsParallelismTuple._1;
final int hbasePutsParallelism = numPutsParallelismTuple._2;
this.numRegionServersForTable = getNumRegionServersAliveForTable();
final float desiredQPSFraction = hBaseIndexQPSResourceAllocator
.calculateQPSFractionForPutsTime(numPuts, this.numRegionServersForTable);
final float desiredQPSFraction =
hBaseIndexQPSResourceAllocator.calculateQPSFractionForPutsTime(numPuts, this.numRegionServersForTable);
logger.info("Desired QPSFraction :" + desiredQPSFraction);
logger.info("Number HBase puts :" + numPuts);
logger.info("Hbase Puts Parallelism :" + hbasePutsParallelism);
final float availableQpsFraction = hBaseIndexQPSResourceAllocator
.acquireQPSResources(desiredQPSFraction, numPuts);
final float availableQpsFraction =
hBaseIndexQPSResourceAllocator.acquireQPSResources(desiredQPSFraction, numPuts);
logger.info("Allocated QPS Fraction :" + availableQpsFraction);
multiPutBatchSize = putBatchSizeCalculator
.getBatchSize(
numRegionServersForTable,
maxQpsPerRegionServer,
hbasePutsParallelism,
maxExecutors,
SLEEP_TIME_MILLISECONDS,
availableQpsFraction);
multiPutBatchSize = putBatchSizeCalculator.getBatchSize(numRegionServersForTable, maxQpsPerRegionServer,
hbasePutsParallelism, maxExecutors, SLEEP_TIME_MILLISECONDS, availableQpsFraction);
logger.info("multiPutBatchSize :" + multiPutBatchSize);
}
}
@VisibleForTesting
public Tuple2<Long, Integer> getHBasePutAccessParallelism(final JavaRDD<WriteStatus> writeStatusRDD) {
final JavaPairRDD<Long, Integer> insertOnlyWriteStatusRDD =
writeStatusRDD.filter(w -> w.getStat().getNumInserts() > 0)
.mapToPair(w -> new Tuple2<>(w.getStat().getNumInserts(), 1));
final JavaPairRDD<Long, Integer> insertOnlyWriteStatusRDD = writeStatusRDD
.filter(w -> w.getStat().getNumInserts() > 0).mapToPair(w -> new Tuple2<>(w.getStat().getNumInserts(), 1));
return insertOnlyWriteStatusRDD.fold(new Tuple2<>(0L, 0), (w, c) -> new Tuple2<>(w._1 + c._1, w._2 + c._2));
}
@@ -460,21 +435,25 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
* 16000. We assume requests get distributed to Region Servers uniformly, so each RS gets 1600 requests which
* happens to be 10% of 16667 (maxQPSPerRegionServer), as expected.
* </p>
* <p> Assumptions made here <li> In a batch, writes get evenly distributed to each RS for that
* table. Since we do writes only in the case of inserts and not updates, for this assumption to fail, inserts would
* have to be skewed towards few RS, likelihood of which is less if Hbase table is pre-split and rowKeys are UUIDs
* (random strings). If this assumption fails, then it is possible for some RS to receive more than
* maxQpsPerRegionServer QPS, but for simplicity, we are going ahead with this model, since this is meant to be a
* lightweight distributed throttling mechanism without maintaining a global context. So if this assumption breaks,
* we are hoping the HBase Master relocates hot-spot regions to new Region Servers.
* <p>
* Assumptions made here
* <li>In a batch, writes get evenly distributed to each RS for that table. Since we do writes only in the case of
* inserts and not updates, for this assumption to fail, inserts would have to be skewed towards few RS, likelihood
* of which is less if Hbase table is pre-split and rowKeys are UUIDs (random strings). If this assumption fails,
* then it is possible for some RS to receive more than maxQpsPerRegionServer QPS, but for simplicity, we are going
* ahead with this model, since this is meant to be a lightweight distributed throttling mechanism without
* maintaining a global context. So if this assumption breaks, we are hoping the HBase Master relocates hot-spot
* regions to new Region Servers.
*
* </li> <li> For Region Server stability, throttling at a second level granularity is fine.
* Although, within a second, the sum of queries might be within maxQpsPerRegionServer, there could be peaks at some
* sub second intervals. So, the assumption is that these peaks are tolerated by the Region Server (which at max can
* be maxQpsPerRegionServer). </li> </p>
* </li>
* <li>For Region Server stability, throttling at a second level granularity is fine. Although, within a second, the
* sum of queries might be within maxQpsPerRegionServer, there could be peaks at some sub second intervals. So, the
* assumption is that these peaks are tolerated by the Region Server (which at max can be maxQpsPerRegionServer).
* </li>
* </p>
*/
public int getBatchSize(int numRegionServersForTable, int maxQpsPerRegionServer,
int numTasksDuringPut, int maxExecutors, int sleepTimeMs, float qpsFraction) {
public int getBatchSize(int numRegionServersForTable, int maxQpsPerRegionServer, int numTasksDuringPut,
int maxExecutors, int sleepTimeMs, float qpsFraction) {
int numRSAlive = numRegionServersForTable;
int maxReqPerSec = (int) (qpsFraction * numRSAlive * maxQpsPerRegionServer);
int numTasks = numTasksDuringPut;
@@ -499,11 +478,9 @@ public class HBaseIndex<T extends HoodieRecordPayload> extends HoodieIndex<T> {
// from the driver, so ok to use a local connection variable.
if (numRegionServersForTable == null) {
try (Connection conn = getHBaseConnection()) {
RegionLocator regionLocator = conn
.getRegionLocator(TableName.valueOf(tableName));
numRegionServersForTable = Math.toIntExact(
regionLocator.getAllRegionLocations().stream().map(e -> e.getServerName()).distinct()
.count());
RegionLocator regionLocator = conn.getRegionLocator(TableName.valueOf(tableName));
numRegionServersForTable = Math
.toIntExact(regionLocator.getAllRegionLocations().stream().map(e -> e.getServerName()).distinct().count());
return numRegionServersForTable;
} catch (IOException e) {
logger.error(e);

View File

@@ -26,11 +26,11 @@ import java.io.Serializable;
public interface HBaseIndexQPSResourceAllocator extends Serializable {
/**
* This method returns the QPS Fraction value that needs to be acquired such that the respective
* HBase index operation can be completed in desiredPutsTime.
* This method returns the QPS Fraction value that needs to be acquired such that the respective HBase index operation
* can be completed in desiredPutsTime.
*
* @param numPuts Number of inserts to be written to HBase index
* @param desiredPutsTimeInSecs Total expected time for the HBase inserts operation
* @param numPuts Number of inserts to be written to HBase index
* @param desiredPutsTimeInSecs Total expected time for the HBase inserts operation
* @return QPS fraction that needs to be acquired.
*/
float calculateQPSFractionForPutsTime(final long numPuts, final int desiredPutsTimeInSecs);
@@ -38,8 +38,8 @@ public interface HBaseIndexQPSResourceAllocator extends Serializable {
/**
* This method acquires the requested QPS Fraction against HBase cluster for index operation.
*
* @param desiredQPSFraction QPS fraction that needs to be requested and acquired
* @param numPuts Number of inserts to be written to HBase index
* @param desiredQPSFraction QPS fraction that needs to be requested and acquired
* @param numPuts Number of inserts to be written to HBase index
* @return value of the acquired QPS Fraction.
*/
float acquireQPSResources(final float desiredQPSFraction, final long numPuts);

View File

@@ -96,8 +96,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
// Total number of new records inserted into the delta file
private long insertRecordsWritten = 0;
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, fileId, hoodieTable);
writeStatus.setStat(new HoodieDeltaWriteStat());
this.fileId = fileId;
@@ -137,10 +137,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
} catch (Exception e) {
logger.error("Error in update task at commit " + instantTime, e);
writeStatus.setGlobalError(e);
throw new HoodieUpsertException(
"Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit "
+ instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath()
+ partitionPath, e);
throw new HoodieUpsertException("Failed to initialize HoodieAppendHandle for FileId: " + fileId + " on commit "
+ instantTime + " on HDFS path " + hoodieTable.getMetaClient().getBasePath() + partitionPath, e);
}
Path path = new Path(partitionPath, writer.getLogFile().getFileName());
writeStatus.getStat().setPath(path.toString());
@@ -155,13 +153,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
if (avroRecord.isPresent()) {
// Convert GenericRecord to GenericRecord with hoodie commit metadata in schema
avroRecord = Option.of(rewriteRecord((GenericRecord) avroRecord.get()));
String seqId = HoodieRecord.generateSequenceId(instantTime, TaskContext.getPartitionId(),
recordIndex.getAndIncrement());
HoodieAvroUtils
.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
hoodieRecord.getPartitionPath(), fileId);
HoodieAvroUtils
.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
String seqId =
HoodieRecord.generateSequenceId(instantTime, TaskContext.getPartitionId(), recordIndex.getAndIncrement());
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord.get(), hoodieRecord.getRecordKey(),
hoodieRecord.getPartitionPath(), fileId);
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord.get(), instantTime, seqId);
// If currentLocation is present, then this is an update
if (hoodieRecord.getCurrentLocation() != null) {
updatedRecordsWritten++;
@@ -208,20 +204,18 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
recordList.clear();
}
if (keysToDelete.size() > 0) {
writer = writer.appendBlock(
new HoodieDeleteBlock(keysToDelete.stream().toArray(HoodieKey[]::new), header));
writer = writer.appendBlock(new HoodieDeleteBlock(keysToDelete.stream().toArray(HoodieKey[]::new), header));
keysToDelete.clear();
}
} catch (Exception e) {
throw new HoodieAppendException(
"Failed while appending records to " + currentLogFile.getPath(), e);
throw new HoodieAppendException("Failed while appending records to " + currentLogFile.getPath(), e);
}
}
@Override
public boolean canWrite(HoodieRecord record) {
return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten * config
.getLogFileToParquetCompressionRatio();
return config.getParquetMaxFileSize() >= estimatedNumberOfBytesWritten
* config.getLogFileToParquetCompressionRatio();
}
@Override
@@ -262,8 +256,8 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
runtimeStats.setTotalUpsertTime(timer.endTimer());
stat.setRuntimeStats(runtimeStats);
logger.info(String.format("AppendHandle for partitionPath %s fileID %s, took %d ms.",
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime()));
logger.info(String.format("AppendHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(),
stat.getFileId(), runtimeStats.getTotalUpsertTime()));
return writeStatus;
} catch (IOException e) {
@@ -282,13 +276,11 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieWri
return HoodieLogFormat.newWriterBuilder()
.onParentPath(FSUtils.getPartitionPath(hoodieTable.getMetaClient().getBasePath(), partitionPath))
.withFileId(fileId).overBaseCommit(baseCommitTime).withLogVersion(
latestLogFile.map(HoodieLogFile::getLogVersion).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withFileId(fileId).overBaseCommit(baseCommitTime)
.withLogVersion(latestLogFile.map(HoodieLogFile::getLogVersion).orElse(HoodieLogFile.LOGFILE_BASE_VERSION))
.withSizeThreshold(config.getLogFileMaxSize()).withFs(fs)
.withLogWriteToken(
latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken))
.withRolloverLogWriteToken(writeToken)
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
.withLogWriteToken(latestLogFile.map(x -> FSUtils.getWriteTokenFromLogPath(x.getPath())).orElse(writeToken))
.withRolloverLogWriteToken(writeToken).withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
}
private void writeToBuffer(HoodieRecord<T> record) {

View File

@@ -45,9 +45,12 @@ import org.apache.log4j.Logger;
/**
* Cleaner is responsible for garbage collecting older files in a given partition path, such that
* <p> 1) It provides sufficient time for existing queries running on older versions, to close <p>
* 2) It bounds the growth of the files in the file system <p> TODO: Should all cleaning be done
* based on {@link HoodieCommitMetadata}
* <p>
* 1) It provides sufficient time for existing queries running on older versions, to close
* <p>
* 2) It bounds the growth of the files in the file system
* <p>
* TODO: Should all cleaning be done based on {@link HoodieCommitMetadata}
*/
public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
@@ -65,23 +68,22 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
this.commitTimeline = hoodieTable.getCompletedCommitTimeline();
this.config = config;
this.fgIdToPendingCompactionOperations =
((SyncableFileSystemView)hoodieTable.getRTFileSystemView()).getPendingCompactionOperations()
.map(entry -> Pair.of(new HoodieFileGroupId(entry.getValue().getPartitionPath(),
entry.getValue().getFileId()), entry.getValue()))
((SyncableFileSystemView) hoodieTable.getRTFileSystemView()).getPendingCompactionOperations()
.map(entry -> Pair.of(
new HoodieFileGroupId(entry.getValue().getPartitionPath(), entry.getValue().getFileId()),
entry.getValue()))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
}
/**
* Selects the older versions of files for cleaning, such that it bounds the number of versions of
* each file. This policy is useful, if you are simply interested in querying the table, and you
* don't want too many versions for a single file (i.e run it with versionsRetained = 1)
* Selects the older versions of files for cleaning, such that it bounds the number of versions of each file. This
* policy is useful, if you are simply interested in querying the table, and you don't want too many versions for a
* single file (i.e run it with versionsRetained = 1)
*/
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath)
throws IOException {
logger.info("Cleaning " + partitionPath + ", retaining latest " + config
.getCleanerFileVersionsRetained() + " file versions. ");
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
private List<String> getFilesToCleanKeepingLatestVersions(String partitionPath) throws IOException {
logger.info("Cleaning " + partitionPath + ", retaining latest " + config.getCleanerFileVersionsRetained()
+ " file versions. ");
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList());
List<String> deletePaths = new ArrayList<>();
// Collect all the datafiles savepointed by all the savepoints
List<String> savepointedFiles = hoodieTable.getSavepoints().stream()
@@ -90,8 +92,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
for (HoodieFileGroup fileGroup : fileGroups) {
int keepVersions = config.getCleanerFileVersionsRetained();
// do not cleanup slice required for pending compaction
Iterator<FileSlice> fileSliceIterator = fileGroup.getAllFileSlices()
.filter(fs -> !isFileSliceNeededForPendingCompaction(fs)).iterator();
Iterator<FileSlice> fileSliceIterator =
fileGroup.getAllFileSlices().filter(fs -> !isFileSliceNeededForPendingCompaction(fs)).iterator();
if (isFileGroupInPendingCompaction(fileGroup)) {
// We have already saved the last version of file-groups for pending compaction Id
keepVersions--;
@@ -116,8 +118,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
}
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
// If merge on read, then clean the log files for the commits as well
deletePaths.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString())
.collect(Collectors.toList()));
deletePaths
.addAll(nextSlice.getLogFiles().map(file -> file.getPath().toString()).collect(Collectors.toList()));
}
}
}
@@ -126,21 +128,21 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
/**
* Selects the versions for file for cleaning, such that it <p> - Leaves the latest version of the
* file untouched - For older versions, - It leaves all the commits untouched which has occured in
* last <code>config.getCleanerCommitsRetained()</code> commits - It leaves ONE commit before this
* window. We assume that the max(query execution time) == commit_batch_time *
* config.getCleanerCommitsRetained(). This is 12 hours by default. This is essential to leave the
* file used by the query thats running for the max time. <p> This provides the effect of having
* lookback into all changes that happened in the last X commits. (eg: if you retain 24 commits,
* and commit batch time is 30 mins, then you have 12 hrs of lookback) <p> This policy is the
* default.
* Selects the versions for file for cleaning, such that it
* <p>
* - Leaves the latest version of the file untouched - For older versions, - It leaves all the commits untouched which
* has occured in last <code>config.getCleanerCommitsRetained()</code> commits - It leaves ONE commit before this
* window. We assume that the max(query execution time) == commit_batch_time * config.getCleanerCommitsRetained().
* This is 12 hours by default. This is essential to leave the file used by the query thats running for the max time.
* <p>
* This provides the effect of having lookback into all changes that happened in the last X commits. (eg: if you
* retain 24 commits, and commit batch time is 30 mins, then you have 12 hrs of lookback)
* <p>
* This policy is the default.
*/
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath)
throws IOException {
private List<String> getFilesToCleanKeepingLatestCommits(String partitionPath) throws IOException {
int commitsRetained = config.getCleanerCommitsRetained();
logger
.info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
logger.info("Cleaning " + partitionPath + ", retaining latest " + commitsRetained + " commits. ");
List<String> deletePaths = new ArrayList<>();
// Collect all the datafiles savepointed by all the savepoints
@@ -150,8 +152,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
// determine if we have enough commits, to start cleaning.
if (commitTimeline.countInstants() > commitsRetained) {
HoodieInstant earliestCommitToRetain = getEarliestCommitToRetain().get();
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath)
.collect(Collectors.toList());
List<HoodieFileGroup> fileGroups = fileSystemView.getAllFileGroups(partitionPath).collect(Collectors.toList());
for (HoodieFileGroup fileGroup : fileGroups) {
List<FileSlice> fileSliceList = fileGroup.getAllFileSlices().collect(Collectors.toList());
@@ -160,8 +161,8 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
}
String lastVersion = fileSliceList.get(0).getBaseInstantTime();
String lastVersionBeforeEarliestCommitToRetain = getLatestVersionBeforeCommit(fileSliceList,
earliestCommitToRetain);
String lastVersionBeforeEarliestCommitToRetain =
getLatestVersionBeforeCommit(fileSliceList, earliestCommitToRetain);
// Ensure there are more than 1 version of the file (we only clean old files from updates)
// i.e always spare the last commit.
@@ -183,16 +184,14 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
}
// Always keep the last commit
if (!isFileSliceNeededForPendingCompaction(aSlice)
&& HoodieTimeline
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
HoodieTimeline.GREATER)) {
if (!isFileSliceNeededForPendingCompaction(aSlice) && HoodieTimeline
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
// this is a commit, that should be cleaned.
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getPath()));
if (hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ) {
// If merge on read, then clean the log files for the commits as well
deletePaths.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString())
.collect(Collectors.toList()));
deletePaths
.addAll(aSlice.getLogFiles().map(file -> file.getPath().toString()).collect(Collectors.toList()));
}
}
}
@@ -205,12 +204,10 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
/**
* Gets the latest version < commitTime. This version file could still be used by queries.
*/
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList,
HoodieInstant commitTime) {
private String getLatestVersionBeforeCommit(List<FileSlice> fileSliceList, HoodieInstant commitTime) {
for (FileSlice file : fileSliceList) {
String fileCommitTime = file.getBaseInstantTime();
if (HoodieTimeline
.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
if (HoodieTimeline.compareTimestamps(commitTime.getTimestamp(), fileCommitTime, HoodieTimeline.GREATER)) {
// fileList is sorted on the reverse, so the first commit we find <= commitTime is the
// one we want
return fileCommitTime;
@@ -246,14 +243,14 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
int commitsRetained = config.getCleanerCommitsRetained();
if (config.getCleanerPolicy() == HoodieCleaningPolicy.KEEP_LATEST_COMMITS
&& commitTimeline.countInstants() > commitsRetained) {
earliestCommitToRetain = commitTimeline
.nthInstant(commitTimeline.countInstants() - commitsRetained);
earliestCommitToRetain = commitTimeline.nthInstant(commitTimeline.countInstants() - commitsRetained);
}
return earliestCommitToRetain;
}
/**
* Determine if file slice needed to be preserved for pending compaction
*
* @param fileSlice File Slice
* @return true if file slice needs to be preserved, false otherwise.
*/

View File

@@ -83,9 +83,8 @@ public class HoodieCommitArchiveLog {
try {
if (this.writer == null) {
return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent())
.withFileId(archiveFilePath.getName())
.withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION).withFs(metaClient.getFs())
.overBaseCommit("").build();
.withFileId(archiveFilePath.getName()).withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION)
.withFs(metaClient.getFs()).overBaseCommit("").build();
} else {
return this.writer;
}
@@ -137,8 +136,7 @@ public class HoodieCommitArchiveLog {
// TODO: Handle ROLLBACK_ACTION in future
// ROLLBACK_ACTION is currently not defined in HoodieActiveTimeline
HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION))
.filterCompletedInstants();
.getTimelineOfActions(Sets.newHashSet(HoodieTimeline.CLEAN_ACTION)).filterCompletedInstants();
Stream<HoodieInstant> instants = cleanAndRollbackTimeline.getInstants()
.collect(Collectors.groupingBy(s -> s.getAction())).entrySet().stream().map(i -> {
if (i.getValue().size() > maxCommitsToKeep) {
@@ -148,7 +146,7 @@ public class HoodieCommitArchiveLog {
}
}).flatMap(i -> i.stream());
//TODO (na) : Add a way to return actions associated with a timeline and then merge/unify
// TODO (na) : Add a way to return actions associated with a timeline and then merge/unify
// with logic above to avoid Stream.concats
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
Option<HoodieInstant> oldestPendingCompactionInstant =
@@ -159,20 +157,16 @@ public class HoodieCommitArchiveLog {
Option<HoodieInstant> firstSavepoint = table.getCompletedSavepointTimeline().firstInstant();
if (!commitTimeline.empty() && commitTimeline.countInstants() > maxCommitsToKeep) {
// Actually do the commits
instants = Stream.concat(instants, commitTimeline.getInstants()
.filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline
.compareTimestamps(firstSavepoint.get().getTimestamp(), s.getTimestamp(),
HoodieTimeline.LESSER_OR_EQUAL));
})
.filter(s -> {
// Ensure commits >= oldest pending compaction commit is retained
return oldestPendingCompactionInstant.map(instant -> {
return HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER);
}).orElse(true);
})
.limit(commitTimeline.countInstants() - minCommitsToKeep));
instants = Stream.concat(instants, commitTimeline.getInstants().filter(s -> {
// if no savepoint present, then dont filter
return !(firstSavepoint.isPresent() && HoodieTimeline.compareTimestamps(firstSavepoint.get().getTimestamp(),
s.getTimestamp(), HoodieTimeline.LESSER_OR_EQUAL));
}).filter(s -> {
// Ensure commits >= oldest pending compaction commit is retained
return oldestPendingCompactionInstant.map(instant -> {
return HoodieTimeline.compareTimestamps(instant.getTimestamp(), s.getTimestamp(), HoodieTimeline.GREATER);
}).orElse(true);
}).limit(commitTimeline.countInstants() - minCommitsToKeep));
}
return instants;
@@ -194,13 +188,10 @@ public class HoodieCommitArchiveLog {
}
// Remove older meta-data from auxiliary path too
Option<HoodieInstant> latestCommitted =
Option.fromJavaOptional(archivedInstants.stream()
.filter(i -> {
return i.isCompleted()
&& (i.getAction().equals(HoodieTimeline.COMMIT_ACTION) || (i.getAction().equals(
HoodieTimeline.DELTA_COMMIT_ACTION)));
}).max(Comparator.comparing(HoodieInstant::getTimestamp)));
Option<HoodieInstant> latestCommitted = Option.fromJavaOptional(archivedInstants.stream().filter(i -> {
return i.isCompleted() && (i.getAction().equals(HoodieTimeline.COMMIT_ACTION)
|| (i.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)));
}).max(Comparator.comparing(HoodieInstant::getTimestamp)));
if (latestCommitted.isPresent()) {
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
}
@@ -214,12 +205,9 @@ public class HoodieCommitArchiveLog {
* @return success if all eligible file deleted successfully
* @throws IOException in case of error
*/
private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant)
throws IOException {
List<HoodieInstant> instants =
HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()),
HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
private boolean deleteAllInstantsOlderorEqualsInAuxMetaFolder(HoodieInstant thresholdInstant) throws IOException {
List<HoodieInstant> instants = HoodieTableMetaClient.scanHoodieInstantsFromFileSystem(metaClient.getFs(),
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE);
List<HoodieInstant> instantsToBeDeleted =
instants.stream().filter(instant1 -> HoodieTimeline.compareTimestamps(instant1.getTimestamp(),
@@ -239,8 +227,7 @@ public class HoodieCommitArchiveLog {
public void archive(List<HoodieInstant> instants) throws HoodieCommitException {
try {
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline()
.filterCompletedInstants();
HoodieTimeline commitTimeline = metaClient.getActiveTimeline().getAllCommitsTimeline().filterCompletedInstants();
Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema();
log.info("Wrapper schema " + wrapperSchema.toString());
List<IndexedRecord> records = new ArrayList<>();
@@ -277,15 +264,14 @@ public class HoodieCommitArchiveLog {
}
}
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline,
HoodieInstant hoodieInstant) throws IOException {
private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieInstant hoodieInstant)
throws IOException {
HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry();
archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp());
switch (hoodieInstant.getAction()) {
case HoodieTimeline.CLEAN_ACTION: {
archivedMetaWrapper.setHoodieCleanMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
HoodieCleanMetadata.class));
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieCleanMetadata.class));
archivedMetaWrapper.setActionType(ActionType.clean.name());
break;
}
@@ -297,16 +283,14 @@ public class HoodieCommitArchiveLog {
break;
}
case HoodieTimeline.ROLLBACK_ACTION: {
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
HoodieRollbackMetadata.class));
archivedMetaWrapper.setHoodieRollbackMetadata(AvroUtils.deserializeAvroMetadata(
commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class));
archivedMetaWrapper.setActionType(ActionType.rollback.name());
break;
}
case HoodieTimeline.SAVEPOINT_ACTION: {
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils
.deserializeAvroMetadata(commitTimeline.getInstantDetails(hoodieInstant).get(),
HoodieSavepointMetadata.class));
archivedMetaWrapper.setHoodieSavePointMetadata(AvroUtils.deserializeAvroMetadata(
commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class));
archivedMetaWrapper.setActionType(ActionType.savepoint.name());
break;
}
@@ -326,10 +310,10 @@ public class HoodieCommitArchiveLog {
private org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadataConverter(
HoodieCommitMetadata hoodieCommitMetadata) {
ObjectMapper mapper = new ObjectMapper();
//Need this to ignore other public get() methods
// Need this to ignore other public get() methods
mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false);
org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData = mapper
.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
org.apache.hudi.avro.model.HoodieCommitMetadata avroMetaData =
mapper.convertValue(hoodieCommitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class);
// Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer
avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, "");
return avroMetaData;

View File

@@ -66,11 +66,10 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
partitionMetadata.trySave(TaskContext.getPartitionId());
createMarkerFile(partitionPath);
this.storageWriter = HoodieStorageWriterFactory
.getStorageWriter(commitTime, path, hoodieTable, config, writerSchema);
this.storageWriter =
HoodieStorageWriterFactory.getStorageWriter(commitTime, path, hoodieTable, config, writerSchema);
} catch (IOException e) {
throw new HoodieInsertException(
"Failed to initialize HoodieStorageWriter for path " + path, e);
throw new HoodieInsertException("Failed to initialize HoodieStorageWriter for path " + path, e);
}
logger.info("New CreateHandle for partition :" + partitionPath + " with fileId " + fileId);
}
@@ -136,8 +135,7 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
}
}
} catch (IOException io) {
throw new HoodieInsertException(
"Failed to insert records for path " + path, io);
throw new HoodieInsertException("Failed to insert records for path " + path, io);
}
}
@@ -151,8 +149,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
*/
@Override
public WriteStatus close() {
logger.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records "
+ recordsWritten);
logger
.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten);
try {
storageWriter.close();
@@ -174,8 +172,8 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload> extends HoodieWri
stat.setRuntimeStats(runtimeStats);
writeStatus.setStat(stat);
logger.info(String.format("CreateHandle for partitionPath %s fileID %s, took %d ms.",
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalCreateTime()));
logger.info(String.format("CreateHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(),
stat.getFileId(), runtimeStats.getTotalCreateTime()));
return writeStatus;
} catch (IOException e) {

View File

@@ -67,15 +67,15 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends Hoodie
/**
* Given a list of row keys and one file, return only row keys existing in that file.
*/
public static List<String> checkCandidatesAgainstFile(Configuration configuration,
List<String> candidateRecordKeys, Path filePath) throws HoodieIndexException {
public static List<String> checkCandidatesAgainstFile(Configuration configuration, List<String> candidateRecordKeys,
Path filePath) throws HoodieIndexException {
List<String> foundRecordKeys = new ArrayList<>();
try {
// Load all rowKeys from the file, to double-confirm
if (!candidateRecordKeys.isEmpty()) {
HoodieTimer timer = new HoodieTimer().startTimer();
Set<String> fileRowKeys = ParquetUtils.filterParquetRowKeys(configuration, filePath,
new HashSet<>(candidateRecordKeys));
Set<String> fileRowKeys =
ParquetUtils.filterParquetRowKeys(configuration, filePath, new HashSet<>(candidateRecordKeys));
foundRecordKeys.addAll(fileRowKeys);
logger.info(String.format("Checked keys against file %s, in %d ms. #candidates (%d) #found (%d)", filePath,
timer.endTimer(), candidateRecordKeys.size(), foundRecordKeys.size()));
@@ -112,11 +112,11 @@ public class HoodieKeyLookupHandle<T extends HoodieRecordPayload> extends Hoodie
}
HoodieDataFile dataFile = getLatestDataFile();
List<String> matchingKeys = checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys,
new Path(dataFile.getPath()));
logger.info(String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)",
totalKeysChecked, candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(),
matchingKeys.size()));
List<String> matchingKeys =
checkCandidatesAgainstFile(hoodieTable.getHadoopConf(), candidateRecordKeys, new Path(dataFile.getPath()));
logger.info(
String.format("Total records (%d), bloom filter candidates (%d)/fp(%d), actual matches (%d)", totalKeysChecked,
candidateRecordKeys.size(), candidateRecordKeys.size() - matchingKeys.size(), matchingKeys.size()));
return new KeyLookupResult(partitionPathFilePair.getRight(), partitionPathFilePair.getLeft(),
dataFile.getCommitTime(), matchingKeys);
}

View File

@@ -71,8 +71,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
Iterator<HoodieRecord<T>> recordItr, String fileId) {
super(config, commitTime, fileId, hoodieTable);
String partitionPath = init(fileId, recordItr);
init(fileId, partitionPath,
hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get());
init(fileId, partitionPath, hoodieTable.getROFileSystemView().getLatestDataFile(partitionPath, fileId).get());
}
/**
@@ -83,8 +82,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
super(config, commitTime, fileId, hoodieTable);
this.keyToNewRecords = keyToNewRecords;
this.useWriterSchema = true;
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get())
.getPartitionPath(), dataFileToBeMerged);
init(fileId, keyToNewRecords.get(keyToNewRecords.keySet().stream().findFirst().get()).getPartitionPath(),
dataFileToBeMerged);
}
@@ -160,15 +159,13 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
new Path(config.getBasePath()), FSUtils.getPartitionPath(config.getBasePath(), partitionPath));
partitionMetadata.trySave(TaskContext.getPartitionId());
oldFilePath = new Path(
config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
oldFilePath = new Path(config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
+ FSUtils.makeDataFileName(instantTime, writeToken, 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()));
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(partitionPath);
@@ -180,14 +177,13 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
createMarkerFile(partitionPath);
// Create the writer for writing the new version file
storageWriter = HoodieStorageWriterFactory
.getStorageWriter(instantTime, newFilePath, hoodieTable, config, writerSchema);
storageWriter =
HoodieStorageWriterFactory.getStorageWriter(instantTime, newFilePath, hoodieTable, config, writerSchema);
} catch (IOException io) {
logger.error("Error in update task at commit " + instantTime, io);
writeStatus.setGlobalError(io);
throw new HoodieUpsertException(
"Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ instantTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io);
throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit "
+ instantTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io);
}
}
@@ -211,16 +207,14 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
record.unseal();
record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
record.seal();
//NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
// NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
keyToNewRecords.put(record.getRecordKey(), record);
}
logger.info("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).getCurrentInMemoryMapSize() + "Number of entries in DiskBasedMap => "
+ ((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries() + "Size of file spilled to disk => "
+ ((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes());
return partitionPath;
}
@@ -258,8 +252,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
}
/**
* 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();
@@ -269,12 +262,12 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
// writing the first record. So make a copy of the record to be merged
HoodieRecord<T> hoodieRecord = new HoodieRecord<>(keyToNewRecords.get(key));
try {
Option<IndexedRecord> combinedAvroRecord = hoodieRecord.getData()
.combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
Option<IndexedRecord> combinedAvroRecord =
hoodieRecord.getData().combineAndGetUpdateValue(oldRecord, useWriterSchema ? writerSchema : originalSchema);
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
/*
* 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.
*/
@@ -282,26 +275,24 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
}
writtenRecordKeys.add(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);
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;
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 writerSchema " + writerSchema
.toString(true));
logger.error("Schema mismatch when rewriting old record " + oldRecord + " from file " + getOldFilePath()
+ " to file " + newFilePath + " with writerSchema " + writerSchema.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);
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++;
@@ -344,8 +335,8 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieWrit
runtimeStats.setTotalUpsertTime(timer.endTimer());
stat.setRuntimeStats(runtimeStats);
logger.info(String.format("MergeHandle for partitionPath %s fileID %s, took %d ms.",
stat.getPartitionPath(), stat.getFileId(), runtimeStats.getTotalUpsertTime()));
logger.info(String.format("MergeHandle for partitionPath %s fileID %s, took %d ms.", stat.getPartitionPath(),
stat.getFileId(), runtimeStats.getTotalUpsertTime()));
return writeStatus;
} catch (IOException e) {

View File

@@ -61,8 +61,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
this.writerSchema = createHoodieWriteSchema(originalSchema);
this.timer = new HoodieTimer().startTimer();
this.writeStatus = (WriteStatus) ReflectionUtils.loadClass(config.getWriteStatusClassName(),
!hoodieTable.getIndex().isImplicitWithStorage(),
config.getWriteStatusFailureFraction());
!hoodieTable.getIndex().isImplicitWithStorage(), config.getWriteStatusFailureFraction());
}
/**
@@ -104,7 +103,7 @@ public abstract class HoodieWriteHandle<T extends HoodieRecordPayload> extends H
}
/**
* THe marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename
* THe marker path will be <base-path>/.hoodie/.temp/<instant_ts>/2019/04/25/filename
*/
private Path makeNewMarkerPath(String partitionPath) {
Path markerRootPath = new Path(hoodieTable.getMetaClient().getMarkerFolderPath(instantTime));

View File

@@ -37,23 +37,20 @@ public interface HoodieCompactor extends Serializable {
/**
* Generate a new compaction plan for scheduling
*
* @param jsc Spark Context
* @param hoodieTable Hoodie Table
* @param config Hoodie Write Configuration
* @param jsc Spark Context
* @param hoodieTable Hoodie Table
* @param config Hoodie Write Configuration
* @param compactionCommitTime scheduled compaction commit time
* @param fgIdsInPendingCompactions partition-fileId pairs for which compaction is pending
* @return Compaction Plan
* @throws IOException when encountering errors
*/
HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
Set<HoodieFileGroupId> fgIdsInPendingCompactions)
throws IOException;
HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException;
/**
* Execute compaction operations and report back status
*/
JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionInstantTime) throws IOException;
JavaRDD<WriteStatus> compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable,
HoodieWriteConfig config, String compactionInstantTime) throws IOException;
}

View File

@@ -63,9 +63,9 @@ import org.apache.spark.util.AccumulatorV2;
import org.apache.spark.util.LongAccumulator;
/**
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all
* possible compactions, passes it through a CompactionFilter and executes all the compactions and
* writes a new version of base files and make a normal commit
* HoodieRealtimeTableCompactor compacts a hoodie table with merge on read storage. Computes all possible compactions,
* passes it through a CompactionFilter and executes all the compactions and writes a new version of base files and make
* a normal commit
*
* @see HoodieCompactor
*/
@@ -78,9 +78,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
private AccumulatorV2<Long, Long> totalFileSlices;
@Override
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc,
HoodieCompactionPlan compactionPlan, HoodieTable hoodieTable, HoodieWriteConfig config,
String compactionInstantTime) throws IOException {
public JavaRDD<WriteStatus> compact(JavaSparkContext jsc, HoodieCompactionPlan compactionPlan,
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionInstantTime) throws IOException {
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|| (compactionPlan.getOperations().isEmpty())) {
return jsc.emptyRDD();
@@ -88,41 +87,36 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
// Compacting is very similar to applying updates to existing file
HoodieCopyOnWriteTable table = new HoodieCopyOnWriteTable(config, jsc);
List<CompactionOperation> operations = compactionPlan.getOperations().stream().map(
CompactionOperation::convertFromAvroRecordInstance).collect(toList());
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
log.info("Compactor compacting " + operations + " files");
return jsc.parallelize(operations, operations.size())
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
.flatMap(List::iterator);
.map(s -> compact(table, metaClient, config, s, compactionInstantTime)).flatMap(List::iterator);
}
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
HoodieWriteConfig config,
CompactionOperation operation, String commitTime) throws IOException {
HoodieWriteConfig config, CompactionOperation operation, String commitTime) throws IOException {
FileSystem fs = metaClient.getFs();
Schema readerSchema = HoodieAvroUtils
.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(config.getSchema()));
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation
.getDeltaFilePaths() + " for commit " + commitTime);
log.info("Compacting base " + operation.getDataFilePath() + " with delta files " + operation.getDeltaFilePaths()
+ " for commit " + commitTime);
// TODO - FIX THIS
// Reads the entire avro file. Always only specific blocks should be read from the avro file
// (failure recover).
// Load all the delta commits since the last compaction commit and get all the blocks to be
// loaded and load it using CompositeAvroLogReader
// Since a DeltaCommit is not defined yet, reading all the records. revisit this soon.
String maxInstantTime = metaClient.getActiveTimeline()
.getTimelineOfActions(
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
HoodieTimeline.DELTA_COMMIT_ACTION))
String maxInstantTime = metaClient
.getActiveTimeline().getTimelineOfActions(Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
log.info("MaxMemoryPerCompaction => " + config.getMaxMemoryPerCompaction());
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs,
metaClient.getBasePath(), operation.getDeltaFilePaths(), readerSchema, maxInstantTime,
config.getMaxMemoryPerCompaction(), config.getCompactionLazyBlockReadEnabled(),
config.getCompactionReverseLogReadEnabled(), config.getMaxDFSStreamBufferSize(),
config.getSpillableMapBasePath());
HoodieMergedLogRecordScanner scanner = new HoodieMergedLogRecordScanner(fs, metaClient.getBasePath(),
operation.getDeltaFilePaths(), readerSchema, maxInstantTime, config.getMaxMemoryPerCompaction(),
config.getCompactionLazyBlockReadEnabled(), config.getCompactionReverseLogReadEnabled(),
config.getMaxDFSStreamBufferSize(), config.getSpillableMapBasePath());
if (!scanner.iterator().hasNext()) {
return Lists.<WriteStatus>newArrayList();
}
@@ -134,53 +128,49 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// If the dataFile is present, there is a base parquet file present, perform updates else perform inserts into a
// new base parquet file.
if (oldDataFileOpt.isPresent()) {
result = hoodieCopyOnWriteTable
.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(), oldDataFileOpt.get());
result = hoodieCopyOnWriteTable.handleUpdate(commitTime, operation.getFileId(), scanner.getRecords(),
oldDataFileOpt.get());
} else {
result = hoodieCopyOnWriteTable
.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(), scanner.iterator());
result = hoodieCopyOnWriteTable.handleInsert(commitTime, operation.getPartitionPath(), operation.getFileId(),
scanner.iterator());
}
Iterable<List<WriteStatus>> resultIterable = () -> result;
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
.peek(s -> {
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
s.getStat().setPartitionPath(operation.getPartitionPath());
s.getStat().setTotalLogSizeCompacted(operation.getMetrics().get(
CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
s.getStat().setRuntimeStats(runtimeStats);
}).collect(toList());
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream).peek(s -> {
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
s.getStat().setPartitionPath(operation.getPartitionPath());
s.getStat()
.setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue());
s.getStat().setTotalLogBlocks(scanner.getTotalLogBlocks());
s.getStat().setTotalCorruptLogBlock(scanner.getTotalCorruptBlocks());
s.getStat().setTotalRollbackBlocks(scanner.getTotalRollbacks());
RuntimeStats runtimeStats = new RuntimeStats();
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
s.getStat().setRuntimeStats(runtimeStats);
}).collect(toList());
}
@Override
public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc,
HoodieTable hoodieTable, HoodieWriteConfig config, String compactionCommitTime,
Set<HoodieFileGroupId> fgIdsInPendingCompactions) throws IOException {
public HoodieCompactionPlan generateCompactionPlan(JavaSparkContext jsc, HoodieTable hoodieTable,
HoodieWriteConfig config, String compactionCommitTime, Set<HoodieFileGroupId> fgIdsInPendingCompactions)
throws IOException {
totalLogFiles = new LongAccumulator();
totalFileSlices = new LongAccumulator();
jsc.sc().register(totalLogFiles);
jsc.sc().register(totalFileSlices);
Preconditions
.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
"HoodieRealtimeTableCompactor can only compact table of type "
+ HoodieTableType.MERGE_ON_READ + " and not " + hoodieTable.getMetaClient()
.getTableType().name());
Preconditions.checkArgument(hoodieTable.getMetaClient().getTableType() == HoodieTableType.MERGE_ON_READ,
"HoodieRealtimeTableCompactor can only compact table of type " + HoodieTableType.MERGE_ON_READ + " and not "
+ hoodieTable.getMetaClient().getTableType().name());
//TODO : check if maxMemory is not greater than JVM or spark.executor memory
// TODO : check if maxMemory is not greater than JVM or spark.executor memory
// TODO - rollback any compactions in flight
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
log.info("Compacting " + metaClient.getBasePath() + " with commit " + compactionCommitTime);
List<String> partitionPaths = FSUtils
.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
List<String> partitionPaths = FSUtils.getAllPartitionPaths(metaClient.getFs(), metaClient.getBasePath(),
config.shouldAssumeDatePartitioning());
// filter the partition paths if needed to reduce list status
partitionPaths = config.getCompactionStrategy().filterPartitionPaths(config, partitionPaths);
@@ -192,28 +182,22 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
RealtimeView fileSystemView = hoodieTable.getRTFileSystemView();
log.info("Compaction looking for files to compact in " + partitionPaths + " partitions");
List<HoodieCompactionOperation> operations =
jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
.getLatestFileSlices(partitionPath)
.filter(slice ->
!fgIdsInPendingCompactions.contains(slice.getFileGroupId()))
.map(
s -> {
List<HoodieLogFile> logFiles = s.getLogFiles().sorted(HoodieLogFile
.getLogFileComparator()).collect(Collectors.toList());
totalLogFiles.add((long) logFiles.size());
totalFileSlices.add(1L);
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
// for spark Map operations and collecting them finally in Avro generated classes for storing
// into meta files.
Option<HoodieDataFile> dataFile = s.getDataFile();
return new CompactionOperation(dataFile, partitionPath, logFiles,
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
})
.filter(c -> !c.getDeltaFilePaths().isEmpty())
.collect(toList()).iterator()).collect().stream().map(CompactionUtils::buildHoodieCompactionOperation)
.collect(toList());
List<HoodieCompactionOperation> operations = jsc.parallelize(partitionPaths, partitionPaths.size())
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
.getLatestFileSlices(partitionPath)
.filter(slice -> !fgIdsInPendingCompactions.contains(slice.getFileGroupId())).map(s -> {
List<HoodieLogFile> logFiles =
s.getLogFiles().sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
totalLogFiles.add((long) logFiles.size());
totalFileSlices.add(1L);
// Avro generated classes are not inheriting Serializable. Using CompactionOperation POJO
// for spark Map operations and collecting them finally in Avro generated classes for storing
// into meta files.
Option<HoodieDataFile> dataFile = s.getDataFile();
return new CompactionOperation(dataFile, partitionPath, logFiles,
config.getCompactionStrategy().captureMetrics(config, dataFile, partitionPath, logFiles));
}).filter(c -> !c.getDeltaFilePaths().isEmpty()).collect(toList()).iterator())
.collect().stream().map(CompactionUtils::buildHoodieCompactionOperation).collect(toList());
log.info("Total of " + operations.size() + " compactions are retrieved");
log.info("Total number of latest files slices " + totalFileSlices.value());
log.info("Total number of log files " + totalLogFiles.value());
@@ -222,11 +206,11 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
// compactions only
HoodieCompactionPlan compactionPlan = config.getCompactionStrategy().generateCompactionPlan(config, operations,
CompactionUtils.getAllPendingCompactionPlans(metaClient).stream().map(Pair::getValue).collect(toList()));
Preconditions.checkArgument(compactionPlan.getOperations().stream().noneMatch(
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
Preconditions.checkArgument(
compactionPlan.getOperations().stream().noneMatch(
op -> fgIdsInPendingCompactions.contains(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()))),
"Bad Compaction Plan. FileId MUST NOT have multiple pending compactions. "
+ "Please fix your strategy implementation."
+ "FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions
+ "Please fix your strategy implementation." + "FileIdsWithPendingCompactions :" + fgIdsInPendingCompactions
+ ", Selected workload :" + compactionPlan);
if (compactionPlan.getOperations().isEmpty()) {
log.warn("After filtering, Nothing to compact for " + metaClient.getBasePath());

View File

@@ -25,8 +25,8 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.config.HoodieWriteConfig;
/**
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and
* limits the list of compactions to be under a configured limit on the IO
* CompactionStrategy which looks at total IO to be done for the compaction (read + write) and limits the list of
* compactions to be under a configured limit on the IO
*
* @see CompactionStrategy
*/

View File

@@ -32,11 +32,10 @@ import org.apache.hudi.config.HoodieWriteConfig;
/**
* This strategy ensures that the last N partitions are picked up even if there are later partitions created for the
* dataset. lastNPartitions is defined as the N partitions before the currentDate.
* currentDay = 2018/01/01
* The dataset has partitions for 2018/02/02 and 2018/03/03 beyond the currentDay
* This strategy will pick up the following partitions for compaction :
* (2018/01/01, allPartitionsInRange[(2018/01/01 - lastNPartitions) to 2018/01/01), 2018/02/02, 2018/03/03)
* dataset. lastNPartitions is defined as the N partitions before the currentDate. currentDay = 2018/01/01 The dataset
* has partitions for 2018/02/02 and 2018/03/03 beyond the currentDay This strategy will pick up the following
* partitions for compaction : (2018/01/01, allPartitionsInRange[(2018/01/01 - lastNPartitions) to 2018/01/01),
* 2018/02/02, 2018/03/03)
*/
public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionStrategy {
@@ -46,15 +45,14 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
// The earliest partition to compact - current day minus the target partitions limit
String earliestPartitionPathToCompact = dateFormat.format(
getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
String earliestPartitionPathToCompact =
dateFormat.format(getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
// Filter out all partitions greater than earliestPartitionPathToCompact
List<HoodieCompactionOperation> eligibleCompactionOperations = operations.stream()
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
.sorted(Map.Entry.comparingByKey(comparator))
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0)
.flatMap(e -> e.getValue().stream())
.collect(Collectors.toList());
List<HoodieCompactionOperation> eligibleCompactionOperations =
operations.stream().collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet()
.stream().sorted(Map.Entry.comparingByKey(comparator))
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e.getKey()) >= 0)
.flatMap(e -> e.getValue().stream()).collect(Collectors.toList());
return eligibleCompactionOperations;
}
@@ -62,13 +60,12 @@ public class BoundedPartitionAwareCompactionStrategy extends DayBasedCompactionS
@Override
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
// The earliest partition to compact - current day minus the target partitions limit
String earliestPartitionPathToCompact = dateFormat.format(
getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
String earliestPartitionPathToCompact =
dateFormat.format(getDateAtOffsetFromToday(-1 * writeConfig.getTargetPartitionsPerDayBasedCompaction()));
// Get all partitions and sort them
List<String> filteredPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e) >= 0)
.collect(Collectors.toList());
.filter(e -> comparator.compare(earliestPartitionPathToCompact, e) >= 0).collect(Collectors.toList());
return filteredPartitionPaths;
}

View File

@@ -32,11 +32,10 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.compact.HoodieRealtimeTableCompactor;
/**
* Strategy for compaction. Pluggable implementation to define how compaction should be done. The
* over-ridden implementations of this abstract class can capture the relevant metrics to order
* and filter the final list of compaction operation to run in a single compaction.
* Implementation of CompactionStrategy cannot hold any state. Difference instantiations can be
* passed in every time
* Strategy for compaction. Pluggable implementation to define how compaction should be done. The over-ridden
* implementations of this abstract class can capture the relevant metrics to order and filter the final list of
* compaction operation to run in a single compaction. Implementation of CompactionStrategy cannot hold any state.
* Difference instantiations can be passed in every time
*
* @see HoodieRealtimeTableCompactor
*/
@@ -49,8 +48,8 @@ public abstract class CompactionStrategy implements Serializable {
public static final String TOTAL_LOG_FILES = "TOTAL_LOG_FILES";
/**
* Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the
* metrics they need to decide on the priority.
* Callback hook when a HoodieCompactionOperation is created. Individual strategies can capture the metrics they need
* to decide on the priority.
*
* @param dataFile - Base file to compact
* @param partitionPath - Partition path
@@ -58,18 +57,18 @@ public abstract class CompactionStrategy implements Serializable {
* @return Map[String, Object] - metrics captured
*/
public Map<String, Double> captureMetrics(HoodieWriteConfig writeConfig, Option<HoodieDataFile> dataFile,
String partitionPath, List<HoodieLogFile> logFiles) {
String partitionPath, List<HoodieLogFile> logFiles) {
Map<String, Double> metrics = Maps.newHashMap();
Long defaultMaxParquetFileSize = writeConfig.getParquetMaxFileSize();
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
.reduce((size1, size2) -> size1 + size2).orElse(0L);
// Total read will be the base file + all the log files
Long totalIORead = FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L)
+ totalLogFileSize);
Long totalIORead =
FSUtils.getSizeInMB((dataFile.isPresent() ? dataFile.get().getFileSize() : 0L) + totalLogFileSize);
// Total write will be similar to the size of the base file
Long totalIOWrite = FSUtils
.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
Long totalIOWrite =
FSUtils.getSizeInMB(dataFile.isPresent() ? dataFile.get().getFileSize() : defaultMaxParquetFileSize);
// Total IO will the the IO for read + write
Long totalIO = totalIORead + totalIOWrite;
// Save these metrics and we will use during the filter
@@ -86,8 +85,8 @@ public abstract class CompactionStrategy implements Serializable {
* implementation takes care of setting compactor Id from configuration allowing subclasses to only worry about
* ordering and filtering compaction operations
*
* @param writeConfig Hoodie Write Config
* @param operations Compaction Operations to be ordered and filtered
* @param writeConfig Hoodie Write Config
* @param operations Compaction Operations to be ordered and filtered
* @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan
* @return Compaction plan to be scheduled.
*/
@@ -95,27 +94,26 @@ public abstract class CompactionStrategy implements Serializable {
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
// Strategy implementation can overload this method to set specific compactor-id
return HoodieCompactionPlan.newBuilder()
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans))
.build();
.setOperations(orderAndFilter(writeConfig, operations, pendingCompactionPlans)).build();
}
/**
* Order and Filter the list of compactions. Use the metrics captured with the captureMetrics to order and filter out
* compactions
*
* @param writeConfig config for this compaction is passed in
* @param operations list of compactions collected
* @param writeConfig config for this compaction is passed in
* @param operations list of compactions collected
* @param pendingCompactionPlans Pending Compaction Plans for strategy to schedule next compaction plan
* @return list of compactions to perform in this run
*/
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<HoodieCompactionOperation> operations,
List<HoodieCompactionPlan> pendingCompactionPlans) {
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
return operations;
}
/**
* Filter the partition paths based on compaction strategy
*
* @param writeConfig
* @param allPartitionPaths
* @return

View File

@@ -34,21 +34,18 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
/**
* This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to
* compact data in latest partitions first and then older capped at the Total_IO allowed.
* This strategy orders compactions in reverse order of creation of Hive Partitions. It helps to compact data in latest
* partitions first and then older capped at the Total_IO allowed.
*/
public class DayBasedCompactionStrategy extends CompactionStrategy {
// For now, use SimpleDateFormat as default partition format
protected static String datePartitionFormat = "yyyy/MM/dd";
// Sorts compaction in LastInFirstCompacted order
protected static Comparator<String> comparator = (String leftPartition,
String rightPartition) -> {
protected static Comparator<String> comparator = (String leftPartition, String rightPartition) -> {
try {
Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
.parse(leftPartition);
Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH)
.parse(rightPartition);
Date left = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH).parse(leftPartition);
Date right = new SimpleDateFormat(datePartitionFormat, Locale.ENGLISH).parse(rightPartition);
return left.after(right) ? -1 : right.after(left) ? 1 : 0;
} catch (ParseException e) {
throw new HoodieException("Invalid Partition Date Format", e);
@@ -68,8 +65,7 @@ public class DayBasedCompactionStrategy extends CompactionStrategy {
List<HoodieCompactionOperation> filteredList = operations.stream()
.collect(Collectors.groupingBy(HoodieCompactionOperation::getPartitionPath)).entrySet().stream()
.sorted(Map.Entry.comparingByKey(comparator)).limit(writeConfig.getTargetPartitionsPerDayBasedCompaction())
.flatMap(e -> e.getValue().stream())
.collect(Collectors.toList());
.flatMap(e -> e.getValue().stream()).collect(Collectors.toList());
return filteredList;
}

View File

@@ -30,14 +30,14 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
/**
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and
* limits the compactions within a configured IO bound
* LogFileSizeBasedCompactionStrategy orders the compactions based on the total log files size and limits the
* compactions within a configured IO bound
*
* @see BoundedIOCompactionStrategy
* @see CompactionStrategy
*/
public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy implements
Comparator<HoodieCompactionOperation> {
public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrategy
implements Comparator<HoodieCompactionOperation> {
private static final String TOTAL_LOG_FILE_SIZE = "TOTAL_LOG_FILE_SIZE";
@@ -47,9 +47,8 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
Map<String, Double> metrics = super.captureMetrics(config, dataFile, partitionPath, logFiles);
// Total size of all the log files
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize)
.filter(size -> size >= 0).reduce((size1, size2) -> size1 + size2)
.orElse(0L);
Long totalLogFileSize = logFiles.stream().map(HoodieLogFile::getFileSize).filter(size -> size >= 0)
.reduce((size1, size2) -> size1 + size2).orElse(0L);
// save the metrics needed during the order
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
return metrics;
@@ -59,9 +58,8 @@ public class LogFileSizeBasedCompactionStrategy extends BoundedIOCompactionStrat
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig writeConfig,
List<HoodieCompactionOperation> operations, List<HoodieCompactionPlan> pendingCompactionPlans) {
// Order the operations based on the reverse size of the logs and limit them by the IO
return super
.orderAndFilter(writeConfig,
operations.stream().sorted(this).collect(Collectors.toList()), pendingCompactionPlans);
return super.orderAndFilter(writeConfig, operations.stream().sorted(this).collect(Collectors.toList()),
pendingCompactionPlans);
}
@Override

View File

@@ -24,9 +24,8 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.config.HoodieWriteConfig;
/**
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a
* pass-through and will compact all the base files which has a log file. This usually means
* no-intelligence on compaction.
* UnBoundedCompactionStrategy will not change ordering or filter any compaction. It is a pass-through and will compact
* all the base files which has a log file. This usually means no-intelligence on compaction.
*
* @see CompactionStrategy
*/

View File

@@ -27,12 +27,11 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.config.HoodieWriteConfig;
/**
* UnBoundedPartitionAwareCompactionStrategy is a custom UnBounded Strategy.
* This will filter all the partitions that are eligible to be compacted by a
* {@link BoundedPartitionAwareCompactionStrategy} and return the result.
* This is done so that a long running UnBoundedPartitionAwareCompactionStrategy does not step over partitions
* in a shorter running BoundedPartitionAwareCompactionStrategy. Essentially, this is an inverse of the
* partitions chosen in BoundedPartitionAwareCompactionStrategy
* UnBoundedPartitionAwareCompactionStrategy is a custom UnBounded Strategy. This will filter all the partitions that
* are eligible to be compacted by a {@link BoundedPartitionAwareCompactionStrategy} and return the result. This is done
* so that a long running UnBoundedPartitionAwareCompactionStrategy does not step over partitions in a shorter running
* BoundedPartitionAwareCompactionStrategy. Essentially, this is an inverse of the partitions chosen in
* BoundedPartitionAwareCompactionStrategy
*
* @see CompactionStrategy
*/
@@ -41,10 +40,10 @@ public class UnBoundedPartitionAwareCompactionStrategy extends CompactionStrateg
@Override
public List<HoodieCompactionOperation> orderAndFilter(HoodieWriteConfig config,
final List<HoodieCompactionOperation> operations, final List<HoodieCompactionPlan> pendingCompactionWorkloads) {
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy
= new BoundedPartitionAwareCompactionStrategy();
List<HoodieCompactionOperation> operationsToExclude = boundedPartitionAwareCompactionStrategy
.orderAndFilter(config, operations, pendingCompactionWorkloads);
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy =
new BoundedPartitionAwareCompactionStrategy();
List<HoodieCompactionOperation> operationsToExclude =
boundedPartitionAwareCompactionStrategy.orderAndFilter(config, operations, pendingCompactionWorkloads);
List<HoodieCompactionOperation> allOperations = new ArrayList<>(operations);
allOperations.removeAll(operationsToExclude);
return allOperations;
@@ -52,13 +51,13 @@ public class UnBoundedPartitionAwareCompactionStrategy extends CompactionStrateg
@Override
public List<String> filterPartitionPaths(HoodieWriteConfig writeConfig, List<String> partitionPaths) {
List<String> allPartitionPaths = partitionPaths.stream().map(partition -> partition.replace("/", "-"))
.sorted(Comparator.reverseOrder()).map(partitionPath -> partitionPath.replace("-", "/"))
.collect(Collectors.toList());
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy
= new BoundedPartitionAwareCompactionStrategy();
List<String> partitionsToExclude = boundedPartitionAwareCompactionStrategy.filterPartitionPaths(writeConfig,
partitionPaths);
List<String> allPartitionPaths =
partitionPaths.stream().map(partition -> partition.replace("/", "-")).sorted(Comparator.reverseOrder())
.map(partitionPath -> partitionPath.replace("-", "/")).collect(Collectors.toList());
BoundedPartitionAwareCompactionStrategy boundedPartitionAwareCompactionStrategy =
new BoundedPartitionAwareCompactionStrategy();
List<String> partitionsToExclude =
boundedPartitionAwareCompactionStrategy.filterPartitionPaths(writeConfig, partitionPaths);
allPartitionPaths.removeAll(partitionsToExclude);
return allPartitionPaths;
}

View File

@@ -32,9 +32,8 @@ public class HoodieParquetConfig {
private Configuration hadoopConf;
private double compressionRatio;
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport,
CompressionCodecName compressionCodecName, int blockSize, int pageSize, long maxFileSize,
Configuration hadoopConf, double compressionRatio) {
public HoodieParquetConfig(HoodieAvroWriteSupport writeSupport, CompressionCodecName compressionCodecName,
int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf, double compressionRatio) {
this.writeSupport = writeSupport;
this.compressionCodecName = compressionCodecName;
this.blockSize = blockSize;

View File

@@ -36,11 +36,11 @@ import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.spark.TaskContext;
/**
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides
* a way to check if the current file can take more records with the <code>canWrite()</code>
* HoodieParquetWriter extends the ParquetWriter to help limit the size of underlying file. Provides a way to check if
* the current file can take more records with the <code>canWrite()</code>
*/
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord> extends
ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
extends ParquetWriter<IndexedRecord> implements HoodieStorageWriter<R> {
private static AtomicLong recordIndex = new AtomicLong(1);
@@ -52,24 +52,22 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
private final Schema schema;
public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig,
Schema schema) throws IOException {
public HoodieParquetWriter(String commitTime, Path file, HoodieParquetConfig parquetConfig, Schema schema)
throws IOException {
super(HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf()),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(),
parquetConfig.getCompressionCodecName(), parquetConfig.getBlockSize(),
parquetConfig.getPageSize(), parquetConfig.getPageSize(),
ParquetFileWriter.Mode.CREATE, parquetConfig.getWriteSupport(), parquetConfig.getCompressionCodecName(),
parquetConfig.getBlockSize(), parquetConfig.getPageSize(), parquetConfig.getPageSize(),
ParquetWriter.DEFAULT_IS_DICTIONARY_ENABLED, ParquetWriter.DEFAULT_IS_VALIDATING_ENABLED,
ParquetWriter.DEFAULT_WRITER_VERSION,
registerFileSystem(file, parquetConfig.getHadoopConf()));
ParquetWriter.DEFAULT_WRITER_VERSION, registerFileSystem(file, parquetConfig.getHadoopConf()));
this.file = HoodieWrapperFileSystem.convertToHoodiePath(file, parquetConfig.getHadoopConf());
this.fs = (HoodieWrapperFileSystem) this.file
.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
this.fs =
(HoodieWrapperFileSystem) this.file.getFileSystem(registerFileSystem(file, parquetConfig.getHadoopConf()));
// We cannot accurately measure the snappy compressed output file size. We are choosing a
// conservative 10%
// TODO - compute this compression ratio dynamically by looking at the bytes written to the
// stream and the actual file size reported by HDFS
this.maxFileSize = parquetConfig.getMaxFileSize() + Math
.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
this.maxFileSize = parquetConfig.getMaxFileSize()
+ Math.round(parquetConfig.getMaxFileSize() * parquetConfig.getCompressionRatio());
this.writeSupport = parquetConfig.getWriteSupport();
this.commitTime = commitTime;
this.schema = schema;
@@ -85,10 +83,10 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
@Override
public void writeAvroWithMetadata(R avroRecord, HoodieRecord record) throws IOException {
String seqId = HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(),
recordIndex.getAndIncrement());
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(),
record.getPartitionPath(), file.getName());
String seqId =
HoodieRecord.generateSequenceId(commitTime, TaskContext.getPartitionId(), recordIndex.getAndIncrement());
HoodieAvroUtils.addHoodieKeyToRecord((GenericRecord) avroRecord, record.getRecordKey(), record.getPartitionPath(),
file.getName());
HoodieAvroUtils.addCommitMetadataToRecord((GenericRecord) avroRecord, commitTime, seqId);
super.write(avroRecord);
writeSupport.add(record.getRecordKey());

View File

@@ -36,8 +36,8 @@ import org.apache.parquet.avro.AvroSchemaConverter;
public class HoodieStorageWriterFactory {
public static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> getStorageWriter(
String commitTime, Path path, HoodieTable<T> hoodieTable,
HoodieWriteConfig config, Schema schema) throws IOException {
String commitTime, Path path, HoodieTable<T> hoodieTable, HoodieWriteConfig config, Schema schema)
throws IOException {
final String name = path.getName();
final String extension = FSUtils.isLogFile(path) ? HOODIE_LOG.getFileExtension() : FSUtils.getFileExtension(name);
if (PARQUET.getFileExtension().equals(extension)) {
@@ -46,19 +46,16 @@ public class HoodieStorageWriterFactory {
throw new UnsupportedOperationException(extension + " format not supported yet.");
}
private static <T extends HoodieRecordPayload,
R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(String commitTime, Path path,
HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable) throws IOException {
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(),
config.getBloomFilterFPP());
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
new AvroSchemaConverter().convert(schema), schema, filter);
private static <T extends HoodieRecordPayload, R extends IndexedRecord> HoodieStorageWriter<R> newParquetStorageWriter(
String commitTime, Path path, HoodieWriteConfig config, Schema schema, HoodieTable hoodieTable)
throws IOException {
BloomFilter filter = new BloomFilter(config.getBloomFilterNumEntries(), config.getBloomFilterFPP());
HoodieAvroWriteSupport writeSupport =
new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema, filter);
HoodieParquetConfig parquetConfig =
new HoodieParquetConfig(writeSupport, config.getParquetCompressionCodec(),
config.getParquetBlockSize(), config.getParquetPageSize(),
config.getParquetMaxFileSize(), hoodieTable.getHadoopConf(),
config.getParquetCompressionRatio());
HoodieParquetConfig parquetConfig = new HoodieParquetConfig(writeSupport, config.getParquetCompressionCodec(),
config.getParquetBlockSize(), config.getParquetPageSize(), config.getParquetMaxFileSize(),
hoodieTable.getHadoopConf(), config.getParquetCompressionRatio());
return new HoodieParquetWriter<>(commitTime, path, parquetConfig, schema);
}

View File

@@ -118,8 +118,8 @@ public class HoodieMetrics {
return indexTimer == null ? null : indexTimer.time();
}
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs,
HoodieCommitMetadata metadata, String actionType) {
public void updateCommitMetrics(long commitEpochTimeInMs, long durationInMs, HoodieCommitMetadata metadata,
String actionType) {
if (config.isMetricsOn()) {
long totalPartitionsWritten = metadata.fetchTotalPartitionsWritten();
long totalFilesInsert = metadata.fetchTotalFilesInsert();
@@ -154,9 +154,8 @@ public class HoodieMetrics {
public void updateRollbackMetrics(long durationInMs, long numFilesDeleted) {
if (config.isMetricsOn()) {
logger.info(String
.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
numFilesDeleted));
logger.info(
String.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs, numFilesDeleted));
Metrics.registerGauge(getMetricsName("rollback", "duration"), durationInMs);
Metrics.registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
}
@@ -164,9 +163,8 @@ public class HoodieMetrics {
public void updateCleanMetrics(long durationInMs, int numFilesDeleted) {
if (config.isMetricsOn()) {
logger.info(String
.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
numFilesDeleted));
logger.info(
String.format("Sending clean metrics (duration=%d, numFilesDeleted=%d)", durationInMs, numFilesDeleted));
Metrics.registerGauge(getMetricsName("clean", "duration"), durationInMs);
Metrics.registerGauge(getMetricsName("clean", "numFilesDeleted"), numFilesDeleted);
}
@@ -174,20 +172,17 @@ public class HoodieMetrics {
public void updateFinalizeWriteMetrics(long durationInMs, long numFilesFinalized) {
if (config.isMetricsOn()) {
logger.info(String
.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)",
durationInMs, numFilesFinalized));
logger.info(String.format("Sending finalize write metrics (duration=%d, numFilesFinalized=%d)", durationInMs,
numFilesFinalized));
Metrics.registerGauge(getMetricsName("finalize", "duration"), durationInMs);
Metrics.registerGauge(getMetricsName("finalize", "numFilesFinalized"), numFilesFinalized);
}
}
public void updateIndexMetrics(final String action,final long durationInMs) {
public void updateIndexMetrics(final String action, final long durationInMs) {
if (config.isMetricsOn()) {
logger.info(String
.format("Sending index metrics (%s.duration, %d)",action, durationInMs));
Metrics.registerGauge(getMetricsName("index", String.format("%s.duration", action)),
durationInMs);
logger.info(String.format("Sending index metrics (%s.duration, %d)", action, durationInMs));
Metrics.registerGauge(getMetricsName("index", String.format("%s.duration", action)), durationInMs);
}
}
@@ -202,4 +197,4 @@ public class HoodieMetrics {
public long getDurationInMs(long ctxDuration) {
return ctxDuration / 1000000;
}
}
}

View File

@@ -26,12 +26,10 @@ import java.io.Closeable;
public class InMemoryMetricsReporter extends MetricsReporter {
@Override
public void start() {
}
public void start() {}
@Override
public void report() {
}
public void report() {}
@Override
public Closeable getReporter() {

View File

@@ -30,8 +30,7 @@ import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
/**
* Implementation of Graphite reporter, which connects to the Graphite server, and send metrics to
* that server.
* Implementation of Graphite reporter, which connects to the Graphite server, and send metrics to that server.
*/
public class MetricsGraphiteReporter extends MetricsReporter {
@@ -50,9 +49,8 @@ public class MetricsGraphiteReporter extends MetricsReporter {
this.serverHost = config.getGraphiteServerHost();
this.serverPort = config.getGraphiteServerPort();
if (serverHost == null || serverPort == 0) {
throw new RuntimeException(String
.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
serverHost, serverPort));
throw new RuntimeException(String.format("Graphite cannot be initialized with serverHost[%s] and serverPort[%s].",
serverHost, serverPort));
}
this.graphiteReporter = createGraphiteReport();
@@ -84,8 +82,7 @@ public class MetricsGraphiteReporter extends MetricsReporter {
private GraphiteReporter createGraphiteReport() {
Graphite graphite = new Graphite(new InetSocketAddress(serverHost, serverPort));
String reporterPrefix = config.getGraphiteMetricPrefix();
return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix)
.convertRatesTo(TimeUnit.SECONDS).convertDurationsTo(TimeUnit.MILLISECONDS)
.filter(MetricFilter.ALL).build(graphite);
return GraphiteReporter.forRegistry(registry).prefixedWith(reporterPrefix).convertRatesTo(TimeUnit.SECONDS)
.convertDurationsTo(TimeUnit.MILLISECONDS).filter(MetricFilter.ALL).build(graphite);
}
}

View File

@@ -19,8 +19,7 @@
package org.apache.hudi.metrics;
/**
* Types of the reporter. Right now we only support Graphite. We can include JMX and CSV in the
* future.
* Types of the reporter. Right now we only support Graphite. We can include JMX and CSV in the future.
*/
public enum MetricsReporterType {
GRAPHITE, INMEMORY

View File

@@ -82,8 +82,7 @@ import scala.Tuple2;
/**
* Implementation of a very heavily read-optimized Hoodie Table where
* <p>
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing
* file, to expand it
* INSERTS - Produce new files, block aligned to desired size (or) Merge with the smallest existing file, to expand it
* <p>
* UPDATES - Produce a new version of the file, just replacing the updated records with new values
*/
@@ -95,31 +94,28 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
super(config, jsc);
}
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String,
PartitionCleanStat> deleteFilesFunc(
private static PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat> deleteFilesFunc(
HoodieTable table) {
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>)
iter -> {
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
return (PairFlatMapFunction<Iterator<Tuple2<String, String>>, String, PartitionCleanStat>) iter -> {
Map<String, PartitionCleanStat> partitionCleanStatMap = new HashMap<>();
FileSystem fs = table.getMetaClient().getFs();
while (iter.hasNext()) {
Tuple2<String, String> partitionDelFileTuple = iter.next();
String partitionPath = partitionDelFileTuple._1();
String deletePathStr = partitionDelFileTuple._2();
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
if (!partitionCleanStatMap.containsKey(partitionPath)) {
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
}
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
}
FileSystem fs = table.getMetaClient().getFs();
while (iter.hasNext()) {
Tuple2<String, String> partitionDelFileTuple = iter.next();
String partitionPath = partitionDelFileTuple._1();
String deletePathStr = partitionDelFileTuple._2();
Boolean deletedFileResult = deleteFileAndGetResult(fs, deletePathStr);
if (!partitionCleanStatMap.containsKey(partitionPath)) {
partitionCleanStatMap.put(partitionPath, new PartitionCleanStat(partitionPath));
}
PartitionCleanStat partitionCleanStat = partitionCleanStatMap.get(partitionPath);
partitionCleanStat.addDeleteFilePatterns(deletePathStr);
partitionCleanStat.addDeletedFileResult(deletePathStr, deletedFileResult);
}
return partitionCleanStatMap.entrySet().stream()
.map(e -> new Tuple2<>(e.getKey(), e.getValue()))
.collect(Collectors.toList()).iterator();
};
return partitionCleanStatMap.entrySet().stream().map(e -> new Tuple2<>(e.getKey(), e.getValue()))
.collect(Collectors.toList()).iterator();
};
}
private static PairFlatMapFunction<String, String, String> getFilesToDeleteFunc(HoodieTable table,
@@ -131,8 +127,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
};
}
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr)
throws IOException {
private static Boolean deleteFileAndGetResult(FileSystem fs, String deletePathStr) throws IOException {
Path deletePath = new Path(deletePathStr);
logger.debug("Working on delete path :" + deletePath);
boolean deleteResult = fs.delete(deletePath, false);
@@ -171,8 +166,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
throw new HoodieNotSupportedException("Compaction is not supported from a CopyOnWrite table");
}
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId, Iterator<HoodieRecord<T>> recordItr)
throws IOException {
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
if (!recordItr.hasNext()) {
logger.info("Empty partition with fileId => " + fileId);
@@ -190,17 +185,16 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
return handleUpdateInternal(upsertHandle, commitTime, fileId);
}
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle,
String commitTime, String fileId)
throws IOException {
protected Iterator<List<WriteStatus>> handleUpdateInternal(HoodieMergeHandle upsertHandle, String commitTime,
String fileId) throws IOException {
if (upsertHandle.getOldFilePath() == null) {
throw new HoodieUpsertException(
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
} else {
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getWriterSchema());
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
try (ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
.withConf(getHadoopConf()).build()) {
try (ParquetReader<IndexedRecord> reader =
AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath()).withConf(getHadoopConf()).build()) {
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
new UpdateHandler(upsertHandle), x -> x);
wrapper.execute();
@@ -214,17 +208,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
}
//TODO(vc): This needs to be revisited
// TODO(vc): This needs to be revisited
if (upsertHandle.getWriteStatus().getPartitionPath() == null) {
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath()
+ ", " + upsertHandle.getWriteStatus());
logger.info("Upsert Handle has partition path as null " + upsertHandle.getOldFilePath() + ", "
+ upsertHandle.getWriteStatus());
}
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus()))
.iterator();
return Collections.singletonList(Collections.singletonList(upsertHandle.getWriteStatus())).iterator();
}
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) {
protected HoodieMergeHandle getUpdateHandle(String commitTime, String fileId, Iterator<HoodieRecord<T>> recordItr) {
return new HoodieMergeHandle<>(config, commitTime, this, recordItr, fileId);
}
@@ -233,8 +225,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
return new HoodieMergeHandle<>(config, commitTime, this, keyToNewRecords, fileId, dataFileToBeMerged);
}
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx, Iterator<HoodieRecord<T>> recordItr)
throws Exception {
// This is needed since sometimes some buckets are never picked in getPartition() and end up with 0 records
if (!recordItr.hasNext()) {
logger.info("Empty partition");
@@ -245,16 +237,16 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String partitionPath, String fileId,
Iterator<HoodieRecord<T>> recordItr) {
HoodieCreateHandle createHandle = new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId,
recordItr);
HoodieCreateHandle createHandle =
new HoodieCreateHandle(config, commitTime, this, partitionPath, fileId, recordItr);
createHandle.write();
return Collections.singletonList(Collections.singletonList(createHandle.close())).iterator();
}
@SuppressWarnings("unchecked")
@Override
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
Iterator recordItr, Partitioner partitioner) {
public Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition, Iterator recordItr,
Partitioner partitioner) {
UpsertPartitioner upsertPartitioner = (UpsertPartitioner) partitioner;
BucketInfo binfo = upsertPartitioner.getBucketInfo(partition);
BucketType btype = binfo.bucketType;
@@ -264,8 +256,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
} else if (btype.equals(BucketType.UPDATE)) {
return handleUpdate(commitTime, binfo.fileIdPrefix, recordItr);
} else {
throw new HoodieUpsertException(
"Unknown bucketType " + btype + " for partition :" + partition);
throw new HoodieUpsertException("Unknown bucketType " + btype + " for partition :" + partition);
}
} catch (Throwable t) {
String msg = "Error upserting bucketType " + btype + " for partition :" + partition;
@@ -275,15 +266,14 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
@Override
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
Iterator recordItr, Partitioner partitioner) {
public Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition, Iterator recordItr,
Partitioner partitioner) {
return handleUpsertPartition(commitTime, partition, recordItr, partitioner);
}
/**
* Performs cleaning of partition paths according to cleaning policy and returns the number of
* files cleaned. Handles skews in partitions to clean by making files to clean as the unit of
* task distribution.
* Performs cleaning of partition paths according to cleaning policy and returns the number of files cleaned. Handles
* skews in partitions to clean by making files to clean as the unit of task distribution.
*
* @throws IllegalArgumentException if unknown cleaning policy is provided
*/
@@ -291,11 +281,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
public List<HoodieCleanStat> clean(JavaSparkContext jsc) {
try {
FileSystem fs = getMetaClient().getFs();
List<String> partitionsToClean = FSUtils
.getAllPartitionPaths(fs, getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning());
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config
.getCleanerPolicy());
List<String> partitionsToClean =
FSUtils.getAllPartitionPaths(fs, getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning());
logger.info("Partitions to clean up : " + partitionsToClean + ", with policy " + config.getCleanerPolicy());
if (partitionsToClean.isEmpty()) {
logger.info("Nothing to clean here mom. It is already clean");
return Collections.emptyList();
@@ -307,12 +295,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
/**
* Common method used for cleaning out parquet files under a partition path during rollback of a
* set of commits
* Common method used for cleaning out parquet files under a partition path during rollback of a set of commits
*/
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String partitionPath,
PathFilter filter)
throws IOException {
PathFilter filter) throws IOException {
logger.info("Cleaning path " + partitionPath);
FileSystem fs = getMetaClient().getFs();
FileStatus[] toBeDeleted = fs.listStatus(FSUtils.getPartitionPath(config.getBasePath(), partitionPath), filter);
@@ -325,12 +311,10 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
/**
* Common method used for cleaning out parquet files under a partition path during rollback of a
* set of commits
* Common method used for cleaning out parquet files under a partition path during rollback of a set of commits
*/
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String commit, String
partitionPath)
throws IOException {
protected Map<FileStatus, Boolean> deleteCleanedFiles(Map<FileStatus, Boolean> results, String commit,
String partitionPath) throws IOException {
logger.info("Cleaning path " + partitionPath);
FileSystem fs = getMetaClient().getFs();
PathFilter filter = (path) -> {
@@ -354,8 +338,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
throws IOException {
String actionType = metaClient.getCommitActionType();
HoodieActiveTimeline activeTimeline = this.getActiveTimeline();
List<String> inflights = this.getInflightCommitTimeline().getInstants()
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
List<String> inflights =
this.getInflightCommitTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
// Atomically unpublish the commits
if (!inflights.contains(commit)) {
activeTimeline.revertToInflight(new HoodieInstant(false, actionType, commit));
@@ -364,27 +348,26 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
// delete all the data files for this commit
logger.info("Clean out all parquet files generated for commit: " + commit);
List<HoodieRollbackStat> stats = jsc.parallelize(FSUtils
.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
// Scan all partitions files with this commit time
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus).build();
}).collect();
List<HoodieRollbackStat> stats =
jsc.parallelize(FSUtils.getAllPartitionPaths(metaClient.getFs(), getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning())).map((Function<String, HoodieRollbackStat>) partitionPath -> {
// Scan all partitions files with this commit time
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus).build();
}).collect();
// Delete Inflight instant if enabled
deleteInflightInstant(deleteInstants, activeTimeline,
new HoodieInstant(true, actionType, commit));
deleteInflightInstant(deleteInstants, activeTimeline, new HoodieInstant(true, actionType, commit));
return stats;
}
/**
* Delete Inflight instant if enabled
*
* @param deleteInstant Enable Deletion of Inflight instant
* @param activeTimeline Hoodie active timeline
* @param activeTimeline Hoodie active timeline
* @param instantToBeDeleted Instant to be deleted
*/
protected void deleteInflightInstant(boolean deleteInstant, HoodieActiveTimeline activeTimeline,
@@ -401,30 +384,27 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
}
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean,
JavaSparkContext jsc) {
private List<HoodieCleanStat> cleanPartitionPaths(List<String> partitionsToClean, JavaSparkContext jsc) {
int cleanerParallelism = Math.min(partitionsToClean.size(), config.getCleanerParallelism());
logger.info("Using cleanerParallelism: " + cleanerParallelism);
List<Tuple2<String, PartitionCleanStat>> partitionCleanStats = jsc
.parallelize(partitionsToClean, cleanerParallelism)
.flatMapToPair(getFilesToDeleteFunc(this, config))
.repartition(cleanerParallelism) // repartition to remove skews
.parallelize(partitionsToClean, cleanerParallelism).flatMapToPair(getFilesToDeleteFunc(this, config))
.repartition(cleanerParallelism) // repartition to remove skews
.mapPartitionsToPair(deleteFilesFunc(this)).reduceByKey(
// merge partition level clean stats below
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1
.merge(e2)).collect();
(Function2<PartitionCleanStat, PartitionCleanStat, PartitionCleanStat>) (e1, e2) -> e1.merge(e2))
.collect();
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
Map<String, PartitionCleanStat> partitionCleanStatsMap =
partitionCleanStats.stream().collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
// Return PartitionCleanStat for each partition passed.
return partitionsToClean.stream().map(partitionPath -> {
PartitionCleanStat partitionCleanStat =
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap
.get(partitionPath) : new PartitionCleanStat(partitionPath);
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy())
.withPartitionPath(partitionPath)
(partitionCleanStatsMap.containsKey(partitionPath)) ? partitionCleanStatsMap.get(partitionPath)
: new PartitionCleanStat(partitionPath);
return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath)
.withEarliestCommitRetained(cleaner.getEarliestCommitToRetain())
.withDeletePathPattern(partitionCleanStat.deletePathPatterns)
.withSuccessfulDeletes(partitionCleanStat.successDeleteFiles)
@@ -453,8 +433,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
@Override
protected void finish() {
}
protected void finish() {}
@Override
protected Void getResult() {
@@ -487,8 +466,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
private PartitionCleanStat merge(PartitionCleanStat other) {
if (!this.partitionPath.equals(other.partitionPath)) {
throw new RuntimeException(String
.format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath));
throw new RuntimeException(
String.format("partitionPath is not a match: (%s, %s)", partitionPath, other.partitionPath));
}
successDeleteFiles.addAll(other.successDeleteFiles);
deletePathPatterns.addAll(other.deletePathPatterns);
@@ -516,8 +495,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
/**
* Helper class for an insert bucket along with the weight [0.0, 0.1] that defines the amount of
* incoming inserts that should be allocated to the bucket
* Helper class for an insert bucket along with the weight [0.0, 0.1] that defines the amount of incoming inserts that
* should be allocated to the bucket
*/
class InsertBucket implements Serializable {
@@ -563,8 +542,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
*/
List<SmallFile> smallFiles = new ArrayList<SmallFile>();
/**
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming
* workload into
* Total number of RDD partitions, is determined by total buckets we want to pack the incoming workload into
*/
private int totalBuckets = 0;
/**
@@ -599,17 +577,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
assignUpdates(profile);
assignInserts(profile);
logger.info(
"Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n"
+ "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n"
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
logger.info("Total Buckets :" + totalBuckets + ", " + "buckets info => " + bucketInfoMap + ", \n"
+ "Partition to insert buckets => " + partitionPathToInsertBuckets + ", \n"
+ "UpdateLocations mapped to buckets =>" + updateLocationToBucket);
}
private void assignUpdates(WorkloadProfile profile) {
// each update location gets a partition
WorkloadStat gStat = profile.getGlobalStat();
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount()
.entrySet()) {
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) {
addUpdateBucket(updateLocEntry.getKey());
}
}
@@ -628,8 +604,9 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
private void assignInserts(WorkloadProfile profile) {
// for new inserts, compute buckets depending on how many records we have for each partition
Set<String> partitionPaths = profile.getPartitionPaths();
long averageRecordSize = averageBytesPerRecord(metaClient.getActiveTimeline().getCommitTimeline()
.filterCompletedInstants(), config.getCopyOnWriteRecordSizeEstimate());
long averageRecordSize =
averageBytesPerRecord(metaClient.getActiveTimeline().getCommitTimeline().filterCompletedInstants(),
config.getCopyOnWriteRecordSizeEstimate());
logger.info("AvgRecordSize => " + averageRecordSize);
for (String partitionPath : partitionPaths) {
WorkloadStat pStat = profile.getWorkloadStat(partitionPath);
@@ -644,20 +621,17 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
// first try packing this into one of the smallFiles
for (SmallFile smallFile : smallFiles) {
long recordsToAppend = Math
.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize,
totalUnassignedInserts);
long recordsToAppend = Math.min((config.getParquetMaxFileSize() - smallFile.sizeBytes) / averageRecordSize,
totalUnassignedInserts);
if (recordsToAppend > 0 && totalUnassignedInserts > 0) {
// create a new bucket or re-use an existing bucket
int bucket;
if (updateLocationToBucket.containsKey(smallFile.location.getFileId())) {
bucket = updateLocationToBucket.get(smallFile.location.getFileId());
logger.info("Assigning " + recordsToAppend + " inserts to existing update bucket "
+ bucket);
logger.info("Assigning " + recordsToAppend + " inserts to existing update bucket " + bucket);
} else {
bucket = addUpdateBucket(smallFile.location.getFileId());
logger.info(
"Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
logger.info("Assigning " + recordsToAppend + " inserts to new update bucket " + bucket);
}
bucketNumbers.add(bucket);
recordsPerBucket.add(recordsToAppend);
@@ -673,10 +647,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
int insertBuckets = (int) Math.ceil((1.0 * totalUnassignedInserts) / insertRecordsPerBucket);
logger.info(
"After small file assignment: unassignedInserts => " + totalUnassignedInserts
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => "
+ insertRecordsPerBucket);
logger.info("After small file assignment: unassignedInserts => " + totalUnassignedInserts
+ ", totalInsertBuckets => " + insertBuckets + ", recordsPerBucket => " + insertRecordsPerBucket);
for (int b = 0; b < insertBuckets; b++) {
bucketNumbers.add(totalBuckets);
recordsPerBucket.add(totalUnassignedInserts / insertBuckets);
@@ -696,15 +668,14 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
bkt.weight = (1.0 * recordsPerBucket.get(i)) / pStat.getNumInserts();
insertBuckets.add(bkt);
}
logger.info(
"Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
logger.info("Total insert buckets for partition path " + partitionPath + " => " + insertBuckets);
partitionPathToInsertBuckets.put(partitionPath, insertBuckets);
}
}
}
/**
* Returns a list of small files in the given partition path
* Returns a list of small files in the given partition path
*/
protected List<SmallFile> getSmallFiles(String partitionPath) {
@@ -716,15 +687,13 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
if (!commitTimeline.empty()) { // if we have some commits
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
List<HoodieDataFile> allFiles = getROFileSystemView()
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp())
.collect(Collectors.toList());
.getLatestDataFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
for (HoodieDataFile file : allFiles) {
if (file.getFileSize() < config.getParquetSmallFileLimit()) {
String filename = file.getFileName();
SmallFile sf = new SmallFile();
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename),
FSUtils.getFileId(filename));
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
sf.sizeBytes = file.getFileSize();
smallFileLocations.add(sf);
// Update the global small files list
@@ -751,19 +720,18 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
@Override
public int getPartition(Object key) {
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation = (Tuple2<HoodieKey,
Option<HoodieRecordLocation>>) key;
Tuple2<HoodieKey, Option<HoodieRecordLocation>> keyLocation =
(Tuple2<HoodieKey, Option<HoodieRecordLocation>>) key;
if (keyLocation._2().isPresent()) {
HoodieRecordLocation location = keyLocation._2().get();
return updateLocationToBucket.get(location.getFileId());
} else {
List<InsertBucket> targetBuckets = partitionPathToInsertBuckets
.get(keyLocation._1().getPartitionPath());
List<InsertBucket> targetBuckets = partitionPathToInsertBuckets.get(keyLocation._1().getPartitionPath());
// pick the target bucket to use based on the weights.
double totalWeight = 0.0;
final long totalInserts = Math.max(1, globalStat.getNumInserts());
final long hashOfKey = Hashing.md5()
.hashString(keyLocation._1().getRecordKey(), StandardCharsets.UTF_8).asLong();
final long hashOfKey =
Hashing.md5().hashString(keyLocation._1().getRecordKey(), StandardCharsets.UTF_8).asLong();
final double r = 1.0 * Math.floorMod(hashOfKey, totalInserts) / totalInserts;
for (InsertBucket insertBucket : targetBuckets) {
totalWeight += insertBucket.weight;
@@ -782,8 +750,8 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
}
/**
* Obtains the average record size based on records written during previous commits. Used for
* estimating how many records pack into one file.
* Obtains the average record size based on records written during previous commits. Used for estimating how many
* records pack into one file.
*/
protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, int defaultRecordSizeEstimate) {
long avgSize = defaultRecordSizeEstimate;

View File

@@ -73,15 +73,21 @@ import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.api.java.function.Function;
/**
* Implementation of a more real-time read-optimized Hoodie Table where <p> INSERTS - Same as
* HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
* smallest existing file, to expand it </p> <p> UPDATES - Appends the changes to a rolling log file
* maintained per file Id. Compaction merges the log file into the base file. </p> <p> WARNING - MOR
* table type does not support nested rollbacks, every rollback must be followed by an attempted
* commit action </p>
* Implementation of a more real-time read-optimized Hoodie Table where
* <p>
* INSERTS - Same as HoodieCopyOnWriteTable - Produce new files, block aligned to desired size (or) Merge with the
* smallest existing file, to expand it
* </p>
* <p>
* UPDATES - Appends the changes to a rolling log file maintained per file Id. Compaction merges the log file into the
* base file.
* </p>
* <p>
* WARNING - MOR table type does not support nested rollbacks, every rollback must be followed by an attempted commit
* action
* </p>
*/
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
HoodieCopyOnWriteTable<T> {
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends HoodieCopyOnWriteTable<T> {
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
@@ -102,27 +108,24 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
@Override
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
Iterator<HoodieRecord<T>> recordItr) throws IOException {
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId, Iterator<HoodieRecord<T>> recordItr)
throws IOException {
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
if (!index.canIndexLogFiles() && mergeOnReadUpsertPartitioner.getSmallFileIds().contains(fileId)) {
logger.info(
"Small file corrections for updates for commit " + commitTime + " for file " + fileId);
logger.info("Small file corrections for updates for commit " + commitTime + " for file " + fileId);
return super.handleUpdate(commitTime, fileId, recordItr);
} else {
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this,
fileId, recordItr);
HoodieAppendHandle<T> appendHandle = new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
appendHandle.doAppend();
appendHandle.close();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus()))
.iterator();
return Collections.singletonList(Collections.singletonList(appendHandle.getWriteStatus())).iterator();
}
}
@Override
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx,
Iterator<HoodieRecord<T>> recordItr) throws Exception {
public Iterator<List<WriteStatus>> handleInsert(String commitTime, String idPfx, Iterator<HoodieRecord<T>> recordItr)
throws Exception {
// If canIndexLogFiles, write inserts to log files else write inserts to parquet files
if (index.canIndexLogFiles()) {
return new MergeOnReadLazyInsertIterable<>(recordItr, config, commitTime, this, idPfx);
@@ -134,8 +137,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
@Override
public HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime) {
logger.info("Checking if compaction needs to be run on " + config.getBasePath());
Option<HoodieInstant> lastCompaction = getActiveTimeline().getCommitTimeline()
.filterCompletedInstants().lastInstant();
Option<HoodieInstant> lastCompaction =
getActiveTimeline().getCommitTimeline().filterCompletedInstants().lastInstant();
String deltaCommitsSinceTs = "0";
if (lastCompaction.isPresent()) {
deltaCommitsSinceTs = lastCompaction.get().getTimestamp();
@@ -145,8 +148,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
.findInstantsAfter(deltaCommitsSinceTs, Integer.MAX_VALUE).countInstants();
if (config.getInlineCompactDeltaCommitMax() > deltaCommitsSinceLastCompaction) {
logger.info("Not running compaction as only " + deltaCommitsSinceLastCompaction
+ " delta commits was found since last compaction " + deltaCommitsSinceTs
+ ". Waiting for " + config.getInlineCompactDeltaCommitMax());
+ " delta commits was found since last compaction " + deltaCommitsSinceTs + ". Waiting for "
+ config.getInlineCompactDeltaCommitMax());
return new HoodieCompactionPlan();
}
@@ -154,7 +157,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
HoodieRealtimeTableCompactor compactor = new HoodieRealtimeTableCompactor();
try {
return compactor.generateCompactionPlan(jsc, this, config, instantTime,
((SyncableFileSystemView)getRTFileSystemView()).getPendingCompactionOperations()
((SyncableFileSystemView) getRTFileSystemView()).getPendingCompactionOperations()
.map(instantTimeCompactionopPair -> instantTimeCompactionopPair.getValue().getFileGroupId())
.collect(Collectors.toSet()));
@@ -186,9 +189,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// Atomically un-publish all non-inflight commits
Option<HoodieInstant> commitOrCompactionOption = Option.fromJavaOptional(this.getActiveTimeline()
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
.filter(i -> commit.equals(i.getTimestamp()))
.findFirst());
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION))
.getInstants().filter(i -> commit.equals(i.getTimestamp())).findFirst());
HoodieInstant instantToRollback = commitOrCompactionOption.get();
// Atomically un-publish all non-inflight commits
if (!instantToRollback.isInflight()) {
@@ -196,128 +198,134 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
logger.info("Unpublished " + commit);
Long startTime = System.currentTimeMillis();
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize(FSUtils
.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning()))
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
HoodieRollbackStat hoodieRollbackStats = null;
// Need to put the path filter here since Filter is not serializable
// PathFilter to get all parquet files and log files that need to be deleted
PathFilter filter = (path) -> {
if (path.toString().contains(".parquet")) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commit.equals(fileCommitTime);
} else if (path.toString().contains(".log")) {
// Since the baseCommitTime is the only commit for new log files, it's okay here
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
return commit.equals(fileCommitTime);
}
return false;
};
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
switch (instantToRollback.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
try {
// Rollback of a commit should delete the newly created parquet files along with any log
// files created with this as baseCommit. This is required to support multi-rollbacks in a MOR table.
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.COMPACTION_ACTION:
try {
// If there is no delta commit present after the current commit (if compaction), no action, else we
// need to make sure that a compaction commit rollback also deletes any log files written as part of the
// succeeding deltacommit.
boolean higherDeltaCommits = !activeTimeline.getDeltaCommitTimeline()
.filterCompletedInstants().findInstantsAfter(commit, 1).empty();
if (higherDeltaCommits) {
// Rollback of a compaction action with no higher deltacommit means that the compaction is scheduled
// and has not yet finished. In this scenario we should delete only the newly created parquet files
// and not corresponding base commit log files created with this as baseCommit since updates would
// have been written to the log files.
super.deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
} else {
// No deltacommits present after this compaction commit (inflight or requested). In this case, we
// can also delete any log files that were created with this compaction commit as base
// commit.
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
.withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus).build();
List<HoodieRollbackStat> allRollbackStats =
jsc.parallelize(FSUtils.getAllPartitionPaths(this.metaClient.getFs(), this.getMetaClient().getBasePath(),
config.shouldAssumeDatePartitioning())).map((Function<String, HoodieRollbackStat>) partitionPath -> {
HoodieActiveTimeline activeTimeline = this.getActiveTimeline().reload();
HoodieRollbackStat hoodieRollbackStats = null;
// Need to put the path filter here since Filter is not serializable
// PathFilter to get all parquet files and log files that need to be deleted
PathFilter filter = (path) -> {
if (path.toString().contains(".parquet")) {
String fileCommitTime = FSUtils.getCommitTime(path.getName());
return commit.equals(fileCommitTime);
} else if (path.toString().contains(".log")) {
// Since the baseCommitTime is the only commit for new log files, it's okay here
String fileCommitTime = FSUtils.getBaseCommitTimeFromLogPath(path);
return commit.equals(fileCommitTime);
}
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.DELTA_COMMIT_ACTION:
// --------------------------------------------------------------------------------------------------
// (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal
// --------------------------------------------------------------------------------------------------
// (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries. In
// this scenario we would want to delete these log files.
// (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario,
// HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks.
// (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is
// being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime and
// and hence will end up deleting these log files. This is done so there are no orphan log files
// lying around.
// (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions
// taken in this scenario is a combination of (A.2) and (A.3)
// ---------------------------------------------------------------------------------------------------
// (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal
// ---------------------------------------------------------------------------------------------------
// (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no entries.
// In this scenario, we delete all the parquet files written for the failed commit.
// (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In
// this scenario, perform (A.1) and for updates written to log files, write rollback blocks.
// (B.3) Rollback triggered for first commit - Same as (B.1)
// (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files
// as well if the base parquet file gets deleted.
try {
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(
metaClient.getCommitTimeline().getInstantDetails(
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()))
.get(), HoodieCommitMetadata.class);
return false;
};
// read commit file and (either append delete blocks or delete file)
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
// In case all data was inserts and the commit failed, delete the file belonging to that commit
// We do not know fileIds for inserts (first inserts are either log files or parquet files),
// delete all files for the corresponding failed commit, if present (same as COW)
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
final Set<String> deletedFiles = filesToDeletedStatus.entrySet().stream()
.map(entry -> {
switch (instantToRollback.getAction()) {
case HoodieTimeline.COMMIT_ACTION:
try {
// Rollback of a commit should delete the newly created parquet files along with any log
// files created with this as baseCommit. This is required to support multi-rollbacks in a MOR
// table.
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus).build();
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.COMPACTION_ACTION:
try {
// If there is no delta commit present after the current commit (if compaction), no action, else we
// need to make sure that a compaction commit rollback also deletes any log files written as part of
// the
// succeeding deltacommit.
boolean higherDeltaCommits = !activeTimeline.getDeltaCommitTimeline().filterCompletedInstants()
.findInstantsAfter(commit, 1).empty();
if (higherDeltaCommits) {
// Rollback of a compaction action with no higher deltacommit means that the compaction is
// scheduled
// and has not yet finished. In this scenario we should delete only the newly created parquet
// files
// and not corresponding base commit log files created with this as baseCommit since updates would
// have been written to the log files.
super.deleteCleanedFiles(filesToDeletedStatus, commit, partitionPath);
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus).build();
} else {
// No deltacommits present after this compaction commit (inflight or requested). In this case, we
// can also delete any log files that were created with this compaction commit as base
// commit.
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus).build();
}
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
case HoodieTimeline.DELTA_COMMIT_ACTION:
// --------------------------------------------------------------------------------------------------
// (A) The following cases are possible if index.canIndexLogFiles and/or index.isGlobal
// --------------------------------------------------------------------------------------------------
// (A.1) Failed first commit - Inserts were written to log files and HoodieWriteStat has no entries.
// In
// this scenario we would want to delete these log files.
// (A.2) Failed recurring commit - Inserts/Updates written to log files. In this scenario,
// HoodieWriteStat will have the baseCommitTime for the first log file written, add rollback blocks.
// (A.3) Rollback triggered for first commit - Inserts were written to the log files but the commit is
// being reverted. In this scenario, HoodieWriteStat will be `null` for the attribute prevCommitTime
// and
// and hence will end up deleting these log files. This is done so there are no orphan log files
// lying around.
// (A.4) Rollback triggered for recurring commits - Inserts/Updates are being rolled back, the actions
// taken in this scenario is a combination of (A.2) and (A.3)
// ---------------------------------------------------------------------------------------------------
// (B) The following cases are possible if !index.canIndexLogFiles and/or !index.isGlobal
// ---------------------------------------------------------------------------------------------------
// (B.1) Failed first commit - Inserts were written to parquet files and HoodieWriteStat has no
// entries.
// In this scenario, we delete all the parquet files written for the failed commit.
// (B.2) Failed recurring commits - Inserts were written to parquet files and updates to log files. In
// this scenario, perform (A.1) and for updates written to log files, write rollback blocks.
// (B.3) Rollback triggered for first commit - Same as (B.1)
// (B.4) Rollback triggered for recurring commits - Same as (B.2) plus we need to delete the log files
// as well if the base parquet file gets deleted.
try {
HoodieCommitMetadata commitMetadata =
HoodieCommitMetadata.fromBytes(
metaClient.getCommitTimeline().getInstantDetails(new HoodieInstant(true,
instantToRollback.getAction(), instantToRollback.getTimestamp())).get(),
HoodieCommitMetadata.class);
// read commit file and (either append delete blocks or delete file)
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
// In case all data was inserts and the commit failed, delete the file belonging to that commit
// We do not know fileIds for inserts (first inserts are either log files or parquet files),
// delete all files for the corresponding failed commit, if present (same as COW)
super.deleteCleanedFiles(filesToDeletedStatus, partitionPath, filter);
final Set<String> deletedFiles = filesToDeletedStatus.entrySet().stream().map(entry -> {
Path filePath = entry.getKey().getPath();
return FSUtils.getFileIdFromFilePath(filePath);
}).collect(Collectors.toSet());
// append rollback blocks for updates
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus,
filesToNumBlocksRollback, deletedFiles);
}
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
// append rollback blocks for updates
if (commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
hoodieRollbackStats = rollback(index, partitionPath, commit, commitMetadata, filesToDeletedStatus,
filesToNumBlocksRollback, deletedFiles);
}
break;
} catch (IOException io) {
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
}
default:
break;
}
default:
break;
}
return hoodieRollbackStats;
}).filter(Objects::nonNull).collect();
return hoodieRollbackStats;
}).filter(Objects::nonNull).collect();
// Delete Inflight instants if enabled
deleteInflightInstant(deleteInstants, this.getActiveTimeline(), new HoodieInstant(true, instantToRollback
.getAction(), instantToRollback.getTimestamp()));
deleteInflightInstant(deleteInstants, this.getActiveTimeline(),
new HoodieInstant(true, instantToRollback.getAction(), instantToRollback.getTimestamp()));
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
@@ -332,8 +340,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
/**
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet
* files to larger ones without the need for an index in the logFile.
* UpsertPartitioner for MergeOnRead table type, this allows auto correction of small parquet files to larger ones
* without the need for an index in the logFile.
*/
class MergeOnReadUpsertPartitioner extends HoodieCopyOnWriteTable.UpsertPartitioner {
@@ -361,21 +369,23 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// TODO : choose last N small files since there can be multiple small files written to a single partition
// by different spark partitions in a single batch
Option<FileSlice> smallFileSlice = Option.fromJavaOptional(getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false).filter(
fileSlice -> fileSlice.getLogFiles().count() < 1
&& fileSlice.getDataFile().get().getFileSize() < config
.getParquetSmallFileLimit()).sorted((FileSlice left, FileSlice right) ->
left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize()
? -1 : 1).findFirst());
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), false)
.filter(fileSlice -> fileSlice.getLogFiles().count() < 1
&& fileSlice.getDataFile().get().getFileSize() < config.getParquetSmallFileLimit())
.sorted((FileSlice left,
FileSlice right) -> left.getDataFile().get().getFileSize() < right.getDataFile().get().getFileSize()
? -1
: 1)
.findFirst());
if (smallFileSlice.isPresent()) {
allSmallFileSlices.add(smallFileSlice.get());
}
} else {
// If we can index log files, we can add more inserts to log files for fileIds including those under
// pending compaction.
List<FileSlice> allFileSlices = getRTFileSystemView()
.getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
.collect(Collectors.toList());
List<FileSlice> allFileSlices =
getRTFileSystemView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
.collect(Collectors.toList());
for (FileSlice fileSlice : allFileSlices) {
if (isSmallFile(partitionPath, fileSlice)) {
allSmallFileSlices.add(fileSlice);
@@ -408,8 +418,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
}
public List<String> getSmallFileIds() {
return (List<String>) smallFiles.stream()
.map(smallFile -> ((SmallFile) smallFile).location.getFileId())
return (List<String>) smallFiles.stream().map(smallFile -> ((SmallFile) smallFile).location.getFileId())
.collect(Collectors.toList());
}
@@ -417,8 +426,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
if (!fileSlice.getDataFile().isPresent()) {
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
} else {
return fileSlice.getDataFile().get().getFileSize() + convertLogFilesSizeToExpectedParquetSize(fileSlice
.getLogFiles().collect(Collectors.toList()));
return fileSlice.getDataFile().get().getFileSize()
+ convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
}
}
@@ -431,13 +440,12 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
@VisibleForTesting
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(hoodieLogFile -> hoodieLogFile.getFileSize())
.filter(size -> size > 0)
.reduce((a, b) -> (a + b)).orElse(0L);
.filter(size -> size > 0).reduce((a, b) -> (a + b)).orElse(0L);
// Here we assume that if there is no base parquet file, all log files contain only inserts.
// We can then just get the parquet equivalent size of these log files, compare that with
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows
long logFilesEquivalentParquetFileSize = (long) (totalSizeOfLogFiles * config
.getLogFileToParquetCompressionRatio());
long logFilesEquivalentParquetFileSize =
(long) (totalSizeOfLogFiles * config.getLogFileToParquetCompressionRatio());
return logFilesEquivalentParquetFileSize;
}
}
@@ -447,8 +455,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
Map<HeaderMetadataType, String> header = Maps.newHashMap();
header.put(HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
header.put(HeaderMetadataType.TARGET_INSTANT_TIME, commit);
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK
.ordinal()));
header.put(HeaderMetadataType.COMMAND_BLOCK_TYPE,
String.valueOf(HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK.ordinal()));
return header;
}
@@ -461,52 +469,47 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
// But the index (global) might store the baseCommit of the parquet and not the requested, hence get the
// baseCommit always by listing the file slice
Map<String, String> fileIdToBaseCommitTimeForLogMap = this.getRTFileSystemView().getLatestFileSlices(partitionPath)
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime));
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
.filter(wStat -> {
// Filter out stats without prevCommit since they are all inserts
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
&& !deletedFiles.contains(wStat.getFileId());
}).forEach(wStat -> {
Writer writer = null;
String baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId());
if (null != baseCommitTime) {
boolean success = false;
try {
writer = HoodieLogFormat.newWriterBuilder().onParentPath(
FSUtils.getPartitionPath(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime)
.withFs(this.metaClient.getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HeaderMetadataType, String> header = generateHeader(commit);
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
success = true;
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException(
"Failed to rollback for commit " + commit, io);
} finally {
try {
if (writer != null) {
writer.close();
}
if (success) {
// This step is intentionally done after writer is closed. Guarantees that
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
// cloud-storage : HUDI-168
filesToNumBlocksRollback.put(this.getMetaClient().getFs()
.getFileStatus(writer.getLogFile().getPath()), 1L);
}
} catch (IOException io) {
throw new UncheckedIOException(io);
}
.collect(Collectors.toMap(FileSlice::getFileId, FileSlice::getBaseInstantTime));
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream().filter(wStat -> {
// Filter out stats without prevCommit since they are all inserts
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
&& !deletedFiles.contains(wStat.getFileId());
}).forEach(wStat -> {
Writer writer = null;
String baseCommitTime = fileIdToBaseCommitTimeForLogMap.get(wStat.getFileId());
if (null != baseCommitTime) {
boolean success = false;
try {
writer = HoodieLogFormat.newWriterBuilder()
.onParentPath(FSUtils.getPartitionPath(this.getMetaClient().getBasePath(), partitionPath))
.withFileId(wStat.getFileId()).overBaseCommit(baseCommitTime).withFs(this.metaClient.getFs())
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
// generate metadata
Map<HeaderMetadataType, String> header = generateHeader(commit);
// if update belongs to an existing log file
writer = writer.appendBlock(new HoodieCommandBlock(header));
success = true;
} catch (IOException | InterruptedException io) {
throw new HoodieRollbackException("Failed to rollback for commit " + commit, io);
} finally {
try {
if (writer != null) {
writer.close();
}
if (success) {
// This step is intentionally done after writer is closed. Guarantees that
// getFileStatus would reflect correct stats and FileNotFoundException is not thrown in
// cloud-storage : HUDI-168
filesToNumBlocksRollback.put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()),
1L);
}
} catch (IOException io) {
throw new UncheckedIOException(io);
}
});
return HoodieRollbackStat.newBuilder()
.withPartitionPath(partitionPath)
.withDeletedFileResults(filesToDeletedStatus)
}
}
});
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath).withDeletedFileResults(filesToDeletedStatus)
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
}

View File

@@ -82,22 +82,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
protected HoodieTable(HoodieWriteConfig config, JavaSparkContext jsc) {
this.config = config;
this.hadoopConfiguration = new SerializableConfiguration(jsc.hadoopConfiguration());
this.viewManager = FileSystemViewManager.createViewManager(
new SerializableConfiguration(jsc.hadoopConfiguration()), config.getViewStorageConfig());
this.viewManager = FileSystemViewManager.createViewManager(new SerializableConfiguration(jsc.hadoopConfiguration()),
config.getViewStorageConfig());
this.metaClient = ClientUtils.createMetaClient(jsc, config, true);
this.index = HoodieIndex.createIndex(config, jsc);
}
private synchronized FileSystemViewManager getViewManager() {
if (null == viewManager) {
viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration,
config.getViewStorageConfig());
viewManager = FileSystemViewManager.createViewManager(hadoopConfiguration, config.getViewStorageConfig());
}
return viewManager;
}
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(
HoodieTableMetaClient metaClient, HoodieWriteConfig config, JavaSparkContext jsc) {
public static <T extends HoodieRecordPayload> HoodieTable<T> getHoodieTable(HoodieTableMetaClient metaClient,
HoodieWriteConfig config, JavaSparkContext jsc) {
switch (metaClient.getTableType()) {
case COPY_ON_WRITE:
return new HoodieCopyOnWriteTable<>(config, jsc);
@@ -202,8 +201,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* Get the list of savepoints in this table
*/
public List<String> getSavepoints() {
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp)
.collect(Collectors.toList());
return getCompletedSavepointTimeline().getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toList());
}
/**
@@ -214,18 +212,14 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
throw new HoodieSavepointException(
"Could not get data files for savepoint " + savepointTime + ". No such savepoint.");
}
HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION,
savepointTime);
HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime);
HoodieSavepointMetadata metadata = null;
try {
metadata = AvroUtils
.deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get());
metadata = AvroUtils.deserializeHoodieSavepointMetadata(getActiveTimeline().getInstantDetails(instant).get());
} catch (IOException e) {
throw new HoodieSavepointException(
"Could not get savepointed data files for savepoint " + savepointTime, e);
throw new HoodieSavepointException("Could not get savepointed data files for savepoint " + savepointTime, e);
}
return metadata.getPartitionMetadata().values().stream()
.flatMap(s -> s.getSavepointDataFile().stream());
return metadata.getPartitionMetadata().values().stream().flatMap(s -> s.getSavepointDataFile().stream());
}
public HoodieActiveTimeline getActiveTimeline() {
@@ -242,30 +236,30 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
/**
* Perform the ultimate IO for a given upserted (RDD) partition
*/
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime,
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
public abstract Iterator<List<WriteStatus>> handleUpsertPartition(String commitTime, Integer partition,
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
/**
* Perform the ultimate IO for a given inserted (RDD) partition
*/
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime,
Integer partition, Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
public abstract Iterator<List<WriteStatus>> handleInsertPartition(String commitTime, Integer partition,
Iterator<HoodieRecord<T>> recordIterator, Partitioner partitioner);
/**
* Schedule compaction for the instant time
* @param jsc Spark Context
*
* @param jsc Spark Context
* @param instantTime Instant Time for scheduling compaction
* @return
*/
public abstract HoodieCompactionPlan scheduleCompaction(JavaSparkContext jsc, String instantTime);
/**
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data
* access
* Run Compaction on the table. Compaction arranges the data so that it is optimized for data access
*
* @param jsc Spark Context
* @param jsc Spark Context
* @param compactionInstantTime Instant Time
* @param compactionPlan Compaction Plan
* @param compactionPlan Compaction Plan
*/
public abstract JavaRDD<WriteStatus> compact(JavaSparkContext jsc, String compactionInstantTime,
HoodieCompactionPlan compactionPlan);
@@ -276,9 +270,9 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
public abstract List<HoodieCleanStat> clean(JavaSparkContext jsc);
/**
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1)
* Atomically unpublish this commit (2) clean indexing data (3) clean new generated parquet files
* / log blocks (4) Finally, delete .<action>.commit or .<action>.inflight file if deleteInstants = true
* Rollback the (inflight/committed) record changes with the given commit time. Four steps: (1) Atomically unpublish
* this commit (2) clean indexing data (3) clean new generated parquet files / log blocks (4) Finally, delete
* .<action>.commit or .<action>.inflight file if deleteInstants = true
*/
public abstract List<HoodieRollbackStat> rollback(JavaSparkContext jsc, String commit, boolean deleteInstants)
throws IOException;
@@ -297,6 +291,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
/**
* Delete Marker directory corresponding to an instant
*
* @param instantTs Instant Time
*/
protected void deleteMarkerDir(String instantTs) {
@@ -317,10 +312,10 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
* Reconciles WriteStats and marker files to detect and safely delete duplicate data files created because of Spark
* retries.
*
* @param jsc Spark Context
* @param jsc Spark Context
* @param instantTs Instant Timestamp
* @param stats Hoodie Write Stat
* @param consistencyCheckEnabled Consistency Check Enabled
* @param stats Hoodie Write Stat
* @param consistencyCheckEnabled Consistency Check Enabled
* @throws HoodieIOException
*/
protected void cleanFailedWrites(JavaSparkContext jsc, String instantTs, List<HoodieWriteStat> stats,
@@ -343,13 +338,12 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
// Contains list of partially created files. These needs to be cleaned up.
invalidDataPaths.removeAll(validDataPaths);
if (!invalidDataPaths.isEmpty()) {
logger.info("Removing duplicate data files created due to spark retries before committing. Paths="
+ invalidDataPaths);
logger.info(
"Removing duplicate data files created due to spark retries before committing. Paths=" + invalidDataPaths);
}
Map<String, List<Pair<String, String>>> groupByPartition = invalidDataPaths.stream()
.map(dp -> Pair.of(new Path(dp).getParent().toString(), dp))
.collect(Collectors.groupingBy(Pair::getKey));
.map(dp -> Pair.of(new Path(dp).getParent().toString(), dp)).collect(Collectors.groupingBy(Pair::getKey));
if (!groupByPartition.isEmpty()) {
// Ensure all files in delete list is actually present. This is mandatory for an eventually consistent FS.
@@ -394,7 +388,8 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
/**
* Ensures all files passed either appear or disappear
* @param jsc JavaSparkContext
*
* @param jsc JavaSparkContext
* @param groupByPartition Files grouped by partition
* @param visibility Appear/Disappear
*/

View File

@@ -23,13 +23,11 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.spark.api.java.JavaRDD;
/**
* Repartition input records into at least expected number of output spark partitions. It should
* give below guarantees - Output spark partition will have records from only one hoodie partition.
* - Average records per output spark partitions should be almost equal to (#inputRecords /
* #outputSparkPartitions) to avoid possible skews.
* Repartition input records into at least expected number of output spark partitions. It should give below guarantees -
* Output spark partition will have records from only one hoodie partition. - Average records per output spark
* partitions should be almost equal to (#inputRecords / #outputSparkPartitions) to avoid possible skews.
*/
public interface UserDefinedBulkInsertPartitioner<T extends HoodieRecordPayload> {
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records,
int outputSparkPartitions);
JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions);
}

View File

@@ -30,8 +30,7 @@ import org.apache.spark.api.java.JavaRDD;
import scala.Tuple2;
/**
* Information about incoming records for upsert/insert obtained either via sampling or
* introspecting the data fully
* Information about incoming records for upsert/insert obtained either via sampling or introspecting the data fully
* <p>
* TODO(vc): Think about obtaining this directly from index.tagLocation
*/
@@ -62,11 +61,10 @@ public class WorkloadProfile<T extends HoodieRecordPayload> implements Serializa
Map<Tuple2<String, Option<HoodieRecordLocation>>, Long> partitionLocationCounts = taggedRecords
.mapToPair(record -> new Tuple2<>(
new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())),
record)).countByKey();
new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record))
.countByKey();
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts
.entrySet()) {
for (Map.Entry<Tuple2<String, Option<HoodieRecordLocation>>, Long> e : partitionLocationCounts.entrySet()) {
String partitionPath = e.getKey()._1();
Long count = e.getValue();
Option<HoodieRecordLocation> locOption = e.getKey()._2();