diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java index 8a1c7c0e96d49..fc5b7a75f7f60 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/lock/ZookeeperBasedLockProvider.java @@ -18,22 +18,25 @@ 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.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.log4j.LogManager; import org.apache.log4j.Logger; import javax.annotation.concurrent.NotThreadSafe; + +import java.io.Serializable; import java.util.concurrent.TimeUnit; import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS; @@ -52,11 +55,11 @@ * using zookeeper. Users need to have a Zookeeper cluster deployed to be able to use this lock. */ @NotThreadSafe -public class ZookeeperBasedLockProvider implements LockProvider { +public class ZookeeperBasedLockProvider implements LockProvider, Serializable { private static final Logger LOG = LogManager.getLogger(ZookeeperBasedLockProvider.class); - private final CuratorFramework curatorFrameworkClient; + private final transient CuratorFramework curatorFrameworkClient; private volatile InterProcessMutex lock = null; protected LockConfiguration lockConfiguration;