[HUDI-3280] Cleaning up Hive-related hierarchies after refactoring (#4743)
This commit is contained in:
@@ -36,6 +36,7 @@ import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
|
||||
import org.apache.hudi.common.util.CollectionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -74,6 +75,7 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
protected final List<Path> queryPaths;
|
||||
|
||||
private final boolean shouldIncludePendingCommits;
|
||||
private final boolean shouldValidateInstant;
|
||||
|
||||
private final HoodieTableType tableType;
|
||||
protected final String basePath;
|
||||
@@ -98,6 +100,7 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
* @param queryPaths target DFS paths being queried
|
||||
* @param specifiedQueryInstant instant as of which table is being queried
|
||||
* @param shouldIncludePendingCommits flags whether file-index should exclude any pending operations
|
||||
* @param shouldValidateInstant flags to validate whether query instant is present in the timeline
|
||||
* @param fileStatusCache transient cache of fetched [[FileStatus]]es
|
||||
*/
|
||||
public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
|
||||
@@ -107,6 +110,7 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
List<Path> queryPaths,
|
||||
Option<String> specifiedQueryInstant,
|
||||
boolean shouldIncludePendingCommits,
|
||||
boolean shouldValidateInstant,
|
||||
FileStatusCache fileStatusCache) {
|
||||
this.partitionColumns = metaClient.getTableConfig().getPartitionFields()
|
||||
.orElse(new String[0]);
|
||||
@@ -122,6 +126,7 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
this.queryPaths = queryPaths;
|
||||
this.specifiedQueryInstant = specifiedQueryInstant;
|
||||
this.shouldIncludePendingCommits = shouldIncludePendingCommits;
|
||||
this.shouldValidateInstant = shouldValidateInstant;
|
||||
|
||||
this.tableType = metaClient.getTableType();
|
||||
this.basePath = metaClient.getBasePath();
|
||||
@@ -142,6 +147,13 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
return getActiveTimeline().filterCompletedInstants().lastInstant();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns table's base-path
|
||||
*/
|
||||
public String getBasePath() {
|
||||
return metaClient.getBasePath();
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch list of latest base files and log files per partition.
|
||||
*
|
||||
@@ -264,6 +276,8 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
Option<String> queryInstant =
|
||||
specifiedQueryInstant.or(() -> latestInstant.map(HoodieInstant::getTimestamp));
|
||||
|
||||
validate(activeTimeline, queryInstant);
|
||||
|
||||
if (tableType.equals(HoodieTableType.MERGE_ON_READ) && queryType.equals(HoodieTableQueryType.SNAPSHOT)) {
|
||||
cachedAllInputFileSlices = partitionFiles.keySet().stream()
|
||||
.collect(Collectors.toMap(
|
||||
@@ -277,15 +291,15 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
)
|
||||
);
|
||||
} else {
|
||||
// TODO re-align with the branch (MOR, snapshot) branch
|
||||
cachedAllInputFileSlices = partitionFiles.keySet().stream()
|
||||
.collect(Collectors.toMap(
|
||||
Function.identity(),
|
||||
partitionPath ->
|
||||
specifiedQueryInstant.map(instant ->
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(partitionPath.path, instant, true))
|
||||
.orElse(fileSystemView.getLatestFileSlices(partitionPath.path))
|
||||
.collect(Collectors.toList())
|
||||
queryInstant.map(instant ->
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(partitionPath.path, instant, true)
|
||||
)
|
||||
.orElse(fileSystemView.getLatestFileSlices(partitionPath.path))
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
);
|
||||
}
|
||||
@@ -303,6 +317,14 @@ public abstract class BaseHoodieTableFileIndex {
|
||||
LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
|
||||
}
|
||||
|
||||
private void validate(HoodieTimeline activeTimeline, Option<String> queryInstant) {
|
||||
if (shouldValidateInstant) {
|
||||
if (queryInstant.isPresent() && !activeTimeline.containsInstant(queryInstant.get())) {
|
||||
throw new HoodieIOException(String.format("Query instant (%s) not found in the timeline", queryInstant.get()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static long fileSliceSize(FileSlice fileSlice) {
|
||||
long logFileSize = fileSlice.getLogFiles().map(HoodieLogFile::getFileSize)
|
||||
.filter(s -> s > 0)
|
||||
|
||||
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.PropertyAccessor;
|
||||
import com.fasterxml.jackson.databind.DeserializationFeature;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -153,10 +154,12 @@ public class HoodieCommitMetadata implements Serializable {
|
||||
* been touched multiple times in the given commits, the return value will keep the one
|
||||
* from the latest commit.
|
||||
*
|
||||
*
|
||||
* @param hadoopConf
|
||||
* @param basePath The base path
|
||||
* @return the file full path to file status mapping
|
||||
*/
|
||||
public Map<String, FileStatus> getFullPathToFileStatus(String basePath) {
|
||||
public Map<String, FileStatus> getFullPathToFileStatus(Configuration hadoopConf, String basePath) {
|
||||
Map<String, FileStatus> fullPathToFileStatus = new HashMap<>();
|
||||
for (List<HoodieWriteStat> stats : getPartitionToWriteStats().values()) {
|
||||
// Iterate through all the written files.
|
||||
@@ -164,7 +167,8 @@ public class HoodieCommitMetadata implements Serializable {
|
||||
String relativeFilePath = stat.getPath();
|
||||
Path fullPath = relativeFilePath != null ? FSUtils.getPartitionPath(basePath, relativeFilePath) : null;
|
||||
if (fullPath != null) {
|
||||
FileStatus fileStatus = new FileStatus(stat.getFileSizeInBytes(), false, 0, 0,
|
||||
long blockSize = FSUtils.getFs(fullPath.toString(), hadoopConf).getDefaultBlockSize(fullPath);
|
||||
FileStatus fileStatus = new FileStatus(stat.getFileSizeInBytes(), false, 0, blockSize,
|
||||
0, fullPath);
|
||||
fullPathToFileStatus.put(fullPath.getName(), fileStatus);
|
||||
}
|
||||
@@ -178,14 +182,16 @@ public class HoodieCommitMetadata implements Serializable {
|
||||
* been touched multiple times in the given commits, the return value will keep the one
|
||||
* from the latest commit by file group ID.
|
||||
*
|
||||
* <p>Note: different with {@link #getFullPathToFileStatus(String)},
|
||||
* <p>Note: different with {@link #getFullPathToFileStatus(Configuration, String)},
|
||||
* only the latest commit file for a file group is returned,
|
||||
* this is an optimization for COPY_ON_WRITE table to eliminate legacy files for filesystem view.
|
||||
*
|
||||
*
|
||||
* @param hadoopConf
|
||||
* @param basePath The base path
|
||||
* @return the file ID to file status mapping
|
||||
*/
|
||||
public Map<String, FileStatus> getFileIdToFileStatus(String basePath) {
|
||||
public Map<String, FileStatus> getFileIdToFileStatus(Configuration hadoopConf, String basePath) {
|
||||
Map<String, FileStatus> fileIdToFileStatus = new HashMap<>();
|
||||
for (List<HoodieWriteStat> stats : getPartitionToWriteStats().values()) {
|
||||
// Iterate through all the written files.
|
||||
|
||||
Reference in New Issue
Block a user