1
0

[HUDI-1646] Provide mechanism to read uncommitted data through InputFormat (#2611)

This commit is contained in:
satishkotha
2021-03-04 17:43:31 -08:00
committed by GitHub
parent 89003bc780
commit 7cc75e0be2
7 changed files with 109 additions and 6 deletions

View File

@@ -19,8 +19,12 @@
package org.apache.hudi.hadoop.utils;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapreduce.JobContext;
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.exception.HoodieIOException;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -40,6 +44,8 @@ public class HoodieHiveUtils {
public static final String HOODIE_CONSUME_MODE_PATTERN = "hoodie.%s.consume.mode";
public static final String HOODIE_START_COMMIT_PATTERN = "hoodie.%s.consume.start.timestamp";
public static final String HOODIE_MAX_COMMIT_PATTERN = "hoodie.%s.consume.max.commits";
public static final String HOODIE_CONSUME_PENDING_COMMITS = "hoodie.%s.consume.pending.commits";
public static final String HOODIE_CONSUME_COMMIT = "hoodie.%s.consume.commit";
public static final Set<String> VIRTUAL_COLUMN_NAMES = CollectionUtils.createImmutableSet(
"INPUT__FILE__NAME", "BLOCK__OFFSET__INSIDE__FILE", "ROW__OFFSET__INSIDE__BLOCK", "RAW__DATA__SIZE",
"ROW__ID", "GROUPING__ID");
@@ -115,4 +121,20 @@ public class HoodieHiveUtils {
}
return result;
}
public static HoodieTimeline getTableTimeline(final String tableName, final JobConf job, final HoodieTableMetaClient metaClient) {
boolean includePendingCommits = job.getBoolean(String.format(HOODIE_CONSUME_PENDING_COMMITS, tableName), false);
if (includePendingCommits) {
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline();
String maxCommit = job.get(String.format(HOODIE_CONSUME_COMMIT, tableName));
if (maxCommit == null || !timeline.containsInstant(maxCommit)) {
LOG.info("Timestamp configured for validation: " + maxCommit + " commits timeline:" + timeline + " table: " + tableName);
throw new HoodieIOException("Valid timestamp is required for " + HOODIE_CONSUME_COMMIT + " in validate mode");
}
return timeline.findInstantsBeforeOrEquals(maxCommit);
}
// by default return all completed commits.
return metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
}
}

View File

@@ -439,8 +439,9 @@ public class HoodieInputFormatUtils {
LOG.debug("Hoodie Metadata initialized with completed commit instant as :" + metaClient);
}
HoodieTimeline timeline = HoodieHiveUtils.getTableTimeline(metaClient.getTableConfig().getTableName(), job, metaClient);
HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient ->
FileSystemViewManager.createInMemoryFileSystemView(engineContext, tableMetaClient, buildMetadataConfig(job)));
FileSystemViewManager.createInMemoryFileSystemViewWithTimeline(engineContext, tableMetaClient, buildMetadataConfig(job), timeline));
List<HoodieBaseFile> filteredBaseFiles = new ArrayList<>();
for (Path p : entry.getValue()) {
String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p);