org.apache.hudi.common.table.view.AbstractTableFileSystemView Maven / Gradle / Ivy
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.common.table.view;
import org.apache.hudi.common.bootstrap.index.BootstrapIndex;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.BootstrapBaseFileMapping;
import org.apache.hudi.common.model.BootstrapFileMapping;
import org.apache.hudi.common.model.CompactionOperation;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieFileGroupId;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodiePartitionMetadata;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.common.table.timeline.CompletionTimeQueryView;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.ClusteringUtils;
import org.apache.hudi.common.util.CompactionUtils;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.storage.StoragePath;
import org.apache.hudi.storage.StoragePathInfo;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.Serializable;
import java.util.AbstractMap;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
import java.util.function.Function;
import java.util.function.Predicate;
import java.util.regex.Matcher;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS;
import static org.apache.hudi.common.table.timeline.InstantComparison.EQUALS;
import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN;
import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS;
import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN;
import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS;
import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps;
/**
* Common thread-safe implementation for multiple TableFileSystemView Implementations.
* Provides uniform handling of:
*
* - Loading file-system views from underlying file-system;
* - Pending compaction operations and changing file-system views based on that;
* - Thread-safety in loading and managing file system views for this table;
* - resetting file-system views.
*
* The actual mechanism of fetching file slices from different view storages is delegated to sub-classes.
*/
public abstract class AbstractTableFileSystemView implements SyncableFileSystemView, Serializable {
private static final Logger LOG = LoggerFactory.getLogger(AbstractTableFileSystemView.class);
protected HoodieTableMetaClient metaClient;
protected CompletionTimeQueryView completionTimeQueryView;
// This is the commits timeline that will be visible for all views extending this view
// This is nothing but the write timeline, which contains both ingestion and compaction(major and minor) writers.
private HoodieTimeline visibleCommitsAndCompactionTimeline;
// Used to concurrently load and populate partition views
private final ConcurrentHashMap addedPartitions = new ConcurrentHashMap<>(4096);
// Locks to control concurrency. Sync operations use write-lock blocking all fetch operations.
// For the common-case, we allow concurrent read of single or multiple partitions
private final ReentrantReadWriteLock globalLock = new ReentrantReadWriteLock();
protected final ReadLock readLock = globalLock.readLock();
protected final WriteLock writeLock = globalLock.writeLock();
private BootstrapIndex bootstrapIndex;
/**
* Initialize the view.
*/
protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) {
this.metaClient = metaClient;
this.completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient);
refreshTimeline(visibleActiveTimeline);
resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline);
this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient);
// Load Pending Compaction Operations
resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream()
.map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
// Load Pending LogCompaction Operations.
resetPendingLogCompactionOperations(CompactionUtils.getAllPendingLogCompactionOperations(metaClient).values().stream()
.map(e -> Pair.of(e.getKey(), CompactionOperation.convertFromAvroRecordInstance(e.getValue()))));
resetBootstrapBaseFileMapping(Stream.empty());
resetFileGroupsInPendingClustering(ClusteringUtils.getAllFileGroupsInPendingClusteringPlans(metaClient));
}
/**
* Refresh commits timeline.
*
* @param visibleActiveTimeline Visible Active Timeline
*/
protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) {
this.visibleCommitsAndCompactionTimeline = visibleActiveTimeline.getWriteTimeline();
}
/**
* Refresh the completion time query view.
*/
protected void refreshCompletionTimeQueryView() {
this.completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient);
}
/**
* Returns the completion time for instant.
*/
public Option getCompletionTime(String instantTime) {
return this.completionTimeQueryView.getCompletionTime(instantTime, instantTime);
}
/**
* Adds the provided statuses into the file system view, and also caches it inside this object.
* If the file statuses are limited to a single partition, use {@link #addFilesToView(String, List)} instead.
*/
public List addFilesToView(List statuses) {
Map> statusesByPartitionPath = statuses.stream()
.collect(Collectors.groupingBy(fileStatus -> FSUtils.getRelativePartitionPath(metaClient.getBasePath(), fileStatus.getPath().getParent())));
return statusesByPartitionPath.entrySet().stream().map(entry -> addFilesToView(entry.getKey(), entry.getValue()))
.flatMap(List::stream).collect(Collectors.toList());
}
/**
* Adds the provided statuses into the file system view for a single partition, and also caches it inside this object.
*/
public List addFilesToView(String partitionPath, List statuses) {
HoodieTimer timer = HoodieTimer.start();
List fileGroups = buildFileGroups(partitionPath, statuses, visibleCommitsAndCompactionTimeline, true);
long fgBuildTimeTakenMs = timer.endTimer();
timer.startTimer();
// Group by partition for efficient updates for both InMemory and DiskBased structures.
fileGroups.stream().collect(Collectors.groupingBy(HoodieFileGroup::getPartitionPath))
.forEach((partition, value) -> {
if (!isPartitionAvailableInStore(partition)) {
if (bootstrapIndex.useIndex()) {
try (BootstrapIndex.IndexReader reader = bootstrapIndex.createReader()) {
LOG.info("Bootstrap Index available for partition " + partition);
List sourceFileMappings =
reader.getSourceFileMappingForPartition(partition);
addBootstrapBaseFileMapping(sourceFileMappings.stream()
.map(s -> new BootstrapBaseFileMapping(new HoodieFileGroupId(s.getPartitionPath(),
s.getFileId()), s.getBootstrapFileStatus())));
}
}
storePartitionView(partition, value);
}
});
long storePartitionsTs = timer.endTimer();
LOG.debug("addFilesToView: NumFiles=" + statuses.size() + ", NumFileGroups=" + fileGroups.size()
+ ", FileGroupsCreationTime=" + fgBuildTimeTakenMs
+ ", StoreTimeTaken=" + storePartitionsTs);
return fileGroups;
}
/**
* Build FileGroups from passed in file-status.
*/
protected List buildFileGroups(String partition, List statuses, HoodieTimeline timeline,
boolean addPendingCompactionFileSlice) {
return buildFileGroups(partition, convertFileStatusesToBaseFiles(statuses), convertFileStatusesToLogFiles(statuses),
timeline,
addPendingCompactionFileSlice);
}
protected List buildFileGroups(String partition, Stream baseFileStream,
Stream logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) {
Map> baseFiles =
baseFileStream.collect(Collectors.groupingBy(HoodieBaseFile::getFileId));
Map> logFiles = logFileStream.collect(Collectors.groupingBy(HoodieLogFile::getFileId));
Set fileIdSet = new HashSet<>(baseFiles.keySet());
fileIdSet.addAll(logFiles.keySet());
List fileGroups = new ArrayList<>(fileIdSet.size());
fileIdSet.forEach(fileId -> {
HoodieFileGroup group = new HoodieFileGroup(partition, fileId, timeline);
if (baseFiles.containsKey(fileId)) {
baseFiles.get(fileId).forEach(group::addBaseFile);
}
if (addPendingCompactionFileSlice) {
// pending compaction file slice must be added before log files so that
// the log files completed later than the compaction instant time could be included
// in the file slice with that compaction instant time as base instant time.
Option> pendingCompaction =
getPendingCompactionOperationWithInstant(group.getFileGroupId());
if (pendingCompaction.isPresent()) {
// If there is no delta-commit after compaction request, this step would ensure a new file-slice appears
// so that any new ingestion uses the correct base-instant
group.addNewFileSliceAtInstant(pendingCompaction.get().getKey());
}
}
if (logFiles.containsKey(fileId)) {
// this should work for both table versions >= 8 and lower.
logFiles.get(fileId).stream().sorted(HoodieLogFile.getLogFileComparator())
.forEach(logFile -> group.addLogFile(completionTimeQueryView, logFile));
}
fileGroups.add(group);
});
return fileGroups;
}
private boolean tableVersion8AndAbove() {
HoodieTableVersion tableVersion = metaClient.getTableConfig().getTableVersion();
return tableVersion.greaterThanOrEquals(HoodieTableVersion.EIGHT);
}
/**
* Get replaced instant for each file group by looking at all commit instants.
*/
private void resetFileGroupsReplaced(HoodieTimeline timeline) {
HoodieTimer hoodieTimer = HoodieTimer.start();
// for each REPLACE instant, get map of (partitionPath -> deleteFileGroup)
HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline();
Stream> resultStream = replacedTimeline.getInstantsAsStream().flatMap(instant -> {
try {
HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(),
HoodieReplaceCommitMetadata.class);
// get replace instant mapping for each partition, fileId
return replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().flatMap(entry -> entry.getValue().stream().map(e ->
new AbstractMap.SimpleEntry<>(new HoodieFileGroupId(entry.getKey(), e), instant)));
} catch (HoodieIOException ex) {
if (ex.getIOException() instanceof FileNotFoundException) {
// Replace instant could be deleted by archive and FileNotFoundException could be threw during getInstantDetails function
// So that we need to catch the FileNotFoundException here and continue
LOG.warn(ex.getMessage());
return Stream.empty();
} else {
throw ex;
}
} catch (IOException e) {
throw new HoodieIOException("error reading commit metadata for " + instant, e);
}
});
// Duplicate key error when insert_overwrite same partition in multi writer, keep the instant with greater timestamp when the file group id conflicts
Map replacedFileGroups = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue,
(instance1, instance2) -> compareTimestamps(instance1.requestedTime(), LESSER_THAN, instance2.requestedTime()) ? instance2 : instance1));
resetReplacedFileGroups(replacedFileGroups);
LOG.info("Took " + hoodieTimer.endTimer() + " ms to read " + replacedTimeline.countInstants() + " instants, "
+ replacedFileGroups.size() + " replaced file groups");
}
@Override
public void close() {
try {
writeLock.lock();
this.metaClient = null;
this.completionTimeQueryView = null;
this.visibleCommitsAndCompactionTimeline = null;
clear();
} finally {
writeLock.unlock();
}
}
/**
* Clears the partition Map and reset view states.
*
* NOTE: The logic MUST BE guarded by the write lock.
*/
@Override
public void reset() {
try {
writeLock.lock();
clear();
// Initialize with new Hoodie timeline.
init(metaClient, getTimeline());
} finally {
writeLock.unlock();
}
}
/**
* Clear the resource.
*/
protected void clear() {
addedPartitions.clear();
resetViewState();
bootstrapIndex = null;
}
/**
* Allows all view metadata in file system view storage to be reset by subclasses.
*/
protected abstract void resetViewState();
/**
* Batch loading all the partitions if needed.
*
* @return A list of relative partition paths of all partitions.
*/
private List ensureAllPartitionsLoadedCorrectly() {
ValidationUtils.checkArgument(!isClosed(), "View is already closed");
try {
List formattedPartitionList = getAllPartitionPaths().stream()
.map(this::formatPartitionKey).collect(Collectors.toList());
ensurePartitionsLoadedCorrectly(formattedPartitionList);
return formattedPartitionList;
} catch (IOException e) {
throw new HoodieIOException("Failed to get all partition paths", e);
}
}
/**
* Allows lazily loading the partitions if needed.
*
* @param partitionList list of partitions to be loaded if not present.
*/
private void ensurePartitionsLoadedCorrectly(List partitionList) {
ValidationUtils.checkArgument(!isClosed(), "View is already closed");
Set partitionSet = new HashSet<>();
synchronized (addedPartitions) {
partitionList.forEach(partition -> {
if (!addedPartitions.containsKey(partition) && !isPartitionAvailableInStore(partition)) {
partitionSet.add(partition);
}
});
if (!partitionSet.isEmpty()) {
long beginTs = System.currentTimeMillis();
// Not loaded yet
try {
LOG.debug("Building file system view for partitions: " + partitionSet);
// Pairs of relative partition path and absolute partition path
List> absolutePartitionPathList = partitionSet.stream()
.map(partition -> Pair.of(
partition, FSUtils.constructAbsolutePath(metaClient.getBasePath(), partition)))
.collect(Collectors.toList());
long beginLsTs = System.currentTimeMillis();
Map, List> pathInfoMap =
listPartitions(absolutePartitionPathList);
long endLsTs = System.currentTimeMillis();
LOG.debug("Time taken to list partitions " + partitionSet + " =" + (endLsTs - beginLsTs));
pathInfoMap.forEach((partitionPair, statuses) -> {
String relativePartitionStr = partitionPair.getLeft();
List groups = addFilesToView(relativePartitionStr, statuses);
if (groups.isEmpty()) {
storePartitionView(relativePartitionStr, Collections.emptyList());
}
LOG.debug("#files found in partition (" + relativePartitionStr + ") =" + statuses.size());
});
} catch (IOException e) {
throw new HoodieIOException("Failed to list base files in partitions " + partitionSet, e);
}
long endTs = System.currentTimeMillis();
LOG.debug("Time to load partition " + partitionSet + " =" + (endTs - beginTs));
}
partitionSet.forEach(partition ->
addedPartitions.computeIfAbsent(partition, partitionPathStr -> true)
);
}
}
/**
* @return A list of relative partition paths of all partitions.
* @throws IOException upon error.
*/
protected List getAllPartitionPaths() throws IOException {
throw new HoodieException("Getting all partition paths with file system listing sequentially "
+ "can be very slow. This should not be invoked.");
}
/**
* @param partitionPathList A list of pairs of the relative and absolute paths of the partitions.
* @return all the files from the partitions.
* @throws IOException upon error.
*/
protected Map, List> listPartitions(
List> partitionPathList) throws IOException {
Map, List> pathInfoMap = new HashMap<>();
for (Pair partitionPair : partitionPathList) {
StoragePath absolutePartitionPath = partitionPair.getRight();
try {
pathInfoMap.put(partitionPair,
metaClient.getStorage().listDirectEntries(absolutePartitionPath));
} catch (IOException e) {
if (!metaClient.getStorage().exists(absolutePartitionPath)) {
pathInfoMap.put(partitionPair, Collections.emptyList());
} else {
// in case the partition path was created by another caller
pathInfoMap.put(partitionPair,
metaClient.getStorage().listDirectEntries(absolutePartitionPath));
}
}
}
return pathInfoMap;
}
/**
* Returns all files situated at the given partition.
*/
private List getAllFilesInPartition(String relativePartitionPath)
throws IOException {
StoragePath partitionPath = FSUtils.constructAbsolutePath(metaClient.getBasePath(),
relativePartitionPath);
long beginLsTs = System.currentTimeMillis();
List pathInfoList = listPartition(partitionPath);
long endLsTs = System.currentTimeMillis();
LOG.debug(
"#files found in partition (" + relativePartitionPath + ") =" + pathInfoList.size()
+ ", " + "Time taken =" + (endLsTs - beginLsTs));
return pathInfoList;
}
/**
* Allows lazily loading the partitions if needed.
*
* @param partition partition to be loaded if not present
*/
protected void ensurePartitionLoadedCorrectly(String partition) {
ValidationUtils.checkArgument(!isClosed(), "View is already closed");
// ensure we list files only once even in the face of concurrency
addedPartitions.computeIfAbsent(partition, (partitionPathStr) -> {
long beginTs = System.currentTimeMillis();
if (!isPartitionAvailableInStore(partitionPathStr)) {
// Not loaded yet
try {
LOG.info("Building file system view for partition (" + partitionPathStr + ")");
List groups = addFilesToView(partitionPathStr, getAllFilesInPartition(partitionPathStr));
if (groups.isEmpty()) {
storePartitionView(partitionPathStr, new ArrayList<>());
}
} catch (IOException e) {
throw new HoodieIOException("Failed to list base files in partition " + partitionPathStr, e);
}
} else {
LOG.debug("View already built for Partition :" + partitionPathStr + ", FOUND is ");
}
long endTs = System.currentTimeMillis();
LOG.debug("Time to load partition (" + partitionPathStr + ") =" + (endTs - beginTs));
return true;
});
}
/**
* Return all the files from the partition.
*
* @param partitionPath The absolute path of the partition
* @throws IOException
*/
protected List listPartition(StoragePath partitionPath) throws IOException {
try {
return metaClient.getStorage().listDirectEntries(partitionPath);
} catch (IOException e) {
if (!metaClient.getStorage().exists(partitionPath)) {
return Collections.emptyList();
} else {
// in case the partition path was created by another caller
return metaClient.getStorage().listDirectEntries(partitionPath);
}
}
}
/**
* Helper to convert file-status to base-files.
*
* @param pathInfoList List of StoragePathInfo
*/
private Stream convertFileStatusesToBaseFiles(List pathInfoList) {
Predicate roFilePredicate = pathInfo -> {
String pathName = pathInfo.getPath().getName();
// Filter base files if:
// 1. file extension equals to table configured file extension
// 2. file is not .hoodie_partition_metadata
if (pathName.startsWith(HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX)) {
return false;
} else if (metaClient.getTableConfig().isMultipleBaseFileFormatsEnabled()) {
return pathName.contains(HoodieFileFormat.PARQUET.getFileExtension())
|| pathName.contains(HoodieFileFormat.ORC.getFileExtension());
} else {
return pathName.contains(metaClient.getTableConfig().getBaseFileFormat().getFileExtension());
}
};
return pathInfoList.stream().filter(roFilePredicate).map(HoodieBaseFile::new);
}
/**
* Helper to convert file-status to log-files.
*
* @param pathInfoList List of StoragePathInfo
*/
private Stream convertFileStatusesToLogFiles(List pathInfoList) {
Predicate rtFilePredicate = pathInfo -> {
String fileName = pathInfo.getPath().getName();
Matcher matcher = FSUtils.LOG_FILE_PATTERN.matcher(fileName);
return matcher.find() && fileName.contains(metaClient.getTableConfig().getLogFileFormat().getFileExtension());
};
return pathInfoList.stream().filter(rtFilePredicate).map(HoodieLogFile::new);
}
/**
* With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those
* base-files.
*
* @param partitionPath partition path for the base file
* @param baseFile base File
*/
protected boolean isBaseFileDueToPendingCompaction(String partitionPath, HoodieBaseFile baseFile) {
Option> compactionWithInstantTime =
getPendingCompactionOperationWithInstant(new HoodieFileGroupId(partitionPath, baseFile.getFileId()));
return (compactionWithInstantTime.isPresent()) && (null != compactionWithInstantTime.get().getKey())
&& baseFile.getCommitTime().equals(compactionWithInstantTime.get().getKey());
}
/**
* With async clustering, it is possible to see partial/complete base-files due to inflight-clustering, Ignore those
* base-files.
*
* @param baseFile base File
*/
protected boolean isBaseFileDueToPendingClustering(HoodieBaseFile baseFile) {
return metaClient.getActiveTimeline().isPendingClusteringInstant(baseFile.getCommitTime());
}
/**
* Returns true if the file-group is under pending-compaction and the file-slice' baseInstant matches compaction
* Instant.
*
* @param fileSlice File Slice
*/
private boolean isFileSliceAfterPendingCompaction(FileSlice fileSlice) {
Option> compactionWithInstantTime =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
return (compactionWithInstantTime.isPresent())
&& fileSlice.getBaseInstantTime().equals(compactionWithInstantTime.get().getKey());
}
/**
* With async compaction, it is possible to see partial/complete base-files due to inflight-compactions, Ignore those
* base-files.
*
* @param fileSlice File Slice
* @param includeEmptyFileSlice include empty file-slice
*/
protected Stream filterBaseFileAfterPendingCompaction(FileSlice fileSlice, boolean includeEmptyFileSlice) {
if (isFileSliceAfterPendingCompaction(fileSlice)) {
LOG.debug("File Slice (" + fileSlice + ") is in pending compaction");
// Base file is filtered out of the file-slice as the corresponding compaction
// instant not completed yet.
FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId());
fileSlice.getLogFiles().forEach(transformed::addLogFile);
if (transformed.isEmpty() && !includeEmptyFileSlice) {
return Stream.of();
}
return Stream.of(transformed);
}
return Stream.of(fileSlice);
}
/**
* Ignores the uncommitted base and log files.
*
* @param fileSlice File Slice
* @param includeEmptyFileSlice include empty file-slice
*/
private Stream filterUncommittedFiles(FileSlice fileSlice, boolean includeEmptyFileSlice) {
Option committedBaseFile = fileSlice.getBaseFile().isPresent() && completionTimeQueryView.isCompleted(fileSlice.getBaseInstantTime()) ? fileSlice.getBaseFile() : Option.empty();
List committedLogFiles = fileSlice.getLogFiles().filter(logFile -> completionTimeQueryView.isCompleted(logFile.getDeltaCommitTime())).collect(Collectors.toList());
if ((fileSlice.getBaseFile().isPresent() && !committedBaseFile.isPresent())
|| committedLogFiles.size() != fileSlice.getLogFiles().count()) {
LOG.debug("File Slice ({}) has uncommitted files.", fileSlice);
// A file is filtered out of the file-slice if the corresponding
// instant has not completed yet.
FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId());
committedBaseFile.ifPresent(transformed::setBaseFile);
committedLogFiles.forEach(transformed::addLogFile);
if (transformed.isEmpty() && !includeEmptyFileSlice) {
return Stream.of();
}
return Stream.of(transformed);
}
return Stream.of(fileSlice);
}
/**
* Ignores the uncommitted log files.
*
* @param fileSlice File Slice
*/
private FileSlice filterUncommittedLogs(FileSlice fileSlice) {
List committedLogFiles = fileSlice.getLogFiles().filter(logFile -> completionTimeQueryView.isCompleted(logFile.getDeltaCommitTime())).collect(Collectors.toList());
if (committedLogFiles.size() != fileSlice.getLogFiles().count()) {
LOG.debug("File Slice (" + fileSlice + ") has uncommitted log files.");
// A file is filtered out of the file-slice if the corresponding
// instant has not completed yet.
FileSlice transformed = new FileSlice(fileSlice.getPartitionPath(), fileSlice.getBaseInstantTime(), fileSlice.getFileId());
fileSlice.getBaseFile().ifPresent(transformed::setBaseFile);
committedLogFiles.forEach(transformed::addLogFile);
return transformed;
}
return fileSlice;
}
protected HoodieFileGroup addBootstrapBaseFileIfPresent(HoodieFileGroup fileGroup) {
return addBootstrapBaseFileIfPresent(fileGroup, this::getBootstrapBaseFile);
}
protected HoodieFileGroup addBootstrapBaseFileIfPresent(HoodieFileGroup fileGroup, Function> bootstrapBaseFileMappingFunc) {
boolean hasBootstrapBaseFile = fileGroup.getAllFileSlices()
.anyMatch(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS));
if (hasBootstrapBaseFile) {
HoodieFileGroup newFileGroup = new HoodieFileGroup(fileGroup);
newFileGroup.getAllFileSlices().filter(fs -> fs.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS))
.forEach(fs -> fs.setBaseFile(
addBootstrapBaseFileIfPresent(fs.getFileGroupId(), fs.getBaseFile().get(), bootstrapBaseFileMappingFunc)));
return newFileGroup;
}
return fileGroup;
}
protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice) {
return addBootstrapBaseFileIfPresent(fileSlice, this::getBootstrapBaseFile);
}
protected FileSlice addBootstrapBaseFileIfPresent(FileSlice fileSlice, Function> bootstrapBaseFileMappingFunc) {
if (fileSlice.getBaseInstantTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) {
FileSlice copy = new FileSlice(fileSlice);
copy.getBaseFile().ifPresent(dataFile -> {
Option edf = getBootstrapBaseFile(copy.getFileGroupId());
bootstrapBaseFileMappingFunc.apply(copy.getFileGroupId()).ifPresent(e -> dataFile.setBootstrapBaseFile(e.getBootstrapBaseFile()));
});
return copy;
}
return fileSlice;
}
protected HoodieBaseFile addBootstrapBaseFileIfPresent(HoodieFileGroupId fileGroupId, HoodieBaseFile baseFile) {
return addBootstrapBaseFileIfPresent(fileGroupId, baseFile, this::getBootstrapBaseFile);
}
protected HoodieBaseFile addBootstrapBaseFileIfPresent(
HoodieFileGroupId fileGroupId,
HoodieBaseFile baseFile,
Function> bootstrapBaseFileMappingFunc) {
if (baseFile.getCommitTime().equals(METADATA_BOOTSTRAP_INSTANT_TS)) {
HoodieBaseFile copy = new HoodieBaseFile(baseFile);
bootstrapBaseFileMappingFunc.apply(fileGroupId).ifPresent(e -> copy.setBootstrapBaseFile(e.getBootstrapBaseFile()));
return copy;
}
return baseFile;
}
@Override
public final Stream> getPendingCompactionOperations() {
try {
readLock.lock();
return fetchPendingCompactionOperations();
} finally {
readLock.unlock();
}
}
public final List getPartitionPaths() {
try {
readLock.lock();
return fetchAllStoredFileGroups()
.filter(fg -> !isFileGroupReplaced(fg))
.map(HoodieFileGroup::getPartitionPath)
.distinct()
.map(name -> name.isEmpty() ? metaClient.getBasePath() : new StoragePath(metaClient.getBasePath(), name))
.collect(Collectors.toList());
} finally {
readLock.unlock();
}
}
public final List getPartitionNames() {
try {
readLock.lock();
return fetchAllStoredFileGroups()
.filter(fg -> !isFileGroupReplaced(fg))
.map(HoodieFileGroup::getPartitionPath)
.distinct()
.collect(Collectors.toList());
} finally {
readLock.unlock();
}
}
@Override
public final Stream> getPendingLogCompactionOperations() {
try {
readLock.lock();
return fetchPendingLogCompactionOperations();
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestBaseFiles(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestBaseFiles(partitionPath)
.filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId()))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestBaseFiles() {
try {
readLock.lock();
return fetchLatestBaseFiles();
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestBaseFilesBeforeOrOn(String partitionStr, String maxCommitTime) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return getLatestBaseFilesBeforeOrOnFromCache(partitionPath, maxCommitTime);
} finally {
readLock.unlock();
}
}
@Override
public final Map> getAllLatestBaseFilesBeforeOrOn(String maxCommitTime) {
try {
readLock.lock();
List formattedPartitionList = ensureAllPartitionsLoadedCorrectly();
return formattedPartitionList.stream().collect(Collectors.toMap(
Function.identity(),
partitionPath -> getLatestBaseFilesBeforeOrOnFromCache(partitionPath, maxCommitTime)
));
} finally {
readLock.unlock();
}
}
private Stream getLatestBaseFilesBeforeOrOnFromCache(String partitionPath, String maxCommitTime) {
return fetchAllStoredFileGroups(partitionPath)
.filter(fileGroup -> !isFileGroupReplacedBeforeOrOn(fileGroup.getFileGroupId(), maxCommitTime))
.map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles()
.filter(baseFile -> compareTimestamps(baseFile.getCommitTime(), LESSER_THAN_OR_EQUALS, maxCommitTime
))
.filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df)).findFirst()))
.filter(Option::isPresent).map(Option::get)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
}
@Override
public final Option getBaseFileOn(String partitionStr, String instantTime, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
if (isFileGroupReplacedBeforeOrOn(new HoodieFileGroupId(partitionPath, fileId), instantTime)) {
return Option.empty();
} else {
return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles()
.filter(baseFile -> compareTimestamps(baseFile.getCommitTime(), EQUALS,
instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df)).findFirst().orElse(null))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
}
} finally {
readLock.unlock();
}
}
/**
* Get Latest base file for a partition and file-Id.
*/
@Override
public final Option getLatestBaseFile(String partitionStr, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
if (isFileGroupReplaced(partitionPath, fileId)) {
return Option.empty();
} else {
return fetchLatestBaseFile(partitionPath, fileId)
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df));
}
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestBaseFilesInRange(List commitsToReturn) {
try {
readLock.lock();
return fetchAllStoredFileGroups()
.filter(fileGroup -> !isFileGroupReplacedBeforeAny(fileGroup.getFileGroupId(), commitsToReturn))
.map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional(
fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime())
&& !isBaseFileDueToPendingCompaction(fileGroup.getPartitionPath(), baseFile) && !isBaseFileDueToPendingClustering(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent())
.map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get()));
} finally {
readLock.unlock();
}
}
@Override
public void loadAllPartitions() {
try {
readLock.lock();
ensureAllPartitionsLoadedCorrectly();
} finally {
readLock.unlock();
}
}
@Override
public void loadPartitions(List partitionPaths) {
try {
readLock.lock();
ensurePartitionsLoadedCorrectly(partitionPaths);
} finally {
readLock.unlock();
}
}
@Override
public final Stream getAllBaseFiles(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllBaseFiles(partitionPath)
.filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId()))
.filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime()))
.filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df))
.map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df));
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestFileSlices(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchLatestFileSlices(partitionPath)
.filter(slice -> !isFileGroupReplaced(slice.getFileGroupId()))
.flatMap(slice -> tableVersion8AndAbove()
? this.filterUncommittedFiles(slice, true)
: this.filterBaseFileAfterPendingCompaction(slice, true))
.map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
}
public Stream getLatestFileSlicesIncludingInflight(String partitionPath) {
try {
readLock.lock();
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.map(HoodieFileGroup::getLatestFileSlicesIncludingInflight)
.filter(Option::isPresent)
.map(Option::get);
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestFileSlicesStateless(String partitionStr) {
String partition = formatPartitionKey(partitionStr);
if (isPartitionAvailableInStore(partition)) {
return getLatestFileSlices(partition);
} else {
try {
Stream fileSliceStream = buildFileGroups(partition, getAllFilesInPartition(partition), visibleCommitsAndCompactionTimeline, true).stream()
.filter(fg -> !isFileGroupReplaced(fg))
.map(HoodieFileGroup::getLatestFileSlice)
.filter(Option::isPresent).map(Option::get)
.flatMap(slice -> tableVersion8AndAbove()
? this.filterUncommittedFiles(slice, true)
: this.filterBaseFileAfterPendingCompaction(slice, true));
if (bootstrapIndex.useIndex()) {
final Map bootstrapBaseFileMappings = getBootstrapBaseFileMappings(partition);
if (!bootstrapBaseFileMappings.isEmpty()) {
return fileSliceStream.map(fileSlice -> addBootstrapBaseFileIfPresent(fileSlice, fileGroupId -> Option.ofNullable(bootstrapBaseFileMappings.get(fileGroupId))));
}
}
return fileSliceStream;
} catch (IOException e) {
throw new HoodieIOException("Failed to fetch all files in partition " + partition, e);
}
}
}
/**
* Get Latest File Slice for a given fileId in a given partition.
*/
@Override
public final Option getLatestFileSlice(String partitionStr, String fileId) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
if (isFileGroupReplaced(partitionPath, fileId)) {
return Option.empty();
} else {
Option fs = fetchLatestFileSlice(partitionPath, fileId);
if (!fs.isPresent()) {
return Option.empty();
}
Stream fileSlices = tableVersion8AndAbove()
? this.filterUncommittedFiles(fs.get(), true)
: this.filterBaseFileAfterPendingCompaction(fs.get(), true);
return Option.ofNullable(fileSlices
.map(this::addBootstrapBaseFileIfPresent)
.findFirst()
.orElse(null)
);
}
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestUnCompactedFileSlices(String partitionStr) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
return fetchAllStoredFileGroups(partitionPath)
.filter(fg -> !isFileGroupReplaced(fg.getFileGroupId()))
.map(fileGroup -> {
FileSlice fileSlice = fileGroup.getLatestFileSlice().get();
// if the file-group is under compaction, pick the latest before compaction instant time.
Option> compactionWithInstantPair =
getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId());
if (compactionWithInstantPair.isPresent()) {
String compactionInstantTime = compactionWithInstantPair.get().getLeft();
return fileGroup.getLatestFileSliceBefore(compactionInstantTime);
}
return Option.of(fileSlice);
}).map(Option::get).map(this::addBootstrapBaseFileIfPresent);
} finally {
readLock.unlock();
}
}
@Override
public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr, String maxCommitTime,
boolean includeFileSlicesInPendingCompaction) {
try {
readLock.lock();
String partitionPath = formatPartitionKey(partitionStr);
ensurePartitionLoadedCorrectly(partitionPath);
Stream> allFileSliceStream = fetchAllStoredFileGroups(partitionPath)
.filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime))
.map(fg -> fg.getAllFileSlicesBeforeOn(maxCommitTime));
if (includeFileSlicesInPendingCompaction) {
Stream