Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions hbase-client/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,16 @@
<groupId>org.jruby.joni</groupId>
<artifactId>joni</artifactId>
</dependency>
<dependency>
<groupId>io.opentelemetry</groupId>
<artifactId>opentelemetry-sdk</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>io.opentelemetry</groupId>
<artifactId>opentelemetry-sdk-testing</artifactId>
<scope>test</scope>
</dependency>
<dependency>
<groupId>org.slf4j</groupId>
<artifactId>jcl-over-slf4j</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,8 +64,8 @@ public interface AsyncConnection extends Closeable {
/**
* Retrieve an {@link AsyncTable} implementation for accessing a table.
* <p>
* The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if
* you want to customize some configs.
* The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if you
* want to customize some configs.
* <p>
* This method no longer checks table existence. An exception will be thrown if the table does not
* exist only when the first operation is attempted.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,9 @@
import static org.apache.hadoop.hbase.client.NonceGenerator.CLIENT_NONCES_ENABLED_KEY;
import static org.apache.hadoop.hbase.util.FutureUtils.addListener;

import io.opentelemetry.api.trace.Span;
import io.opentelemetry.context.Scope;
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;
Expand All @@ -53,14 +53,15 @@
import org.apache.hadoop.hbase.ipc.RpcClientFactory;
import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.trace.TraceUtil;
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;
Expand Down Expand Up @@ -123,7 +124,7 @@ class AsyncConnectionImpl implements AsyncConnection {
private volatile ConnectionOverAsyncConnection conn;

public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, String clusterId,
SocketAddress localAddress, User user) {
SocketAddress localAddress, User user) {
this.conf = conf;
this.user = user;

Expand All @@ -137,8 +138,8 @@ public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, Stri
} else {
this.metrics = Optional.empty();
}
this.rpcClient = RpcClientFactory.createClient(
conf, clusterId, localAddress, metrics.orElse(null));
this.rpcClient =
RpcClientFactory.createClient(conf, clusterId, localAddress, metrics.orElse(null));
this.rpcControllerFactory = RpcControllerFactory.instantiate(conf);
this.rpcTimeout =
(int) Math.min(Integer.MAX_VALUE, TimeUnit.NANOSECONDS.toMillis(connConf.getRpcTimeoutNs()));
Expand All @@ -161,14 +162,13 @@ public AsyncConnectionImpl(Configuration conf, ConnectionRegistry registry, Stri
LOG.warn("{} is true, but {} is not set", STATUS_PUBLISHED, STATUS_LISTENER_CLASS);
} else {
try {
listener = new ClusterStatusListener(
new ClusterStatusListener.DeadServerHandler() {
@Override
public void newDead(ServerName sn) {
locator.clearCache(sn);
rpcClient.cancelConnections(sn);
}
}, conf, listenerClass);
listener = new ClusterStatusListener(new ClusterStatusListener.DeadServerHandler() {
@Override
public void newDead(ServerName sn) {
locator.clearCache(sn);
rpcClient.cancelConnections(sn);
}
}, conf, listenerClass);
} catch (IOException e) {
LOG.warn("Failed create of ClusterStatusListener, not a critical, ignoring...", e);
}
Expand Down Expand Up @@ -205,24 +205,26 @@ public boolean isClosed() {

@Override
public void close() {
if (!closed.compareAndSet(false, true)) {
return;
}
LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
if(LOG.isDebugEnabled()){
logCallStack(Thread.currentThread().getStackTrace());
}
IOUtils.closeQuietly(clusterStatusListener);
IOUtils.closeQuietly(rpcClient);
IOUtils.closeQuietly(registry);
if (choreService != null) {
choreService.shutdown();
}
metrics.ifPresent(MetricsConnection::shutdown);
ConnectionOverAsyncConnection c = this.conn;
if (c != null) {
c.closePool();
}
TraceUtil.trace(() -> {
if (!closed.compareAndSet(false, true)) {
return;
}
LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
if (LOG.isDebugEnabled()) {
logCallStack(Thread.currentThread().getStackTrace());
}
IOUtils.closeQuietly(clusterStatusListener);
IOUtils.closeQuietly(rpcClient);
IOUtils.closeQuietly(registry);
if (choreService != null) {
choreService.shutdown();
}
metrics.ifPresent(MetricsConnection::shutdown);
ConnectionOverAsyncConnection c = this.conn;
if (c != null) {
c.closePool();
}
}, "AsyncConnection.close");
}

private void logCallStack(StackTraceElement[] stackTraceElements) {
Expand Down Expand Up @@ -336,7 +338,7 @@ public AsyncTable<AdvancedScanResultConsumer> build() {

@Override
public AsyncTableBuilder<ScanResultConsumer> getTableBuilder(TableName tableName,
ExecutorService pool) {
ExecutorService pool) {
return new AsyncTableBuilderBase<ScanResultConsumer>(tableName, connConf) {

@Override
Expand Down Expand Up @@ -377,7 +379,7 @@ public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName

@Override
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
ExecutorService pool) {
ExecutorService pool) {
return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
RETRY_TIMER);
}
Expand All @@ -401,28 +403,36 @@ public Connection toConnection() {

@Override
public CompletableFuture<Hbck> getHbck() {
CompletableFuture<Hbck> future = new CompletableFuture<>();
addListener(registry.getActiveMaster(), (sn, error) -> {
if (error != null) {
future.completeExceptionally(error);
} else {
try {
future.complete(getHbck(sn));
} catch (IOException e) {
future.completeExceptionally(e);
return TraceUtil.tracedFuture(() -> {
CompletableFuture<Hbck> future = new CompletableFuture<>();
addListener(registry.getActiveMaster(), (sn, error) -> {
if (error != null) {
future.completeExceptionally(error);
} else {
try {
future.complete(getHbck(sn));
} catch (IOException e) {
future.completeExceptionally(e);
}
}
}
});
return future;
});
return future;
}, getClass().getName() + ".getHbck");
}

@Override
public Hbck getHbck(ServerName masterServer) throws IOException {
// we will not create a new connection when creating a new protobuf stub, and for hbck there
// will be no performance consideration, so for simplification we will create a new stub every
// time instead of caching the stub here.
return new HBaseHbck(MasterProtos.HbckService.newBlockingStub(
rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)), rpcControllerFactory);
Span span = TraceUtil.createSpan(getClass().getName() + ".getHbck")
.setAttribute(TraceUtil.SERVER_NAME_KEY, masterServer.getServerName());
try (Scope scope = span.makeCurrent()) {
// we will not create a new connection when creating a new protobuf stub, and for hbck there
// will be no performance consideration, so for simplification we will create a new stub every
// time instead of caching the stub here.
return new HBaseHbck(
MasterProtos.HbckService
.newBlockingStub(rpcClient.createBlockingRpcChannel(masterServer, user, rpcTimeout)),
rpcControllerFactory);
}
}

Optional<MetricsConnection> getConnectionMetrics() {
Expand Down
Loading