[HUDI-3239] Convert BaseHoodieTableFileIndex to Java (#4669)
Converting BaseHoodieTableFileIndex to Java, removing Scala as a dependency from "hudi-common"
This commit is contained in:
@@ -0,0 +1,351 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.BaseFile;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieTableQueryType;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
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.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Common (engine-agnostic) File Index implementation enabling individual query engines to
|
||||
* list Hudi Table contents based on the
|
||||
*
|
||||
* <ul>
|
||||
* <li>Table type (MOR, COW)</li>
|
||||
* <li>Query type (snapshot, read_optimized, incremental)</li>
|
||||
* <li>Query instant/range</li>
|
||||
* </ul>
|
||||
*/
|
||||
public abstract class BaseHoodieTableFileIndex {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BaseHoodieTableFileIndex.class);
|
||||
|
||||
private final String[] partitionColumns;
|
||||
|
||||
private final FileSystemViewStorageConfig fileSystemStorageConfig;
|
||||
private final HoodieMetadataConfig metadataConfig;
|
||||
|
||||
private final HoodieTableQueryType queryType;
|
||||
private final Option<String> specifiedQueryInstant;
|
||||
protected final List<Path> queryPaths;
|
||||
|
||||
private final boolean shouldIncludePendingCommits;
|
||||
|
||||
private final HoodieTableType tableType;
|
||||
protected final String basePath;
|
||||
|
||||
private final HoodieTableMetaClient metaClient;
|
||||
private final HoodieEngineContext engineContext;
|
||||
|
||||
private final transient FileStatusCache fileStatusCache;
|
||||
|
||||
protected transient volatile long cachedFileSize = 0L;
|
||||
protected transient volatile Map<PartitionPath, List<FileSlice>> cachedAllInputFileSlices;
|
||||
|
||||
protected volatile boolean queryAsNonePartitionedTable = false;
|
||||
|
||||
private transient volatile HoodieTableFileSystemView fileSystemView = null;
|
||||
|
||||
/**
|
||||
* @param engineContext Hudi engine-specific context
|
||||
* @param metaClient Hudi table's meta-client
|
||||
* @param configProperties unifying configuration (in the form of generic properties)
|
||||
* @param queryType target query type
|
||||
* @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 fileStatusCache transient cache of fetched [[FileStatus]]es
|
||||
*/
|
||||
public BaseHoodieTableFileIndex(HoodieEngineContext engineContext,
|
||||
HoodieTableMetaClient metaClient,
|
||||
TypedProperties configProperties,
|
||||
HoodieTableQueryType queryType,
|
||||
List<Path> queryPaths,
|
||||
Option<String> specifiedQueryInstant,
|
||||
boolean shouldIncludePendingCommits,
|
||||
FileStatusCache fileStatusCache) {
|
||||
this.partitionColumns = metaClient.getTableConfig().getPartitionFields()
|
||||
.orElse(new String[0]);
|
||||
|
||||
this.fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
|
||||
.fromProperties(configProperties)
|
||||
.build();
|
||||
this.metadataConfig = HoodieMetadataConfig.newBuilder()
|
||||
.fromProperties(configProperties)
|
||||
.build();
|
||||
|
||||
this.queryType = queryType;
|
||||
this.queryPaths = queryPaths;
|
||||
this.specifiedQueryInstant = specifiedQueryInstant;
|
||||
this.shouldIncludePendingCommits = shouldIncludePendingCommits;
|
||||
|
||||
this.tableType = metaClient.getTableType();
|
||||
this.basePath = metaClient.getBasePath();
|
||||
|
||||
this.metaClient = metaClient;
|
||||
this.engineContext = engineContext;
|
||||
this.fileStatusCache = fileStatusCache;
|
||||
|
||||
doRefresh();
|
||||
}
|
||||
|
||||
protected abstract Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath);
|
||||
|
||||
/**
|
||||
* Returns latest completed instant as seen by this instance of the file-index
|
||||
*/
|
||||
public Option<HoodieInstant> getLatestCompletedInstant() {
|
||||
return getActiveTimeline().filterCompletedInstants().lastInstant();
|
||||
}
|
||||
|
||||
/**
|
||||
* Fetch list of latest base files and log files per partition.
|
||||
*
|
||||
* @return mapping from string partition paths to its base/log files
|
||||
*/
|
||||
public Map<String, List<FileSlice>> listFileSlices() {
|
||||
return cachedAllInputFileSlices.entrySet()
|
||||
.stream()
|
||||
.collect(Collectors.toMap(e -> e.getKey().path, Map.Entry::getValue));
|
||||
}
|
||||
|
||||
protected List<PartitionPath> getAllQueryPartitionPaths() {
|
||||
List<String> queryRelativePartitionPaths = queryPaths.stream()
|
||||
.map(path -> FSUtils.getRelativePartitionPath(new Path(basePath), path))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Load all the partition path from the basePath, and filter by the query partition path.
|
||||
// TODO load files from the queryRelativePartitionPaths directly.
|
||||
List<String> matchedPartitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath)
|
||||
.stream()
|
||||
.filter(path -> queryRelativePartitionPaths.stream().anyMatch(path::startsWith))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
// Convert partition's path into partition descriptor
|
||||
return matchedPartitionPaths.stream()
|
||||
.map(partitionPath -> {
|
||||
Object[] partitionColumnValues = parsePartitionColumnValues(partitionColumns, partitionPath);
|
||||
return new PartitionPath(partitionPath, partitionColumnValues);
|
||||
})
|
||||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
protected void refresh() {
|
||||
fileStatusCache.invalidate();
|
||||
doRefresh();
|
||||
}
|
||||
|
||||
protected HoodieTimeline getActiveTimeline() {
|
||||
// NOTE: We have to use commits and compactions timeline, to make sure that we're properly
|
||||
// handling the following case: when records are inserted into the new log-file w/in the file-group
|
||||
// that is under the pending compaction process, new log-file will bear the compaction's instant (on the
|
||||
// timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering
|
||||
// such log-file we have to _always_ include pending compaction instants into consideration
|
||||
// TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here
|
||||
HoodieTimeline timeline = metaClient.getCommitsAndCompactionTimeline();
|
||||
if (shouldIncludePendingCommits) {
|
||||
return timeline;
|
||||
} else {
|
||||
return timeline.filterCompletedAndCompactionInstants();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all partition paths and it's files under the query table path.
|
||||
*/
|
||||
private Map<PartitionPath, FileStatus[]> loadPartitionPathFiles() {
|
||||
// List files in all partition paths
|
||||
List<PartitionPath> pathToFetch = new ArrayList<>();
|
||||
Map<PartitionPath, FileStatus[]> cachedPartitionToFiles = new HashMap<>();
|
||||
|
||||
// Fetch from the FileStatusCache
|
||||
List<PartitionPath> partitionPaths = getAllQueryPartitionPaths();
|
||||
partitionPaths.forEach(partitionPath -> {
|
||||
Option<FileStatus[]> filesInPartition = fileStatusCache.get(partitionPath.fullPartitionPath(basePath));
|
||||
if (filesInPartition.isPresent()) {
|
||||
cachedPartitionToFiles.put(partitionPath, filesInPartition.get());
|
||||
} else {
|
||||
pathToFetch.add(partitionPath);
|
||||
}
|
||||
});
|
||||
|
||||
Map<PartitionPath, FileStatus[]> fetchedPartitionToFiles;
|
||||
|
||||
if (pathToFetch.isEmpty()) {
|
||||
fetchedPartitionToFiles = Collections.emptyMap();
|
||||
} else {
|
||||
Map<String, PartitionPath> fullPartitionPathsMapToFetch = pathToFetch.stream()
|
||||
.collect(Collectors.toMap(
|
||||
partitionPath -> partitionPath.fullPartitionPath(basePath).toString(),
|
||||
Function.identity())
|
||||
);
|
||||
|
||||
fetchedPartitionToFiles =
|
||||
FSUtils.getFilesInPartitions(
|
||||
engineContext,
|
||||
metadataConfig,
|
||||
basePath,
|
||||
fullPartitionPathsMapToFetch.keySet().toArray(new String[0]),
|
||||
fileSystemStorageConfig.getSpillableDir())
|
||||
.entrySet()
|
||||
.stream()
|
||||
.collect(Collectors.toMap(e -> fullPartitionPathsMapToFetch.get(e.getKey()), e -> e.getValue()));
|
||||
|
||||
}
|
||||
|
||||
// Update the fileStatusCache
|
||||
fetchedPartitionToFiles.forEach((partitionPath, filesInPartition) -> {
|
||||
fileStatusCache.put(partitionPath.fullPartitionPath(basePath), filesInPartition);
|
||||
});
|
||||
|
||||
return CollectionUtils.combine(cachedPartitionToFiles, fetchedPartitionToFiles);
|
||||
}
|
||||
|
||||
private void doRefresh() {
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
Map<PartitionPath, FileStatus[]> partitionFiles = loadPartitionPathFiles();
|
||||
FileStatus[] allFiles = partitionFiles.values().stream().flatMap(Arrays::stream).toArray(FileStatus[]::new);
|
||||
|
||||
metaClient.reloadActiveTimeline();
|
||||
|
||||
HoodieTimeline activeTimeline = getActiveTimeline();
|
||||
Option<HoodieInstant> latestInstant = activeTimeline.lastInstant();
|
||||
|
||||
// TODO we can optimize the flow by:
|
||||
// - First fetch list of files from instants of interest
|
||||
// - Load FileStatus's
|
||||
fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles);
|
||||
|
||||
Option<String> queryInstant =
|
||||
specifiedQueryInstant.or(() -> latestInstant.map(HoodieInstant::getTimestamp));
|
||||
|
||||
if (tableType.equals(HoodieTableType.MERGE_ON_READ) && queryType.equals(HoodieTableQueryType.SNAPSHOT)) {
|
||||
cachedAllInputFileSlices = partitionFiles.keySet().stream()
|
||||
.collect(Collectors.toMap(
|
||||
Function.identity(),
|
||||
partitionPath ->
|
||||
queryInstant.map(instant ->
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(partitionPath.path, queryInstant.get())
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
.orElse(Collections.emptyList())
|
||||
)
|
||||
);
|
||||
} 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())
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
cachedFileSize = cachedAllInputFileSlices.values().stream()
|
||||
.flatMap(Collection::stream)
|
||||
.mapToLong(BaseHoodieTableFileIndex::fileSliceSize)
|
||||
.sum();
|
||||
|
||||
// If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
|
||||
queryAsNonePartitionedTable = partitionFiles.keySet().stream().anyMatch(p -> p.values.length == 0);
|
||||
|
||||
long duration = System.currentTimeMillis() - startTime;
|
||||
|
||||
LOG.info(String.format("Refresh table %s, spent: %d ms", metaClient.getTableConfig().getTableName(), duration));
|
||||
}
|
||||
|
||||
private static long fileSliceSize(FileSlice fileSlice) {
|
||||
long logFileSize = fileSlice.getLogFiles().map(HoodieLogFile::getFileSize)
|
||||
.filter(s -> s > 0)
|
||||
.reduce(0L, Long::sum);
|
||||
|
||||
return fileSlice.getBaseFile().map(BaseFile::getFileLen).orElse(0L) + logFileSize;
|
||||
}
|
||||
|
||||
protected static final class PartitionPath {
|
||||
final String path;
|
||||
final Object[] values;
|
||||
|
||||
public PartitionPath(String path, Object[] values) {
|
||||
this.path = path;
|
||||
this.values = values;
|
||||
}
|
||||
|
||||
Path fullPartitionPath(String basePath) {
|
||||
if (!path.isEmpty()) {
|
||||
return new Path(basePath, path);
|
||||
}
|
||||
|
||||
return new Path(basePath);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
return other instanceof PartitionPath
|
||||
&& Objects.equals(path, ((PartitionPath) other).path)
|
||||
&& Arrays.equals(values, ((PartitionPath) other).values);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return path.hashCode() * 1103 + Arrays.hashCode(values);
|
||||
}
|
||||
}
|
||||
|
||||
protected interface FileStatusCache {
|
||||
Option<FileStatus[]> get(Path path);
|
||||
|
||||
void put(Path path, FileStatus[] leafFiles);
|
||||
|
||||
void invalidate();
|
||||
}
|
||||
}
|
||||
@@ -30,7 +30,7 @@ package org.apache.hudi.common.model;
|
||||
* </ol>
|
||||
*/
|
||||
public enum HoodieTableQueryType {
|
||||
QUERY_TYPE_SNAPSHOT,
|
||||
QUERY_TYPE_INCREMENTAL,
|
||||
QUERY_TYPE_READ_OPTIMIZED
|
||||
SNAPSHOT,
|
||||
INCREMENTAL,
|
||||
READ_OPTIMIZED
|
||||
}
|
||||
|
||||
@@ -62,7 +62,7 @@ public class CollectionUtils {
|
||||
|
||||
|
||||
/**
|
||||
* Combines provided {@link List}s into one
|
||||
* Combines provided {@link List}s into one, returning new instance of {@link ArrayList}
|
||||
*/
|
||||
public static <E> List<E> combine(List<E> one, List<E> another) {
|
||||
ArrayList<E> combined = new ArrayList<>(one.size() + another.size());
|
||||
@@ -71,6 +71,19 @@ public class CollectionUtils {
|
||||
return combined;
|
||||
}
|
||||
|
||||
/**
|
||||
* Combines provided {@link Map}s into one, returning new instance of {@link HashMap}.
|
||||
*
|
||||
* NOTE: That values associated with overlapping keys from the second map, will override
|
||||
* values from the first one
|
||||
*/
|
||||
public static <K, V> Map<K, V> combine(Map<K, V> one, Map<K, V> another) {
|
||||
Map<K, V> combined = new HashMap<>(one.size() + another.size());
|
||||
combined.putAll(one);
|
||||
combined.putAll(another);
|
||||
return combined;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns difference b/w {@code one} {@link Set} of elements and {@code another}
|
||||
*/
|
||||
|
||||
@@ -1,324 +0,0 @@
|
||||
/*
|
||||
* 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
|
||||
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
|
||||
import org.apache.hudi.common.model.{FileSlice, HoodieTableQueryType}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant
|
||||
import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
import scala.language.implicitConversions
|
||||
|
||||
/**
|
||||
* Common (engine-agnostic) File Index implementation enabling individual query engines to
|
||||
* list Hudi Table contents based on the
|
||||
*
|
||||
* <ul>
|
||||
* <li>Table type (MOR, COW)</li>
|
||||
* <li>Query type (snapshot, read_optimized, incremental)</li>
|
||||
* <li>Query instant/range</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param engineContext Hudi engine-specific context
|
||||
* @param metaClient Hudi table's meta-client
|
||||
* @param configProperties unifying configuration (in the form of generic properties)
|
||||
* @param queryType target query type
|
||||
* @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 fileStatusCache transient cache of fetched [[FileStatus]]es
|
||||
*/
|
||||
abstract class HoodieTableFileIndexBase(engineContext: HoodieEngineContext,
|
||||
metaClient: HoodieTableMetaClient,
|
||||
configProperties: TypedProperties,
|
||||
queryType: HoodieTableQueryType,
|
||||
protected val queryPaths: Seq[Path],
|
||||
specifiedQueryInstant: Option[String] = None,
|
||||
shouldIncludePendingCommits: Boolean = false,
|
||||
@transient fileStatusCache: FileStatusCacheTrait) {
|
||||
/**
|
||||
* Get all completeCommits.
|
||||
*/
|
||||
lazy val completedCommits = metaClient.getCommitsTimeline
|
||||
.filterCompletedInstants().getInstants.iterator().toList.map(_.getTimestamp)
|
||||
|
||||
private lazy val _partitionColumns: Array[String] =
|
||||
metaClient.getTableConfig.getPartitionFields.orElse(Array[String]())
|
||||
|
||||
private lazy val fileSystemStorageConfig = FileSystemViewStorageConfig.newBuilder()
|
||||
.fromProperties(configProperties)
|
||||
.build()
|
||||
private lazy val metadataConfig = HoodieMetadataConfig.newBuilder
|
||||
.fromProperties(configProperties)
|
||||
.build()
|
||||
|
||||
private val tableType = metaClient.getTableType
|
||||
|
||||
protected val basePath: String = metaClient.getBasePath
|
||||
|
||||
@transient
|
||||
@volatile protected var cachedFileSize: Long = 0L
|
||||
@transient
|
||||
@volatile protected var cachedAllInputFileSlices: Map[PartitionPath, Seq[FileSlice]] = _
|
||||
@volatile protected var queryAsNonePartitionedTable: Boolean = _
|
||||
@transient
|
||||
@volatile private var fileSystemView: HoodieTableFileSystemView = _
|
||||
|
||||
refresh0()
|
||||
|
||||
/**
|
||||
* Returns latest completed instant as seen by this instance of the file-index
|
||||
*/
|
||||
def latestCompletedInstant(): Option[HoodieInstant] =
|
||||
getActiveTimeline.filterCompletedInstants().lastInstant()
|
||||
|
||||
/**
|
||||
* Fetch list of latest base files and log files per partition.
|
||||
*
|
||||
* @return mapping from string partition paths to its base/log files
|
||||
*/
|
||||
def listFileSlices(): Map[String, Seq[FileSlice]] = {
|
||||
if (queryAsNonePartitionedTable) {
|
||||
// Read as Non-Partitioned table.
|
||||
cachedAllInputFileSlices.map(entry => (entry._1.path, entry._2))
|
||||
} else {
|
||||
cachedAllInputFileSlices.keys.toSeq.map(partition => {
|
||||
(partition.path, cachedAllInputFileSlices(partition))
|
||||
}).toMap
|
||||
}
|
||||
}
|
||||
|
||||
private def refresh0(): Unit = {
|
||||
val startTime = System.currentTimeMillis()
|
||||
val partitionFiles = loadPartitionPathFiles()
|
||||
val allFiles = partitionFiles.values.reduceOption(_ ++ _)
|
||||
.getOrElse(Array.empty[FileStatus])
|
||||
|
||||
metaClient.reloadActiveTimeline()
|
||||
|
||||
val activeTimeline = getActiveTimeline
|
||||
val latestInstant = activeTimeline.lastInstant()
|
||||
// TODO we can optimize the flow by:
|
||||
// - First fetch list of files from instants of interest
|
||||
// - Load FileStatus's
|
||||
fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles)
|
||||
val queryInstant = if (specifiedQueryInstant.isDefined) {
|
||||
specifiedQueryInstant
|
||||
} else if (latestInstant.isPresent) {
|
||||
Some(latestInstant.get.getTimestamp)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
(tableType, queryType) match {
|
||||
case (MERGE_ON_READ, HoodieTableQueryType.QUERY_TYPE_SNAPSHOT) =>
|
||||
// Fetch and store latest base and log files, and their sizes
|
||||
cachedAllInputFileSlices = partitionFiles.map(p => {
|
||||
val latestSlices = if (queryInstant.isDefined) {
|
||||
fileSystemView.getLatestMergedFileSlicesBeforeOrOn(p._1.path, queryInstant.get)
|
||||
.iterator().asScala.toSeq
|
||||
} else {
|
||||
Seq()
|
||||
}
|
||||
(p._1, latestSlices)
|
||||
})
|
||||
cachedFileSize = cachedAllInputFileSlices.values.flatten.map(fileSlice => {
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileLen + fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).sum
|
||||
} else {
|
||||
fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).sum
|
||||
}
|
||||
}).sum
|
||||
case (_, _) =>
|
||||
// Fetch and store latest base files and its sizes
|
||||
cachedAllInputFileSlices = partitionFiles.map(p => {
|
||||
val fileSlices = specifiedQueryInstant
|
||||
.map(instant =>
|
||||
fileSystemView.getLatestFileSlicesBeforeOrOn(p._1.path, instant, true))
|
||||
.getOrElse(fileSystemView.getLatestFileSlices(p._1.path))
|
||||
.iterator().asScala.toSeq
|
||||
(p._1, fileSlices)
|
||||
})
|
||||
cachedFileSize = cachedAllInputFileSlices.values.flatten.map(fileSliceSize).sum
|
||||
}
|
||||
|
||||
// If the partition value contains InternalRow.empty, we query it as a non-partitioned table.
|
||||
queryAsNonePartitionedTable = partitionFiles.keys.exists(p => p.values.isEmpty)
|
||||
val flushSpend = System.currentTimeMillis() - startTime
|
||||
|
||||
logInfo(s"Refresh table ${metaClient.getTableConfig.getTableName}," +
|
||||
s" spend: $flushSpend ms")
|
||||
}
|
||||
|
||||
protected def refresh(): Unit = {
|
||||
fileStatusCache.invalidate()
|
||||
refresh0()
|
||||
}
|
||||
|
||||
private def getActiveTimeline = {
|
||||
// NOTE: We have to use commits and compactions timeline, to make sure that we're properly
|
||||
// handling the following case: when records are inserted into the new log-file w/in the file-group
|
||||
// that is under the pending compaction process, new log-file will bear the compaction's instant (on the
|
||||
// timeline) in its name, as opposed to the base-file's commit instant. To make sure we're not filtering
|
||||
// such log-file we have to _always_ include pending compaction instants into consideration
|
||||
// TODO(HUDI-3302) re-evaluate whether we should not filter any commits in here
|
||||
val timeline = metaClient.getCommitsAndCompactionTimeline
|
||||
if (shouldIncludePendingCommits) {
|
||||
timeline
|
||||
} else {
|
||||
timeline.filterCompletedAndCompactionInstants()
|
||||
}
|
||||
}
|
||||
|
||||
private def fileSliceSize(fileSlice: FileSlice): Long = {
|
||||
val logFileSize = fileSlice.getLogFiles.iterator().asScala.map(_.getFileSize).filter(_ > 0).sum
|
||||
if (fileSlice.getBaseFile.isPresent) {
|
||||
fileSlice.getBaseFile.get().getFileLen + logFileSize
|
||||
} else {
|
||||
logFileSize
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Load all partition paths and it's files under the query table path.
|
||||
*/
|
||||
private def loadPartitionPathFiles(): Map[PartitionPath, Array[FileStatus]] = {
|
||||
val partitionPaths = getAllQueryPartitionPaths
|
||||
// List files in all of the partition path.
|
||||
val pathToFetch = mutable.ArrayBuffer[PartitionPath]()
|
||||
val cachePartitionToFiles = mutable.Map[PartitionPath, Array[FileStatus]]()
|
||||
// Fetch from the FileStatusCache
|
||||
partitionPaths.foreach { partitionPath =>
|
||||
fileStatusCache.get(partitionPath.fullPartitionPath(basePath)) match {
|
||||
case Some(filesInPartition) =>
|
||||
cachePartitionToFiles.put(partitionPath, filesInPartition)
|
||||
|
||||
case None => pathToFetch.append(partitionPath)
|
||||
}
|
||||
}
|
||||
|
||||
val fetchedPartitionToFiles =
|
||||
if (pathToFetch.nonEmpty) {
|
||||
val fullPartitionPathsToFetch = pathToFetch.map(p => (p, p.fullPartitionPath(basePath).toString)).toMap
|
||||
val partitionToFilesMap = FSUtils.getFilesInPartitions(engineContext, metadataConfig, basePath,
|
||||
fullPartitionPathsToFetch.values.toArray, fileSystemStorageConfig.getSpillableDir)
|
||||
fullPartitionPathsToFetch.map(p => {
|
||||
(p._1, partitionToFilesMap.get(p._2))
|
||||
})
|
||||
} else {
|
||||
Map.empty[PartitionPath, Array[FileStatus]]
|
||||
}
|
||||
|
||||
// Update the fileStatusCache
|
||||
fetchedPartitionToFiles.foreach {
|
||||
case (partitionRowPath, filesInPartition) =>
|
||||
fileStatusCache.put(partitionRowPath.fullPartitionPath(basePath), filesInPartition)
|
||||
}
|
||||
cachePartitionToFiles.toMap ++ fetchedPartitionToFiles
|
||||
}
|
||||
|
||||
def getAllQueryPartitionPaths: Seq[PartitionPath] = {
|
||||
val queryRelativePartitionPaths = queryPaths.map(FSUtils.getRelativePartitionPath(new Path(basePath), _))
|
||||
// Load all the partition path from the basePath, and filter by the query partition path.
|
||||
// TODO load files from the queryRelativePartitionPaths directly.
|
||||
val partitionPaths = FSUtils.getAllPartitionPaths(engineContext, metadataConfig, basePath).asScala
|
||||
.filter(path => queryRelativePartitionPaths.exists(path.startsWith))
|
||||
|
||||
val partitionSchema = _partitionColumns
|
||||
|
||||
// Convert partition's path into partition descriptor
|
||||
partitionPaths.map { partitionPath =>
|
||||
val partitionColumnValues = parsePartitionColumnValues(partitionSchema, partitionPath)
|
||||
PartitionPath(partitionPath, partitionColumnValues)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses partition columns' values from the provided partition's path, returning list of
|
||||
* values (that might have engine-specific representation)
|
||||
*
|
||||
* @param partitionColumns partitioning columns identifying the partition
|
||||
* @param partitionPath partition's path to parse partitioning columns' values from
|
||||
*/
|
||||
protected def parsePartitionColumnValues(partitionColumns: Array[String], partitionPath: String): Array[Any]
|
||||
|
||||
// TODO eval whether we should just use logger directly
|
||||
protected def logWarning(str: => String): Unit
|
||||
protected def logInfo(str: => String): Unit
|
||||
|
||||
/**
|
||||
* Represents a partition as a tuple of
|
||||
* <ul>
|
||||
* <li>Actual partition path (relative to the table's base path)</li>
|
||||
* <li>Values of the corresponding columns table is being partitioned by (partitioning columns)</li>
|
||||
* </ul>
|
||||
*
|
||||
* E.g. PartitionPath("2021/02/01", Array("2021","02","01"))
|
||||
*
|
||||
* NOTE: Partitioning column values might have engine specific representation (for ex,
|
||||
* {@code UTF8String} for Spark, etc) and are solely used in partition pruning in an very
|
||||
* engine-specific ways
|
||||
*
|
||||
* @param values values of the corresponding partitioning columns
|
||||
* @param path partition's path
|
||||
*
|
||||
* TODO expose as a trait and make impls engine-specific (current impl is tailored for Spark)
|
||||
*/
|
||||
case class PartitionPath(path: String, values: Array[Any]) {
|
||||
override def equals(other: Any): Boolean = other match {
|
||||
case PartitionPath(otherPath, _) => path == otherPath
|
||||
case _ => false
|
||||
}
|
||||
|
||||
override def hashCode(): Int = {
|
||||
path.hashCode
|
||||
}
|
||||
|
||||
def fullPartitionPath(basePath: String): Path = {
|
||||
if (path.isEmpty) {
|
||||
new Path(basePath) // This is a non-partition path
|
||||
} else {
|
||||
new Path(basePath, path)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Converts Hudi's internal representation of the {@code Option} into Scala's default one
|
||||
*/
|
||||
implicit def asScalaOption[T](opt: org.apache.hudi.common.util.Option[T]): Option[T] =
|
||||
if (opt.isPresent) {
|
||||
Some(opt.get)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
trait FileStatusCacheTrait {
|
||||
def get(path: Path): Option[Array[FileStatus]]
|
||||
def put(path: Path, leafFiles: Array[FileStatus]): Unit
|
||||
def invalidate(): Unit
|
||||
}
|
||||
Reference in New Issue
Block a user