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


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantFormat.java:
##########
@@ -0,0 +1,240 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hudi.common.util.StringUtils;
+
+import java.io.Serializable;
+import java.util.Date;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HoodieInstantFormat implements Serializable {

Review Comment:
   Can we add some doc for it.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java:
##########
@@ -34,11 +34,14 @@
 import java.io.IOException;
 import java.util.Properties;
 
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
 import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
 import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_WAIT_TIMEOUT_MS;
 import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
 import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
 import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_FILE_NAME_PROP_KEY;
 import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
 import static 
org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP_KEY;

Review Comment:
   We may need to revert this change.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGeneratorBase.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieLockException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_WAIT_TIMEOUT_MS;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
+
+public abstract class TimeGeneratorBase implements TimeGenerator, Serializable 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TimeGeneratorBase.class);
+
+  private volatile LockProvider lockProvider;
+  private final int maxRetries;
+  private final long maxWaitTimeInMs;
+  private final int lockAcquireWaitTimeInMs;
+
+  private final HoodieTimeGeneratorConfig config;
+  private final LockConfiguration lockConfiguration;
+
+  private final Configuration hadoopConf;
+
+  public TimeGeneratorBase(HoodieTimeGeneratorConfig config, Configuration 
hadoopConf) {
+    this.config = config;

Review Comment:
   The lock configuration should be resolved from the `HoodieWriteConfig` 
instead, so I guess the parameters should be `HoodieWriteConfig config, 
Configuration hadoopConf`



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1462,8 +1467,11 @@ public static String 
deleteMetadataTablePartition(HoodieTableMetaClient dataMeta
     }
 
     if (backup) {
+      HoodieTimeGeneratorConfig timeGeneratorConfig = 
HoodieTimeGeneratorConfig.newBuilder()
+          .withPath(dataMetaClient.getBasePathV2().toString()).build();
       final Path metadataPartitionBackupPath = new 
Path(metadataTablePartitionPath.getParent().getParent(),
-          String.format(".metadata_%s_%s", partitionType.getPartitionPath(), 
HoodieActiveTimeline.createNewInstantTime()));
+          String.format(".metadata_%s_%s", partitionType.getPartitionPath(),
+              HoodieActiveTimeline.createNewInstantTime(timeGeneratorConfig, 
dataMetaClient.getHadoopConf())));

Review Comment:
   We can ignore the lock.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java:
##########
@@ -248,10 +271,11 @@ public void deleteCompletedRollback(HoodieInstant 
instant) {
   }
 
   public static void deleteInstantFile(FileSystem fs, String metaPath, 
HoodieInstant instant) {
+    String instantFileName = 
HoodieInstantFormat.getInstantFormat().getInstantFileName(instant);
     try {
-      fs.delete(new Path(metaPath, instant.getFileName()), false);
+      fs.delete(new Path(metaPath, instantFileName), false);

Review Comment:
   I feel, if we made the `HoodieInstantFormat` static, we can then fix the 
`instant.getFileName()` directly, there is no need to do any change change here 
and all other places.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantFormat.java:
##########
@@ -0,0 +1,240 @@
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hudi.common.util.StringUtils;
+
+import java.io.Serializable;
+import java.util.Date;
+import java.util.Objects;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class HoodieInstantFormat implements Serializable {
+  private static final String DELIMITER = ".";
+
+  private static final String EMPTY_FILE_EXTENSION = "";
+
+  private static final String FILE_NAME_FORMAT_ERROR =
+      "The provided file name %s does not conform to the required format";
+
+  // Instant like 20230104152218702.commit.request, 20230104152218702.inflight
+  protected static final Pattern NAME_FORMAT_V0 =
+      Pattern.compile("^(\\d+)(\\.\\w+)(\\.\\D+)?$");
+
+  // Instant like 20230104152218702_20230104152630238.commit
+  private static final Pattern COMPLETE_INSTANT_FORMAT =
+      Pattern.compile("^(\\d+)_(\\d+)\\.(\\w+)$");
+
+  private HoodieInstantFormat() {
+
+  }
+
+  private static final HoodieInstantFormat INSTANCE = new 
HoodieInstantFormat();
+  
+  public static HoodieInstantFormat getInstantFormat() {
+    return INSTANCE;

Review Comment:
   At least for now, we can make all the methods static and there is no need to 
maintain this singleton instance.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java:
##########
@@ -79,8 +79,11 @@ public FileSystemBasedLockProvider(final LockConfiguration 
lockConfiguration, fi
       lockDirectory = 
lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key())
           + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
     }
+    lockFileName = 
lockConfiguration.getConfig().getString(HoodieLockConfig.FILESYSTEM_LOCK_FILE_NAME.key(),
+        HoodieLockConfig.FILESYSTEM_LOCK_FILE_NAME.defaultValue());
+
     this.lockTimeoutMinutes = 
lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY);
-    this.lockFile = new Path(lockDirectory + Path.SEPARATOR + LOCK_FILE_NAME);
+    this.lockFile = new Path(lockDirectory + Path.SEPARATOR + lockFileName);
     this.lockInfo = new LockInfo();

Review Comment:
   We may need to revert the change to this class.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeGeneratorConfig.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+
+import java.util.Properties;
+
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
+
+public class HoodieTimeGeneratorConfig extends HoodieConfig {

Review Comment:
   Add some doc to this class.



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -955,8 +956,9 @@ public static HoodieTableFileSystemView 
getFileSystemView(HoodieTableMetaClient
     // though we may have initialized them.
     HoodieTimeline timeline = metaClient.getActiveTimeline();
     if (timeline.empty()) {
+      HoodieTimeGeneratorConfig timeGeneratorConfig = 
HoodieTimeGeneratorConfig.newBuilder().withPath(metaClient.getBasePathV2().toString()).build();
       final HoodieInstant instant = new HoodieInstant(false, 
HoodieTimeline.DELTA_COMMIT_ACTION,

Review Comment:
   The write config/lock provider config does not pass around correctly, not 
sure whether we should take a lock here as this is a metadata table.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeGeneratorConfig.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+
+import java.util.Properties;
+
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
+
+public class HoodieTimeGeneratorConfig extends HoodieConfig {
+
+  private static final String BASE_PATH = "hoodie.base.path";
+
+  public static final ConfigProperty<String> TRUE_TIME_GENERATOR_IMPL = 
ConfigProperty
+      .key("hoodie.true.time.generator.impl")
+      .defaultValue(WaitBasedTimeGenerator.class.getName())
+      .sinceVersion("1.0.0")
+      .markAdvanced()
+      .withDocumentation("TrueTime generator class name");
+
+  public static final ConfigProperty<String> TRUE_TIME_LOCK_PROVIDER = 
ConfigProperty
+      .key(LOCK_PREFIX + "provider")
+      
.defaultValue("org.apache.hudi.client.transaction.lock.InProcessLockProvider")
+      .sinceVersion("1.0.0")

Review Comment:
   Do we still need this?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerator.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+public interface TimeGenerator {
+
+  void lock();
+
+
+  /**
+   * Generate a globally monotonically increasing timestamp.All implements 
must ensure
+   * 1. any caller can compute a timestamp T that is guaranteed to be greater 
than
+   * any timestamp T' if T' finished being generated before T started being 
generated.
+   * 2. This guarantee holds across all different callers.
+   *
+   * @param shouldLock Whether to hold a new lock inside this method. Some 
callers(like
+   * completion time generation) already holds lock, whereas here doesn't need 
to lock again.
+   *
+   * @return Current true time as millis.
+   */
+  long nowAsMillis(boolean shouldLock);

Review Comment:
   How about we follow the Java method name `currentTimeMillis`



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java:
##########
@@ -198,29 +211,39 @@ private void readObject(java.io.ObjectInputStream in) 
throws IOException, ClassN
     in.defaultReadObject();
   }
 
-  public void createNewInstant(HoodieInstant instant) {
+  public void createCompleteInstant(boolean shouldLock, 
HoodieTimeGeneratorConfig config, HoodieInstant instant) {
+    LOG.info("Creating a new complete instant " + instant);
+    createCompleteFileInMetaPath(shouldLock, config, instant, Option.empty());
+  }
+
+  public void createIncompleteInstant(HoodieInstant instant) {
     LOG.info("Creating a new instant " + instant);

Review Comment:
   To follow Hoodie naming convention, rename to `createPendingInstant`



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -1146,6 +1149,12 @@ protected HoodieWriteConfig(EngineType engineType, 
Properties props) {
     this.tableServiceManagerConfig = 
HoodieTableServiceManagerConfig.newBuilder().fromProperties(props).build();
     this.commonConfig = 
HoodieCommonConfig.newBuilder().fromProperties(props).build();
     this.storageConfig = 
HoodieStorageConfig.newBuilder().fromProperties(props).build();
+    if (isLockRequired()) {
+      this.timeGeneratorConfig = 
HoodieTimeGeneratorConfig.newBuilder().fromProperties(props).build();
+    } else {
+      this.timeGeneratorConfig = 
HoodieTimeGeneratorConfig.newBuilder().fromProperties(props)
+          .withDefaultLockProvider().build();

Review Comment:
   Can be simplified to one line:
   
   ```java
   this.timeGeneratorConfig = 
HoodieTimeGeneratorConfig.newBuilder().fromProperties(props)
             .withDefaultLockProvider(!isLockRequired()).build()
   ```



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGeneratorBase.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieLockException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_WAIT_TIMEOUT_MS;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
+
+public abstract class TimeGeneratorBase implements TimeGenerator, Serializable 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TimeGeneratorBase.class);
+
+  private volatile LockProvider lockProvider;
+  private final int maxRetries;
+  private final long maxWaitTimeInMs;
+  private final int lockAcquireWaitTimeInMs;
+
+  private final HoodieTimeGeneratorConfig config;
+  private final LockConfiguration lockConfiguration;
+
+  private final Configuration hadoopConf;
+
+  public TimeGeneratorBase(HoodieTimeGeneratorConfig config, Configuration 
hadoopConf) {
+    this.config = config;
+    this.lockConfiguration = config.getLockConfiguration();
+    this.hadoopConf = hadoopConf;
+
+    maxRetries = 
lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY,
+        Integer.parseInt(DEFAULT_LOCK_ACQUIRE_NUM_RETRIES));
+    lockAcquireWaitTimeInMs = 
lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY,
+        Integer.parseInt(DEFAULT_LOCK_ACQUIRE_WAIT_TIMEOUT_MS));
+    maxWaitTimeInMs = 
lockConfiguration.getConfig().getLong(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY,
+        Long.parseLong(DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS));
+  }
+
+  protected LockProvider getLockProvider() {
+    // Perform lazy initialization of lock provider only if needed
+    if (lockProvider == null) {
+      synchronized (this) {
+        if (lockProvider == null) {
+          String lockProviderClass = 
lockConfiguration.getConfig().getString("hoodie.write.lock.provider");
+          LOG.info("LockProvider for TrueTimeGenerator: " + lockProviderClass);
+          lockProvider = (LockProvider) 
ReflectionUtils.loadClass(lockProviderClass,
+              lockConfiguration, hadoopConf);
+        }
+      }
+    }
+    return lockProvider;
+  }
+
+  @Override
+  public void lock() {
+    LockProvider lockProvider = getLockProvider();
+    int retryCount = 0;
+    boolean acquired = false;
+    while (retryCount <= maxRetries) {
+      try {

Review Comment:
   The lock provider should handle the failure retries, is it?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeGeneratorConfig.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+
+import java.util.Properties;
+
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
+
+public class HoodieTimeGeneratorConfig extends HoodieConfig {
+
+  private static final String BASE_PATH = "hoodie.base.path";
+
+  public static final ConfigProperty<String> TRUE_TIME_GENERATOR_IMPL = 
ConfigProperty
+      .key("hoodie.true.time.generator.impl")
+      .defaultValue(WaitBasedTimeGenerator.class.getName())

Review Comment:
   rename to `hoodie.time.generator.type`, it's value is a Enumeration, 
`LOCK_PROVIDER` and `TRUE_TIME`.



##########
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java:
##########
@@ -1406,7 +1408,10 @@ public static String 
deleteMetadataTable(HoodieTableMetaClient dataMetaClient, H
     }
 
     if (backup) {
-      final Path metadataBackupPath = new Path(metadataTablePath.getParent(), 
".metadata_" + HoodieActiveTimeline.createNewInstantTime());
+      HoodieTimeGeneratorConfig timeGeneratorConfig = 
HoodieTimeGeneratorConfig.newBuilder()
+          .withPath(dataMetaClient.getBasePathV2().toString()).build();
+      final Path metadataBackupPath = new Path(metadataTablePath.getParent(),
+          ".metadata_" + 
HoodieActiveTimeline.createNewInstantTime(timeGeneratorConfig, 
dataMetaClient.getHadoopConf()));

Review Comment:
   We can ignore the lock.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGenerators.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import com.github.benmanes.caffeine.cache.Cache;
+import com.github.benmanes.caffeine.cache.Caffeine;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.common.util.ReflectionUtils;
+
+public class TimeGenerators {
+
+  private static final Cache<String, TimeGenerator>
+      TIME_GENERATOR_CACHE = 
Caffeine.newBuilder().maximumSize(10).weakValues().build();
+
+  public static TimeGenerator getTimeGenerator(HoodieTimeGeneratorConfig 
trueTimeConfig, Configuration hadoopConf) {
+    return TIME_GENERATOR_CACHE.get(trueTimeConfig.getBasePath(), s -> 
(TimeGenerator) ReflectionUtils.loadClass(

Review Comment:
   `trueTimeConfig` -> `timeGeneratorConfig`



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeGeneratorConfig.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hudi.common.config.ConfigProperty;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+
+import java.util.Properties;
+
+import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
+
+public class HoodieTimeGeneratorConfig extends HoodieConfig {
+
+  private static final String BASE_PATH = "hoodie.base.path";
+
+  public static final ConfigProperty<String> TRUE_TIME_GENERATOR_IMPL = 
ConfigProperty
+      .key("hoodie.true.time.generator.impl")
+      .defaultValue(WaitBasedTimeGenerator.class.getName())
+      .sinceVersion("1.0.0")
+      .markAdvanced()
+      .withDocumentation("TrueTime generator class name");
+
+  public static final ConfigProperty<String> TRUE_TIME_LOCK_PROVIDER = 
ConfigProperty
+      .key(LOCK_PREFIX + "provider")
+      
.defaultValue("org.apache.hudi.client.transaction.lock.InProcessLockProvider")
+      .sinceVersion("1.0.0")
+      .markAdvanced()
+      .withDocumentation("Lock provider class name used in TrueTime, user can 
provide "
+          + "their own implementation of LockProvider which should be subclass 
of "
+          + "org.apache.hudi.common.lock.LockProvider");
+
+  public static final ConfigProperty<Long> MAX_EXPECTED_CLOCK_SKEW_MS = 
ConfigProperty
+      .key("hoodie.true.time.generator.max_expected_clock_skew_ms")
+      .defaultValue(500L)
+      .withInferFunction(cfg -> {
+        if 
(cfg.getStringOrDefault(TRUE_TIME_LOCK_PROVIDER).equals(TRUE_TIME_LOCK_PROVIDER.defaultValue()))
 {
+          return Option.of(1L);
+        }
+        return Option.empty();
+      })
+      .sinceVersion("1.0.0")
+      .markAdvanced()
+      .withDocumentation("The max expected clock skew time for 
WaitBasedTrueTimeGenerator");
+
+  private HoodieTimeGeneratorConfig() {
+    super();
+  }
+
+  public String getTrueTimeGeneratorImpl() {
+    return props.getProperty(TRUE_TIME_GENERATOR_IMPL.key(), 
TRUE_TIME_GENERATOR_IMPL.defaultValue());
+  }
+
+  public long getMaxExpectedClockSkewMs() {
+    return props.getLong(MAX_EXPECTED_CLOCK_SKEW_MS.key(), 
MAX_EXPECTED_CLOCK_SKEW_MS.defaultValue());
+  }
+
+  public String getBasePath() {
+    return props.getString(BASE_PATH);
+  }
+
+  public static HoodieTimeGeneratorConfig.Builder newBuilder() {
+    return new HoodieTimeGeneratorConfig.Builder();
+  }
+
+  public static class Builder {
+    private final HoodieTimeGeneratorConfig timeGeneratorConfig = new 
HoodieTimeGeneratorConfig();

Review Comment:
   Maintains a `TypedProperties` instead and finally construct the 
`HoodieTimeGeneratorConfig` from there.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimeGeneratorBase.java:
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.common.table.timeline;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieLockException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
+import static 
org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_WAIT_TIMEOUT_MS;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
+
+public abstract class TimeGeneratorBase implements TimeGenerator, Serializable 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TimeGeneratorBase.class);
+
+  private volatile LockProvider lockProvider;
+  private final int maxRetries;
+  private final long maxWaitTimeInMs;
+  private final int lockAcquireWaitTimeInMs;
+
+  private final HoodieTimeGeneratorConfig config;
+  private final LockConfiguration lockConfiguration;
+
+  private final Configuration hadoopConf;

Review Comment:
   Should use `SerializableConfiguration hadoopConf` instead.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to