1
0

[HUDI 1308] Harden RFC-15 Implementation based on production testing (#2441)

Addresses leaks, perf degradation observed during testing. These were regressions from the original rfc-15 PoC implementation.

* Pass a single instance of HoodieTableMetadata everywhere
* Fix tests and add config for enabling metrics
 - Removed special casing of assumeDatePartitioning inside FSUtils#getAllPartitionPaths()
 - Consequently, IOException is never thrown and many files had to be adjusted
- More diligent handling of open file handles in metadata table
 - Added config for controlling reuse of connections
 - Added config for turning off fallback to listing, so we can see tests fail
 - Changed all ipf listing code to cache/amortize the open/close for better performance
 - Timelineserver also reuses connections, for better performance
 - Without timelineserver, when metadata table is opened from executors, reuse is not allowed
 - HoodieMetadataConfig passed into HoodieTableMetadata#create as argument.
 -  Fix TestHoodieBackedTableMetadata#testSync
This commit is contained in:
vinoth chandar
2021-01-19 21:20:28 -08:00
committed by GitHub
parent e23967b9e9
commit 5ca0625b27
55 changed files with 767 additions and 570 deletions

View File

@@ -96,17 +96,9 @@ public class HoodieHFileInputFormat extends FileInputFormat<NullWritable, ArrayW
// process snapshot queries next.
List<Path> snapshotPaths = inputPathHandler.getSnapshotPaths();
if (snapshotPaths.size() > 0) {
Map<HoodieTableMetaClient, List<Path>> groupedPaths =
HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths);
LOG.info("Found a total of " + groupedPaths.size() + " groups");
for (Map.Entry<HoodieTableMetaClient, List<Path>> entry : groupedPaths.entrySet()) {
List<FileStatus> result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue());
if (result != null) {
returns.addAll(result);
}
}
returns.addAll(HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths));
}
return returns.toArray(new FileStatus[returns.size()]);
return returns.toArray(new FileStatus[0]);
}
/**

View File

@@ -108,18 +108,9 @@ public class HoodieParquetInputFormat extends MapredParquetInputFormat implement
// process snapshot queries next.
List<Path> snapshotPaths = inputPathHandler.getSnapshotPaths();
if (snapshotPaths.size() > 0) {
Map<HoodieTableMetaClient, List<Path>> groupedPaths =
HoodieInputFormatUtils.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths);
LOG.info("Found a total of " + groupedPaths.size() + " groups");
for (Map.Entry<HoodieTableMetaClient, List<Path>> entry : groupedPaths.entrySet()) {
HoodieTableMetaClient metaClient = entry.getKey();
List<FileStatus> result = HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, entry.getKey(), entry.getValue());
if (result != null) {
returns.addAll(result);
}
}
returns.addAll(HoodieInputFormatUtils.filterFileStatusForSnapshotMode(job, tableMetaClientMap, snapshotPaths));
}
return returns.toArray(new FileStatus[returns.size()]);
return returns.toArray(new FileStatus[0]);
}

View File

@@ -18,9 +18,6 @@
package org.apache.hudi.hadoop;
import java.util.Map;
import java.util.Set;
import org.apache.hadoop.conf.Configurable;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
@@ -32,7 +29,9 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.hadoop.utils.HoodieHiveUtils;
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
@@ -44,13 +43,10 @@ import java.io.Serializable;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP;
/**
* Given a path is a part of - Hoodie table = accepts ONLY the latest version of each path - Non-Hoodie table = then
* always accept
@@ -87,6 +83,9 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
*/
private SerializableConfiguration conf;
private transient HoodieLocalEngineContext engineContext;
private transient FileSystem fs;
public HoodieROTablePathFilter() {
@@ -116,6 +115,10 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
@Override
public boolean accept(Path path) {
if (engineContext == null) {
this.engineContext = new HoodieLocalEngineContext(this.conf.get());
}
if (LOG.isDebugEnabled()) {
LOG.debug("Checking acceptance for path " + path);
}
@@ -164,6 +167,7 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
}
if (baseDir != null) {
HoodieTableFileSystemView fsView = null;
try {
HoodieTableMetaClient metaClient = metaClientCache.get(baseDir.toString());
if (null == metaClient) {
@@ -171,13 +175,9 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
metaClientCache.put(baseDir.toString(), metaClient);
}
boolean useFileListingFromMetadata = getConf().getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS);
boolean verifyFileListing = getConf().getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE);
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf.get());
HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext,
metaClient, useFileListingFromMetadata, verifyFileListing);
fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext,
metaClient, HoodieInputFormatUtils.buildMetadataConfig(getConf()));
String partition = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), folder);
List<HoodieBaseFile> latestFiles = fsView.getLatestBaseFiles(partition).collect(Collectors.toList());
// populate the cache
if (!hoodiePathCache.containsKey(folder.toString())) {
@@ -202,6 +202,10 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
}
nonHoodiePathCache.add(folder.toString());
return true;
} finally {
if (fsView != null) {
fsView.close();
}
}
} else {
// files is at < 3 level depth in FS tree, can't be hoodie dataset

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.hadoop.utils;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.engine.HoodieLocalEngineContext;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieBaseFile;
@@ -41,6 +42,7 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.LocatedFileStatusWithBootstrapBaseFile;
import org.apache.hudi.hadoop.realtime.HoodieHFileRealtimeInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
@@ -65,10 +67,10 @@ import java.util.Set;
import java.util.function.Function;
import java.util.stream.Collectors;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP;
import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP;
import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP;
public class HoodieInputFormatUtils {
@@ -413,40 +415,50 @@ public class HoodieInputFormatUtils {
return grouped;
}
/**
* Filters data files under @param paths for a snapshot queried table.
* @param job
* @param metaClient
* @param paths
* @return
*/
public static List<FileStatus> filterFileStatusForSnapshotMode(
JobConf job, HoodieTableMetaClient metaClient, List<Path> paths) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Hoodie Metadata initialized with completed commit Ts as :" + metaClient);
}
public static HoodieMetadataConfig buildMetadataConfig(Configuration conf) {
return HoodieMetadataConfig.newBuilder()
.enable(conf.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS))
.validate(conf.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE))
.build();
}
boolean useFileListingFromMetadata = job.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS);
boolean verifyFileListing = job.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE);
public static List<FileStatus> filterFileStatusForSnapshotMode(JobConf job, Map<String, HoodieTableMetaClient> tableMetaClientMap,
List<Path> snapshotPaths) throws IOException {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(job);
HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext,
metaClient, useFileListingFromMetadata, verifyFileListing);
List<HoodieBaseFile> filteredBaseFiles = new ArrayList<>();
for (Path p : paths) {
String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p);
List<HoodieBaseFile> matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList());
filteredBaseFiles.addAll(matched);
}
LOG.info("Total paths to process after hoodie filter " + filteredBaseFiles.size());
List<FileStatus> returns = new ArrayList<>();
for (HoodieBaseFile filteredFile : filteredBaseFiles) {
if (LOG.isDebugEnabled()) {
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
Map<HoodieTableMetaClient, List<Path>> groupedPaths = HoodieInputFormatUtils
.groupSnapshotPathsByMetaClient(tableMetaClientMap.values(), snapshotPaths);
Map<HoodieTableMetaClient, HoodieTableFileSystemView> fsViewCache = new HashMap<>();
LOG.info("Found a total of " + groupedPaths.size() + " groups");
try {
for (Map.Entry<HoodieTableMetaClient, List<Path>> entry : groupedPaths.entrySet()) {
HoodieTableMetaClient metaClient = entry.getKey();
if (LOG.isDebugEnabled()) {
LOG.debug("Hoodie Metadata initialized with completed commit instant as :" + metaClient);
}
HoodieTableFileSystemView fsView = fsViewCache.computeIfAbsent(metaClient, tableMetaClient ->
FileSystemViewManager.createInMemoryFileSystemView(engineContext, tableMetaClient, buildMetadataConfig(job)));
List<HoodieBaseFile> filteredBaseFiles = new ArrayList<>();
for (Path p : entry.getValue()) {
String relativePartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), p);
List<HoodieBaseFile> matched = fsView.getLatestBaseFiles(relativePartitionPath).collect(Collectors.toList());
filteredBaseFiles.addAll(matched);
}
LOG.info("Total paths to process after hoodie filter " + filteredBaseFiles.size());
for (HoodieBaseFile filteredFile : filteredBaseFiles) {
if (LOG.isDebugEnabled()) {
LOG.debug("Processing latest hoodie file - " + filteredFile.getPath());
}
filteredFile = refreshFileStatus(job, filteredFile);
returns.add(getFileStatus(filteredFile));
}
}
filteredFile = refreshFileStatus(job, filteredFile);
returns.add(getFileStatus(filteredFile));
} finally {
fsViewCache.forEach(((metaClient, fsView) -> fsView.close()));
}
return returns;
}

View File

@@ -55,11 +55,6 @@ import java.util.Map;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP;
import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
import static org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP;
import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE;
public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
private static final Logger LOG = LogManager.getLogger(HoodieRealtimeInputFormatUtils.class);
@@ -70,28 +65,25 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
// TODO(vc): Should we handle also non-hoodie splits here?
Map<Path, HoodieTableMetaClient> partitionsToMetaClient = getTableMetaClientByBasePath(conf, partitionsToParquetSplits.keySet());
boolean useFileListingFromMetadata = conf.getBoolean(METADATA_ENABLE_PROP, DEFAULT_METADATA_ENABLE_FOR_READERS);
boolean verifyFileListing = conf.getBoolean(METADATA_VALIDATE_PROP, DEFAULT_METADATA_VALIDATE);
// Create file system cache so metadata table is only instantiated once. Also can benefit normal file listing if
// partition path is listed twice so file groups will already be loaded in file system
Map<HoodieTableMetaClient, HoodieTableFileSystemView> fsCache = new HashMap<>();
// for all unique split parents, obtain all delta files based on delta commit timeline,
// grouped on file id
List<InputSplit> rtSplits = new ArrayList<>();
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
// for each partition path obtain the data & log file groupings, then map back to inputsplits
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
if (!fsCache.containsKey(metaClient)) {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf);
HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext,
metaClient, useFileListingFromMetadata, verifyFileListing);
fsCache.put(metaClient, fsView);
}
HoodieTableFileSystemView fsView = fsCache.get(metaClient);
try {
partitionsToParquetSplits.keySet().forEach(partitionPath -> {
// for each partition path obtain the data & log file groupings, then map back to inputsplits
HoodieTableMetaClient metaClient = partitionsToMetaClient.get(partitionPath);
if (!fsCache.containsKey(metaClient)) {
HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(conf);
HoodieTableFileSystemView fsView = FileSystemViewManager.createInMemoryFileSystemView(engineContext,
metaClient, HoodieInputFormatUtils.buildMetadataConfig(conf));
fsCache.put(metaClient, fsView);
}
HoodieTableFileSystemView fsView = fsCache.get(metaClient);
String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath);
try {
String relPartitionPath = FSUtils.getRelativePartitionPath(new Path(metaClient.getBasePath()), partitionPath);
// Both commit and delta-commits are included - pick the latest completed one
Option<HoodieInstant> latestCompletedInstant =
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();
@@ -105,7 +97,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
.collect(Collectors.groupingBy(split -> FSUtils.getFileId(split.getPath().getName())));
// Get the maxCommit from the last delta or compaction or commit - when bootstrapped from COW table
String maxCommitTime = metaClient.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION,
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION))
HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION))
.filterCompletedInstants().lastInstant().get().getTimestamp();
latestFileSlices.forEach(fileSlice -> {
List<FileSplit> dataFileSplits = groupedInputSplits.get(fileSlice.getFileId());
@@ -121,7 +113,7 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
.filter(SplitLocationInfo::isInMemory).toArray(String[]::new) : new String[0];
FileSplit baseSplit = new FileSplit(eSplit.getPath(), eSplit.getStart(), eSplit.getLength(),
hosts, inMemoryHosts);
rtSplits.add(new RealtimeBootstrapBaseFileSplit(baseSplit,metaClient.getBasePath(),
rtSplits.add(new RealtimeBootstrapBaseFileSplit(baseSplit, metaClient.getBasePath(),
logFilePaths, maxCommitTime, eSplit.getBootstrapFileSplit()));
} else {
rtSplits.add(new HoodieRealtimeFileSplit(split, metaClient.getBasePath(), logFilePaths, maxCommitTime));
@@ -131,10 +123,13 @@ public class HoodieRealtimeInputFormatUtils extends HoodieInputFormatUtils {
}
});
});
} catch (Exception e) {
throw new HoodieException("Error obtaining data file/log file grouping: " + partitionPath, e);
}
});
});
} catch (Exception e) {
throw new HoodieException("Error obtaining data file/log file grouping ", e);
} finally {
// close all the open fs views.
fsCache.forEach((k, view) -> view.close());
}
LOG.info("Returning a total splits of " + rtSplits.size());
return rtSplits.toArray(new InputSplit[0]);
}