1
0

[HUDI-3239] Convert BaseHoodieTableFileIndex to Java (#4669)

Converting BaseHoodieTableFileIndex to Java, removing Scala as a dependency from "hudi-common"
This commit is contained in:
Alexey Kudinkin
2022-02-09 15:42:08 -08:00
committed by GitHub
parent 973087f385
commit 464027ec37
15 changed files with 443 additions and 554 deletions

View File

@@ -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();
}
}

View File

@@ -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
}

View File

@@ -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}
*/