Fixing Rollback for compaction/commit operation, added check for null commit
- Fallback to old way of rollback by listing all partitions
- Added null check to ensure only partitions which are to be rolledback are considered
- Added location (committime) to workload stat
- Added checks in CompactedScanner to guard against task retries
- Introduce new logic for rollback (bounded by instant_time and target_instant time)
- Reversed logfiles order
This commit is contained in:
committed by
vinoth chandar
parent
be0b1f3e57
commit
2116815261
@@ -318,7 +318,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
partitionStat.getUpdateLocationToCount().entrySet().stream().forEach(entry -> {
|
||||
HoodieWriteStat writeStat = new HoodieWriteStat();
|
||||
writeStat.setFileId(entry.getKey());
|
||||
writeStat.setNumUpdateWrites(entry.getValue());
|
||||
writeStat.setPrevCommit(entry.getValue().getKey());
|
||||
writeStat.setNumUpdateWrites(entry.getValue().getValue());
|
||||
metadata.addWriteStat(path.toString(), writeStat);
|
||||
});
|
||||
});
|
||||
|
||||
@@ -187,7 +187,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
}
|
||||
} catch (Exception e) {
|
||||
throw new HoodieAppendException(
|
||||
"Failed while appeding records to " + currentLogFile.getPath(), e);
|
||||
"Failed while appending records to " + currentLogFile.getPath(), e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -23,6 +23,7 @@ import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.uber.hoodie.WriteStatus;
|
||||
import com.uber.hoodie.common.model.HoodieCommitMetadata;
|
||||
import com.uber.hoodie.common.model.HoodieLogFile;
|
||||
import com.uber.hoodie.common.model.HoodieTableType;
|
||||
import com.uber.hoodie.common.model.HoodieWriteStat;
|
||||
import com.uber.hoodie.common.table.HoodieTableMetaClient;
|
||||
@@ -86,7 +87,8 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
.flatMap((FlatMapFunction<String, CompactionOperation>) partitionPath -> fileSystemView
|
||||
.getLatestFileSlices(partitionPath)
|
||||
.map(s -> new CompactionOperation(s.getDataFile().get(),
|
||||
partitionPath, s.getLogFiles().collect(Collectors.toList()), config))
|
||||
partitionPath, s.getLogFiles().sorted(HoodieLogFile.getLogVersionComparator().reversed())
|
||||
.collect(Collectors.toList()), config))
|
||||
.filter(c -> !c.getDeltaFilePaths().isEmpty())
|
||||
.collect(toList()).iterator()).collect();
|
||||
log.info("Total of " + operations.size() + " compactions are retrieved");
|
||||
@@ -144,7 +146,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
|
||||
String maxInstantTime = metaClient.getActiveTimeline()
|
||||
.getTimelineOfActions(
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION,
|
||||
Sets.newHashSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION,
|
||||
HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.filterCompletedInstants().lastInstant().get().getTimestamp();
|
||||
|
||||
|
||||
@@ -54,6 +54,7 @@ import java.util.Set;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.avro.generic.IndexedRecord;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -199,7 +200,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
private void assignUpdates(WorkloadProfile profile) {
|
||||
// each update location gets a partition
|
||||
WorkloadStat gStat = profile.getGlobalStat();
|
||||
for (Map.Entry<String, Long> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) {
|
||||
for (Map.Entry<String, Pair<String, Long>> updateLocEntry : gStat.getUpdateLocationToCount().entrySet()) {
|
||||
addUpdateBucket(updateLocEntry.getKey());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -32,11 +32,20 @@ import com.uber.hoodie.common.table.log.block.HoodieCommandBlock;
|
||||
import com.uber.hoodie.common.table.log.block.HoodieLogBlock;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieActiveTimeline;
|
||||
import com.uber.hoodie.common.table.timeline.HoodieInstant;
|
||||
import com.uber.hoodie.common.util.FSUtils;
|
||||
import com.uber.hoodie.config.HoodieWriteConfig;
|
||||
import com.uber.hoodie.exception.HoodieCompactionException;
|
||||
import com.uber.hoodie.exception.HoodieRollbackException;
|
||||
import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
import org.apache.spark.api.java.function.Function;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.Arrays;
|
||||
@@ -47,38 +56,36 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
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>
|
||||
*/
|
||||
public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
HoodieCopyOnWriteTable<T> {
|
||||
|
||||
private static Logger logger = LogManager.getLogger(HoodieMergeOnReadTable.class);
|
||||
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config, HoodieTableMetaClient metaClient) {
|
||||
public HoodieMergeOnReadTable(HoodieWriteConfig config,
|
||||
HoodieTableMetaClient metaClient) {
|
||||
super(config, metaClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Iterator<List<WriteStatus>> handleUpdate(String commitTime, String fileId,
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
Iterator<HoodieRecord<T>> recordItr) throws IOException {
|
||||
logger.info("Merging updates for commit " + commitTime + " for file " + fileId);
|
||||
HoodieAppendHandle<T> appendHandle =
|
||||
new HoodieAppendHandle<>(config, commitTime, this, fileId, recordItr);
|
||||
@@ -126,7 +133,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
Map<String, HoodieInstant> commitsAndCompactions =
|
||||
this.getActiveTimeline()
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION, HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION))
|
||||
.getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
@@ -139,41 +147,28 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
Long startTime = System.currentTimeMillis();
|
||||
|
||||
List<HoodieRollbackStat> allRollbackStats = commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
List<HoodieRollbackStat> stats = null;
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback Commit/Compaction " + instant);
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitsTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||
.collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
Map<FileStatus, Boolean> results = super
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
List<HoodieRollbackStat> allRollbackStats = jsc.parallelize
|
||||
(FSUtils.getAllPartitionPaths(this.metaClient.getFs(),
|
||||
this.getMetaClient().getBasePath(), config.shouldAssumeDatePartitioning()))
|
||||
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> {
|
||||
return commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
HoodieRollbackStat hoodieRollbackStats = null;
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
try {
|
||||
Map<FileStatus, Boolean> results = super.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(results).build();
|
||||
}).collect();
|
||||
logger.info("Finished rollback of Commit/Compaction " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
try {
|
||||
logger.info("Starting to rollback delta commit " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
case HoodieTimeline.DELTA_COMMIT_ACTION:
|
||||
try {
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitTimeline().getInstantDetails(new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
HoodieCommitMetadata commitMetadata = HoodieCommitMetadata
|
||||
.fromBytes(this.getCommitsTimeline().getInstantDetails(
|
||||
new HoodieInstant(true, instant.getAction(), instant.getTimestamp())).get());
|
||||
|
||||
stats = jsc.parallelize(commitMetadata.getPartitionToWriteStats().keySet().stream()
|
||||
.collect(Collectors.toList()))
|
||||
.map((Function<String, HoodieRollbackStat>) partitionPath -> {
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
Map<FileStatus, Long> filesToNumBlocksRollback = new HashMap<>();
|
||||
@@ -183,69 +178,64 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
|
||||
// append rollback blocks for updates
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT)
|
||||
.forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(
|
||||
new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(getMetaClient().getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME,
|
||||
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
// if update belongs to an existing log file
|
||||
writer.appendBlock(new HoodieCommandBlock(
|
||||
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
|
||||
metadata));
|
||||
numRollbackBlocks++;
|
||||
if (wStat.getNumDeletes() > 0) {
|
||||
writer.appendBlock(new HoodieCommandBlock(
|
||||
if(commitMetadata.getPartitionToWriteStats().containsKey(partitionPath)) {
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> {
|
||||
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT
|
||||
&& wStat.getPrevCommit() != null;
|
||||
})
|
||||
.forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
try {
|
||||
writer = HoodieLogFormat.newWriterBuilder()
|
||||
.onParentPath(new Path(this.getMetaClient().getBasePath(), partitionPath))
|
||||
.withFileId(wStat.getFileId()).overBaseCommit(wStat.getPrevCommit())
|
||||
.withFs(this.metaClient.getFs())
|
||||
.withFileExtension(HoodieLogFile.DELTA_EXTENSION).build();
|
||||
Long numRollbackBlocks = 0L;
|
||||
// generate metadata
|
||||
Map<HoodieLogBlock.LogMetadataType, String> metadata = Maps.newHashMap();
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.INSTANT_TIME,
|
||||
metaClient.getActiveTimeline().lastInstant().get().getTimestamp());
|
||||
metadata.put(HoodieLogBlock.LogMetadataType.TARGET_INSTANT_TIME, commit);
|
||||
// if update belongs to an existing log file
|
||||
writer = writer.appendBlock(new HoodieCommandBlock(
|
||||
HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_PREVIOUS_BLOCK,
|
||||
metadata));
|
||||
numRollbackBlocks++;
|
||||
filesToNumBlocksRollback
|
||||
.put(this.getMetaClient().getFs().getFileStatus(writer.getLogFile().getPath()),
|
||||
numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
filesToNumBlocksRollback
|
||||
.put(getMetaClient().getFs()
|
||||
.getFileStatus(writer.getLogFile().getPath()),
|
||||
numRollbackBlocks);
|
||||
} catch (IOException | InterruptedException io) {
|
||||
throw new HoodieRollbackException(
|
||||
"Failed to rollback for commit " + commit, io);
|
||||
} finally {
|
||||
try {
|
||||
writer.close();
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException(io);
|
||||
}
|
||||
}
|
||||
});
|
||||
return HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}).collect();
|
||||
logger.info("Fnished rollback of delta commit " + instant);
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
}
|
||||
return stats;
|
||||
}).flatMap(x -> x.stream()).collect(Collectors.toList());
|
||||
});
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder().withPartitionPath(partitionPath)
|
||||
.withDeletedFileResults(filesToDeletedStatus)
|
||||
.withRollbackBlockAppendResults(filesToNumBlocksRollback).build();
|
||||
}
|
||||
break;
|
||||
} catch (IOException io) {
|
||||
throw new UncheckedIOException("Failed to rollback for commit " + commit, io);
|
||||
}
|
||||
}
|
||||
return hoodieRollbackStats;
|
||||
}).collect(Collectors.toList());
|
||||
}).flatMap(x -> x.iterator()).filter(x -> x != null).collect();
|
||||
|
||||
commitsAndCompactions.entrySet().stream()
|
||||
.map(entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||
entry.getValue().getTimestamp()))
|
||||
.forEach(this.getActiveTimeline()::deleteInflight);
|
||||
|
||||
logger
|
||||
.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
logger.debug("Time(in ms) taken to finish rollback " + (System.currentTimeMillis() - startTime));
|
||||
|
||||
return allRollbackStats;
|
||||
}
|
||||
|
||||
@@ -17,6 +17,8 @@
|
||||
package com.uber.hoodie.table;
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieRecordLocation;
|
||||
import org.apache.commons.lang3.tuple.Pair;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.HashMap;
|
||||
|
||||
@@ -29,7 +31,7 @@ public class WorkloadStat implements Serializable {
|
||||
|
||||
private long numUpdates = 0L;
|
||||
|
||||
private HashMap<String, Long> updateLocationToCount;
|
||||
private HashMap<String, Pair<String, Long>> updateLocationToCount;
|
||||
|
||||
public WorkloadStat() {
|
||||
updateLocationToCount = new HashMap<>();
|
||||
@@ -40,7 +42,7 @@ public class WorkloadStat implements Serializable {
|
||||
}
|
||||
|
||||
long addUpdates(HoodieRecordLocation location, long numUpdates) {
|
||||
updateLocationToCount.put(location.getFileId(), numUpdates);
|
||||
updateLocationToCount.put(location.getFileId(), Pair.of(location.getCommitTime(), numUpdates));
|
||||
return this.numUpdates += numUpdates;
|
||||
}
|
||||
|
||||
@@ -52,7 +54,7 @@ public class WorkloadStat implements Serializable {
|
||||
return numInserts;
|
||||
}
|
||||
|
||||
public HashMap<String, Long> getUpdateLocationToCount() {
|
||||
public HashMap<String, Pair<String, Long>> getUpdateLocationToCount() {
|
||||
return updateLocationToCount;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user