You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by zh...@apache.org on 2021/01/04 15:30:59 UTC
[hbase] branch master updated: HBASE-25457 Possible race in AsyncConnectionImpl between getChoreServ… (#2839)
This is an automated email from the ASF dual-hosted git repository.
zhangduo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase.git
The following commit(s) were added to refs/heads/master by this push:
new 8a0b023 HBASE-25457 Possible race in AsyncConnectionImpl between getChoreServ… (#2839)
8a0b023 is described below
commit 8a0b023023554ba0d65c72f57b242d84bfe2b132
Author: Duo Zhang <zh...@apache.org>
AuthorDate: Mon Jan 4 23:30:32 2021 +0800
HBASE-25457 Possible race in AsyncConnectionImpl between getChoreServ… (#2839)
Signed-off-by: Viraj Jasani <vj...@apache.org>
---
.../apache/hadoop/hbase/client/AsyncConnectionImpl.java | 14 +++++++++-----
1 file changed, 9 insertions(+), 5 deletions(-)
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
index 1dbb7e6..8a1ac5a 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncConnectionImpl.java
@@ -28,8 +28,6 @@ import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLE
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
import java.net.SocketAddress;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
@@ -56,11 +54,11 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.ConcurrentMapUtils;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import org.apache.hbase.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hbase.thirdparty.io.netty.util.HashedWheelTimer;
import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
@@ -187,6 +185,9 @@ class AsyncConnectionImpl implements AsyncConnection {
* @return ChoreService
*/
synchronized ChoreService getChoreService() {
+ if (isClosed()) {
+ throw new IllegalStateException("connection is already closed");
+ }
if (choreService == null) {
choreService = new ChoreService("AsyncConn Chore Service");
}
@@ -216,8 +217,11 @@ class AsyncConnectionImpl implements AsyncConnection {
e -> LOG.warn("failed to close clusterStatusListener", e));
IOUtils.closeQuietly(rpcClient, e -> LOG.warn("failed to close rpcClient", e));
IOUtils.closeQuietly(registry, e -> LOG.warn("failed to close registry", e));
- if (choreService != null) {
- choreService.shutdown();
+ synchronized (this) {
+ if (choreService != null) {
+ choreService.shutdown();
+ choreService = null;
+ }
}
metrics.ifPresent(MetricsConnection::shutdown);
ConnectionOverAsyncConnection c = this.conn;