yanghua commented on code in PR #6071:
URL: https://github.com/apache/hudi/pull/6071#discussion_r923298238


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.client.transaction.lock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.lock.LockState;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
+
+/**
+ * A FileSystem based lock. This {@link LockProvider} implementation allows to 
lock table operations
+ * using DFS. Users might need to manually clean the Locker's path if 
writeClient crash and never run again.
+ * NOTE: This only works for DFS with atomic create/delete operation
+ */
+public class FileSystemBasedLockProvider implements LockProvider<String>, 
Serializable {
+
+  private static final Logger LOG = 
LogManager.getLogger(FileSystemBasedLockProvider.class);
+
+  private static final String LOCK_FILE_NAME = "lock";
+
+  private final int lockTimeoutMinutes;
+  private transient FileSystem fs;
+  private transient Path lockFile;
+  protected LockConfiguration lockConfiguration;
+
+  public FileSystemBasedLockProvider(final LockConfiguration 
lockConfiguration, final Configuration configuration) {
+    checkRequiredProps(lockConfiguration);
+    this.lockConfiguration = lockConfiguration;
+    String lockDirectory = 
lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null);
+    if (StringUtils.isNullOrEmpty(lockDirectory)) {
+      lockDirectory = 
lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key(), null)

Review Comment:
   You have checked the two config items. There should be must one non-null. So 
IMO, here we do not need to give a default `null` value right?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.client.transaction.lock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.lock.LockState;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
+
+/**
+ * A FileSystem based lock. This {@link LockProvider} implementation allows to 
lock table operations
+ * using DFS. Users might need to manually clean the Locker's path if 
writeClient crash and never run again.
+ * NOTE: This only works for DFS with atomic create/delete operation
+ */
+public class FileSystemBasedLockProvider implements LockProvider<String>, 
Serializable {
+
+  private static final Logger LOG = 
LogManager.getLogger(FileSystemBasedLockProvider.class);
+
+  private static final String LOCK_FILE_NAME = "lock";
+
+  private final int lockTimeoutMinutes;
+  private transient FileSystem fs;
+  private transient Path lockFile;
+  protected LockConfiguration lockConfiguration;
+
+  public FileSystemBasedLockProvider(final LockConfiguration 
lockConfiguration, final Configuration configuration) {
+    checkRequiredProps(lockConfiguration);
+    this.lockConfiguration = lockConfiguration;
+    String lockDirectory = 
lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null);
+    if (StringUtils.isNullOrEmpty(lockDirectory)) {
+      lockDirectory = 
lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key(), null)
+            + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
+    }
+    this.lockTimeoutMinutes = 
lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY);
+    this.lockFile = new Path(lockDirectory + Path.SEPARATOR + LOCK_FILE_NAME);
+    this.fs = FSUtils.getFs(this.lockFile.toString(), configuration);
+  }
+
+  @Override
+  public void close() {
+    synchronized (LOCK_FILE_NAME) {
+      try {
+        fs.delete(this.lockFile, true);
+      } catch (IOException e) {
+        throw new 
HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE), e);
+      }
+    }
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) {
+    try {
+      synchronized (LOCK_FILE_NAME) {
+        if (fs.exists(this.lockFile)) {
+          // Check whether lock is already expired or not, if so try to delete 
lock file
+          if (checkIfExpired()) {

Review Comment:
   Can we merge them into one condition? e.g. `a && b`



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieLockConfig.java:
##########
@@ -106,7 +107,13 @@ public class HoodieLockConfig extends HoodieConfig {
       .key(FILESYSTEM_LOCK_PATH_PROP_KEY)
       .noDefaultValue()
       .sinceVersion("0.8.0")
-      .withDocumentation("For DFS based lock providers, path to store the 
locks under.");
+      .withDocumentation("For DFS based lock providers, path to store the 
locks under. If don't provide one, will use Table's meta path as default");

Review Comment:
   Here we call `.noDefaultValue()`, so can we remove `If don't provide one, `, 
just say: `use Table's meta path as default.`?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.client.transaction.lock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.lock.LockState;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
+
+/**
+ * A FileSystem based lock. This {@link LockProvider} implementation allows to 
lock table operations
+ * using DFS. Users might need to manually clean the Locker's path if 
writeClient crash and never run again.
+ * NOTE: This only works for DFS with atomic create/delete operation
+ */
+public class FileSystemBasedLockProvider implements LockProvider<String>, 
Serializable {
+
+  private static final Logger LOG = 
LogManager.getLogger(FileSystemBasedLockProvider.class);
+
+  private static final String LOCK_FILE_NAME = "lock";
+
+  private final int lockTimeoutMinutes;
+  private transient FileSystem fs;
+  private transient Path lockFile;
+  protected LockConfiguration lockConfiguration;
+
+  public FileSystemBasedLockProvider(final LockConfiguration 
lockConfiguration, final Configuration configuration) {
+    checkRequiredProps(lockConfiguration);
+    this.lockConfiguration = lockConfiguration;
+    String lockDirectory = 
lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null);
+    if (StringUtils.isNullOrEmpty(lockDirectory)) {
+      lockDirectory = 
lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key(), null)
+            + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
+    }
+    this.lockTimeoutMinutes = 
lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY);
+    this.lockFile = new Path(lockDirectory + Path.SEPARATOR + LOCK_FILE_NAME);
+    this.fs = FSUtils.getFs(this.lockFile.toString(), configuration);
+  }
+
+  @Override
+  public void close() {
+    synchronized (LOCK_FILE_NAME) {
+      try {
+        fs.delete(this.lockFile, true);
+      } catch (IOException e) {
+        throw new 
HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE), e);
+      }
+    }
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) {
+    try {
+      synchronized (LOCK_FILE_NAME) {
+        if (fs.exists(this.lockFile)) {
+          // Check whether lock is already expired or not, if so try to delete 
lock file

Review Comment:
   Remove `or not` looks more reasonable?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/FileSystemBasedLockProvider.java:
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.client.transaction.lock;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.lock.LockProvider;
+import org.apache.hudi.common.lock.LockState;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.concurrent.TimeUnit;
+
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_EXPIRE_PROP_KEY;
+import static 
org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
+
+/**
+ * A FileSystem based lock. This {@link LockProvider} implementation allows to 
lock table operations
+ * using DFS. Users might need to manually clean the Locker's path if 
writeClient crash and never run again.
+ * NOTE: This only works for DFS with atomic create/delete operation
+ */
+public class FileSystemBasedLockProvider implements LockProvider<String>, 
Serializable {
+
+  private static final Logger LOG = 
LogManager.getLogger(FileSystemBasedLockProvider.class);
+
+  private static final String LOCK_FILE_NAME = "lock";
+
+  private final int lockTimeoutMinutes;
+  private transient FileSystem fs;
+  private transient Path lockFile;
+  protected LockConfiguration lockConfiguration;
+
+  public FileSystemBasedLockProvider(final LockConfiguration 
lockConfiguration, final Configuration configuration) {
+    checkRequiredProps(lockConfiguration);
+    this.lockConfiguration = lockConfiguration;
+    String lockDirectory = 
lockConfiguration.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY, null);
+    if (StringUtils.isNullOrEmpty(lockDirectory)) {
+      lockDirectory = 
lockConfiguration.getConfig().getString(HoodieWriteConfig.BASE_PATH.key(), null)
+            + Path.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME;
+    }
+    this.lockTimeoutMinutes = 
lockConfiguration.getConfig().getInteger(FILESYSTEM_LOCK_EXPIRE_PROP_KEY);
+    this.lockFile = new Path(lockDirectory + Path.SEPARATOR + LOCK_FILE_NAME);
+    this.fs = FSUtils.getFs(this.lockFile.toString(), configuration);
+  }
+
+  @Override
+  public void close() {
+    synchronized (LOCK_FILE_NAME) {
+      try {
+        fs.delete(this.lockFile, true);
+      } catch (IOException e) {
+        throw new 
HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE), e);
+      }
+    }
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) {
+    try {
+      synchronized (LOCK_FILE_NAME) {
+        if (fs.exists(this.lockFile)) {
+          // Check whether lock is already expired or not, if so try to delete 
lock file
+          if (checkIfExpired()) {
+            fs.delete(this.lockFile, true);
+          }
+        }
+        acquireLock();
+        return fs.exists(this.lockFile);
+      }
+    } catch (IOException | HoodieIOException e) {
+      LOG.info(generateLogStatement(LockState.FAILED_TO_ACQUIRE), e);
+      return false;
+    }
+  }
+
+  @Override
+  public void unlock() {
+    synchronized (LOCK_FILE_NAME) {
+      try {
+        if (fs.exists(this.lockFile)) {
+          fs.delete(this.lockFile, true);
+        }
+      } catch (IOException io) {
+        throw new 
HoodieIOException(generateLogStatement(LockState.FAILED_TO_RELEASE), io);
+      }
+    }
+  }
+
+  @Override
+  public String getLock() {
+    return this.lockFile.toString();
+  }
+
+  private boolean checkIfExpired() {
+    if (lockTimeoutMinutes == 0) {
+      return false;
+    }
+    try {
+      long modificationTime = 
fs.getFileStatus(this.lockFile).getModificationTime();
+      if (System.currentTimeMillis() - modificationTime > lockTimeoutMinutes * 
60 * 1000) {
+        return true;
+      }
+    } catch (IOException | HoodieIOException e) {
+      LOG.error(generateLogStatement(LockState.ALREADY_RELEASED) + " failed to 
get lockFile's modification time", e);
+    }
+    return false;
+  }
+
+  private void acquireLock() {
+    try {
+      fs.create(this.lockFile, false).close();
+    } catch (IOException e) {
+      throw new 
HoodieIOException(generateLogStatement(LockState.FAILED_TO_ACQUIRE), e);
+    }
+  }
+
+  protected String generateLogStatement(LockState state) {
+    return StringUtils.join(state.name(), " lock at: ", getLock());
+  }
+
+  private void checkRequiredProps(final LockConfiguration config) {
+    
ValidationUtils.checkArgument(config.getConfig().getString(FILESYSTEM_LOCK_PATH_PROP_KEY,
 null) != null
+          || config.getConfig().getString(HoodieWriteConfig.BASE_PATH.key(), 
null) != null);

Review Comment:
   Here can we  just call the `config.getConfig().getString(key)` method?



-- 
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-unsubscr...@hudi.apache.org

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

Reply via email to