You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/05/23 02:55:25 UTC

[GitHub] [hudi] danny0405 commented on a diff in pull request #5572: [HUDI-3654] Preparations for hudi metastore.

danny0405 commented on code in PR #5572:
URL: https://github.com/apache/hudi/pull/5572#discussion_r878986268


##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##########
@@ -98,21 +100,22 @@ public class HoodieTableMetaClient implements Serializable {
   // NOTE: Since those two parameters lay on the hot-path of a lot of computations, we
   //       use tailored extension of the {@code Path} class allowing to avoid repetitive
   //       computations secured by its immutability
-  private SerializablePath basePath;
-  private SerializablePath metaPath;
+  protected SerializablePath basePath;
+  protected SerializablePath metaPath;
 
   private transient HoodieWrapperFileSystem fs;
   private boolean loadActiveTimelineOnLoad;
-  private SerializableConfiguration hadoopConf;
+  protected SerializableConfiguration hadoopConf;
   private HoodieTableType tableType;
   private TimelineLayoutVersion timelineLayoutVersion;
-  private HoodieTableConfig tableConfig;
-  private HoodieActiveTimeline activeTimeline;
+  protected HoodieTableConfig tableConfig;
+  protected HoodieActiveTimeline activeTimeline;
   private HoodieArchivedTimeline archivedTimeline;
   private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build();
   private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build();
+  protected HoodieMetastoreConfig metastoreConfig;
 
-  private HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
+  protected HoodieTableMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
                                 ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion,

Review Comment:
   Why make these variables protected scope ? The class has no sub-classes ?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java:
##########
@@ -566,7 +566,7 @@ private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant,
     }
   }
 
-  private void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
+  protected void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) {
     ValidationUtils.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp()));

Review Comment:
   Why make these methods protected ?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##########
@@ -620,6 +631,21 @@ public void initializeBootstrapDirsIfNotExists() throws IOException {
     initializeBootstrapDirsIfNotExists(getHadoopConf(), basePath.toString(), getFs());
   }
 
+  private static HoodieTableMetaClient newMetaClient(Configuration conf, String basePath, boolean loadActiveTimelineOnLoad,
+      ConsistencyGuardConfig consistencyGuardConfig, Option<TimelineLayoutVersion> layoutVersion,
+      String payloadClassName, FileSystemRetryConfig fileSystemRetryConfig, Properties props) {
+    HoodieMetastoreConfig metastoreConfig = null == props
+        ? new HoodieMetastoreConfig.Builder().build()
+        : new HoodieMetastoreConfig.Builder().fromProperties(props).build();
+    return metastoreConfig.enableMetastore()
+        ? (HoodieTableMetaClient) ReflectionUtils.loadClass("org.apache.hudi.common.table.HoodieTableMetastoreClient",
+            new Class<?>[]{Configuration.class, ConsistencyGuardConfig.class, FileSystemRetryConfig.class, String.class, String.class, HoodieMetastoreConfig.class},
+            conf, consistencyGuardConfig, fileSystemRetryConfig,
+            props.getProperty(HoodieTableConfig.DATABASE_NAME.key()), props.getProperty(HoodieTableConfig.NAME.key()), metastoreConfig)

Review Comment:
   Did you check the NPE here ?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##########
@@ -367,6 +370,13 @@ public synchronized HoodieArchivedTimeline getArchivedTimeline() {
     return archivedTimeline;
   }
 
+  public HoodieMetastoreConfig getMetastoreConfig() {
+    if (metastoreConfig == null) {

Review Comment:
   Is this method thread safe ?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org