1
0

[HUDI-3191] Rebasing Hive's FileInputFormat onto AbstractHoodieTableFileIndex (#4531)

This commit is contained in:
Alexey Kudinkin
2022-01-18 14:54:51 -08:00
committed by GitHub
parent caeea946fb
commit 4bea758738
21 changed files with 623 additions and 78 deletions

View File

@@ -0,0 +1,100 @@
/*
* 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.hadoop;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.HoodieTableFileIndexBase;
import org.apache.hudi.FileStatusCacheTrait;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieTableQueryType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.Option;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import scala.Function0;
import scala.collection.JavaConverters;
import java.util.List;
/**
* Implementation of {@link HoodieTableFileIndexBase} for Hive-based query engines
*/
public class HiveHoodieTableFileIndex extends HoodieTableFileIndexBase {
public static final Logger LOG = LoggerFactory.getLogger(HiveHoodieTableFileIndex.class);
public HiveHoodieTableFileIndex(HoodieEngineContext engineContext,
HoodieTableMetaClient metaClient,
TypedProperties configProperties,
HoodieTableQueryType queryType,
List<Path> queryPaths,
Option<String> specifiedQueryInstant,
boolean shouldIncludePendingCommits
) {
super(engineContext,
metaClient,
configProperties,
queryType,
JavaConverters.asScalaBufferConverter(queryPaths).asScala(),
toScalaOption(specifiedQueryInstant),
shouldIncludePendingCommits,
new NoopCache());
}
private static scala.Option<String> toScalaOption(Option<String> opt) {
return scala.Option.apply(opt.orElse(null));
}
@Override
public Object[] parsePartitionColumnValues(String[] partitionColumns, String partitionPath) {
// NOTE: Parsing partition path into partition column values isn't required on Hive,
// since Hive does partition pruning in a different way (based on the input-path being
// fetched by the query engine)
return new Object[0];
}
@Override
public void logInfo(Function0<String> lazyStr) {
LOG.info(lazyStr.apply());
}
@Override
public void logWarning(Function0<String> lazyStr) {
LOG.info(lazyStr.apply());
}
static class NoopCache implements FileStatusCacheTrait {
@Override
public scala.Option<FileStatus[]> get(Path path) {
return scala.Option.empty();
}
@Override
public void put(Path path, FileStatus[] leafFiles) {
// no-op
}
@Override
public void invalidate() {
// no-op
}
}
}

View File

@@ -27,18 +27,33 @@ import org.apache.hadoop.io.NullWritable;
import org.apache.hadoop.mapred.FileInputFormat;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieTableQueryType;
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.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import scala.collection.JavaConverters;
import scala.collection.Seq;
import javax.annotation.Nonnull;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.util.ValidationUtils.checkState;
/**
* Base implementation of the Hive's {@link FileInputFormat} allowing for reading of Hudi's
@@ -67,6 +82,27 @@ public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWrit
this.conf = conf;
}
@Nonnull
private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFile latestLogFile, Stream<HoodieLogFile> logFiles) {
List<HoodieLogFile> sortedLogFiles = logFiles.sorted(HoodieLogFile.getLogFileComparator()).collect(Collectors.toList());
try {
RealtimeFileStatus rtFileStatus = new RealtimeFileStatus(latestLogFile.getFileStatus());
rtFileStatus.setDeltaLogFiles(sortedLogFiles);
return rtFileStatus;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
@Nonnull
private static FileStatus getFileStatusUnchecked(Option<HoodieBaseFile> baseFileOpt) {
try {
return HoodieInputFormatUtils.getFileStatus(baseFileOpt.get());
} catch (IOException ioe) {
throw new RuntimeException(ioe);
}
}
@Override
public FileStatus[] listStatus(JobConf job) throws IOException {
// Segregate inputPaths[] to incremental, snapshot and non hoodie paths
@@ -102,18 +138,98 @@ public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWrit
// process snapshot queries next.
List<Path> snapshotPaths = inputPathHandler.getSnapshotPaths();
if (snapshotPaths.size() > 0) {
returns.addAll(HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths, includeLogFilesForSnapShotView()));
returns.addAll(listStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths));
}
return returns.toArray(new FileStatus[0]);
}
@Nonnull
private List<FileStatus> listStatusForSnapshotMode(JobConf job,
Map<String, HoodieTableMetaClient> tableMetaClientMap,
List<Path> snapshotPaths) throws IOException {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(job);
List<FileStatus> targetFiles = new ArrayList<>();
TypedProperties props = new TypedProperties(new Properties());
Map<HoodieTableMetaClient, List<Path>> groupedPaths =
HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths);
for (Map.Entry<HoodieTableMetaClient, List<Path>> entry : groupedPaths.entrySet()) {
HoodieTableMetaClient tableMetaClient = entry.getKey();
List<Path> partitionPaths = entry.getValue();
// Hive job might specify a max commit instant up to which table's state
// should be examined. We simply pass it as query's instant to the file-index
Option<String> queryCommitInstant =
HoodieHiveUtils.getMaxCommit(job, tableMetaClient.getTableConfig().getTableName());
boolean shouldIncludePendingCommits =
HoodieHiveUtils.shouldIncludePendingCommits(job, tableMetaClient.getTableConfig().getTableName());
HiveHoodieTableFileIndex fileIndex =
new HiveHoodieTableFileIndex(
engineContext,
tableMetaClient,
props,
HoodieTableQueryType.QUERY_TYPE_SNAPSHOT,
partitionPaths,
queryCommitInstant,
shouldIncludePendingCommits);
Map<String, Seq<FileSlice>> partitionedFileSlices =
JavaConverters.mapAsJavaMapConverter(fileIndex.listFileSlices()).asJava();
targetFiles.addAll(
partitionedFileSlices.values()
.stream()
.flatMap(seq -> JavaConverters.seqAsJavaListConverter(seq).asJava().stream())
.map(fileSlice -> {
Option<HoodieBaseFile> baseFileOpt = fileSlice.getBaseFile();
Option<HoodieLogFile> latestLogFileOpt = fileSlice.getLatestLogFile();
if (baseFileOpt.isPresent()) {
return getFileStatusUnchecked(baseFileOpt);
} else if (includeLogFilesForSnapShotView() && latestLogFileOpt.isPresent()) {
return createRealtimeFileStatusUnchecked(latestLogFileOpt.get(), fileSlice.getLogFiles());
} else {
throw new IllegalStateException("Invalid state: either base-file or log-file should be present");
}
})
.collect(Collectors.toList())
);
}
// TODO cleanup
validate(targetFiles, listStatusForSnapshotModeLegacy(job, tableMetaClientMap, snapshotPaths));
return targetFiles;
}
private void validate(List<FileStatus> targetFiles, List<FileStatus> legacyFileStatuses) {
List<FileStatus> diff = CollectionUtils.diff(targetFiles, legacyFileStatuses);
checkState(diff.isEmpty(), "Should be empty");
}
@Nonnull
private List<FileStatus> listStatusForSnapshotModeLegacy(JobConf job, Map<String, HoodieTableMetaClient> tableMetaClientMap, List<Path> snapshotPaths) throws IOException {
return HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths, includeLogFilesForSnapShotView());
}
/**
* Abstracts and exposes {@link FileInputFormat#listStatus(JobConf)} operation to subclasses that
* lists files (returning an array of {@link FileStatus}) corresponding to the input paths specified
* as part of provided {@link JobConf}
*/
protected final FileStatus[] doListStatus(JobConf job) throws IOException {
return super.listStatus(job);
}
/**
* Achieves listStatus functionality for an incrementally queried table. Instead of listing all
* partitions and then filtering based on the commits of interest, this logic first extracts the
* partitions touched by the desired commits and then lists only those partitions.
*/
protected List<FileStatus> listStatusForIncrementalMode(
JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
protected List<FileStatus> listStatusForIncrementalMode(JobConf job, HoodieTableMetaClient tableMetaClient, List<Path> inputPaths) throws IOException {
String tableName = tableMetaClient.getTableConfig().getTableName();
Job jobContext = Job.getInstance(job);
Option<HoodieTimeline> timeline = HoodieInputFormatUtils.getFilteredCommitsTimeline(jobContext, tableMetaClient);
@@ -133,13 +249,4 @@ public abstract class HoodieFileInputFormatBase extends FileInputFormat<NullWrit
FileStatus[] fileStatuses = doListStatus(job);
return HoodieInputFormatUtils.filterIncrementalFileStatus(jobContext, tableMetaClient, timeline.get(), fileStatuses, commitsToCheck.get());
}
/**
* Abstracts and exposes {@link FileInputFormat#listStatus(JobConf)} operation to subclasses that
* lists files (returning an array of {@link FileStatus}) corresponding to the input paths specified
* as part of provided {@link JobConf}
*/
protected final FileStatus[] doListStatus(JobConf job) throws IOException {
return super.listStatus(job);
}
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.hadoop.utils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CollectionUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.Path;
@@ -76,6 +77,14 @@ public class HoodieHiveUtils {
public static final Pattern HOODIE_CONSUME_MODE_PATTERN_STRING = Pattern.compile("hoodie\\.(.*)\\.consume\\.mode");
public static final String GLOBALLY_CONSISTENT_READ_TIMESTAMP = "last_replication_timestamp";
public static boolean shouldIncludePendingCommits(JobConf job, String tableName) {
return job.getBoolean(String.format(HOODIE_CONSUME_PENDING_COMMITS, tableName), false);
}
public static Option<String> getMaxCommit(JobConf job, String tableName) {
return Option.ofNullable(job.get(String.format(HOODIE_CONSUME_COMMIT, tableName)));
}
public static boolean stopAtCompaction(JobContext job, String tableName) {
String compactionPropName = String.format(HOODIE_STOP_AT_COMPACTION_PATTERN, tableName);
boolean stopAtCompaction = job.getConfiguration().getBoolean(compactionPropName, true);
@@ -149,23 +158,17 @@ public class HoodieHiveUtils {
* (false or notSet, notSet) -> returns completedTimeline unfiltered
*
* validCommit is one which exists in the timeline being checked and vice versa
*
* @param tableName
* @param job
* @param metaClient
* @return
*/
public static HoodieTimeline getTableTimeline(final String tableName, final JobConf job, final HoodieTableMetaClient metaClient) {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline();
boolean includePendingCommits = job.getBoolean(String.format(HOODIE_CONSUME_PENDING_COMMITS, tableName), false);
String maxCommit = job.get(String.format(HOODIE_CONSUME_COMMIT, tableName));
boolean includePendingCommits = shouldIncludePendingCommits(job, tableName);
Option<String> maxCommit = getMaxCommit(job, tableName);
if (!includePendingCommits && maxCommit == null) {
return timeline.filterCompletedInstants();
}
HoodieTimeline finalizedTimeline = includePendingCommits ? timeline : timeline.filterCompletedInstants();
return !maxCommit.isPresent() ? finalizedTimeline : filterIfInstantExists(tableName, finalizedTimeline, maxCommit.get());
return filterIfInstantExists(tableName, includePendingCommits ? timeline : timeline.filterCompletedInstants(), maxCommit);
}
private static HoodieTimeline filterIfInstantExists(String tableName, HoodieTimeline timeline, String maxCommit) {