[ 
https://issues.apache.org/jira/browse/HUDI-2098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17377458#comment-17377458
 ] 

ASF GitHub Bot commented on HUDI-2098:
--------------------------------------

n3nash commented on a change in pull request #3189:
URL: https://github.com/apache/hudi/pull/3189#discussion_r666296765



##########
File path: 
hudi-common/src/main/java/org/apache/hudi/common/lock/HdfsFileLockProvider.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.lock;
+
+import org.apache.hudi.common.config.LockConfiguration;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieLockException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A file based lock. This {@link LockProvider} implementation allows to lock 
table operations using HDFS
+ * This is achieved by using the concept of acquiring the data out stream 
using Append option.
+ */
+public class HdfsFileLockProvider implements LockProvider<String> {
+  private static final Logger LOG = 
LogManager.getLogger(HdfsFileLockProvider.class);
+
+  private String lockDir;
+  private FileSystem fs;
+  protected LockConfiguration lockConfiguration;
+  private DataOutputStream dataOutputStream;
+  private String lockFile;
+
+  public HdfsFileLockProvider(final LockConfiguration lockConfiguration, final 
Configuration conf) {
+    this.lockConfiguration = lockConfiguration;
+    this.lockDir = 
lockConfiguration.getConfig().getString(LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY);
+    this.fs = FSUtils.getFs(lockDir, conf);
+    this.lockFile = new Path(lockDir, "lockFile").toUri().toString();
+  }
+
+  @Override
+  public boolean tryLock(long time, TimeUnit unit) {
+    try {
+      int retryCount = 
lockConfiguration.getConfig().getInteger(LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY);
+      int retryTimeout = 
lockConfiguration.getConfig().getInteger(LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY);
+      for (int i = 0; i < retryCount; i++) {
+        if (acquireLock()) {
+          LOG.info(generateLogStatement(LockState.ACQUIRED, 
generateLogSuffixString()));
+          return true;
+        } else {
+          Thread.sleep(retryTimeout);
+        }
+      }
+    } catch (InterruptedException e) {
+      throw new HoodieLockException("Failed to acquire file lock", e);
+    }
+    return false;
+  }
+
+  @Override
+  public void unlock() {
+    LOG.info(generateLogStatement(LockState.RELEASING, 
generateLogSuffixString()));
+    if (dataOutputStream != null) {
+      try {
+        dataOutputStream.close();
+        dataOutputStream = null;
+      } catch (IOException e) {
+        LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, 
generateLogSuffixString()));
+        throw new 
HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, 
generateLogSuffixString()), e);
+      }
+    }
+    LOG.info(generateLogStatement(LockState.RELEASED, 
generateLogSuffixString()));
+  }
+
+  private boolean acquireLock() {
+    if (lockDir == null || lockFile == null) {
+      return false;
+    }
+    try {
+      Path lockPath = new Path(lockDir);
+      Path lockFilePath = new Path(lockFile);
+      if (!fs.exists(lockPath)) {
+        fs.mkdirs(lockPath);
+      }
+      if (!fs.exists(lockFilePath)) {
+        // Pass the permissions during file creation itself
+        fs.create(lockFilePath, new FsPermission(FsAction.ALL, FsAction.ALL, 
FsAction.ALL),

Review comment:
       Why do we need to pass permissions ? 




-- 
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]


> add Hdfs file lock for HUDI
> ---------------------------
>
>                 Key: HUDI-2098
>                 URL: https://issues.apache.org/jira/browse/HUDI-2098
>             Project: Apache Hudi
>          Issue Type: Bug
>          Components: Usability
>    Affects Versions: 0.8.0
>         Environment: spark3.1.1
> hive3.1.1
> hadoop3.1.1
>            Reporter: tao meng
>            Assignee: tao meng
>            Priority: Minor
>              Labels: features, pull-request-available
>             Fix For: 0.9.0
>
>
> now hudi support hive/zk lock for concurrency write,  we introduce a new lock 
> type hdfs lock
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to