vinothchandar commented on a change in pull request #2374: URL: https://github.com/apache/hudi/pull/2374#discussion_r588682179
########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java ########## @@ -0,0 +1,161 @@ +/* + * 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.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.CuratorFrameworkState; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +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.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.TimeUnit; + +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.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations + * using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock. + */ +@NotThreadSafe +public class ZookeeperBasedLockProvider extends LockProvider { + + private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); + + private final CuratorFramework curatorFrameworkClient; + private volatile InterProcessMutex lock = null; + + public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = CuratorFrameworkFactory.builder() + .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)) + .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP), + 5000, lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) + .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS)) + .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS)) + .build(); + this.curatorFrameworkClient.start(); + } + + // Only used for testing + public ZookeeperBasedLockProvider( + final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = curatorFrameworkClient; + synchronized (this.curatorFrameworkClient) { + if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) { + this.curatorFrameworkClient.start(); + } + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } + return lock != null && lock.isAcquiredInThisProcess(); + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString())); + if (lock == null || !lock.isAcquiredInThisProcess()) { + return; + } + lock.release(); + lock = null; + LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + @Override + public void close() { + try { + if (lock != null) { + lock.release(); + lock = null; + } + this.curatorFrameworkClient.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + @Override + public InterProcessMutex getLock() { + return this.lock; + } + + private void acquireLock(long time, TimeUnit unit) throws Exception { + ValidationUtils.checkArgument(this.lock == null, LockState.ALREADY_ACQUIRED.name()); + InterProcessMutex newLock = new InterProcessMutex( + this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/" + + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP)); + boolean acquired = newLock.acquire(time, unit); + if (!acquired) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString())); Review comment: Can we handle this specially and just rethrow the lock exception without the wrapping. ########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java ########## @@ -0,0 +1,161 @@ +/* + * 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.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.CuratorFrameworkState; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +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.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.TimeUnit; + +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.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations + * using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock. + */ +@NotThreadSafe +public class ZookeeperBasedLockProvider extends LockProvider { + + private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); + + private final CuratorFramework curatorFrameworkClient; + private volatile InterProcessMutex lock = null; + + public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = CuratorFrameworkFactory.builder() + .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)) + .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP), + 5000, lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) + .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS)) + .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS)) + .build(); + this.curatorFrameworkClient.start(); + } + + // Only used for testing + public ZookeeperBasedLockProvider( + final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = curatorFrameworkClient; + synchronized (this.curatorFrameworkClient) { + if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) { + this.curatorFrameworkClient.start(); + } + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } + return lock != null && lock.isAcquiredInThisProcess(); + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString())); + if (lock == null || !lock.isAcquiredInThisProcess()) { + return; + } + lock.release(); + lock = null; + LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + @Override + public void close() { + try { + if (lock != null) { + lock.release(); + lock = null; + } + this.curatorFrameworkClient.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + @Override + public InterProcessMutex getLock() { + return this.lock; + } + + private void acquireLock(long time, TimeUnit unit) throws Exception { + ValidationUtils.checkArgument(this.lock == null, LockState.ALREADY_ACQUIRED.name()); Review comment: a better validation message, than just the enum name? (applicable for any such usage) ########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java ########## @@ -0,0 +1,161 @@ +/* + * 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.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.CuratorFrameworkState; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +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.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.TimeUnit; + +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.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations + * using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock. + */ +@NotThreadSafe +public class ZookeeperBasedLockProvider extends LockProvider { + + private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); + + private final CuratorFramework curatorFrameworkClient; + private volatile InterProcessMutex lock = null; + + public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = CuratorFrameworkFactory.builder() + .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)) + .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP), + 5000, lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) + .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS)) + .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS)) + .build(); + this.curatorFrameworkClient.start(); + } + + // Only used for testing + public ZookeeperBasedLockProvider( + final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + this.curatorFrameworkClient = curatorFrameworkClient; + synchronized (this.curatorFrameworkClient) { + if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) { + this.curatorFrameworkClient.start(); + } + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); + } + return lock != null && lock.isAcquiredInThisProcess(); + } + + @Override + public void unlock() { + try { + LOG.info(generateLogStatement(LockState.RELEASING, generateLogSuffixString())); + if (lock == null || !lock.isAcquiredInThisProcess()) { + return; + } + lock.release(); + lock = null; + LOG.info(generateLogStatement(LockState.RELEASED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString()), e); + } + } + + @Override + public void close() { + try { + if (lock != null) { + lock.release(); + lock = null; + } + this.curatorFrameworkClient.close(); + } catch (Exception e) { + LOG.error(generateLogStatement(LockState.FAILED_TO_RELEASE, generateLogSuffixString())); + } + } + + @Override + public InterProcessMutex getLock() { + return this.lock; + } + + private void acquireLock(long time, TimeUnit unit) throws Exception { + ValidationUtils.checkArgument(this.lock == null, LockState.ALREADY_ACQUIRED.name()); + InterProcessMutex newLock = new InterProcessMutex( + this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/" + + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP)); + boolean acquired = newLock.acquire(time, unit); + if (!acquired) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString())); Review comment: this will lead to a LockException being wrapped inside another LockException in tryLock()? ########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/lock/ZookeeperBasedLockProvider.java ########## @@ -0,0 +1,157 @@ +/* + * 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.lock; + +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.CuratorFrameworkFactory; +import org.apache.curator.framework.imps.CuratorFrameworkState; +import org.apache.curator.framework.recipes.locks.InterProcessMutex; +import org.apache.curator.retry.BoundedExponentialBackoffRetry; +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.lock.LockState; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import javax.annotation.concurrent.NotThreadSafe; +import java.util.concurrent.TimeUnit; + +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.LOCK_ACQUIRE_NUM_RETRIES_PROP; +import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP; +import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP; + +/** + * A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations + * using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock. + */ +@NotThreadSafe +public class ZookeeperBasedLockProvider extends LockProvider { + + private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); + + private CuratorFramework curatorFrameworkClient; + private volatile InterProcessMutex lock = null; + + public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final Configuration conf) { + this(lockConfiguration); + this.curatorFrameworkClient = CuratorFrameworkFactory.builder() + .connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP)) + .retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP), + 5000, lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP))) + .sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS)) + .connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS)) + .build(); + this.curatorFrameworkClient.start(); + } + + public ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration, final CuratorFramework curatorFrameworkClient) { + this(lockConfiguration); + this.curatorFrameworkClient = curatorFrameworkClient; + synchronized (this.curatorFrameworkClient) { + if (this.curatorFrameworkClient.getState() != CuratorFrameworkState.STARTED) { + this.curatorFrameworkClient.start(); + } + } + } + + ZookeeperBasedLockProvider(final LockConfiguration lockConfiguration) { + checkRequiredProps(lockConfiguration); + this.lockConfiguration = lockConfiguration; + } + + public void acquireLock(long time, TimeUnit unit) throws Exception { + ValidationUtils.checkArgument(this.lock == null, LockState.ALREADY_ACQUIRED.name()); + InterProcessMutex newLock = new InterProcessMutex( + this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/" + + this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP)); + newLock.acquire(time, unit); + if (newLock.isAcquiredInThisProcess()) { + lock = newLock; + } + } + + @Override + public boolean tryLock(long time, TimeUnit unit) { + LOG.info(generateLogStatement(LockState.ACQUIRING, generateLogSuffixString())); + try { + acquireLock(time, unit); + LOG.info(generateLogStatement(LockState.ACQUIRED, generateLogSuffixString())); + } catch (Exception e) { + throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()), e); Review comment: I think the actual code you have know handles this. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected]
