General enhancements
This commit is contained in:
committed by
vinoth chandar
parent
30c5f8b7bd
commit
6946dd7557
@@ -238,8 +238,8 @@ public class CompactionAdminClient implements Serializable {
|
||||
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).map(x -> x).orElse(0);
|
||||
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 -> {
|
||||
@@ -322,8 +322,7 @@ public class CompactionAdminClient implements Serializable {
|
||||
Set<HoodieLogFile> diff =
|
||||
logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf))
|
||||
.collect(Collectors.toSet());
|
||||
Preconditions.checkArgument(diff.stream()
|
||||
.filter(lf -> !lf.getBaseCommitTime().equals(compactionInstant)).count() == 0,
|
||||
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 {
|
||||
@@ -438,14 +437,14 @@ public class CompactionAdminClient implements Serializable {
|
||||
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))
|
||||
.collect(Collectors.toList());
|
||||
logFilesToRepair.sort(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed());
|
||||
merged.getLogFiles().filter(lf -> lf.getBaseCommitTime().equals(compactionInstant))
|
||||
.sorted(HoodieLogFile.getBaseInstantAndLogVersionComparator().reversed())
|
||||
.collect(Collectors.toList());
|
||||
FileSlice fileSliceForCompaction =
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(operation.getPartitionPath(), operation.getBaseInstantTime())
|
||||
.filter(fs -> fs.getFileId().equals(operation.getFileId())).findFirst().get();
|
||||
int maxUsedVersion =
|
||||
fileSliceForCompaction.getLogFiles().findFirst().map(lf -> lf.getLogVersion())
|
||||
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);
|
||||
|
||||
@@ -440,7 +440,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
} else {
|
||||
return hoodieTable.handleInsertPartition(commitTime, partition, recordItr, partitioner);
|
||||
}
|
||||
}, true).flatMap(writeStatuses -> writeStatuses.iterator());
|
||||
}, true).flatMap(List::iterator);
|
||||
|
||||
return updateIndexAndCommitIfNeeded(writeStatusRDD, hoodieTable, commitTime);
|
||||
}
|
||||
@@ -469,7 +469,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
Partitioner partitioner) {
|
||||
return dedupedRecords.mapToPair(record -> new Tuple2<>(
|
||||
new Tuple2<>(record.getKey(), Option.apply(record.getCurrentLocation())), record))
|
||||
.partitionBy(partitioner).map(tuple -> tuple._2());
|
||||
.partitionBy(partitioner).map(Tuple2::_2);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -499,7 +499,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata();
|
||||
|
||||
List<HoodieWriteStat> stats = writeStatuses.map(status -> status.getStat()).collect();
|
||||
List<HoodieWriteStat> stats = writeStatuses.map(WriteStatus::getStat).collect();
|
||||
|
||||
updateMetadataAndRollingStats(actionType, metadata, stats);
|
||||
|
||||
@@ -522,7 +522,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
// add in extra metadata
|
||||
if (extraMetadata.isPresent()) {
|
||||
extraMetadata.get().forEach((k, v) -> metadata.addMetadata(k, v));
|
||||
extraMetadata.get().forEach(metadata::addMetadata);
|
||||
}
|
||||
|
||||
try {
|
||||
@@ -806,7 +806,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
});
|
||||
|
||||
List<String> pendingCompactionToRollback =
|
||||
commits.stream().filter(c -> pendingCompactions.contains(c)).collect(Collectors.toList());
|
||||
commits.stream().filter(pendingCompactions::contains).collect(Collectors.toList());
|
||||
List<String> commitsToRollback =
|
||||
commits.stream().filter(c -> !pendingCompactions.contains(c)).collect(Collectors.toList());
|
||||
|
||||
@@ -837,12 +837,12 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
}
|
||||
|
||||
// Remove interleaving pending compactions before rolling back commits
|
||||
pendingCompactionToRollback.stream().forEach(this::deletePendingCompaction);
|
||||
pendingCompactionToRollback.forEach(this::deletePendingCompaction);
|
||||
|
||||
List<HoodieRollbackStat> stats = table.rollback(jsc, commitsToRollback);
|
||||
|
||||
// cleanup index entries
|
||||
commitsToRollback.stream().forEach(s -> {
|
||||
commitsToRollback.forEach(s -> {
|
||||
if (!index.rollbackCommit(s)) {
|
||||
throw new HoodieRollbackException("Rollback index changes failed, for time :" + s);
|
||||
}
|
||||
@@ -1076,7 +1076,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
// everything
|
||||
// so pick it from one of the records.
|
||||
return new HoodieRecord<T>(rec1.getKey(), reducedData);
|
||||
}, parallelism).map(recordTuple -> recordTuple._2());
|
||||
}, parallelism).map(Tuple2::_2);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -1099,7 +1099,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
// Create a Hoodie table which encapsulated the commits and files visible
|
||||
HoodieTable table = HoodieTable.getHoodieTable(
|
||||
new HoodieTableMetaClient(jsc.hadoopConfiguration(), config.getBasePath(), true), config, jsc);
|
||||
if (table.getMetaClient().getCommitActionType() == HoodieTimeline.COMMIT_ACTION) {
|
||||
if (table.getMetaClient().getCommitActionType().equals(HoodieTimeline.COMMIT_ACTION)) {
|
||||
writeContext = metrics.getCommitCtx();
|
||||
} else {
|
||||
writeContext = metrics.getDeltaCommitCtx();
|
||||
@@ -1214,7 +1214,7 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> implements Seriali
|
||||
|
||||
private HoodieCommitMetadata doCompactionCommit(JavaRDD<WriteStatus> writeStatuses,
|
||||
HoodieTableMetaClient metaClient, String compactionCommitTime, Optional<Map<String, String>> extraMetadata) {
|
||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(writeStatus -> writeStatus.getStat())
|
||||
List<HoodieWriteStat> updateStatusMap = writeStatuses.map(WriteStatus::getStat)
|
||||
.collect();
|
||||
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||
|
||||
@@ -117,10 +117,9 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
|
||||
if (SparkEnv.get() != null) {
|
||||
// 1 GB is the default conf used by Spark, look at SparkContext.scala
|
||||
long executorMemoryInBytes = Long.valueOf(
|
||||
Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
|
||||
long executorMemoryInBytes = Utils.memoryStringToMb(SparkEnv.get().conf().get(SPARK_EXECUTOR_MEMORY_PROP,
|
||||
DEFAULT_SPARK_EXECUTOR_MEMORY_MB)) * 1024
|
||||
* 1024L);
|
||||
* 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}
|
||||
|
||||
@@ -187,13 +187,11 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
for (String partitionPath : recordsPerPartition.keySet()) {
|
||||
long numRecords = recordsPerPartition.get(partitionPath);
|
||||
long numFiles =
|
||||
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
||||
: 1L;
|
||||
filesPerPartition.getOrDefault(partitionPath, 1L);
|
||||
|
||||
totalComparisons += numFiles * numRecords;
|
||||
totalFiles +=
|
||||
filesPerPartition.containsKey(partitionPath) ? filesPerPartition.get(partitionPath)
|
||||
: 0L;
|
||||
filesPerPartition.getOrDefault(partitionPath, 0L);
|
||||
totalRecords += numRecords;
|
||||
}
|
||||
logger.info("TotalRecords: " + totalRecords + ", TotalFiles: " + totalFiles
|
||||
@@ -340,7 +338,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
}
|
||||
}
|
||||
return recordComparisons;
|
||||
}).flatMapToPair(t -> t.iterator());
|
||||
}).flatMapToPair(List::iterator);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -369,7 +367,7 @@ public class HoodieBloomIndex<T extends HoodieRecordPayload> extends HoodieIndex
|
||||
|
||||
return fileSortedTripletRDD.mapPartitionsWithIndex(
|
||||
new HoodieBloomIndexCheckFunction(metaClient, config.getBasePath()), true)
|
||||
.flatMap(indexLookupResults -> indexLookupResults.iterator())
|
||||
.flatMap(List::iterator)
|
||||
.filter(lookupResult -> lookupResult.getMatchingRecordKeys().size() > 0)
|
||||
.flatMapToPair(lookupResult -> {
|
||||
List<Tuple2<String, String>> vals = new ArrayList<>();
|
||||
|
||||
@@ -104,7 +104,7 @@ public class HoodieGlobalBloomIndex<T extends HoodieRecordPayload> extends Hoodi
|
||||
}
|
||||
}
|
||||
return recordComparisons;
|
||||
}).flatMapToPair(t -> t.iterator());
|
||||
}).flatMapToPair(List::iterator);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -97,7 +97,7 @@ public class ConsistencyCheck implements Serializable {
|
||||
.filter(p -> !fileNames.contains(new Path(basePath, p).getName()))
|
||||
.collect(Collectors.toList());
|
||||
})
|
||||
.flatMap(itr -> itr.iterator()).collect();
|
||||
.flatMap(List::iterator).collect();
|
||||
if (remainingPaths.size() == 0) {
|
||||
break; // we are done.
|
||||
}
|
||||
|
||||
@@ -65,9 +65,9 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload> extends HoodieIOH
|
||||
private final WriteStatus writeStatus;
|
||||
private final String fileId;
|
||||
// Buffer for holding records in memory before they are flushed to disk
|
||||
List<IndexedRecord> recordList = new ArrayList<>();
|
||||
private List<IndexedRecord> recordList = new ArrayList<>();
|
||||
// Buffer for holding records (to be deleted) in memory before they are flushed to disk
|
||||
List<String> keysToDelete = new ArrayList<>();
|
||||
private List<String> keysToDelete = new ArrayList<>();
|
||||
private TableFileSystemView.RealtimeView fileSystemView;
|
||||
private String partitionPath;
|
||||
private Iterator<HoodieRecord<T>> recordItr;
|
||||
|
||||
@@ -168,8 +168,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
// The window of commit retain == max query run time. So a query could be running which
|
||||
// still
|
||||
// uses this file.
|
||||
if (fileCommitTime.equals(lastVersion) || (lastVersionBeforeEarliestCommitToRetain != null
|
||||
&& fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
if (fileCommitTime.equals(lastVersion) || (fileCommitTime.equals(lastVersionBeforeEarliestCommitToRetain))) {
|
||||
// move on to the next file
|
||||
continue;
|
||||
}
|
||||
@@ -180,9 +179,7 @@ public class HoodieCleanHelper<T extends HoodieRecordPayload<T>> {
|
||||
.compareTimestamps(earliestCommitToRetain.getTimestamp(), fileCommitTime,
|
||||
HoodieTimeline.GREATER)) {
|
||||
// this is a commit, that should be cleaned.
|
||||
if (aFile.isPresent()) {
|
||||
deletePaths.add(aFile.get().getFileStatus().getPath().toString());
|
||||
}
|
||||
aFile.ifPresent(hoodieDataFile -> deletePaths.add(hoodieDataFile.getFileStatus().getPath().toString()));
|
||||
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())
|
||||
|
||||
@@ -195,12 +195,11 @@ public class HoodieCommitArchiveLog {
|
||||
|
||||
// Remove older meta-data from auxiliary path too
|
||||
Optional<HoodieInstant> latestCommitted =
|
||||
archivedInstants.stream()
|
||||
.filter(i -> {
|
||||
return i.isCompleted()
|
||||
&& (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION)));
|
||||
})
|
||||
.sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()).findFirst();
|
||||
archivedInstants.stream()
|
||||
.filter(i -> {
|
||||
return i.isCompleted()
|
||||
&& (i.getAction().equals(COMMIT_ACTION) || (i.getAction().equals(DELTA_COMMIT_ACTION)));
|
||||
}).max(Comparator.comparing(HoodieInstant::getTimestamp));
|
||||
if (latestCommitted.isPresent()) {
|
||||
success &= deleteAllInstantsOlderorEqualsInAuxMetaFolder(latestCommitted.get());
|
||||
}
|
||||
|
||||
@@ -254,9 +254,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload> extends HoodieIOHa
|
||||
public WriteStatus close() {
|
||||
try {
|
||||
// write out any pending records (this can happen when inserts are turned into updates)
|
||||
Iterator<String> pendingRecordsItr = keyToNewRecords.keySet().iterator();
|
||||
while (pendingRecordsItr.hasNext()) {
|
||||
String key = pendingRecordsItr.next();
|
||||
for (String key : keyToNewRecords.keySet()) {
|
||||
if (!writtenRecordKeys.contains(key)) {
|
||||
HoodieRecord<T> hoodieRecord = keyToNewRecords.get(key);
|
||||
writeRecord(hoodieRecord, hoodieRecord.getData().getInsertValue(schema));
|
||||
|
||||
@@ -90,7 +90,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
log.info("Compactor compacting " + operations + " files");
|
||||
return jsc.parallelize(operations, operations.size())
|
||||
.map(s -> compact(table, metaClient, config, s, compactionInstantTime))
|
||||
.flatMap(writeStatusesItr -> writeStatusesItr.iterator());
|
||||
.flatMap(List::iterator);
|
||||
}
|
||||
|
||||
private List<WriteStatus> compact(HoodieCopyOnWriteTable hoodieCopyOnWriteTable, HoodieTableMetaClient metaClient,
|
||||
@@ -141,7 +141,7 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
}
|
||||
Iterable<List<WriteStatus>> resultIterable = () -> result;
|
||||
return StreamSupport.stream(resultIterable.spliterator(), false).flatMap(Collection::stream)
|
||||
.map(s -> {
|
||||
.peek(s -> {
|
||||
s.getStat().setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
|
||||
s.getStat().setTotalLogFilesCompacted(scanner.getTotalLogFiles());
|
||||
s.getStat().setTotalLogRecords(scanner.getTotalLogRecords());
|
||||
@@ -154,7 +154,6 @@ public class HoodieRealtimeTableCompactor implements HoodieCompactor {
|
||||
RuntimeStats runtimeStats = new RuntimeStats();
|
||||
runtimeStats.setTotalScanTime(scanner.getTotalTimeTakenToReadAndMergeBlocks());
|
||||
s.getStat().setRuntimeStats(runtimeStats);
|
||||
return s;
|
||||
}).collect(toList());
|
||||
}
|
||||
|
||||
|
||||
@@ -74,7 +74,7 @@ public abstract class CompactionStrategy implements Serializable {
|
||||
metrics.put(TOTAL_IO_WRITE_MB, totalIOWrite.doubleValue());
|
||||
metrics.put(TOTAL_IO_MB, totalIO.doubleValue());
|
||||
metrics.put(TOTAL_LOG_FILE_SIZE, totalLogFileSize.doubleValue());
|
||||
metrics.put(TOTAL_LOG_FILES, Double.valueOf(logFiles.size()));
|
||||
metrics.put(TOTAL_LOG_FILES, (double) logFiles.size());
|
||||
return metrics;
|
||||
}
|
||||
|
||||
|
||||
@@ -139,13 +139,8 @@ public class HoodieWrapperFileSystem extends FileSystem {
|
||||
return fsDataOutputStream;
|
||||
}
|
||||
|
||||
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(fsDataOutputStream,
|
||||
new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
openStreams.remove(path.getName());
|
||||
}
|
||||
});
|
||||
SizeAwareFSDataOutputStream os = new SizeAwareFSDataOutputStream(
|
||||
fsDataOutputStream, () -> openStreams.remove(path.getName()));
|
||||
openStreams.put(path.getName(), os);
|
||||
return os;
|
||||
}
|
||||
|
||||
@@ -145,7 +145,7 @@ 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,
|
||||
.format("Sending rollback metrics (duration=%d, numFilesDeleted=%d)", durationInMs,
|
||||
numFilesDeleted));
|
||||
registerGauge(getMetricsName("rollback", "duration"), durationInMs);
|
||||
registerGauge(getMetricsName("rollback", "numFilesDeleted"), numFilesDeleted);
|
||||
@@ -180,12 +180,7 @@ public class HoodieMetrics {
|
||||
void registerGauge(String metricName, final long value) {
|
||||
try {
|
||||
MetricRegistry registry = Metrics.getInstance().getRegistry();
|
||||
registry.register(metricName, new Gauge<Long>() {
|
||||
@Override
|
||||
public Long getValue() {
|
||||
return value;
|
||||
}
|
||||
});
|
||||
registry.register(metricName, (Gauge<Long>) () -> value);
|
||||
} catch (Exception e) {
|
||||
// Here we catch all exception, so the major upsert pipeline will not be affected if the
|
||||
// metrics system
|
||||
|
||||
@@ -201,17 +201,15 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
"Error in finding the old file path at commit " + commitTime + " for fileId: " + fileId);
|
||||
} else {
|
||||
AvroReadSupport.setAvroReadSchema(getHadoopConf(), upsertHandle.getSchema());
|
||||
ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
||||
.withConf(getHadoopConf()).build();
|
||||
BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
|
||||
try {
|
||||
try (ParquetReader<IndexedRecord> reader = AvroParquetReader.<IndexedRecord>builder(upsertHandle.getOldFilePath())
|
||||
.withConf(getHadoopConf()).build()) {
|
||||
wrapper = new SparkBoundedInMemoryExecutor(config, new ParquetReaderIterator(reader),
|
||||
new UpdateHandler(upsertHandle), x -> x);
|
||||
new UpdateHandler(upsertHandle), x -> x);
|
||||
wrapper.execute();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException(e);
|
||||
} finally {
|
||||
reader.close();
|
||||
upsertHandle.close();
|
||||
if (null != wrapper) {
|
||||
wrapper.shutdownNow();
|
||||
@@ -480,7 +478,7 @@ public class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
|
||||
.merge(e2)).collect();
|
||||
|
||||
Map<String, PartitionCleanStat> partitionCleanStatsMap = partitionCleanStats.stream()
|
||||
.collect(Collectors.toMap(e -> e._1(), e -> e._2()));
|
||||
.collect(Collectors.toMap(Tuple2::_1, Tuple2::_2));
|
||||
|
||||
HoodieCleanHelper cleaner = new HoodieCleanHelper(this, config);
|
||||
// Return PartitionCleanStat for each partition passed.
|
||||
|
||||
@@ -52,16 +52,7 @@ import com.uber.hoodie.io.HoodieAppendHandle;
|
||||
import com.uber.hoodie.io.compact.HoodieRealtimeTableCompactor;
|
||||
import java.io.IOException;
|
||||
import java.io.UncheckedIOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.*;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -185,105 +176,103 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
.getTimelineOfActions(Sets.newHashSet(HoodieActiveTimeline.COMMIT_ACTION,
|
||||
HoodieActiveTimeline.DELTA_COMMIT_ACTION, HoodieActiveTimeline.COMPACTION_ACTION)).getInstants()
|
||||
.filter(i -> commits.contains(i.getTimestamp()))
|
||||
.collect(Collectors.toMap(i -> i.getTimestamp(), i -> i));
|
||||
.collect(Collectors.toMap(HoodieInstant::getTimestamp, i -> i));
|
||||
|
||||
// Atomically un-publish all non-inflight commits
|
||||
commitsAndCompactions.entrySet().stream().map(entry -> entry.getValue())
|
||||
commitsAndCompactions.entrySet().stream().map(Map.Entry::getValue)
|
||||
.filter(i -> !i.isInflight()).forEach(this.getActiveTimeline()::revertToInflight);
|
||||
logger.info("Unpublished " + commits);
|
||||
Long startTime = System.currentTimeMillis();
|
||||
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;
|
||||
// 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 commits.contains(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 commits.contains(fileCommitTime);
|
||||
}
|
||||
return false;
|
||||
};
|
||||
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
Map<FileStatus, Boolean> results = super
|
||||
.deleteCleanedFiles(partitionPath, Arrays.asList(commit));
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(results).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, instant.getAction(), instant.getTimestamp()))
|
||||
.get(), HoodieCommitMetadata.class);
|
||||
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
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);
|
||||
}
|
||||
default:
|
||||
break;
|
||||
.map((Function<String, List<HoodieRollbackStat>>) partitionPath -> commits.stream().map(commit -> {
|
||||
HoodieInstant instant = commitsAndCompactions.get(commit);
|
||||
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 commits.contains(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 commits.contains(fileCommitTime);
|
||||
}
|
||||
return hoodieRollbackStats;
|
||||
}).collect(Collectors.toList());
|
||||
}).flatMap(x -> x.iterator()).filter(x -> x != null).collect();
|
||||
return false;
|
||||
};
|
||||
|
||||
switch (instant.getAction()) {
|
||||
case HoodieTimeline.COMMIT_ACTION:
|
||||
case HoodieTimeline.COMPACTION_ACTION:
|
||||
try {
|
||||
Map<FileStatus, Boolean> results = super
|
||||
.deleteCleanedFiles(partitionPath, Collections.singletonList(commit));
|
||||
hoodieRollbackStats = HoodieRollbackStat.newBuilder()
|
||||
.withPartitionPath(partitionPath).withDeletedFileResults(results).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, instant.getAction(), instant.getTimestamp()))
|
||||
.get(), HoodieCommitMetadata.class);
|
||||
|
||||
// read commit file and (either append delete blocks or delete file)
|
||||
final Map<FileStatus, Boolean> filesToDeletedStatus = new HashMap<>();
|
||||
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);
|
||||
}
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return hoodieRollbackStats;
|
||||
}).collect(Collectors.toList())).flatMap(List::iterator).filter(Objects::nonNull).collect();
|
||||
|
||||
commitsAndCompactions.entrySet().stream().map(
|
||||
entry -> new HoodieInstant(true, entry.getValue().getAction(),
|
||||
@@ -312,9 +301,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
Optional<String> lastRollingStat = Optional.ofNullable(commitMetadata.getExtraMetadata()
|
||||
.get(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY));
|
||||
if (lastRollingStat.isPresent()) {
|
||||
HoodieRollingStatMetadata rollingStatMetadata = HoodieCommitMetadata
|
||||
return HoodieCommitMetadata
|
||||
.fromBytes(lastRollingStat.get().getBytes(), HoodieRollingStatMetadata.class);
|
||||
return rollingStatMetadata;
|
||||
}
|
||||
}
|
||||
return null;
|
||||
@@ -411,8 +399,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
HoodieRollingStat rollingStatForFile = partitionRollingStats.get(fileSlice.getFileId());
|
||||
if (rollingStatForFile != null) {
|
||||
long inserts = rollingStatForFile.getInserts();
|
||||
long totalSize = averageRecordSize * inserts;
|
||||
return totalSize;
|
||||
return averageRecordSize * inserts;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -427,10 +414,7 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
private boolean isSmallFile(String partitionPath, FileSlice fileSlice) {
|
||||
long totalSize = getTotalFileSize(partitionPath, fileSlice);
|
||||
if (totalSize < config.getParquetMaxFileSize()) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
return totalSize < config.getParquetMaxFileSize();
|
||||
}
|
||||
|
||||
// TODO (NA) : Make this static part of utility
|
||||
@@ -470,11 +454,8 @@ public class HoodieMergeOnReadTable<T extends HoodieRecordPayload> extends
|
||||
commitMetadata.getPartitionToWriteStats().get(partitionPath).stream()
|
||||
.filter(wStat -> {
|
||||
// Filter out stats without prevCommit since they are all inserts
|
||||
if (wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
|
||||
&& !deletedFiles.contains(wStat.getFileId())) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
return wStat != null && wStat.getPrevCommit() != HoodieWriteStat.NULL_COMMIT && wStat.getPrevCommit() != null
|
||||
&& !deletedFiles.contains(wStat.getFileId());
|
||||
}).forEach(wStat -> {
|
||||
HoodieLogFormat.Writer writer = null;
|
||||
String baseCommitTime = wStat.getPrevCommit();
|
||||
|
||||
Reference in New Issue
Block a user