Skip to content

HBASE-26127 Backport HBASE-23898 "Add trace support for simple apis i… #3556

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 2 commits into from
Aug 5, 2021
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 @@ -156,6 +156,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,6 +27,8 @@
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.util.Optional;
import java.util.concurrent.CompletableFuture;
Expand All @@ -48,6 +50,7 @@
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;
Expand Down Expand Up @@ -153,14 +156,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 @@ -195,27 +197,29 @@ public Configuration getConfiguration() {

@Override
public void close() {
// As the code below is safe to be executed in parallel, here we do not use CAS or lock, just a
// simple volatile flag.
if (closed) {
return;
}
LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
if(LOG.isDebugEnabled()){
logCallStack(Thread.currentThread().getStackTrace());
}
IOUtils.closeQuietly(clusterStatusListener,
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));
synchronized (this) {
if (choreService != null) {
choreService.shutdown();
choreService = null;
TraceUtil.trace(() -> {
// As the code below is safe to be executed in parallel, here we do not use CAS or lock,
// just a simple volatile flag.
if (closed) {
return;
}
}
metrics.ifPresent(MetricsConnection::shutdown);
closed = true;
LOG.info("Connection has been closed by {}.", Thread.currentThread().getName());
if (LOG.isDebugEnabled()) {
logCallStack(Thread.currentThread().getStackTrace());
}
IOUtils.closeQuietly(clusterStatusListener,
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));
synchronized (this) {
if (choreService != null) {
choreService.shutdown();
choreService = null;
}
}
metrics.ifPresent(MetricsConnection::shutdown);
closed = true;
}, "AsyncConnection.close");
}

private void logCallStack(StackTraceElement[] stackTraceElements) {
Expand Down Expand Up @@ -320,7 +324,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 @@ -361,35 +365,43 @@ public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName

@Override
public AsyncBufferedMutatorBuilder getBufferedMutatorBuilder(TableName tableName,
ExecutorService pool) {
ExecutorService pool) {
return new AsyncBufferedMutatorBuilderImpl(connConf, getTableBuilder(tableName, pool),
RETRY_TIMER);
}

@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);
}
}

@Override
Expand Down
Loading