1
0

[HUDI-1138] Add timeline-server-based marker file strategy for improving marker-related latency (#3233)

- Can be enabled for cloud stores like S3. Not supported for hdfs yet, due to partial write failures.
This commit is contained in:
Y Ethan Guo
2021-08-11 08:48:13 -07:00
committed by GitHub
parent 29332498af
commit 4783176554
52 changed files with 2144 additions and 353 deletions

View File

@@ -51,9 +51,9 @@ import org.apache.hudi.metrics.DistributedRegistry;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade;
import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
@@ -375,7 +375,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
} catch (IOException e) {
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
}
new MarkerFiles(table, clusteringCommitTime).quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
WriteMarkersFactory.get(config.getMarkersType(), table, clusteringCommitTime)
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
if (clusteringTimer != null) {
long durationInMs = metrics.getDurationInMs(clusteringTimer.stop());
try {

View File

@@ -31,7 +31,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieInsertException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -187,8 +187,8 @@ public class HoodieRowCreateHandle implements Serializable {
* @param partitionPath Partition path
*/
private void createMarkerFile(String partitionPath, String dataFileName) {
MarkerFiles markerFiles = new MarkerFiles(table, instantTime);
markerFiles.create(partitionPath, dataFileName, IOType.CREATE);
WriteMarkersFactory.get(writeConfig.getMarkersType(), table, instantTime)
.create(partitionPath, dataFileName, IOType.CREATE);
}
private String getWriteToken() {

View File

@@ -33,13 +33,15 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.hadoop.fs.FileStatus;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
@@ -56,8 +58,8 @@ public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> ext
public List<HoodieRollbackStat> execute(HoodieInstant instantToRollback) {
JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context);
try {
MarkerFiles markerFiles = new MarkerFiles(table, instantToRollback.getTimestamp());
List<String> markerFilePaths = markerFiles.allMarkerFilePaths();
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp());
List<String> markerFilePaths = new ArrayList<>(writeMarkers.allMarkerFilePaths());
int parallelism = Math.max(Math.min(markerFilePaths.size(), config.getRollbackParallelism()), 1);
jsc.setJobGroup(this.getClass().getSimpleName(), "Rolling back using marker files");
return jsc.parallelize(markerFilePaths, parallelism)
@@ -66,11 +68,11 @@ public class SparkMarkerBasedRollbackStrategy<T extends HoodieRecordPayload> ext
IOType type = IOType.valueOf(typeStr);
switch (type) {
case MERGE:
return undoMerge(MarkerFiles.stripMarkerSuffix(markerFilePath));
return undoMerge(WriteMarkers.stripMarkerSuffix(markerFilePath));
case APPEND:
return undoAppend(MarkerFiles.stripMarkerSuffix(markerFilePath), instantToRollback);
return undoAppend(WriteMarkers.stripMarkerSuffix(markerFilePath), instantToRollback);
case CREATE:
return undoCreate(MarkerFiles.stripMarkerSuffix(markerFilePath));
return undoCreate(WriteMarkers.stripMarkerSuffix(markerFilePath));
default:
throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback);
}

View File

@@ -23,7 +23,8 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import java.util.List;
import java.util.stream.Collectors;
@@ -40,9 +41,9 @@ public class OneToZeroDowngradeHandler implements DowngradeHandler {
HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompaction();
List<HoodieInstant> commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList());
for (HoodieInstant commitInstant : commits) {
// delete existing marker files
MarkerFiles markerFiles = new MarkerFiles(table, commitInstant.getTimestamp());
markerFiles.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
// delete existing markers
WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, commitInstant.getTimestamp());
writeMarkers.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
}
}
}

View File

@@ -31,13 +31,15 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieRollbackException;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.MarkerFiles;
import org.apache.hudi.table.action.rollback.ListingBasedRollbackHelper;
import org.apache.hudi.table.action.rollback.ListingBasedRollbackRequest;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.table.action.rollback.RollbackUtils;
import org.apache.hudi.table.marker.WriteMarkers;
import org.apache.hudi.table.marker.WriteMarkersFactory;
import org.apache.hudi.table.marker.MarkerType;
import java.util.List;
import java.util.stream.Collectors;
@@ -59,35 +61,37 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
commits.remove(instantTime);
}
for (String commit : commits) {
// for every pending commit, delete old marker files and re-create marker files in new format
recreateMarkerFiles(commit, table, context, config.getMarkersDeleteParallelism());
// for every pending commit, delete old markers and re-create markers in new format
recreateMarkers(commit, table, context, config.getMarkersType(), config.getMarkersDeleteParallelism());
}
}
/**
* Recreate marker files in new format.
* Step1: Delete existing marker files
* Recreate markers in new format.
* Step1: Delete existing markers
* Step2: Collect all rollback file info.
* Step3: recreate marker files for all interested files.
* Step3: recreate markers for all interested files.
*
* @param commitInstantTime instant of interest for which marker files need to be recreated.
* @param commitInstantTime instant of interest for which markers need to be recreated.
* @param table instance of {@link HoodieSparkTable} to use
* @param context instance of {@link HoodieEngineContext} to use
* @param markerType marker type to use
* @throws HoodieRollbackException on any exception during upgrade.
*/
private static void recreateMarkerFiles(final String commitInstantTime,
HoodieSparkTable table,
HoodieEngineContext context,
int parallelism) throws HoodieRollbackException {
private static void recreateMarkers(final String commitInstantTime,
HoodieSparkTable table,
HoodieEngineContext context,
MarkerType markerType,
int parallelism) throws HoodieRollbackException {
try {
// fetch hoodie instant
Option<HoodieInstant> commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstants()
.filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime))
.findFirst());
if (commitInstantOpt.isPresent()) {
// delete existing marker files
MarkerFiles markerFiles = new MarkerFiles(table, commitInstantTime);
markerFiles.quietDeleteMarkerDir(context, parallelism);
// delete existing markers
WriteMarkers writeMarkers = WriteMarkersFactory.get(markerType, table, commitInstantTime);
writeMarkers.quietDeleteMarkerDir(context, parallelism);
// generate rollback stats
List<ListingBasedRollbackRequest> rollbackRequests;
@@ -99,15 +103,15 @@ public class ZeroToOneUpgradeHandler implements UpgradeHandler {
List<HoodieRollbackStat> rollbackStats = new ListingBasedRollbackHelper(table.getMetaClient(), table.getConfig())
.collectRollbackStats(context, commitInstantOpt.get(), rollbackRequests);
// recreate marker files adhering to marker based rollback
// recreate markers adhering to marker based rollback
for (HoodieRollbackStat rollbackStat : rollbackStats) {
for (String path : rollbackStat.getSuccessDeleteFiles()) {
String dataFileName = path.substring(path.lastIndexOf("/") + 1);
// not feasible to differentiate MERGE from CREATE. hence creating with MERGE IOType for all base files.
markerFiles.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
writeMarkers.create(rollbackStat.getPartitionPath(), dataFileName, IOType.MERGE);
}
for (FileStatus fileStatus : rollbackStat.getCommandBlocksCount().keySet()) {
markerFiles.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
writeMarkers.create(rollbackStat.getPartitionPath(), getFileNameForMarkerFromLogFile(fileStatus.getPath().toString(), table), IOType.APPEND);
}
}
}