You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by si...@apache.org on 2020/05/28 00:19:15 UTC

[hudi] 32/40: [HUDI-895] Remove unnecessary listing .hoodie folder when using timeline server (#1636)

This is an automated email from the ASF dual-hosted git repository.

sivabalan pushed a commit to branch release-0.5.3
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 0fb02e7ad1111f1b84d48c8983e9a4cd7eb8b91e
Author: Balaji Varadarajan <va...@uber.com>
AuthorDate: Sun May 17 18:18:53 2020 -0700

    [HUDI-895] Remove unnecessary listing .hoodie folder when using timeline server (#1636)
---
 .../org/apache/hudi/client/HoodieWriteClient.java  |  6 +--
 .../java/org/apache/hudi/table/HoodieTable.java    |  6 +--
 .../common/table/view/FileSystemViewManager.java   | 59 +++++++++++++---------
 3 files changed, 40 insertions(+), 31 deletions(-)

diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
index 90bc9b3..ba39b9b 100644
--- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
+++ b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java
@@ -872,10 +872,8 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
             HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), instantTime, HoodieTimeline.LESSER),
         "Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :"
             + latestPending + ",  Ingesting at " + instantTime));
-    HoodieTable<T> table = HoodieTable.getHoodieTable(metaClient, config, jsc);
-    HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
-    String commitActionType = table.getMetaClient().getCommitActionType();
-    activeTimeline.createNewInstant(new HoodieInstant(State.REQUESTED, commitActionType, instantTime));
+    metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(),
+        instantTime));
   }
 
   /**
diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
index ad0196c..2f106c0 100644
--- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
+++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java
@@ -150,21 +150,21 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
    * Get the base file only view of the file system for this table.
    */
   public BaseFileOnlyView getBaseFileOnlyView() {
-    return getViewManager().getFileSystemView(metaClient.getBasePath());
+    return getViewManager().getFileSystemView(metaClient);
   }
 
   /**
    * Get the full view of the file system for this table.
    */
   public SliceView getSliceView() {
-    return getViewManager().getFileSystemView(metaClient.getBasePath());
+    return getViewManager().getFileSystemView(metaClient);
   }
 
   /**
    * Get complete view of the file system for this table with ability to force sync.
    */
   public SyncableFileSystemView getHoodieView() {
-    return getViewManager().getFileSystemView(metaClient.getBasePath());
+    return getViewManager().getFileSystemView(metaClient);
   }
 
   /**
diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java
index 8d50ef5..ffeafaa 100644
--- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java
+++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewManager.java
@@ -59,10 +59,10 @@ public class FileSystemViewManager {
   // Map from Base-Path to View
   private final ConcurrentHashMap<String, SyncableFileSystemView> globalViewMap;
   // Factory Map to create file-system views
-  private final Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
+  private final Function2<HoodieTableMetaClient, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator;
 
   public FileSystemViewManager(SerializableConfiguration conf, FileSystemViewStorageConfig viewStorageConfig,
-      Function2<String, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
+      Function2<HoodieTableMetaClient, FileSystemViewStorageConfig, SyncableFileSystemView> viewCreator) {
     this.conf = new SerializableConfiguration(conf);
     this.viewStorageConfig = viewStorageConfig;
     this.globalViewMap = new ConcurrentHashMap<>();
@@ -88,7 +88,21 @@ public class FileSystemViewManager {
    * @return
    */
   public SyncableFileSystemView getFileSystemView(String basePath) {
-    return globalViewMap.computeIfAbsent(basePath, (path) -> viewCreator.apply(path, viewStorageConfig));
+    return globalViewMap.computeIfAbsent(basePath, (path) -> {
+      HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), path);
+      return viewCreator.apply(metaClient, viewStorageConfig);
+    });
+  }
+
+  /**
+   * Main API to get the file-system view for the base-path.
+   *
+   * @param metaClient HoodieTableMetaClient
+   * @return
+   */
+  public SyncableFileSystemView getFileSystemView(HoodieTableMetaClient metaClient) {
+    return globalViewMap.computeIfAbsent(metaClient.getBasePath(),
+        (path) -> viewCreator.apply(metaClient, viewStorageConfig));
   }
 
   /**
@@ -106,12 +120,11 @@ public class FileSystemViewManager {
    * 
    * @param conf Hadoop Configuration
    * @param viewConf View Storage Configuration
-   * @param basePath Base Path of table
+   * @param metaClient HoodieTableMetaClient
    * @return
    */
   private static RocksDbBasedFileSystemView createRocksDBBasedFileSystemView(SerializableConfiguration conf,
-      FileSystemViewStorageConfig viewConf, String basePath) {
-    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
+      FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
     HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
     return new RocksDbBasedFileSystemView(metaClient, timeline, viewConf);
   }
@@ -121,13 +134,12 @@ public class FileSystemViewManager {
    * 
    * @param conf Hadoop Configuration
    * @param viewConf View Storage Configuration
-   * @param basePath Base Path of table
+   * @param metaClient HoodieTableMetaClient
    * @return
    */
   private static SpillableMapBasedFileSystemView createSpillableMapBasedFileSystemView(SerializableConfiguration conf,
-      FileSystemViewStorageConfig viewConf, String basePath) {
-    LOG.info("Creating SpillableMap based view for basePath " + basePath);
-    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
+      FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
+    LOG.info("Creating SpillableMap based view for basePath " + metaClient.getBasePath());
     HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
     return new SpillableMapBasedFileSystemView(metaClient, timeline, viewConf);
   }
@@ -137,13 +149,12 @@ public class FileSystemViewManager {
    * 
    * @param conf Hadoop Configuration
    * @param viewConf View Storage Configuration
-   * @param basePath Base Path of table
+   * @param metaClient HoodieTableMetaClient
    * @return
    */
   private static HoodieTableFileSystemView createInMemoryFileSystemView(SerializableConfiguration conf,
-      FileSystemViewStorageConfig viewConf, String basePath) {
-    LOG.info("Creating InMemory based view for basePath " + basePath);
-    HoodieTableMetaClient metaClient = new HoodieTableMetaClient(conf.newCopy(), basePath, true);
+      FileSystemViewStorageConfig viewConf, HoodieTableMetaClient metaClient) {
+    LOG.info("Creating InMemory based view for basePath " + metaClient.getBasePath());
     HoodieTimeline timeline = metaClient.getActiveTimeline().filterCompletedAndCompactionInstants();
     return new HoodieTableFileSystemView(metaClient, timeline, viewConf.isIncrementalTimelineSyncEnabled());
   }
@@ -178,34 +189,34 @@ public class FileSystemViewManager {
       case EMBEDDED_KV_STORE:
         LOG.info("Creating embedded rocks-db based Table View");
         return new FileSystemViewManager(conf, config,
-            (basePath, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, basePath));
+            (metaClient, viewConf) -> createRocksDBBasedFileSystemView(conf, viewConf, metaClient));
       case SPILLABLE_DISK:
         LOG.info("Creating Spillable Disk based Table View");
         return new FileSystemViewManager(conf, config,
-            (basePath, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, basePath));
+            (metaClient, viewConf) -> createSpillableMapBasedFileSystemView(conf, viewConf, metaClient));
       case MEMORY:
         LOG.info("Creating in-memory based Table View");
         return new FileSystemViewManager(conf, config,
-            (basePath, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, basePath));
+            (metaClient, viewConfig) -> createInMemoryFileSystemView(conf, viewConfig, metaClient));
       case REMOTE_ONLY:
         LOG.info("Creating remote only table view");
-        return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> createRemoteFileSystemView(conf,
-            viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath)));
+        return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> createRemoteFileSystemView(conf,
+            viewConfig, metaClient));
       case REMOTE_FIRST:
         LOG.info("Creating remote first table view");
-        return new FileSystemViewManager(conf, config, (basePath, viewConfig) -> {
+        return new FileSystemViewManager(conf, config, (metaClient, viewConfig) -> {
           RemoteHoodieTableFileSystemView remoteFileSystemView =
-              createRemoteFileSystemView(conf, viewConfig, new HoodieTableMetaClient(conf.newCopy(), basePath));
+              createRemoteFileSystemView(conf, viewConfig, metaClient);
           SyncableFileSystemView secondaryView;
           switch (viewConfig.getSecondaryStorageType()) {
             case MEMORY:
-              secondaryView = createInMemoryFileSystemView(conf, viewConfig, basePath);
+              secondaryView = createInMemoryFileSystemView(conf, viewConfig, metaClient);
               break;
             case EMBEDDED_KV_STORE:
-              secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, basePath);
+              secondaryView = createRocksDBBasedFileSystemView(conf, viewConfig, metaClient);
               break;
             case SPILLABLE_DISK:
-              secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, basePath);
+              secondaryView = createSpillableMapBasedFileSystemView(conf, viewConfig, metaClient);
               break;
             default:
               throw new IllegalArgumentException("Secondary Storage type can only be in-memory or spillable. Was :"