1
0

[HUDI-1144] Speedup spark read queries by caching metaclient in HoodieROPathFilter (#1919)

This commit is contained in:
Balaji Varadarajan
2020-08-05 09:19:10 -07:00
committed by GitHub
parent 7a2429f5ba
commit 9bcd3221fd
2 changed files with 22 additions and 8 deletions

View File

@@ -18,6 +18,8 @@
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.model.HoodieBaseFile;
@@ -60,12 +62,17 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
* Its quite common, to have all files from a given partition path be passed into accept(), cache the check for hoodie
* metadata for known partition paths and the latest versions of files.
*/
private HashMap<String, HashSet<Path>> hoodiePathCache;
private Map<String, HashSet<Path>> hoodiePathCache;
/**
* Paths that are known to be non-hoodie tables.
*/
private HashSet<String> nonHoodiePathCache;
private Set<String> nonHoodiePathCache;
/**
* Table Meta Client Cache.
*/
Map<String, HoodieTableMetaClient> metaClientCache;
/**
* Hadoop configurations for the FileSystem.
@@ -82,6 +89,7 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
this.hoodiePathCache = new HashMap<>();
this.nonHoodiePathCache = new HashSet<>();
this.conf = new SerializableConfiguration(conf);
this.metaClientCache = new HashMap<>();
}
/**
@@ -149,7 +157,12 @@ public class HoodieROTablePathFilter implements Configurable, PathFilter, Serial
if (baseDir != null) {
try {
HoodieTableMetaClient metaClient = new HoodieTableMetaClient(fs.getConf(), baseDir.toString());
HoodieTableMetaClient metaClient = metaClientCache.get(baseDir.toString());
if (null == metaClient) {
metaClient = new HoodieTableMetaClient(fs.getConf(), baseDir.toString(), true);
metaClientCache.put(baseDir.toString(), metaClient);
}
HoodieTableFileSystemView fsView = new HoodieTableFileSystemView(metaClient,
metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(), fs.listStatus(folder));
List<HoodieBaseFile> latestFiles = fsView.getLatestBaseFiles().collect(Collectors.toList());