From da5eba09fb905009d01237ad8dc51de8d03cce5f Mon Sep 17 00:00:00 2001 From: stack Date: Fri, 29 Apr 2016 16:10:11 -0700 Subject: [PATCH 1/2] First cut --- conf/log4j.properties | 1 + hbase-client/pom.xml | 2 +- .../apache/hadoop/hbase/client/AsyncProcess.java | 13 +- .../hadoop/hbase/client/BufferedMutatorImpl.java | 5 +- .../hadoop/hbase/client/ClusterConnection.java | 11 +- .../org/apache/hadoop/hbase/client/Connection.java | 2 +- .../hbase/client/ConnectionImplementation.java | 32 ++- .../org/apache/hadoop/hbase/client/HBaseAdmin.java | 8 +- .../org/apache/hadoop/hbase/client/HTable.java | 220 +++++++++++---------- .../client/ResultBoundedCompletionService.java | 5 +- .../hbase/client/ZooKeeperKeepAliveConnection.java | 8 +- .../hbase/client/replication/ReplicationAdmin.java | 7 +- .../apache/hadoop/hbase/ipc/AsyncRpcChannel.java | 12 +- .../org/apache/hadoop/hbase/ipc/RpcClientImpl.java | 62 +++--- .../hbase/zookeeper/RecoverableZooKeeper.java | 55 +++--- .../org/apache/hadoop/hbase/zookeeper/ZKUtil.java | 14 +- .../apache/hadoop/hbase/zookeeper/ZkAclReset.java | 15 +- .../hadoop/hbase/zookeeper/ZooKeeperWatcher.java | 41 +++- .../apache/hadoop/hbase/zookeeper/TestZKUtil.java | 2 +- .../hbase/zookeeper/TestZooKeeperWatcher.java | 2 +- hbase-common/pom.xml | 2 +- .../hbase/trace/HBaseHTraceConfiguration.java | 9 +- .../hadoop/hbase/trace/SpanReceiverHost.java | 110 ----------- hbase-common/src/main/resources/hbase-default.xml | 18 ++ .../hadoop/hbase/HBaseCommonTestingUtility.java | 2 +- .../hadoop/hbase/io/hfile/MemcachedBlockCache.java | 12 +- hbase-it/pom.xml | 2 +- .../hadoop/hbase/mttr/IntegrationTestMTTR.java | 17 +- .../test/IntegrationTestZKAndFSPermissions.java | 2 +- .../trace/IntegrationTestSendTraceRequests.java | 32 +-- hbase-protocol/src/main/protobuf/Tracing.proto | 13 +- hbase-rsgroup/pom.xml | 2 +- hbase-server/pom.xml | 2 +- .../org/apache/hadoop/hbase/LocalHBaseCluster.java | 6 +- .../apache/hadoop/hbase/executor/EventHandler.java | 18 +- .../hadoop/hbase/io/hfile/HFileReaderImpl.java | 17 +- .../org/apache/hadoop/hbase/ipc/CallRunner.java | 12 +- .../org/apache/hadoop/hbase/ipc/RpcServer.java | 11 +- .../hadoop/hbase/mapreduce/TableMapReduceUtil.java | 2 +- .../org/apache/hadoop/hbase/master/HMaster.java | 42 ++-- .../apache/hadoop/hbase/master/ServerManager.java | 2 +- .../hadoop/hbase/master/cleaner/LogCleaner.java | 2 +- .../hbase/master/handler/DisableTableHandler.java | 10 +- .../master/procedure/DisableTableProcedure.java | 9 +- .../apache/hadoop/hbase/regionserver/HRegion.java | 9 +- .../hadoop/hbase/regionserver/HRegionServer.java | 31 +-- .../hadoop/hbase/regionserver/MemStoreFlusher.java | 14 +- .../hbase/regionserver/wal/AbstractFSWAL.java | 23 ++- .../hadoop/hbase/regionserver/wal/AsyncFSWAL.java | 37 ++-- .../hadoop/hbase/regionserver/wal/FSHLog.java | 120 ++++++----- .../hbase/regionserver/wal/RingBufferTruck.java | 23 +-- .../hadoop/hbase/regionserver/wal/SyncFuture.java | 67 +++---- .../replication/HBaseReplicationEndpoint.java | 2 +- .../apache/hadoop/hbase/HBaseTestingUtility.java | 51 ++--- .../apache/hadoop/hbase/PerformanceEvaluation.java | 31 ++- .../hbase/client/TestReplicaWithCluster.java | 18 +- .../hbase/procedure/TestProcedureManager.java | 2 +- .../apache/hadoop/hbase/trace/TestHTraceHooks.java | 142 ------------- .../hadoop/hbase/wal/WALPerformanceEvaluation.java | 36 ++-- .../hbase/zookeeper/TestRecoverableZooKeeper.java | 2 +- hbase-shell/pom.xml | 2 +- hbase-shell/src/main/ruby/shell/commands/trace.rb | 23 ++- hbase-thrift/pom.xml | 2 +- pom.xml | 4 +- src/main/asciidoc/_chapters/tracing.adoc | 16 +- 65 files changed, 706 insertions(+), 820 deletions(-) delete mode 100644 hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java delete mode 100644 hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java diff --git a/conf/log4j.properties b/conf/log4j.properties index d36a22e..33d35b8 100644 --- a/conf/log4j.properties +++ b/conf/log4j.properties @@ -92,6 +92,7 @@ log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %m% log4j.logger.org.apache.zookeeper=INFO #log4j.logger.org.apache.hadoop.fs.FSNamesystem=DEBUG log4j.logger.org.apache.hadoop.hbase=INFO +#log4j.logger.org.apache.htrace=TRACE log4j.logger.org.apache.hadoop.hbase.META=INFO # Make these two classes INFO-level. Make them DEBUG to see more zk debug. log4j.logger.org.apache.hadoop.hbase.zookeeper.ZKUtil=INFO diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml index e74e0d5..77ad28e 100644 --- a/hbase-client/pom.xml +++ b/hbase-client/pom.xml @@ -170,7 +170,7 @@ org.apache.htrace - htrace-core + htrace-core4 org.codehaus.jackson diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index b2c758d..5f34aa3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -58,7 +58,6 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.htrace.Trace; import com.google.common.annotations.VisibleForTesting; @@ -1037,7 +1036,7 @@ class AsyncProcess { runnable.run(); } else { try { - pool.submit(runnable); + pool.submit(connection.getTracer().wrap(runnable, "sendMultiAction")); } catch (Throwable t) { if (t instanceof RejectedExecutionException) { // This should never happen. But as the pool is provided by the end user, @@ -1070,8 +1069,10 @@ class AsyncProcess { if (connection.getConnectionMetrics() != null) { connection.getConnectionMetrics().incrNormalRunners(); } - return Collections.singletonList(Trace.wrap("AsyncProcess.sendMultiAction", - new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress))); + Runnable runnable = + new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress); + return Collections.singletonList( + connection.getTracer().wrap(runnable, "AsyncProcess.sendMultiAction")); } // group the actions by the amount of delay @@ -1109,7 +1110,7 @@ class AsyncProcess { connection.getConnectionMetrics().incrNormalRunners(); } } - runnable = Trace.wrap(traceText, runnable); + runnable = connection.getTracer().wrap(runnable, traceText); toReturn.add(runnable); } @@ -1143,7 +1144,7 @@ class AsyncProcess { // Start the thread that may kick off replica gets. // TODO: we could do it on the same thread, but it's a user thread, might be a bad idea. try { - pool.submit(replicaRunnable); + pool.submit(connection.getTracer().wrap(replicaRunnable, "startWaitingForReplicaCalls")); } catch (RejectedExecutionException ree) { LOG.warn("#" + id + ", replica task was rejected by the pool - no replica calls", ree); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java index 01aaec5..8b107b2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/BufferedMutatorImpl.java @@ -23,6 +23,7 @@ import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.ipc.RpcControllerFactory; +import org.apache.htrace.core.TraceScope; import java.io.IOException; import java.io.InterruptedIOException; @@ -207,7 +208,7 @@ public class BufferedMutatorImpl implements BufferedMutator { // Keep track of the size so that this thread doesn't spin forever long dequeuedSize = 0; - try { + try (TraceScope traceScope = this.connection.getTracer().newScope("flushCommits")) { // Grab all of the available mutations. Mutation m; @@ -278,4 +279,4 @@ public class BufferedMutatorImpl implements BufferedMutator { public long getWriteBufferSize() { return this.writeBufferSize; } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java index 3027761..f768158 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ClusterConnection.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hbase.ipc.RpcControllerFactory; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ClientService; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.MasterService; +import org.apache.htrace.core.Tracer; /** Internal methods on Connection that should not be used by user code. */ @InterfaceAudience.Private @@ -320,11 +321,19 @@ public interface ClusterConnection extends HConnection { /** * @return the MetricsConnection instance associated with this connection. */ - public MetricsConnection getConnectionMetrics(); + MetricsConnection getConnectionMetrics(); /** * @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so * supports cell blocks. */ boolean hasCellBlockSupport(); + + /** + * @return This Connection's non-null Tracer instance. + */ + // All the 'extras' in here are annotated Private, as for internal use only but adding + // this annotation here to underline this fact. + @InterfaceAudience.Private + Tracer getTracer(); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java index a3f6fe6..1ae5b10 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/Connection.java @@ -46,7 +46,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; * thread will obtain its own Table instance. Caching or pooling of {@link Table} and {@link Admin} * is not recommended. * - *

This class replaces {@link HConnection}, which is now deprecated. + *

This class replaces HConnection, which is now deprecated. * @see ConnectionFactory * @since 0.99.0 */ diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index 9a7dfc7..e7df3ab 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -87,6 +87,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunn import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.SetNormalizerRunningResponse; import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException; import org.apache.hadoop.hbase.security.User; +import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.ExceptionUtil; @@ -96,6 +97,10 @@ import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.ipc.RemoteException; +import org.apache.htrace.core.HTraceConfiguration; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; +import org.apache.htrace.core.TracerId; import org.apache.zookeeper.KeeperException; @@ -183,14 +188,25 @@ class ConnectionImplementation implements ClusterConnection, Closeable { private final ClientBackoffPolicy backoffPolicy; /** + * Connection tracer. + */ + private final Tracer tracer; + + /** * constructor * @param conf Configuration object */ - ConnectionImplementation(Configuration conf, - ExecutorService pool, User user) throws IOException { + ConnectionImplementation(Configuration conf, ExecutorService pool, User user) + throws IOException { this.conf = conf; this.user = user; this.batchPool = pool; + + // Set up tracer. Add Connection hash to the TraceId so it comes out as "HConnection,hash". + conf.set(HBaseHTraceConfiguration.KEY_PREFIX + TracerId.TRACER_ID_KEY, toString()); + HTraceConfiguration htraceConfiguration = new HBaseHTraceConfiguration(conf); + this.tracer = new Tracer.Builder("").conf(htraceConfiguration).build(); + this.connectionConfig = new ConnectionConfiguration(conf); this.closed = false; this.pause = conf.getLong(HConstants.HBASE_CLIENT_PAUSE, @@ -731,7 +747,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable { "table name cannot be null or zero length"); } if (tableName.equals(TableName.META_TABLE_NAME)) { - return locateMeta(tableName, useCache, replicaId); + try (TraceScope traceScope = getTracer().newScope("locateMeta")) { + return locateMeta(tableName, useCache, replicaId); + } } else { // Region not in the cache - have to go to the meta RS return locateRegionInMeta(tableName, row, useCache, retry, replicaId); @@ -1330,7 +1348,8 @@ class ConnectionImplementation implements ClusterConnection, Closeable { } // We don't check that our link to ZooKeeper is still valid // But there is a retry mechanism in the ZooKeeperWatcher itself - keepAliveZookeeper = new ZooKeeperKeepAliveConnection(conf, this.toString(), this); + keepAliveZookeeper = + new ZooKeeperKeepAliveConnection(conf, this.toString(), this, getTracer()); } keepAliveZookeeperUserCount.addAndGet(1); return keepAliveZookeeper; @@ -2299,4 +2318,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable { public RpcControllerFactory getRpcControllerFactory() { return this.rpcControllerFactory; } + + @Override + public Tracer getTracer() { + return this.tracer; + } } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java index 7371f03..e3cfff7 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java @@ -1210,7 +1210,7 @@ public class HBaseAdmin implements Admin { try { checkTableExists(tableName); zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); + new ThrowableAbortable(), connection.getTracer()); List> pairs; if (TableName.META_TABLE_NAME.equals(tableName)) { pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper); @@ -1586,7 +1586,7 @@ public class HBaseAdmin implements Admin { try { checkTableExists(tableName); zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); + new ThrowableAbortable(), this.connection.getTracer()); List> pairs; if (TableName.META_TABLE_NAME.equals(tableName)) { pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper); @@ -2107,7 +2107,7 @@ public class HBaseAdmin implements Admin { throws IOException { ZooKeeperWatcher zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); + new ThrowableAbortable(), this.connection.getTracer()); List regions = null; try { if (TableName.META_TABLE_NAME.equals(tableName)) { @@ -3065,7 +3065,7 @@ public class HBaseAdmin implements Admin { List> pairs; if (TableName.META_TABLE_NAME.equals(tableName)) { zookeeper = new ZooKeeperWatcher(conf, ZK_IDENTIFIER_PREFIX + connection.toString(), - new ThrowableAbortable()); + new ThrowableAbortable(), this.connection.getTracer()); pairs = new MetaTableLocator().getMetaRegionsAndLocations(zookeeper); } else { pairs = MetaTableAccessor.getTableRegionsAndLocations(connection, tableName); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index bf9ec22..0160c34 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -67,6 +67,7 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.ReflectionUtils; import org.apache.hadoop.hbase.util.Threads; +import org.apache.htrace.core.TraceScope; import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.Descriptors; @@ -118,6 +119,7 @@ public class HTable implements HTableInterface { private final boolean cleanupConnectionOnClose; // close the connection in close() private Consistency defaultConsistency = Consistency.STRONG; private HRegionLocator locator; + private TraceScope tableTracerScope; /** The Async process for batch */ protected AsyncProcess multiAp; @@ -163,6 +165,7 @@ public class HTable implements HTableInterface { throw new IllegalArgumentException("Connection is null or closed."); } this.tableName = tableName; + connection.getTracer().newScope(getName().toString()); this.cleanupConnectionOnClose = false; this.connection = connection; this.configuration = connection.getConfiguration(); @@ -189,6 +192,7 @@ public class HTable implements HTableInterface { protected HTable(ClusterConnection conn, BufferedMutatorParams params) throws IOException { connection = conn; tableName = params.getTableName(); + conn.getTracer().newScope(getName().toString()); connConfiguration = new ConnectionConfiguration(connection.getConfiguration()); cleanupPoolOnClose = false; cleanupConnectionOnClose = false; @@ -268,12 +272,14 @@ public class HTable implements HTableInterface { */ @Override public HTableDescriptor getTableDescriptor() throws IOException { - HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory, - rpcControllerFactory, operationTimeout, rpcTimeout); - if (htd != null) { - return new UnmodifyableHTableDescriptor(htd); + try (TraceScope traceScope = this.connection.getTracer().newScope("getTableDescriptor")) { + HTableDescriptor htd = HBaseAdmin.getTableDescriptor(tableName, connection, rpcCallerFactory, + rpcControllerFactory, operationTimeout, rpcTimeout); + if (htd != null) { + return new UnmodifyableHTableDescriptor(htd); + } + return null; } - return null; } /** @@ -317,65 +323,68 @@ public class HTable implements HTableInterface { List keysInRange = new ArrayList(); List regionsInRange = new ArrayList(); byte[] currentKey = startKey; - do { - HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload); - keysInRange.add(currentKey); - regionsInRange.add(regionLocation); - currentKey = regionLocation.getRegionInfo().getEndKey(); - } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) - && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0 - || (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0))); - return new Pair, List>(keysInRange, - regionsInRange); + try (TraceScope traceScope = this.connection.getTracer().newScope("getKeysAndRegionsInRange")) { + do { + HRegionLocation regionLocation = getRegionLocator().getRegionLocation(currentKey, reload); + keysInRange.add(currentKey); + regionsInRange.add(regionLocation); + currentKey = regionLocation.getRegionInfo().getEndKey(); + } while (!Bytes.equals(currentKey, HConstants.EMPTY_END_ROW) + && (endKeyIsEndOfTable || Bytes.compareTo(currentKey, endKey) < 0 + || (includeEndKey && Bytes.compareTo(currentKey, endKey) == 0))); + return new Pair, List>(keysInRange, regionsInRange); + } } /** * The underlying {@link HTable} must not be closed. - * {@link HTableInterface#getScanner(Scan)} has other usage details. + * HTableInterface#getScanner(Scan) has other usage details. */ @Override public ResultScanner getScanner(final Scan scan) throws IOException { - if (scan.getBatch() > 0 && scan.isSmall()) { - throw new IllegalArgumentException("Small scan should not be used with batching"); - } + try (TraceScope traceScope = this.connection.getTracer().newScope("getScanner")) { + if (scan.getBatch() > 0 && scan.isSmall()) { + throw new IllegalArgumentException("Small scan should not be used with batching"); + } - if (scan.getCaching() <= 0) { - scan.setCaching(scannerCaching); - } - if (scan.getMaxResultSize() <= 0) { - scan.setMaxResultSize(scannerMaxResultSize); - } + if (scan.getCaching() <= 0) { + scan.setCaching(scannerCaching); + } + if (scan.getMaxResultSize() <= 0) { + scan.setMaxResultSize(scannerMaxResultSize); + } - Boolean async = scan.isAsyncPrefetch(); - if (async == null) { - async = connConfiguration.isClientScannerAsyncPrefetch(); - } + Boolean async = scan.isAsyncPrefetch(); + if (async == null) { + async = connConfiguration.isClientScannerAsyncPrefetch(); + } - if (scan.isReversed()) { - if (scan.isSmall()) { - return new ClientSmallReversedScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } else { - return new ReversedClientScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + if (scan.isReversed()) { + if (scan.isSmall()) { + return new ClientSmallReversedScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, this.rpcControllerFactory, + pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } else { + return new ReversedClientScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, this.rpcControllerFactory, + pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } } - } - if (scan.isSmall()) { - return new ClientSmallScanner(getConfiguration(), scan, getName(), - this.connection, this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); - } else { - if (async) { - return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, + if (scan.isSmall()) { + return new ClientSmallScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, this.rpcControllerFactory, pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); } else { - return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection, - this.rpcCallerFactory, this.rpcControllerFactory, - pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + if (async) { + return new ClientAsyncPrefetchScanner(getConfiguration(), scan, getName(), + this.connection, this.rpcCallerFactory, this.rpcControllerFactory, + pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } else { + return new ClientSimpleScanner(getConfiguration(), scan, getName(), this.connection, + this.rpcCallerFactory, this.rpcControllerFactory, + pool, connConfiguration.getReplicaCallTimeoutMicroSecondScan()); + } } } } @@ -412,38 +421,40 @@ public class HTable implements HTableInterface { } private Result get(Get get, final boolean checkExistenceOnly) throws IOException { - // if we are changing settings to the get, clone it. - if (get.isCheckExistenceOnly() != checkExistenceOnly || get.getConsistency() == null) { - get = ReflectionUtils.newInstance(get.getClass(), get); - get.setCheckExistenceOnly(checkExistenceOnly); - if (get.getConsistency() == null){ - get.setConsistency(defaultConsistency); + try (TraceScope traceScope = this.connection.getTracer().newScope("get")) { + // if we are changing settings to the get, clone it. + if (get.isCheckExistenceOnly() != checkExistenceOnly || get.getConsistency() == null) { + get = ReflectionUtils.newInstance(get.getClass(), get); + get.setCheckExistenceOnly(checkExistenceOnly); + if (get.getConsistency() == null){ + get.setConsistency(defaultConsistency); + } } - } - if (get.getConsistency() == Consistency.STRONG) { - // Good old call. - final Get getReq = get; - RegionServerCallable callable = new RegionServerCallable(this.connection, - getName(), get.getRow()) { - @Override - public Result call(int callTimeout) throws IOException { - ClientProtos.GetRequest request = - RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); - controller.setPriority(tableName); - controller.setCallTimeout(callTimeout); - try { - ClientProtos.GetResponse response = getStub().get(controller, request); - if (response == null) return null; - return ProtobufUtil.toResult(response.getResult(), controller.cellScanner()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); + if (get.getConsistency() == Consistency.STRONG) { + // Good old call. + final Get getReq = get; + RegionServerCallable callable = new RegionServerCallable(this.connection, + getName(), get.getRow()) { + @Override + public Result call(int callTimeout) throws IOException { + ClientProtos.GetRequest request = + RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq); + PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + controller.setPriority(tableName); + controller.setCallTimeout(callTimeout); + try { + ClientProtos.GetResponse response = getStub().get(controller, request); + if (response == null) return null; + return ProtobufUtil.toResult(response.getResult(), controller.cellScanner()); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); + } } - } - }; - return rpcCallerFactory.newCaller(rpcTimeout).callWithRetries(callable, - this.operationTimeout); + }; + return rpcCallerFactory.newCaller(rpcTimeout).callWithRetries(callable, + this.operationTimeout); + } } // Call that takes into account the replica @@ -464,13 +475,13 @@ public class HTable implements HTableInterface { if (gets.size() == 1) { return new Result[]{get(gets.get(0))}; } - try { + try (TraceScope traceScope = this.connection.getTracer().newScope("gets")) { Object[] r1 = new Object[gets.size()]; batch((List) gets, r1); // translate. Result [] results = new Result[r1.length]; - int i=0; + int i = 0; for (Object o : r1) { // batch ensures if there is a failure we get an exception instead results[i++] = (Result) o; @@ -488,10 +499,12 @@ public class HTable implements HTableInterface { @Override public void batch(final List actions, final Object[] results) throws InterruptedException, IOException { - AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, actions, null, results); - ars.waitUntilDone(); - if (ars.hasError()) { - throw ars.getErrors(); + try (TraceScope traceScope = this.connection.getTracer().newScope("batch")) { + AsyncRequestFuture ars = multiAp.submitAll(pool, tableName, actions, null, results); + ars.waitUntilDone(); + if (ars.hasError()) { + throw ars.getErrors(); + } } } @@ -529,8 +542,10 @@ public class HTable implements HTableInterface { } } }; - rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, + try (TraceScope traceScope = this.connection.getTracer().newScope("delete")) { + rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); + } } /** @@ -539,19 +554,21 @@ public class HTable implements HTableInterface { @Override public void delete(final List deletes) throws IOException { - Object[] results = new Object[deletes.size()]; - try { - batch(deletes, results); - } catch (InterruptedException e) { - throw (InterruptedIOException)new InterruptedIOException().initCause(e); - } finally { - // mutate list so that it is empty for complete success, or contains only failed records - // results are returned in the same order as the requests in list walk the list backwards, - // so we can remove from list without impacting the indexes of earlier members - for (int i = results.length - 1; i>=0; i--) { - // if result is not null, it succeeded - if (results[i] instanceof Result) { - deletes.remove(i); + try (TraceScope traceScope = this.connection.getTracer().newScope("deletes")) { + Object[] results = new Object[deletes.size()]; + try { + batch(deletes, results); + } catch (InterruptedException e) { + throw (InterruptedIOException)new InterruptedIOException().initCause(e); + } finally { + // mutate list so that it is empty for complete success, or contains only failed records + // results are returned in the same order as the requests in list walk the list backwards, + // so we can remove from list without impacting the indexes of earlier members + for (int i = results.length - 1; i>=0; i--) { + // if result is not null, it succeeded + if (results[i] instanceof Result) { + deletes.remove(i); + } } } } @@ -659,8 +676,10 @@ public class HTable implements HTableInterface { } } }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, + try (TraceScope traceScope = this.connection.getTracer().newScope("append")) { + return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); + } } /** @@ -1032,6 +1051,7 @@ public class HTable implements HTableInterface { LOG.warn("waitForTermination interrupted"); } } + if (this.tableTracerScope != null) this.tableTracerScope.close(); if (cleanupConnectionOnClose) { if (this.connection != null) { this.connection.close(); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java index 9b32e93..6ef451b 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java @@ -25,7 +25,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.htrace.Trace; +import org.apache.htrace.core.Tracer; /** * A completion service for the RpcRetryingCallerFactory. @@ -143,7 +143,8 @@ public class ResultBoundedCompletionService { public void submit(RetryingCallable task, int callTimeout, int id) { QueueingFuture newFuture = new QueueingFuture(task, callTimeout); - executor.execute(Trace.wrap(newFuture)); + Tracer tracer = Tracer.curThreadTracer(); + executor.execute(tracer == null? newFuture: tracer.wrap(newFuture, "submit")); tasks[id] = newFuture; } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java index 04d4b41..58fcb30 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ZooKeeperKeepAliveConnection.java @@ -24,6 +24,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.htrace.core.Tracer; /** * We inherit the current ZooKeeperWatcher implementation to change the semantic @@ -39,8 +40,9 @@ import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{ ZooKeeperKeepAliveConnection( Configuration conf, String descriptor, - ConnectionImplementation conn) throws IOException { - super(conf, descriptor, conn); + ConnectionImplementation conn, Tracer tracer) + throws IOException { + super(conf, descriptor, conn, tracer); } @Override @@ -53,4 +55,4 @@ class ZooKeeperKeepAliveConnection extends ZooKeeperWatcher{ void internalClose(){ super.close(); } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java index a2ad2e7..a59f52f 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/replication/ReplicationAdmin.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hbase.TableNotFoundException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; import org.apache.hadoop.hbase.client.Admin; +import org.apache.hadoop.hbase.client.ClusterConnection; import org.apache.hadoop.hbase.client.Connection; import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.hbase.client.RegionLocator; @@ -94,7 +95,7 @@ public class ReplicationAdmin implements Closeable { public static final String REPLICATIONGLOBAL = Integer .toString(HConstants.REPLICATION_SCOPE_GLOBAL); - private final Connection connection; + private final ClusterConnection connection; // TODO: replication should be managed by master. All the classes except ReplicationAdmin should // be moved to hbase-server. Resolve it in HBASE-11392. private final ReplicationQueuesClient replicationQueuesClient; @@ -117,7 +118,7 @@ public class ReplicationAdmin implements Closeable { throw new RuntimeException("hbase.replication isn't true, please " + "enable it in order to use replication"); } - this.connection = ConnectionFactory.createConnection(conf); + this.connection = (ClusterConnection)ConnectionFactory.createConnection(conf); try { zkw = createZooKeeperWatcher(); try { @@ -161,7 +162,7 @@ public class ReplicationAdmin implements Closeable { public boolean isAborted() { return false; } - }); + }, connection.getTracer()); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java index 53eb824..87f8444 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/AsyncRpcChannel.java @@ -57,8 +57,8 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenSelector; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.Tracer; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; @@ -386,10 +386,10 @@ public class AsyncRpcChannel { requestHeaderBuilder.setCallId(call.id).setMethodName(call.method.getName()) .setRequestParam(call.param != null); - if (Trace.isTracing()) { - Span s = Trace.currentSpan(); - requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder() - .setParentId(s.getSpanId()).setTraceId(s.getTraceId())); + SpanId spanId = Tracer.getCurrentSpanId(); + if (spanId.isValid()) { + requestHeaderBuilder.setTraceInfo(TracingProtos.RPCTInfo.newBuilder(). + setParentId(spanId.getLow()).setTraceId(spanId.getHigh())); } ByteBuffer cellBlock = client.buildCellBlock(call.controller.cellScanner()); diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java index 83d4adf..e173d58 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/ipc/RpcClientImpl.java @@ -92,9 +92,9 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenSelector; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.Descriptors.MethodDescriptor; @@ -144,15 +144,15 @@ public class RpcClientImpl extends AbstractRpcClient { private static class CallFuture { final Call call; final int priority; - final Span span; + final SpanId parentSpanId; // We will use this to stop the writer - final static CallFuture DEATH_PILL = new CallFuture(null, -1, null); + final static CallFuture DEATH_PILL = new CallFuture(null, -1); - CallFuture(Call call, int priority, Span span) { + CallFuture(Call call, int priority) { this.call = call; this.priority = priority; - this.span = span; + this.parentSpanId = Tracer.getCurrentSpanId(); } } @@ -206,9 +206,9 @@ public class RpcClientImpl extends AbstractRpcClient { protected final BlockingQueue callsToWrite; - public CallFuture sendCall(Call call, int priority, Span span) + public CallFuture sendCall(Call call, int priority) throws InterruptedException, IOException { - CallFuture cts = new CallFuture(call, priority, span); + CallFuture cts = new CallFuture(call, priority); if (!callsToWrite.offer(cts)) { throw new IOException("Can't add the call " + call.id + " to the write queue. callsToWrite.size()=" + callsToWrite.size()); @@ -267,9 +267,10 @@ public class RpcClientImpl extends AbstractRpcClient { if (cts.call.checkAndSetTimeout()) { continue; } - + Tracer tracer = Tracer.curThreadTracer(); + TraceScope traceScope = tracer == null? null: tracer.newScope("call"); try { - Connection.this.tracedWriteRequest(cts.call, cts.priority, cts.span); + Connection.this.writeRequest(cts.call, cts.priority); } catch (IOException e) { if (LOG.isDebugEnabled()) { LOG.debug("call write error for call #" + cts.call.id @@ -277,6 +278,8 @@ public class RpcClientImpl extends AbstractRpcClient { } cts.call.setException(e); markClosed(e); + } finally { + if (traceScope != null) traceScope.close(); } } @@ -867,15 +870,6 @@ public class RpcClientImpl extends AbstractRpcClient { } } - protected void tracedWriteRequest(Call call, int priority, Span span) throws IOException { - TraceScope ts = Trace.continueSpan(span); - try { - writeRequest(call, priority, span); - } finally { - ts.close(); - } - } - /** * Initiates a call by sending the parameter to the remote server. * Note: this is not called from the Connection thread, but by other @@ -884,12 +878,16 @@ public class RpcClientImpl extends AbstractRpcClient { */ @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="IS2_INCONSISTENT_SYNC", justification="Findbugs is misinterpreting locking missing fact that this.outLock is held") - private void writeRequest(Call call, final int priority, Span span) throws IOException { + private void writeRequest(Call call, final int priority) throws IOException { RequestHeader.Builder builder = RequestHeader.newBuilder(); builder.setCallId(call.id); - if (span != null) { - builder.setTraceInfo( - RPCTInfo.newBuilder().setParentId(span.getSpanId()).setTraceId(span.getTraceId())); + SpanId spanId = Tracer.getCurrentSpanId(); + if (spanId.isValid()) { + // Pre-4.0.0 htrace, we had parent and trace id. In 4.0.0, there is one id only and it s + // 128 bits. Rather than add new fields or change field names, just set the high and low of + // the 128 integer into the old parent id and trace id fields. + builder.setTraceInfo(RPCTInfo.newBuilder(). + setParentId(spanId.getLow()).setTraceId(spanId.getHigh())); } builder.setMethodName(call.md.getName()); builder.setRequestParam(call.param != null); @@ -1226,8 +1224,12 @@ public class RpcClientImpl extends AbstractRpcClient { final Connection connection = getConnection(ticket, call, addr); final CallFuture cts; + Tracer tracer = Tracer.curThreadTracer(); if (connection.callSender != null) { - cts = connection.callSender.sendCall(call, pcrc.getPriority(), Trace.currentSpan()); + TraceScope traceScope = tracer == null? null: + tracer.newScope("call " + connection.callSender.getName()); + try { + cts = connection.callSender.sendCall(call, pcrc.getPriority()); pcrc.notifyOnCancel(new RpcCallback() { @Override public void run(Object parameter) { @@ -1239,9 +1241,17 @@ public class RpcClientImpl extends AbstractRpcClient { call.callComplete(); return new Pair(call.response, call.cells); } + } finally { + if (traceScope != null) traceScope.close(); + } } else { cts = null; - connection.tracedWriteRequest(call, pcrc.getPriority(), Trace.currentSpan()); + TraceScope traceScope = tracer == null? null: tracer.newScope("call"); + try { + connection.writeRequest(call, pcrc.getPriority()); + } finally { + if (traceScope != null) traceScope.close(); + } } while (!call.done) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java index c1eb214..0614f54 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/RecoverableZooKeeper.java @@ -31,6 +31,8 @@ import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.RetryCounter; import org.apache.hadoop.hbase.util.RetryCounterFactory; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -44,8 +46,6 @@ import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.apache.zookeeper.proto.CreateRequest; import org.apache.zookeeper.proto.SetDataRequest; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; /** * A zookeeper that can handle 'recoverable' errors. @@ -97,21 +97,24 @@ public class RecoverableZooKeeper { private static final int ID_LENGTH_OFFSET = MAGIC_SIZE; private static final int ID_LENGTH_SIZE = Bytes.SIZEOF_INT; + private final Tracer tracer; + public RecoverableZooKeeper(String quorumServers, int sessionTimeout, - Watcher watcher, int maxRetries, int retryIntervalMillis) + Watcher watcher, int maxRetries, int retryIntervalMillis, final Tracer tracer) throws IOException { - this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, - null); + this(quorumServers, sessionTimeout, watcher, maxRetries, retryIntervalMillis, tracer, null); } @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DE_MIGHT_IGNORE", justification="None. Its always been this way.") public RecoverableZooKeeper(String quorumServers, int sessionTimeout, - Watcher watcher, int maxRetries, int retryIntervalMillis, String identifier) + Watcher watcher, int maxRetries, int retryIntervalMillis, final Tracer tracer, + String identifier) throws IOException { + this.tracer = tracer; + if (this.tracer == null) throw new NullPointerException(); // TODO: Add support for zk 'chroot'; we don't add it to the quorumServers String as we should. - this.retryCounterFactory = - new RetryCounterFactory(maxRetries+1, retryIntervalMillis); + this.retryCounterFactory = new RetryCounterFactory(maxRetries+1, retryIntervalMillis); if (identifier == null || identifier.length() == 0) { // the identifier = processID@hostName @@ -168,9 +171,8 @@ public class RecoverableZooKeeper { */ public void delete(String path, int version) throws InterruptedException, KeeperException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.delete"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.delete"); RetryCounter retryCounter = retryCounterFactory.create(); boolean isRetry = false; // False for first attempt, true for all retries. while (true) { @@ -212,9 +214,8 @@ public class RecoverableZooKeeper { */ public Stat exists(String path, Watcher watcher) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.exists"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.exists"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -244,9 +245,8 @@ public class RecoverableZooKeeper { */ public Stat exists(String path, boolean watch) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.exists"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.exists"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -285,10 +285,9 @@ public class RecoverableZooKeeper { * @return List of children znodes */ public List getChildren(String path, Watcher watcher) - throws KeeperException, InterruptedException { - TraceScope traceScope = null; + throws KeeperException, InterruptedException { + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.getChildren"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.getChildren"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -318,9 +317,8 @@ public class RecoverableZooKeeper { */ public List getChildren(String path, boolean watch) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.getChildren"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.getChildren"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -350,9 +348,8 @@ public class RecoverableZooKeeper { */ public byte[] getData(String path, Watcher watcher, Stat stat) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.getData"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.getData"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -383,9 +380,8 @@ public class RecoverableZooKeeper { */ public byte[] getData(String path, boolean watch, Stat stat) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.getData"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.getData"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -418,9 +414,8 @@ public class RecoverableZooKeeper { */ public Stat setData(String path, byte[] data, int version) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.setData"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.setData"); RetryCounter retryCounter = retryCounterFactory.create(); byte[] newData = appendMetaData(data); boolean isRetry = false; @@ -470,7 +465,7 @@ public class RecoverableZooKeeper { throws KeeperException, InterruptedException { TraceScope traceScope = null; try { - traceScope = Trace.startSpan("RecoverableZookeeper.getAcl"); + traceScope = this.tracer.newScope("RecoverableZookeeper.getAcl"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -502,7 +497,7 @@ public class RecoverableZooKeeper { throws KeeperException, InterruptedException { TraceScope traceScope = null; try { - traceScope = Trace.startSpan("RecoverableZookeeper.setAcl"); + traceScope = this.tracer.newScope("RecoverableZookeeper.setAcl"); RetryCounter retryCounter = retryCounterFactory.create(); while (true) { try { @@ -544,9 +539,8 @@ public class RecoverableZooKeeper { public String create(String path, byte[] data, List acl, CreateMode createMode) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.create"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.create"); byte[] newData = appendMetaData(data); switch (createMode) { case EPHEMERAL: @@ -674,9 +668,8 @@ public class RecoverableZooKeeper { */ public List multi(Iterable ops) throws KeeperException, InterruptedException { - TraceScope traceScope = null; + TraceScope traceScope = this.tracer.newScope("RecoverableZookeeper.multi"); try { - traceScope = Trace.startSpan("RecoverableZookeeper.multi"); RetryCounter retryCounter = retryCounterFactory.create(); Iterable multiOps = prepareZKMulti(ops); while (true) { diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java index 9e01d09..1df6b03 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZKUtil.java @@ -59,6 +59,7 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.DeleteNodeFailSilent; import org.apache.hadoop.hbase.zookeeper.ZKUtil.ZKUtilOp.SetData; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.authentication.util.KerberosUtil; +import org.apache.htrace.core.Tracer; import org.apache.zookeeper.AsyncCallback; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.KeeperException; @@ -107,20 +108,21 @@ public class ZKUtil { * @return connection to zookeeper * @throws IOException if unable to connect to zk or config problem */ - public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher) + public static RecoverableZooKeeper connect(Configuration conf, Watcher watcher, + final Tracer tracer) throws IOException { String ensemble = ZKConfig.getZKQuorumServersString(conf); - return connect(conf, ensemble, watcher); + return connect(conf, ensemble, watcher, tracer); } public static RecoverableZooKeeper connect(Configuration conf, String ensemble, - Watcher watcher) + Watcher watcher, final Tracer tracer) throws IOException { - return connect(conf, ensemble, watcher, null); + return connect(conf, ensemble, watcher, tracer, null); } public static RecoverableZooKeeper connect(Configuration conf, String ensemble, - Watcher watcher, final String identifier) + Watcher watcher, final Tracer tracer, final String identifier) throws IOException { if(ensemble == null) { throw new IOException("Unable to determine ZooKeeper ensemble"); @@ -136,7 +138,7 @@ public class ZKUtil { zkDumpConnectionTimeOut = conf.getInt("zookeeper.dump.connection.timeout", 1000); return new RecoverableZooKeeper(ensemble, timeout, watcher, - retry, retryIntervalMillis, identifier); + retry, retryIntervalMillis, tracer, identifier); } /** diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java index 9bb6bf7..2252944 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZkAclReset.java @@ -27,8 +27,10 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.classification.InterfaceAudience; +import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.htrace.core.Tracer; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.ZooKeeper; @@ -47,7 +49,8 @@ public class ZkAclReset extends Configured implements Tool { private static final Log LOG = LogFactory.getLog(ZkAclReset.class); private static void resetAcls(final ZooKeeperWatcher zkw, final String znode, - final boolean eraseAcls) throws Exception { + final boolean eraseAcls) + throws Exception { List children = ZKUtil.listChildrenNoWatch(zkw, znode); if (children != null) { for (String child: children) { @@ -65,9 +68,9 @@ public class ZkAclReset extends Configured implements Tool { } } - private static void resetAcls(final Configuration conf, boolean eraseAcls) + private static void resetAcls(final Configuration conf, boolean eraseAcls, Tracer tracer) throws Exception { - ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "ZkAclReset", null); + ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "ZkAclReset", null, tracer); try { LOG.info((eraseAcls ? "Erase" : "Set") + " HBase ACLs for " + zkw.getQuorum() + " " + zkw.getBaseZNode()); @@ -105,8 +108,10 @@ public class ZkAclReset extends Configured implements Tool { printUsageAndExit(); } } - - resetAcls(getConf(), eraseAcls); + try (Tracer tracer = new Tracer.Builder("ZkAclReset"). + conf(new HBaseHTraceConfiguration(getConf())).build();) { + resetAcls(getConf(), eraseAcls, tracer); + } return(0); } diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index 205d397..934ea56 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.ZooKeeperConnectionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.htrace.core.Tracer; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.WatchedEvent; import org.apache.zookeeper.Watcher; @@ -142,9 +143,22 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @throws IOException * @throws ZooKeeperConnectionException */ - public ZooKeeperWatcher(Configuration conf, String identifier, - Abortable abortable) throws ZooKeeperConnectionException, IOException { - this(conf, identifier, abortable, false); + public ZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable) + throws ZooKeeperConnectionException, IOException { + this(conf, identifier, abortable, Tracer.curThreadTracer()); + } + + /** + * Instantiate a ZooKeeper connection and watcher. + * @param identifier string that is passed to RecoverableZookeeper to be used as + * identifier for this instance. Use null for default. + * @throws IOException + * @throws ZooKeeperConnectionException + */ + public ZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable, + final Tracer tracer) + throws ZooKeeperConnectionException, IOException { + this(conf, identifier, abortable, tracer, false); } /** @@ -152,8 +166,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { * @param conf * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for * this instance. Use null for default. - * @param abortable Can be null if there is on error there is no host to abort: e.g. client - * context. + * @param abortable What to call when abort. * @param canCreateBaseZNode * @throws IOException * @throws ZooKeeperConnectionException @@ -161,6 +174,22 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { public ZooKeeperWatcher(Configuration conf, String identifier, Abortable abortable, boolean canCreateBaseZNode) throws IOException, ZooKeeperConnectionException { + this(conf, identifier, abortable, null, canCreateBaseZNode); + } + + /** + * Instantiate a ZooKeeper connection and watcher. + * @param conf + * @param identifier string that is passed to RecoverableZookeeper to be used as identifier for + * this instance. Use null for default. + * @param abortable What to call when abort. + * @param canCreateBaseZNode + * @throws IOException + * @throws ZooKeeperConnectionException + */ + public ZooKeeperWatcher(Configuration conf, String identifier, + Abortable abortable, Tracer tracer, boolean canCreateBaseZNode) + throws IOException, ZooKeeperConnectionException { this.conf = conf; // Capture a stack trace now. Will print it out later if problem so we can // distingush amongst the myriad ZKWs. @@ -176,7 +205,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { this.identifier = identifier + "0x0"; this.abortable = abortable; setNodeNames(conf); - this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, identifier); + this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, tracer, identifier); if (canCreateBaseZNode) { createBaseZNodes(); } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java index 2795472..2288537 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZKUtil.java @@ -44,7 +44,7 @@ public class TestZKUtil { Configuration conf = HBaseConfiguration.create(); conf.set(Superusers.SUPERUSER_CONF_KEY, "user1,@group1,user2,@group2,user3"); String node = "/hbase/testCreateACL"; - ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, false); + ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, node, null, null, false); List aclList = ZKUtil.createACL(watcher, node, true); Assert.assertEquals(aclList.size(), 4); // 3+1, since ACL will be set for the creator by default Assert.assertTrue(!aclList.contains(new ACL(Perms.ALL, new Id("auth", "@group1"))) diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java index 10a3816..f2dbdd7 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestZooKeeperWatcher.java @@ -34,7 +34,7 @@ public class TestZooKeeperWatcher { @Test public void testIsClientReadable() throws ZooKeeperConnectionException, IOException { ZooKeeperWatcher watcher = new ZooKeeperWatcher(HBaseConfiguration.create(), - "testIsClientReadable", null, false); + "testIsClientReadable", null, null, false); assertTrue(watcher.isClientReadable(watcher.baseZNode)); assertTrue(watcher.isClientReadable(watcher.getZNodeForReplica(0))); diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml index 5b43553..4b971f0 100644 --- a/hbase-common/pom.xml +++ b/hbase-common/pom.xml @@ -277,7 +277,7 @@ org.apache.htrace - htrace-core + htrace-core4 diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java index 56de264..0f09364 100644 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java +++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/HBaseHTraceConfiguration.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.trace; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.htrace.HTraceConfiguration; +import org.apache.htrace.core.HTraceConfiguration; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -65,17 +65,16 @@ public class HBaseHTraceConfiguration extends HTraceConfiguration { @Override public String get(String key) { - return conf.get(KEY_PREFIX +key); + return conf.get(KEY_PREFIX + key); } @Override public String get(String key, String defaultValue) { - return conf.get(KEY_PREFIX + key,defaultValue); - + return conf.get(KEY_PREFIX + key, defaultValue); } @Override public boolean getBoolean(String key, boolean defaultValue) { return conf.getBoolean(KEY_PREFIX + key, defaultValue); } -} +} \ No newline at end of file diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java deleted file mode 100644 index b90d191..0000000 --- a/hbase-common/src/main/java/org/apache/hadoop/hbase/trace/SpanReceiverHost.java +++ /dev/null @@ -1,110 +0,0 @@ -/** - * 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.hadoop.hbase.trace; - -import java.io.IOException; -import java.util.Collection; -import java.util.HashSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.htrace.SpanReceiver; -import org.apache.htrace.SpanReceiverBuilder; -import org.apache.htrace.Trace; - -/** - * This class provides functions for reading the names of SpanReceivers from - * hbase-site.xml, adding those SpanReceivers to the Tracer, and closing those - * SpanReceivers when appropriate. - */ -@InterfaceAudience.Private -public class SpanReceiverHost { - public static final String SPAN_RECEIVERS_CONF_KEY = "hbase.trace.spanreceiver.classes"; - private static final Log LOG = LogFactory.getLog(SpanReceiverHost.class); - private Collection receivers; - private Configuration conf; - private boolean closed = false; - - private static enum SingletonHolder { - INSTANCE; - Object lock = new Object(); - SpanReceiverHost host = null; - } - - public static SpanReceiverHost getInstance(Configuration conf) { - synchronized (SingletonHolder.INSTANCE.lock) { - if (SingletonHolder.INSTANCE.host != null) { - return SingletonHolder.INSTANCE.host; - } - - SpanReceiverHost host = new SpanReceiverHost(conf); - host.loadSpanReceivers(); - SingletonHolder.INSTANCE.host = host; - return SingletonHolder.INSTANCE.host; - } - - } - - SpanReceiverHost(Configuration conf) { - receivers = new HashSet(); - this.conf = conf; - } - - /** - * Reads the names of classes specified in the {@code hbase.trace.spanreceiver.classes} property - * and instantiates and registers them with the Tracer. - * - */ - public void loadSpanReceivers() { - String[] receiverNames = conf.getStrings(SPAN_RECEIVERS_CONF_KEY); - if (receiverNames == null || receiverNames.length == 0) { - return; - } - - SpanReceiverBuilder builder = new SpanReceiverBuilder(new HBaseHTraceConfiguration(conf)); - for (String className : receiverNames) { - className = className.trim(); - - SpanReceiver receiver = builder.spanReceiverClass(className).build(); - if (receiver != null) { - receivers.add(receiver); - LOG.info("SpanReceiver " + className + " was loaded successfully."); - } - } - for (SpanReceiver rcvr : receivers) { - Trace.addReceiver(rcvr); - } - } - - /** - * Calls close() on all SpanReceivers created by this SpanReceiverHost. - */ - public synchronized void closeReceivers() { - if (closed) return; - closed = true; - for (SpanReceiver rcvr : receivers) { - try { - rcvr.close(); - } catch (IOException e) { - LOG.warn("Unable to close SpanReceiver correctly: " + e.getMessage(), e); - } - } - } -} diff --git a/hbase-common/src/main/resources/hbase-default.xml b/hbase-common/src/main/resources/hbase-default.xml index e19cbf8..eea32d0 100644 --- a/hbase-common/src/main/resources/hbase-default.xml +++ b/hbase-common/src/main/resources/hbase-default.xml @@ -1770,4 +1770,22 @@ possible configurations would overwhelm and obscure the important. Timeout for regionservers to keep threads in snapshot request pool waiting + + hbase.htrace.span.receiver.classes + + + The class name of the HTrace SpanReceivers to use inside HBase. If + there are no class names supplied here, tracings will not be emitted. + + + + hbase.htrace.sampler.classes + org.apache.htrace.core.AlwaysSampler + Sampler to use when tracing. Default is + org.apache.htrace.core.NeverSampler. Other options are + org.apache.htrace.core.AlwaysSampler and + org.apache.htrace.core.ProbabilitySampler. See htrace-core + for options provided by htrace. + + diff --git a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java index 3cae4d2..a6d0f01 100644 --- a/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java +++ b/hbase-common/src/test/java/org/apache/hadoop/hbase/HBaseCommonTestingUtility.java @@ -32,7 +32,7 @@ import org.apache.hadoop.hbase.classification.InterfaceStability; /** * Common helpers for testing HBase that do not depend on specific server/etc. things. - * @see {@link HBaseTestingUtility} + * Like HBaseTestingUtility but with just common-module dependencies. * */ @InterfaceAudience.Public diff --git a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java index 69d8521..266b045 100644 --- a/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java +++ b/hbase-external-blockcache/src/main/java/org/apache/hadoop/hbase/io/hfile/MemcachedBlockCache.java @@ -33,9 +33,8 @@ import org.apache.hadoop.hbase.io.hfile.Cacheable.MemoryType; import org.apache.hadoop.hbase.nio.ByteBuff; import org.apache.hadoop.hbase.nio.SingleByteBuff; import org.apache.hadoop.hbase.util.Addressing; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; - +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import java.io.IOException; import java.net.InetSocketAddress; @@ -133,8 +132,10 @@ public class MemcachedBlockCache implements BlockCache { boolean repeat, boolean updateCacheMetrics) { // Assume that nothing is the block cache HFileBlock result = null; - - try (TraceScope traceScope = Trace.startSpan("MemcachedBlockCache.getBlock")) { + Tracer tracer = Tracer.curThreadTracer(); + TraceScope traceScope = tracer == null? null: + tracer.newScope("MemcachedBlockCache.getBlock"); + try { result = client.get(cacheKey.toString(), tc); } catch (Exception e) { // Catch a pretty broad set of exceptions to limit any changes in the memecache client @@ -146,6 +147,7 @@ public class MemcachedBlockCache implements BlockCache { } result = null; } finally { + traceScope.close(); // Update stats if this request doesn't have it turned off 100% of the time if (updateCacheMetrics) { if (result == null) { diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml index 70d13ad..c18733c 100644 --- a/hbase-it/pom.xml +++ b/hbase-it/pom.xml @@ -265,7 +265,7 @@ org.apache.htrace - htrace-core + htrace-core4 diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java index 437f200..9af7aae 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/mttr/IntegrationTestMTTR.java @@ -63,9 +63,10 @@ import org.apache.hadoop.hbase.regionserver.NoSuchColumnFamilyException; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.LoadTestTool; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.Span; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import org.apache.htrace.impl.AlwaysSampler; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -158,6 +159,9 @@ public class IntegrationTestMTTR { @BeforeClass public static void setUp() throws Exception { + // TODO: Tracing had the ALWAYS Sampler turned on. The refactor to come up on htrace 4.0 did + // not add in this. Fix. + // Set up the integration test util if (util == null) { util = new IntegrationTestingUtility(); @@ -357,7 +361,7 @@ public class IntegrationTestMTTR { */ private static class TimingResult { DescriptiveStatistics stats = new DescriptiveStatistics(); - ArrayList traces = new ArrayList(10); + ArrayList traces = new ArrayList(10); /** * Add a result to this aggregate result. @@ -367,7 +371,7 @@ public class IntegrationTestMTTR { public void addResult(long time, Span span) { stats.addValue(TimeUnit.MILLISECONDS.convert(time, TimeUnit.NANOSECONDS)); if (TimeUnit.SECONDS.convert(time, TimeUnit.NANOSECONDS) >= 1) { - traces.add(span.getTraceId()); + traces.add(span.getSpanId()); } } @@ -412,7 +416,8 @@ public class IntegrationTestMTTR { long start = System.nanoTime(); TraceScope scope = null; try { - scope = Trace.startSpan(getSpanName(), AlwaysSampler.INSTANCE); + scope = Tracer.curThreadTracer() == null? null: + Tracer.curThreadTracer().newScope(getSpanName()); boolean actionResult = doAction(); if (actionResult && future.isDone()) { numAfterDone++; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java index 3845846..80431a2 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/test/IntegrationTestZKAndFSPermissions.java @@ -138,7 +138,7 @@ public class IntegrationTestZKAndFSPermissions extends AbstractHBaseTool { private void testZNodeACLs() throws IOException, KeeperException, InterruptedException { ZooKeeperWatcher watcher = new ZooKeeperWatcher(conf, "IntegrationTestZnodeACLs", null); - RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher); + RecoverableZooKeeper zk = ZKUtil.connect(this.conf, watcher, null); String baseZNode = watcher.baseZNode; diff --git a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java index f325aac..a93cf11 100644 --- a/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java +++ b/hbase-it/src/test/java/org/apache/hadoop/hbase/trace/IntegrationTestSendTraceRequests.java @@ -35,9 +35,8 @@ import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.util.AbstractHBaseTool; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.util.ToolRunner; -import org.apache.htrace.Sampler; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -61,9 +60,10 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { private IntegrationTestingUtility util; private Random random = new Random(); private Admin admin; - private SpanReceiverHost receiverHost; public static void main(String[] args) throws Exception { + // TODO: Tracing had the ALWAYS Sampler turned on. The refactor to come up on htrace 4.0 did + // not add in this. Fix. Configuration configuration = HBaseConfiguration.create(); IntegrationTestingUtility.setUseDistributedCluster(configuration); IntegrationTestSendTraceRequests tool = new IntegrationTestSendTraceRequests(); @@ -108,13 +108,11 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { service.shutdown(); service.awaitTermination(100, TimeUnit.SECONDS); Thread.sleep(90000); - receiverHost.closeReceivers(); util.restoreCluster(); util = null; } private void doScans(ExecutorService service, final LinkedBlockingQueue rks) { - for (int i = 0; i < 100; i++) { Runnable runnable = new Runnable() { private TraceScope innerScope = null; @@ -123,7 +121,8 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { public void run() { ResultScanner rs = null; try { - innerScope = Trace.startSpan("Scan", Sampler.ALWAYS); + Tracer tracer = Tracer.curThreadTracer(); + innerScope = tracer == null? null: tracer.newScope("Scan"); Table ht = util.getConnection().getTable(tableName); Scan s = new Scan(); s.setStartRow(Bytes.toBytes(rowKeyQueue.take())); @@ -144,9 +143,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { } catch (IOException e) { e.printStackTrace(); - innerScope.getSpan().addKVAnnotation( - Bytes.toBytes("exception"), - Bytes.toBytes(e.getClass().getSimpleName())); + innerScope.getSpan().addKVAnnotation("exception", e.getClass().getSimpleName()); } catch (Exception e) { } finally { @@ -182,7 +179,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { long accum = 0; for (int x = 0; x < 5; x++) { try { - innerScope = Trace.startSpan("gets", Sampler.ALWAYS); + innerScope = Tracer.curThreadTracer().newScope("gets"); long rk = rowKeyQueue.take(); Result r1 = ht.get(new Get(Bytes.toBytes(rk))); if (r1 != null) { @@ -212,7 +209,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { private void createTable() throws IOException { TraceScope createScope = null; try { - createScope = Trace.startSpan("createTable", Sampler.ALWAYS); + createScope = Tracer.curThreadTracer().newScope("createTable"); util.createTable(tableName, familyName); } finally { if (createScope != null) createScope.close(); @@ -224,7 +221,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { try { if (admin.tableExists(tableName)) { - deleteScope = Trace.startSpan("deleteTable", Sampler.ALWAYS); + deleteScope = Tracer.curThreadTracer().newScope("deleteTable"); util.deleteTable(tableName); } } finally { @@ -237,7 +234,8 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { BufferedMutator ht = util.getConnection().getBufferedMutator(this.tableName); byte[] value = new byte[300]; for (int x = 0; x < 5000; x++) { - TraceScope traceScope = Trace.startSpan("insertData", Sampler.ALWAYS); + Tracer tracer = Tracer.curThreadTracer(); + TraceScope traceScope = tracer == null? null: tracer.newScope("insertData"); try { for (int i = 0; i < 5; i++) { long rk = random.nextLong(); @@ -253,7 +251,7 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { admin.flush(tableName); } } finally { - traceScope.close(); + if (traceScope != null) traceScope.close(); } } admin.flush(tableName); @@ -280,6 +278,8 @@ public class IntegrationTestSendTraceRequests extends AbstractHBaseTool { Configuration conf = new Configuration(util.getConfiguration()); conf.setBoolean("hbase.zipkin.is-in-client-mode", true); - this.receiverHost = SpanReceiverHost.getInstance(conf); + /** this.receiverHost = SpanReceiverHost.getInstance(conf); + * TODO: FIX!! What does this mean in htrace 4.0? + */ } } diff --git a/hbase-protocol/src/main/protobuf/Tracing.proto b/hbase-protocol/src/main/protobuf/Tracing.proto index 5a64cfc..b129504 100644 --- a/hbase-protocol/src/main/protobuf/Tracing.proto +++ b/hbase-protocol/src/main/protobuf/Tracing.proto @@ -22,12 +22,15 @@ option java_outer_classname = "TracingProtos"; option java_generate_equals_and_hash = true; option optimize_for = SPEED; -//Used to pass through the information necessary to continue -//a trace after an RPC is made. All we need is the traceid -//(so we know the overarching trace this message is a part of), and -//the id of the current span when this message was sent, so we know -//what span caused the new span we will create when this message is received. +// Used to pass through the information necessary to continue +// a trace after an RPC is made. All we need is the traceid +// (so we know the overarching trace this message is a part of). +// Pre-4.0.0 htrace, we had parent and trace id. In 4.0.0, there is one id only and it is +// 128 bits. Rather than add new fields or change field names, just set the high and low of +// the 128 integer into the old parent id and trace id fields. message RPCTInfo { + // In 4.0.0 htrace, trace_id holds the high part of the 128 traceid optional int64 trace_id = 1; + // In 4.0.0 htrace, trace_id holds the low part of the 128 traceid optional int64 parent_id = 2; } diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml index aff49b7..53c6539 100644 --- a/hbase-rsgroup/pom.xml +++ b/hbase-rsgroup/pom.xml @@ -183,7 +183,7 @@ org.apache.htrace - htrace-core + htrace-core4 org.codehaus.jackson diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml index 2631417..d226fe9 100644 --- a/hbase-server/pom.xml +++ b/hbase-server/pom.xml @@ -539,7 +539,7 @@ org.apache.htrace - htrace-core + htrace-core4 com.lmax diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java index 82dc934..0ac18c2 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/LocalHBaseCluster.java @@ -113,7 +113,8 @@ public class LocalHBaseCluster { } @SuppressWarnings("unchecked") - private static Class getRegionServerImplementation(final Configuration conf) { + private static Class + getRegionServerImplementation(final Configuration conf) { return (Class)conf.getClass(HConstants.REGION_SERVER_IMPL, HRegionServer.class); } @@ -420,7 +421,8 @@ public class LocalHBaseCluster { * @return True if a 'local' address in hbase.master value. */ public static boolean isLocal(final Configuration c) { - boolean mode = c.getBoolean(HConstants.CLUSTER_DISTRIBUTED, HConstants.DEFAULT_CLUSTER_DISTRIBUTED); + boolean mode = c.getBoolean(HConstants.CLUSTER_DISTRIBUTED, + HConstants.DEFAULT_CLUSTER_DISTRIBUTED); return(mode == HConstants.CLUSTER_IS_LOCAL); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java index ba4bee9..9136e06 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/executor/EventHandler.java @@ -25,9 +25,9 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.Server; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; /** * Abstract base class for all HBase event handlers. Subclasses should @@ -68,13 +68,13 @@ public abstract class EventHandler implements Runnable, Comparable { // Time to wait for events to happen, should be kept short protected int waitingTimeForEvents; - private final Span parent; + private final SpanId parentSpanId; /** * Default base class constructor. */ public EventHandler(Server server, EventType eventType) { - this.parent = Trace.currentSpan(); + this.parentSpanId = Tracer.getCurrentSpanId(); this.server = server; this.eventType = eventType; seqid = seqids.incrementAndGet(); @@ -99,13 +99,17 @@ public abstract class EventHandler implements Runnable, Comparable { @Override public void run() { - TraceScope chunk = Trace.startSpan(this.getClass().getSimpleName(), parent); + Tracer tracer = Tracer.curThreadTracer(); + TraceScope traceScope = tracer == null? null: + this.parentSpanId.isValid()? + tracer.newScope(this.getClass().getSimpleName(), this.parentSpanId): + tracer.newScope(this.getClass().getSimpleName()); try { process(); } catch(Throwable t) { handleException(t); } finally { - chunk.close(); + if (traceScope != null) traceScope.close(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index fc1c04e..9797f02 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -57,8 +57,8 @@ import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.IdLock; import org.apache.hadoop.hbase.util.ObjectIntPair; import org.apache.hadoop.io.WritableUtils; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import com.google.common.annotations.VisibleForTesting; @@ -864,7 +864,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { return false; } - // The first key in the current block 'seekToBlock' is greater than the given + // The first key in the current block 'seekToBlock' is greater than the given // seekBefore key. We will go ahead by reading the next block that satisfies the // given key. Return the current block before reading the next one. reader.returnBlock(seekToBlock); @@ -978,7 +978,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { + keyPair.getSecond(), currKeyLen); } else { // Better to do a copy here instead of holding on to this BB so that - // we could release the blocks referring to this key. This key is specifically used + // we could release the blocks referring to this key. This key is specifically used // in HalfStoreFileReader to get the firstkey and lastkey by creating a new scanner // every time. So holding onto the BB (incase of DBB) is not advised here. byte[] key = new byte[currKeyLen]; @@ -1419,7 +1419,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { boolean useLock = false; IdLock.Entry lockEntry = null; - TraceScope traceScope = Trace.startSpan("HFileReaderImpl.readBlock"); + Tracer tracer = Tracer.curThreadTracer(); + TraceScope traceScope = tracer == null? null: tracer.newScope("HFileReaderImpl.readBlock"); try { while (true) { // Check cache for block. If found return. @@ -1435,7 +1436,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { if (LOG.isTraceEnabled()) { LOG.trace("From Cache " + cachedBlock); } - if (Trace.isTracing()) { + if (traceScope != null) { traceScope.getSpan().addTimelineAnnotation("blockCacheHit"); } assert cachedBlock.isUnpacked() : "Packed block leak."; @@ -1463,7 +1464,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { // Carry on, please load. } - if (Trace.isTracing()) { + if (traceScope != null) { traceScope.getSpan().addTimelineAnnotation("blockCacheMiss"); } // Load block from filesystem. @@ -1487,7 +1488,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { return unpacked; } } finally { - traceScope.close(); + if (traceScope != null) traceScope.close(); if (lockEntry != null) { offsetLock.releaseLockEntry(lockEntry); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java index 3514245..ed205d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/CallRunner.java @@ -31,8 +31,9 @@ import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.Span; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import com.google.protobuf.Message; @@ -109,8 +110,11 @@ public class CallRunner { throw new ServerNotRunningYetException("Server " + (address != null ? address : "(channel closed)") + " is not running yet"); } - if (call.tinfo != null) { - traceScope = Trace.startSpan(call.toTraceString(), call.tinfo); + Span span = Tracer.getCurrentSpan(); + if (span != null) { + Tracer tracer = Tracer.curThreadTracer(); + traceScope = tracer == null? null: + tracer.newScope(call.toTraceString(), span.getSpanId()); } // make the call resultPair = this.rpcServer.call(call.service, call.md, call.param, call.cellScanner, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java index b9a9b26..ebf9943 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java @@ -96,6 +96,7 @@ import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ExceptionResponse; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.RequestHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.ResponseHeader; import org.apache.hadoop.hbase.protobuf.generated.RPCProtos.UserInformation; +import org.apache.hadoop.hbase.protobuf.generated.TracingProtos.RPCTInfo; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.security.AccessDeniedException; import org.apache.hadoop.hbase.security.AuthMethod; @@ -127,7 +128,6 @@ import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.util.StringUtils; -import org.apache.htrace.TraceInfo; import org.codehaus.jackson.map.ObjectMapper; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -310,7 +310,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { protected long size; // size of current call protected boolean isError; - protected TraceInfo tinfo; + protected RPCTInfo tinfo; private ByteBuffer cellBlock = null; private User user; @@ -325,7 +325,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { justification="Can't figure why this complaint is happening... see below") Call(int id, final BlockingService service, final MethodDescriptor md, RequestHeader header, Message param, CellScanner cellScanner, Connection connection, Responder responder, - long size, TraceInfo tinfo, final InetAddress remoteAddress) { + long size, RPCTInfo tinfo, final InetAddress remoteAddress) { this.id = id; this.service = service; this.md = md; @@ -1931,8 +1931,9 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver { return; } - TraceInfo traceInfo = header.hasTraceInfo() - ? new TraceInfo(header.getTraceInfo().getTraceId(), header.getTraceInfo().getParentId()) + RPCTInfo traceInfo = header.hasTraceInfo() + ? RPCTInfo.newBuilder().setTraceId(header.getTraceInfo().getTraceId()). + setParentId(header.getTraceInfo().getParentId()).build() : null; Call call = new Call(id, this.service, md, header, param, cellScanner, this, responder, totalRequestSize, traceInfo, this.addr); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java index 37e4e44..3dc87ce 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mapreduce/TableMapReduceUtil.java @@ -800,7 +800,7 @@ public class TableMapReduceUtil { io.netty.channel.Channel.class, com.google.protobuf.Message.class, com.google.common.collect.Lists.class, - org.apache.htrace.Trace.class, + org.apache.htrace.core.Tracer.class, com.codahale.metrics.MetricRegistry.class); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 2f1cd3c..8abc41d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -168,6 +168,7 @@ import org.apache.hadoop.hbase.zookeeper.SplitOrMergeTracker; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.htrace.core.Tracer; import org.apache.zookeeper.KeeperException; import org.mortbay.jetty.Connector; import org.mortbay.jetty.nio.SelectChannelConnector; @@ -268,6 +269,7 @@ public class HMaster extends HRegionServer implements MasterServices { // Metrics for the HMaster final MetricsMaster metricsMaster; + // file system manager for the master FS operations private MasterFileSystem fileSystemManager; @@ -431,7 +433,7 @@ public class HMaster extends HRegionServer implements MasterServices { } } - // Some unit tests don't need a cluster, so no zookeeper at all + // Some unit tests don't need a cluster, so no zookeeper at all in this case. if (!conf.getBoolean("hbase.testing.nocluster", false)) { activeMasterManager = new ActiveMasterManager(zooKeeper, this.serverName, this); int infoPort = putUpJettyServer(); @@ -861,8 +863,7 @@ public class HMaster extends HRegionServer implements MasterServices { /** * Create a {@link ServerManager} instance. */ - ServerManager createServerManager(final Server master, - final MasterServices services) + ServerManager createServerManager(final Server master, final MasterServices services) throws IOException { // We put this out here in a method so can do a Mockito.spy and stub it out // w/ a mocked up ServerManager. @@ -1729,22 +1730,31 @@ public class HMaster extends HRegionServer implements MasterServices { } activeMasterManager.setInfoPort(infoPort); - // Start a thread to try to become the active master, so we won't block here - Threads.setDaemonThreadRunning(new Thread(new Runnable() { + // Start a thread to try to become the active master, so we won't block here. Wrap it if + // tracing so we pass off the tracer context. + Tracer tracer = Tracer.curThreadTracer(); + Threads.setDaemonThreadRunning( + new Thread(tracer.wrap(createMasterStartupRunnable(), "ActiveMasterManager")), + getServerName().toShortString() + ".activeMasterManager"); + } + + /** + * Thread to run the Master startup. + */ + private Runnable createMasterStartupRunnable() { + return new Runnable() { @Override public void run() { int timeout = conf.getInt(HConstants.ZK_SESSION_TIMEOUT, - HConstants.DEFAULT_ZK_SESSION_TIMEOUT); + HConstants.DEFAULT_ZK_SESSION_TIMEOUT); // If we're a backup master, stall until a primary to writes his address - if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, - HConstants.DEFAULT_MASTER_TYPE_BACKUP)) { - LOG.debug("HMaster started in backup mode. " - + "Stalling until master znode is written."); + if (conf.getBoolean(HConstants.MASTER_TYPE_BACKUP, HConstants.DEFAULT_MASTER_TYPE_BACKUP)) { + LOG.debug("HMaster started in backup mode. Stalling until master znode is written."); // This will only be a minute or so while the cluster starts up, // so don't worry about setting watches on the parent znode while (!activeMasterManager.hasActiveMaster()) { LOG.debug("Waiting for master address ZNode to be written " - + "(Also watching cluster state node)"); + + "(Also watching cluster state node)"); Threads.sleep(timeout); } } @@ -1759,13 +1769,13 @@ public class HMaster extends HRegionServer implements MasterServices { LOG.fatal("Failed to become active master", t); // HBASE-5680: Likely hadoop23 vs hadoop 20.x/1.x incompatibility if (t instanceof NoClassDefFoundError && - t.getMessage() + t.getMessage() .contains("org/apache/hadoop/hdfs/protocol/HdfsConstants$SafeModeAction")) { // improved error message for this special case abort("HBase is having a problem with its Hadoop jars. You may need to " - + "recompile HBase against Hadoop version " - + org.apache.hadoop.util.VersionInfo.getVersion() - + " or change your hadoop jars to start properly", t); + + "recompile HBase against Hadoop version " + + org.apache.hadoop.util.VersionInfo.getVersion() + + " or change your hadoop jars to start properly", t); } else { abort("Unhandled exception. Starting shutdown.", t); } @@ -1773,7 +1783,7 @@ public class HMaster extends HRegionServer implements MasterServices { status.cleanup(); } } - }, getServerName().toShortString() + ".activeMasterManager")); + }; } private void checkCompression(final HTableDescriptor htd) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java index d69c7aa..c72fb95 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java @@ -62,9 +62,9 @@ import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.AdminService; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.OpenRegionResponse; import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.ServerInfo; -import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionStoreSequenceIds; import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId; +import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest; import org.apache.hadoop.hbase.protobuf.generated.ZooKeeperProtos.SplitLogTask.RecoveryMode; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.RegionOpeningState; diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java index 1cc8d48..9bdb967 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/LogCleaner.java @@ -53,4 +53,4 @@ public class LogCleaner extends CleanerChore { protected boolean validate(Path file) { return DefaultWALProvider.validateWALFilename(file.getName()); } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java index d34f25e..e105689 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java @@ -43,7 +43,7 @@ import org.apache.hadoop.hbase.master.RegionState; import org.apache.hadoop.hbase.master.RegionStates; import org.apache.hadoop.hbase.master.TableLockManager; import org.apache.hadoop.hbase.master.TableLockManager.TableLock; -import org.apache.htrace.Trace; +import org.apache.htrace.core.Tracer; /** * Handler to run disable of a table. @@ -205,11 +205,15 @@ public class DisableTableHandler extends EventHandler { continue; } final HRegionInfo hri = region; - pool.execute(Trace.wrap("DisableTableHandler.BulkDisabler",new Runnable() { + Tracer tracer = Tracer.curThreadTracer(); + Runnable runnable = new Runnable() { public void run() { assignmentManager.unassign(hri); } - })); + }; + pool.execute(tracer != null? + tracer.wrap(runnable, "DisableTableHandler.BulkDisabler"): + runnable); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index 005069d..51b4e05 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -48,7 +48,7 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterProcedureProtos.DisableTableState; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.htrace.Trace; +import org.apache.htrace.core.Tracer; @InterfaceAudience.Private public class DisableTableProcedure @@ -506,12 +506,15 @@ public class DisableTableProcedure && !regionStates.isRegionInState(region, RegionState.State.FAILED_CLOSE)) { continue; } - pool.execute(Trace.wrap("DisableTableHandler.BulkDisabler", new Runnable() { + Tracer tracer = Tracer.curThreadTracer(); + Runnable runnable = new Runnable() { @Override public void run() { assignmentManager.unassign(region); } - })); + }; + pool.execute(tracer == null? runnable: + tracer.wrap(runnable, "DisableTableHandler.BulkDisabler")); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index 5676e15..d54b240 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -178,8 +178,8 @@ import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay; import org.apache.hadoop.io.MultipleIOException; import org.apache.hadoop.util.StringUtils; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; @@ -5121,8 +5121,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi TraceScope traceScope = null; // If we're tracing start a span to show how long this took. - if (Trace.isTracing()) { - traceScope = Trace.startSpan("HRegion.getRowLock"); + Tracer tracer = Tracer.curThreadTracer(); + if (tracer != null) { + traceScope = tracer.newScope("HRegion.getRowLock"); traceScope.getSpan().addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock")); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 06a2aff..6764a92 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -147,7 +147,7 @@ import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.replication.regionserver.ReplicationLoad; import org.apache.hadoop.hbase.security.Superusers; import org.apache.hadoop.hbase.security.UserProvider; -import org.apache.hadoop.hbase.trace.SpanReceiverHost; +import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; import org.apache.hadoop.hbase.util.Addressing; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; @@ -178,6 +178,8 @@ import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.metrics.util.MBeanUtil; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.StringUtils; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import org.apache.zookeeper.KeeperException; import org.apache.zookeeper.KeeperException.NoNodeException; import org.apache.zookeeper.data.Stat; @@ -201,7 +203,7 @@ import sun.misc.SignalHandler; * the HMaster. There are many HRegionServers in a single HBase deployment. */ @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) -@SuppressWarnings("deprecation") +@SuppressWarnings({ "deprecation", "restriction" }) public class HRegionServer extends HasThread implements RegionServerServices, LastSequenceId, ConfigurationObserver { @@ -347,7 +349,8 @@ public class HRegionServer extends HasThread implements MetricsRegionServer metricsRegionServer; MetricsTable metricsTable; - private SpanReceiverHost spanReceiverHost; + + private final Tracer tracer; /** * ChoreService used to schedule tasks that we want to run periodically @@ -515,6 +518,15 @@ public class HRegionServer extends HasThread implements throws IOException { this.fsOk = true; this.conf = conf; + + // Set up tracer. Add ServerName to the TraceId so it comes out as "RegionServer,host,port,ts". + // Start up a scope immediately. This sets the current tracer into the the current thread so + // can be picked up down the stack during various initializations. + LOG.info("Setting up tracer " + conf.get("hbase.htrace.htraced.receiver.address")); + this.tracer = new Tracer.Builder("Server").conf(new HBaseHTraceConfiguration(conf)).build(); + this.tracer.newScope("Server"); + if (LOG.isDebugEnabled()) LOG.debug("Tracer created " + this.tracer); + HFile.checkHFileVersion(this.conf); checkCodecs(this.conf); this.userProvider = UserProvider.instantiate(conf); @@ -592,13 +604,12 @@ public class HRegionServer extends HasThread implements this.tableDescriptors = getFsTableDescriptors(); service = new ExecutorService(getServerName().toShortString()); - spanReceiverHost = SpanReceiverHost.getInstance(getConfiguration()); // Some unit tests don't need a cluster, so no zookeeper at all if (!conf.getBoolean("hbase.testing.nocluster", false)) { // Open connection to zookeeper and set primary watcher zooKeeper = new ZooKeeperWatcher(conf, getProcessName() + ":" + - rpcServices.isa.getPort(), this, canCreateBaseZNode()); + rpcServices.isa.getPort(), this, this.tracer, canCreateBaseZNode()); this.csm = (BaseCoordinatedStateManager) csm; this.csm.initialize(this); @@ -762,7 +773,7 @@ public class HRegionServer extends HasThread implements * @throws IOException * @throws InterruptedException */ - private void preRegistrationInitialization(){ + private void preRegistrationInitialization() { try { setupClusterConnection(); @@ -917,7 +928,7 @@ public class HRegionServer extends HasThread implements */ @Override public void run() { - try { + try (TraceScope traceScope = this.tracer.newScope("Setup Zookeeper")) { // Do pre-registration initializations; zookeeper, lease threads, etc. preRegistrationInitialization(); } catch (Throwable e) { @@ -1135,7 +1146,7 @@ public class HRegionServer extends HasThread implements } LOG.info("stopping server " + this.serverName + "; zookeeper connection closed."); - + this.tracer.close(); LOG.info(Thread.currentThread().getName() + " exiting"); } @@ -2198,10 +2209,6 @@ public class HRegionServer extends HasThread implements if (this.cacheFlusher != null) { this.cacheFlusher.join(); } - - if (this.spanReceiverHost != null) { - this.spanReceiverHost.closeReceivers(); - } if (this.walRoller != null) { Threads.shutdown(this.walRoller.getThread()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index a69d8c0..ef312ba 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -57,8 +57,8 @@ import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import com.google.common.base.Preconditions; @@ -566,11 +566,11 @@ class MemStoreFlusher implements FlushRequester { * amount of memstore consumption. */ public void reclaimMemStoreMemory() { - TraceScope scope = Trace.startSpan("MemStoreFluser.reclaimMemStoreMemory"); + Tracer tracer = Tracer.curThreadTracer(); + TraceScope scope = tracer == null? null: + tracer.newScope("MemStoreFluser.reclaimMemStoreMemory"); if (isAboveHighWaterMark()) { - if (Trace.isTracing()) { - scope.getSpan().addTimelineAnnotation("Force Flush. We're above high water mark."); - } + if (scope != null) scope.addTimelineAnnotation("Force Flush. We're above high water mark."); long start = EnvironmentEdgeManager.currentTime(); synchronized (this.blockSignal) { boolean blocked = false; @@ -617,7 +617,7 @@ class MemStoreFlusher implements FlushRequester { } else if (isAboveLowWaterMark()) { wakeupFlushThread(); } - scope.close(); + if (scope != null) scope.close(); } @Override public String toString() { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 9ef161a..72f6c1a 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -41,8 +41,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; -import com.google.common.annotations.VisibleForTesting; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -66,9 +64,11 @@ import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.util.StringUtils; import org.apache.htrace.NullScope; -import org.apache.htrace.Span; import org.apache.htrace.Trace; import org.apache.htrace.TraceScope; +import org.apache.htrace.core.Tracer; + +import com.google.common.annotations.VisibleForTesting; /** * Implementation of {@link WAL} to go against {@link FileSystem}; i.e. keep WALs in HDFS. Only one @@ -98,12 +98,16 @@ import org.apache.htrace.TraceScope; */ @InterfaceAudience.Private public abstract class AbstractFSWAL implements WAL { - private static final Log LOG = LogFactory.getLog(AbstractFSWAL.class); protected static final int DEFAULT_SLOW_SYNC_TIME_MS = 100; // in ms /** + * Tracer to use. Can be null. + */ + protected final Tracer tracer; + + /** * file system instance */ protected final FileSystem fs; @@ -270,6 +274,7 @@ public abstract class AbstractFSWAL implements WAL { final boolean failIfWALExists, final String prefix, final String suffix) throws FailedLogCloseException, IOException { this.fs = fs; + this.tracer = Tracer.curThreadTracer(); this.walDir = new Path(rootDir, logDir); this.walArchiveDir = new Path(rootDir, archiveDir); this.conf = conf; @@ -631,11 +636,11 @@ public abstract class AbstractFSWAL implements WAL { } } - protected Span blockOnSync(final SyncFuture syncFuture) throws IOException { + protected SyncFuture blockOnSync(final SyncFuture syncFuture) throws IOException { // Now we have published the ringbuffer, halt the current thread until we get an answer back. try { syncFuture.get(); - return syncFuture.getSpan(); + return syncFuture; } catch (InterruptedException ie) { LOG.warn("Interrupted", ie); throw convertInterruptedExceptionToIOException(ie); @@ -772,13 +777,13 @@ public abstract class AbstractFSWAL implements WAL { LOG.info("Closed WAL: " + toString()); } - protected SyncFuture getSyncFuture(final long sequence, Span span) { + protected SyncFuture getSyncFuture(final long txid) { SyncFuture syncFuture = this.syncFuturesByHandler.get(Thread.currentThread()); if (syncFuture == null) { - syncFuture = new SyncFuture(sequence, span); + syncFuture = new SyncFuture(txid, Tracer.getCurrentSpanId()); this.syncFuturesByHandler.put(Thread.currentThread(), syncFuture); } else { - syncFuture.reset(sequence, span); + syncFuture.reset(txid, Tracer.getCurrentSpanId()); } return syncFuture; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java index d5bccf0..04778e8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AsyncFSWAL.java @@ -56,9 +56,7 @@ import org.apache.hadoop.hbase.wal.WALKey; import org.apache.hadoop.hbase.wal.WALProvider.AsyncWriter; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.ipc.RemoteException; -import org.apache.htrace.NullScope; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.TraceScope; /** * An asynchronous implementation of FSWAL. @@ -74,7 +72,7 @@ import org.apache.htrace.TraceScope; *
    *
  1. Lock 'waitingConsumePayloads', bump nextTxid, and insert the entry to * 'waitingConsumePayloads'.
  2. - *
  3. Schedule the consumer task if needed. See {@link #shouldScheduleConsumer()} for more details. + *
  4. Schedule the consumer task if needed. See #shouldScheduleConsumer() for more details. *
  5. *
* @@ -83,7 +81,7 @@ import org.apache.htrace.TraceScope; *
  • Poll the entry from 'waitingConsumePayloads' and insert it into 'waitingAppendEntries'
  • *
  • Poll the entry from 'waitingAppendEntries', append it to the AsyncWriter, and insert it into * 'unackedEntries'
  • - *
  • If the buffered size reaches {@link #batchSize}, or there is a sync request, then we call + *
  • If the buffered size reaches #batchSize, or there is a sync request, then we call * sync on the AsyncWriter.
  • *
  • In the callback methods(CompletionHandler): *
      @@ -105,7 +103,7 @@ import org.apache.htrace.TraceScope; * For a normal roll request(for example, we have reached the log roll size): *
        *
      1. In the log roller thread, we add a roll payload to 'waitingConsumePayloads', and then wait on - * the rollPromise(see {@link #waitForSafePoint()}).
      2. + * the rollPromise(see #waitForSafePoint()). *
      3. In the consumer thread, we will stop polling entries from 'waitingConsumePayloads' if we hit * a Payload which contains a roll request.
      4. *
      5. Append all entries to current writer, issue a sync request if possible.
      6. @@ -389,9 +387,10 @@ public class AsyncFSWAL extends AbstractFSWAL { } private void addTimeAnnotation(SyncFuture future, String annotation) { - TraceScope scope = Trace.continueSpan(future.getSpan()); - Trace.addTimelineAnnotation(annotation); - future.setSpan(scope.detach()); + try (TraceScope traceScope = tracer == null? + null: tracer.newScope("annotation", future.getSpanId());) { + if (traceScope != null) traceScope.addTimelineAnnotation(annotation); + } } private int finishSync(boolean addSyncTrace) { @@ -551,22 +550,18 @@ public class AsyncFSWAL extends AbstractFSWAL { @Override public void sync() throws IOException { - TraceScope scope = Trace.startSpan("AsyncFSWAL.sync"); - try { + try (TraceScope scope = tracer.newScope("AsyncFSWAL.sync")) { SyncFuture future; boolean scheduleTask; synchronized (waitingConsumePayloads) { scheduleTask = shouldScheduleConsumer(); - future = getSyncFuture(nextTxid - 1, scope.detach()); + future = getSyncFuture(nextTxid - 1); waitingConsumePayloads.addLast(new Payload(future)); } if (scheduleTask) { eventLoop.execute(consumer); } - scope = Trace.continueSpan(blockOnSync(future)); - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); + blockOnSync(future); } } @@ -575,9 +570,8 @@ public class AsyncFSWAL extends AbstractFSWAL { if (highestSyncedTxid.get() >= txid) { return; } - TraceScope scope = Trace.startSpan("AsyncFSWAL.sync"); - try { - SyncFuture future = getSyncFuture(txid, scope.detach()); + try (TraceScope traceScope = tracer.newScope("AsyncFSWAL.sync")) { + SyncFuture future = getSyncFuture(txid); boolean scheduleTask; synchronized (waitingConsumePayloads) { scheduleTask = shouldScheduleConsumer(); @@ -586,10 +580,7 @@ public class AsyncFSWAL extends AbstractFSWAL { if (scheduleTask) { eventLoop.execute(consumer); } - scope = Trace.continueSpan(blockOnSync(future)); - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); + blockOnSync(future); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java index 67c2b93..0976179 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.io.FileNotFoundException; import java.io.IOException; import java.io.OutputStream; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.BlockingQueue; @@ -30,15 +31,6 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; -import com.google.common.annotations.VisibleForTesting; -import com.lmax.disruptor.BlockingWaitStrategy; -import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.ExceptionHandler; -import com.lmax.disruptor.LifecycleAware; -import com.lmax.disruptor.TimeoutException; -import com.lmax.disruptor.dsl.Disruptor; -import com.lmax.disruptor.dsl.ProducerType; - import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -63,10 +55,19 @@ import org.apache.hadoop.hbase.wal.WALSplitter; import org.apache.hadoop.hdfs.DFSOutputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; -import org.apache.htrace.NullScope; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; +import org.apache.htrace.core.Span; +import org.apache.htrace.core.SpanId; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; + +import com.google.common.annotations.VisibleForTesting; +import com.lmax.disruptor.BlockingWaitStrategy; +import com.lmax.disruptor.EventHandler; +import com.lmax.disruptor.ExceptionHandler; +import com.lmax.disruptor.LifecycleAware; +import com.lmax.disruptor.TimeoutException; +import com.lmax.disruptor.dsl.Disruptor; +import com.lmax.disruptor.dsl.ProducerType; /** * The default implementation of FSWAL. @@ -271,10 +272,11 @@ public class FSHLog extends AbstractFSWAL { /** * Run a sync after opening to set up the pipeline. */ - private void preemptiveSync(final ProtobufLogWriter nextWriter) { + private void preemptiveSync(final ProtobufLogWriter nextWriter, final TraceScope scope) { long startTimeNanos = System.nanoTime(); try { nextWriter.sync(); + if (scope != null) scope.addTimelineAnnotation("preemptive"); postSync(System.nanoTime() - startTimeNanos, 0); } catch (IOException e) { // optimization failed, no need to abort here. @@ -290,7 +292,10 @@ public class FSHLog extends AbstractFSWAL { protected Writer createWriterInstance(final Path path) throws IOException { Writer writer = DefaultWALProvider.createWriter(conf, fs, path, false); if (writer instanceof ProtobufLogWriter) { - preemptiveSync((ProtobufLogWriter) writer); + try (TraceScope traceScope = this.tracer == null? + null: this.tracer.newScope("FSHLog.preemptive.sync");) { + preemptiveSync((ProtobufLogWriter) writer, traceScope); + } } return writer; } @@ -327,7 +332,8 @@ public class FSHLog extends AbstractFSWAL { : this.ringBufferEventHandler.attainSafePoint(); afterCreatingZigZagLatch(); long oldFileLen = 0L; - try { + try (TraceScope traceScope = this.tracer == null? + null: this.tracer.newScope("FSHFile.replaceWriter");) { // Wait on the safe point to be achieved. Send in a sync in case nothing has hit the // ring buffer between the above notification of writer that we want it to go to // 'safe point' and then here where we are waiting on it to attain safe point. Use @@ -335,7 +341,7 @@ public class FSHLog extends AbstractFSWAL { // to come back. Cleanup this syncFuture down below after we are ready to run again. try { if (zigzagLatch != null) { - Trace.addTimelineAnnotation("awaiting safepoint"); + if (traceScope != null) traceScope.addTimelineAnnotation("awaiting safepoint"); syncFuture = zigzagLatch.waitSafePoint(publishSyncOnRingBuffer()); } } catch (FailedSyncBeforeLogCloseException e) { @@ -351,9 +357,10 @@ public class FSHLog extends AbstractFSWAL { if (this.writer != null) { oldFileLen = this.writer.getLength(); try { - Trace.addTimelineAnnotation("closing writer"); + Span span = Tracer.getCurrentSpan(); + if (span != null) span.addTimelineAnnotation("closing writer"); this.writer.close(); - Trace.addTimelineAnnotation("writer closed"); + if (span != null) span.addTimelineAnnotation("writer closed"); this.closeErrorCount.set(0); } catch (IOException ioe) { int errors = closeErrorCount.incrementAndGet(); @@ -446,9 +453,6 @@ public class FSHLog extends AbstractFSWAL { if (this.closed) { throw new IOException("Cannot append; log is closed"); } - // Make a trace scope for the append. It is closed on other side of the ring buffer by the - // single consuming thread. Don't have to worry about it. - TraceScope scope = Trace.startSpan("FSHLog.append"); // This is crazy how much it takes to make an edit. Do we need all this stuff!!!!???? We need // all this to make a key and then below to append the edit, we need to carry htd, info, @@ -461,7 +465,7 @@ public class FSHLog extends AbstractFSWAL { // edit with its edit/sequence id. // TODO: reuse FSWALEntry as we do SyncFuture rather create per append. entry = new FSWALEntry(sequence, key, edits, hri, inMemstore); - truck.loadPayload(entry, scope.detach()); + truck.loadPayload(entry); } finally { this.disruptor.getRingBuffer().publish(sequence); } @@ -489,6 +493,9 @@ public class FSHLog extends AbstractFSWAL { // Keep around last exception thrown. Clear on successful sync. private final BlockingQueue syncFutures; + // Rather than create each time, create once and reuse. + private SpanId [] oneSpanId = new SpanId[1]; + /** * UPDATE! * @param syncs the batch of calls to sync that arrived as this thread was starting; when done, @@ -560,6 +567,19 @@ public class FSHLog extends AbstractFSWAL { } /** + * @return SpanIds of all futures in the queue that are less than currentSequence + */ + private List getSpanIds(final long currentSequence) { + List spanIds = null; + for (SyncFuture f: this.syncFutures) { + if (f.getTxid() > currentSequence) break; + if (spanIds == null) spanIds = new ArrayList(this.syncFutures.size()); + spanIds.add(f.getSpanId()); + } + return spanIds; + } + + /** * @param sequence The sequence we ran the filesystem sync against. * @return Current highest synced sequence. */ @@ -603,14 +623,27 @@ public class FSHLog extends AbstractFSWAL { break; } // I got something. Lets run. Save off current sequence number in case it changes - // while we run. - TraceScope scope = Trace.continueSpan(takeSyncFuture.getSpan()); + // while we run. Do trace work if enabled. + TraceScope traceScope = null; + if (Tracer.getCurrentSpan() != null) { + traceScope = tracer.newScope("syncer"); + // Set parents. We have at least takeSyncFuture as parent but likely more. + List spanIds = getSpanIds(currentSequence); + if (spanIds != null) { + spanIds.add(takeSyncFuture.getSpanId()); + traceScope.getSpan().setParents(spanIds.toArray(this.oneSpanId)); + } else { + this.oneSpanId[0] = takeSyncFuture.getSpanId(); + traceScope.getSpan().setParents(this.oneSpanId); + } + } long start = System.nanoTime(); Throwable lastException = null; try { - Trace.addTimelineAnnotation("syncing writer"); + Span span = Tracer.getCurrentSpan(); + if (span != null) span.addTimelineAnnotation("syncing writer"); writer.sync(); - Trace.addTimelineAnnotation("writer synced"); + if (span != null) span.addTimelineAnnotation("writer synced"); currentSequence = updateHighestSyncedSequence(currentSequence); } catch (IOException e) { LOG.error("Error syncing, request close of WAL", e); @@ -619,8 +652,6 @@ public class FSHLog extends AbstractFSWAL { LOG.warn("UNEXPECTED", e); lastException = e; } finally { - // reattach the span to the future before releasing. - takeSyncFuture.setSpan(scope.detach()); // First release what we 'took' from the queue. syncCount += releaseSyncFuture(takeSyncFuture, currentSequence, lastException); // Can we release other syncs? @@ -714,13 +745,9 @@ public class FSHLog extends AbstractFSWAL { } private SyncFuture publishSyncOnRingBuffer() { - return publishSyncOnRingBuffer(null); - } - - private SyncFuture publishSyncOnRingBuffer(Span span) { long sequence = this.disruptor.getRingBuffer().next(); // here we use ring buffer sequence as transaction id - SyncFuture syncFuture = getSyncFuture(sequence, span); + SyncFuture syncFuture = getSyncFuture(sequence); try { RingBufferTruck truck = this.disruptor.getRingBuffer().get(sequence); truck.loadPayload(syncFuture); @@ -731,8 +758,8 @@ public class FSHLog extends AbstractFSWAL { } // Sync all known transactions - private Span publishSyncThenBlockOnCompletion(Span span) throws IOException { - return blockOnSync(publishSyncOnRingBuffer(span)); + private SyncFuture publishSyncThenBlockOnCompletion() throws IOException { + return blockOnSync(publishSyncOnRingBuffer()); } /** @@ -757,12 +784,8 @@ public class FSHLog extends AbstractFSWAL { @Override public void sync() throws IOException { - TraceScope scope = Trace.startSpan("FSHLog.sync"); - try { - scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach())); - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); + try (TraceScope traceScope = this.tracer == null? null: this.tracer.newScope("FSHLog.sync");) { + publishSyncThenBlockOnCompletion(); } } @@ -772,12 +795,9 @@ public class FSHLog extends AbstractFSWAL { // Already sync'd. return; } - TraceScope scope = Trace.startSpan("FSHLog.sync"); - try { - scope = Trace.continueSpan(publishSyncThenBlockOnCompletion(scope.detach())); - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); + try (TraceScope traceScope = this.tracer == null? null: + this.tracer.newScope("FSHLog.sync txid=" + txid)) { + publishSyncThenBlockOnCompletion(); } } @@ -984,7 +1004,6 @@ public class FSHLog extends AbstractFSWAL { endOfBatch = true; } } else if (truck.hasFSWALEntryPayload()) { - TraceScope scope = Trace.continueSpan(truck.unloadSpanPayload()); try { FSWALEntry entry = truck.unloadFSWALEntryPayload(); if (this.exception != null) { @@ -1003,9 +1022,6 @@ public class FSHLog extends AbstractFSWAL { this.exception = e; // Return to keep processing events coming off the ringbuffer return; - } finally { - assert scope == NullScope.INSTANCE || !scope.isDetached(); - scope.close(); // append scope is complete } } else { // What is this if not an append or sync. Fail all up to this!!! diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java index 25c2111..4334974 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java @@ -25,7 +25,7 @@ import com.lmax.disruptor.EventFactory; /** * A 'truck' to carry a payload across the {@link FSHLog} ring buffer from Handler to WAL. - * Has EITHER a {@link FSWALEntry} for making an append OR it has a {@link SyncFuture} to + * Has EITHER a FSWALEntry for making an append OR it has a SyncFuture to * represent a 'sync' invocation. Truck instances are reused by the disruptor when it gets * around to it so their payload references must be discarded on consumption to release them * to GC. @@ -39,17 +39,10 @@ class RingBufferTruck { private FSWALEntry entry; /** - * The tracing span for this entry. Can be null. - * TODO: Fix up tracing. - */ - private Span span; - - /** * Load the truck with a {@link FSWALEntry} and associated {@link Span}. */ - void loadPayload(final FSWALEntry entry, final Span span) { + void loadPayload(final FSWALEntry entry) { this.entry = entry; - this.span = span; this.syncFuture = null; } @@ -59,7 +52,6 @@ class RingBufferTruck { void loadPayload(final SyncFuture syncFuture) { this.syncFuture = syncFuture; this.entry = null; - this.span = null; } /** @@ -97,15 +89,6 @@ class RingBufferTruck { } /** - * Unload the truck of its {@link Span} payload. The internal reference is released. - */ - Span unloadSpanPayload() { - Span ret = this.span; - this.span = null; - return ret; - } - - /** * Factory for making a bunch of these. Needed by the ringbuffer/disruptor. */ final static EventFactory EVENT_FACTORY = new EventFactory() { @@ -113,4 +96,4 @@ class RingBufferTruck { return new RingBufferTruck(); } }; -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java index 5ec218a..22d34fe 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java @@ -20,7 +20,7 @@ package org.apache.hadoop.hbase.regionserver.wal; import java.util.concurrent.ExecutionException; import org.apache.hadoop.hbase.classification.InterfaceAudience; -import org.apache.htrace.Span; +import org.apache.htrace.core.SpanId; /** * A Future on a filesystem sync call. It given to a client or 'Handler' for it to wait on till the @@ -32,7 +32,7 @@ import org.apache.htrace.Span; * goes across a queue and is handled by a background thread; when it completes, it finishes up the * future, the handler get or failed check completes and the Handler can then progress. *

        - * This is just a partial implementation of Future; we just implement get and failure. + * NOTE: This is just a partial implementation of Future; we just implement get and failure. *

        * There is not a one-to-one correlation between dfs sync invocations and instances of this class. A * single dfs sync call may complete and mark many SyncFutures as done; i.e. we batch up sync calls @@ -49,7 +49,7 @@ class SyncFuture { private static final long NOT_DONE = -1L; /** - * The transaction id of this operation, monotonically increases. + * The transaction id of this operation. Monotonically increases. */ private long txid; @@ -64,41 +64,44 @@ class SyncFuture { */ private Throwable throwable; - private Thread t; + private Thread currentThread; /** - * Optionally carry a disconnected scope to the SyncRunner. + * Optionally carry current SpanId. */ - private Span span; + private SpanId spanId = SpanId.INVALID; - SyncFuture(long txid, Span span) { - this.t = Thread.currentThread(); - this.txid = txid; - this.span = span; - this.doneTxid = NOT_DONE; + SyncFuture(long txid, final SpanId spanId) { + init(txid, spanId); } /** * Call this method to clear old usage and get it ready for new deploy. - * @param txid the new transaction id - * @param span current span, detached from caller. Don't forget to attach it when resuming after a - * call to {@link #get()}. * @return this */ - synchronized SyncFuture reset(final long txid, Span span) { - if (t != null && t != Thread.currentThread()) { - throw new IllegalStateException(); - } - t = Thread.currentThread(); + synchronized SyncFuture reset(final long txid, final SpanId spanId) { if (!isDone()) { throw new IllegalStateException("" + txid + " " + Thread.currentThread()); } - this.doneTxid = NOT_DONE; - this.txid = txid; - this.span = span; + init(txid, spanId); return this; } + /** + * Called from constructor and from reset. Does initializations in common. + * @param txid + * @param spanId + */ + private void init(final long txid, final SpanId spanId) { + if (currentThread != null && currentThread != Thread.currentThread()) { + throw new IllegalStateException(); + } + currentThread = Thread.currentThread(); + this.txid = txid; + this.doneTxid = NOT_DONE; + this.spanId = spanId; + } + @Override public synchronized String toString() { return "done=" + isDone() + ", txid=" + this.txid; @@ -109,20 +112,12 @@ class SyncFuture { } /** - * Retrieve the {@code span} instance from this Future. EventHandler calls this method to continue - * the span. Thread waiting on this Future musn't call this method until AFTER calling - * {@link #get()} and the future has been released back to the originating thread. - */ - synchronized Span getSpan() { - return this.span; - } - - /** - * Used to re-attach a {@code span} to the Future. Called by the EventHandler after a it has - * completed processing and detached the span from its scope. + * Retrieve the {@code SpanId} instance from this Future. Thread waiting on this Future musn't + * call this method until AFTER calling {@link #get()} and the future has been + * released back to the originating thread. */ - synchronized void setSpan(Span span) { - this.span = span; + synchronized SpanId getSpanId() { + return this.spanId; } /** @@ -175,4 +170,4 @@ class SyncFuture { synchronized Throwable getThrowable() { return this.throwable; } -} +} \ No newline at end of file diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java index 7f2d2f9..d057cbc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/HBaseReplicationEndpoint.java @@ -223,4 +223,4 @@ public abstract class HBaseReplicationEndpoint extends BaseReplicationEndpoint } } } -} +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java index 010e184..da01b13 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/HBaseTestingUtility.java @@ -358,7 +358,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Returns this classes's instance of {@link Configuration}. Be careful how - * you use the returned Configuration since {@link HConnection} instances + * you use the returned Configuration since HConnection instances * can be shared. The Map of HConnections is keyed by the Configuration. If * say, a Connection was being used against a cluster that had been shutdown, * see {@link #shutdownMiniCluster()}, then the Connection will no longer @@ -583,7 +583,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Start a minidfscluster. * @param servers How many DNs to start. * @throws Exception - * @see {@link #shutdownMiniDFSCluster()} + * @see #shutdownMiniCluster() * @return The mini dfs cluster created. */ public MiniDFSCluster startMiniDFSCluster(int servers) throws Exception { @@ -598,7 +598,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * datanodes will have the same host name. * @param hosts hostnames DNs to run on. * @throws Exception - * @see {@link #shutdownMiniDFSCluster()} + * @see #shutdownMiniDFSCluster() * @return The mini dfs cluster created. */ public MiniDFSCluster startMiniDFSCluster(final String hosts[]) @@ -616,7 +616,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @param servers How many DNs to start. * @param hosts hostnames DNs to run on. * @throws Exception - * @see {@link #shutdownMiniDFSCluster()} + * @see #shutdownMiniDFSCluster() * @return The mini dfs cluster created. */ public MiniDFSCluster startMiniDFSCluster(int servers, final String hosts[]) @@ -825,7 +825,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } /** - * Shuts down zk cluster created by call to {@link #startMiniZKCluster(File)} + * Shuts down zk cluster created by call to #startMiniZKCluster(File) * or does nothing. * @throws IOException * @see #startMiniZKCluster() @@ -841,7 +841,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Start up a minicluster of hbase, dfs, and zookeeper. * @throws Exception * @return Mini hbase cluster instance created. - * @see {@link #shutdownMiniDFSCluster()} + * @see #shutdownMiniDFSCluster() */ public MiniHBaseCluster startMiniCluster() throws Exception { return startMiniCluster(1, 1); @@ -853,7 +853,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * (will overwrite if dir already exists) * @throws Exception * @return Mini hbase cluster instance created. - * @see {@link #shutdownMiniDFSCluster()} + * @see #shutdownMiniDFSCluster() */ public MiniHBaseCluster startMiniCluster(final int numSlaves, boolean create) throws Exception { @@ -870,7 +870,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * hbase.regionserver.info.port is -1 (i.e. no ui per regionserver) otherwise * bind errors. * @throws Exception - * @see {@link #shutdownMiniCluster()} + * @see #shutdownMiniCluster() * @return Mini hbase cluster instance created. */ public MiniHBaseCluster startMiniCluster(final int numSlaves) @@ -882,7 +882,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * Start minicluster. Whether to create a new root or data dir path even if such a path * has been created earlier is decided based on flag create * @throws Exception - * @see {@link #shutdownMiniCluster()} + * @see #shutdownMiniCluster() * @return Mini hbase cluster instance created. */ public MiniHBaseCluster startMiniCluster(final int numMasters, @@ -894,7 +894,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * start minicluster * @throws Exception - * @see {@link #shutdownMiniCluster()} + * @see #shutdownMiniCluster() * @return Mini hbase cluster instance created. */ public MiniHBaseCluster startMiniCluster(final int numMasters, @@ -931,7 +931,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * If you start MiniDFSCluster without host names, * all instances of the datanodes will have the same host name. * @throws Exception - * @see {@link #shutdownMiniCluster()} + * @see #shutdownMiniCluster() * @return Mini hbase cluster instance created. */ public MiniHBaseCluster startMiniCluster(final int numMasters, @@ -973,7 +973,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @param regionserverClass The class to use as HRegionServer, or null for * default * @throws Exception - * @see {@link #shutdownMiniCluster()} + * @see #shutdownMiniCluster() * @return Mini hbase cluster instance created. */ public MiniHBaseCluster startMiniCluster(final int numMasters, @@ -1054,7 +1054,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @return Reference to the hbase mini hbase cluster. * @throws IOException * @throws InterruptedException - * @see {@link #startMiniCluster()} + * @see #startMiniCluster() */ public MiniHBaseCluster startMiniHBaseCluster(final int numMasters, final int numSlaves, Class masterClass, @@ -1136,7 +1136,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Stops mini hbase, zk, and hdfs clusters. * @throws IOException - * @see {@link #startMiniCluster(int)} + * @see #startMiniCluster(int) */ public void shutdownMiniCluster() throws Exception { LOG.info("Shutting down minicluster"); @@ -1761,7 +1761,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @param desc * @param startKey * @param endKey - * @return + * @return A Region instance * @throws IOException */ public HRegion createLocalHRegion(HTableDescriptor desc, byte [] startKey, @@ -1804,7 +1804,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * @return A region on which you must call {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. * @deprecated use - * {@link #createLocalHRegion(TableName, byte[], byte[], boolean, Durability, WAL, byte[]...)} + * #createLocalHRegion(TableName, byte[], byte[], boolean, Durability, WAL, byte[]...) */ @Deprecated public HRegion createLocalHRegion(byte[] tableName, byte[] startKey, byte[] stopKey, @@ -1816,13 +1816,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } /** - * @param tableName - * @param startKey - * @param stopKey - * @param callingMethod - * @param conf - * @param isReadOnly - * @param families * @return A region on which you must call * {@link HBaseTestingUtility#closeRegionAndWAL(HRegion)} when done. * @throws IOException @@ -1963,7 +1956,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { } /** A tracker for tracking and validating table rows - * generated with {@link HBaseTestingUtility#loadTable(HTable, byte[])} + * generated with HBaseTestingUtility#loadTable(HTable, byte[]) */ public static class SeenRowTracker { int dim = 'z' - 'a' + 1; @@ -2198,7 +2191,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { return digest.toString(); } - /** All the row values for the data loaded by {@link #loadTable(HTable, byte[])} */ + /** All the row values for the data loaded by #loadTable(HTable, byte[]) */ public static final byte[][] ROWS = new byte[(int) Math.pow('z' - 'a' + 1, 3)][3]; // ~52KB static { int i = 0; @@ -2959,7 +2952,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * regions have been all assigned. Will timeout after default period (30 seconds) * Tolerates nonexistent table. * @param table Table to wait on. - * @param table * @throws InterruptedException * @throws IOException */ @@ -2971,7 +2963,7 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { /** * Waits for a table to be 'enabled'. Enabled means that table is set as 'enabled' and the * regions have been all assigned. - * @see #waitTableEnabled(Admin, byte[], long) + * @see #waitTableEnabled(TableName, long) * @param table Table to wait on. * @param timeoutMillis Time to wait on it being marked enabled. * @throws InterruptedException @@ -3127,11 +3119,6 @@ public class HBaseTestingUtility extends HBaseCommonTestingUtility { * 2010-06-15 11:52:28,511 WARN [DataStreamer for file /hbase/.logs/wal.1276627923013 block blk_928005470262850423_1021] hdfs.DFSClient$DFSOutputStream(2657): Error Recovery for block blk_928005470262850423_1021 failed because recovery from primary datanode 127.0.0.1:53683 failed 4 times. Pipeline was 127.0.0.1:53687, 127.0.0.1:53683. Will retry... * * @param stream A DFSClient.DFSOutputStream. - * @param max - * @throws NoSuchFieldException - * @throws SecurityException - * @throws IllegalAccessException - * @throws IllegalArgumentException */ public static void setMaxRecoveryErrorCount(final OutputStream stream, final int max) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java index 02b994a..cb3d5bb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/PerformanceEvaluation.java @@ -79,7 +79,6 @@ import org.apache.hadoop.hbase.io.hfile.RandomDistribution; import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; -import org.apache.hadoop.hbase.trace.SpanReceiverHost; import org.apache.hadoop.hbase.util.*; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; @@ -90,11 +89,12 @@ import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; import org.apache.hadoop.mapreduce.lib.reduce.LongSumReducer; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; +import org.apache.htrace.core.HTraceConfiguration; +import org.apache.htrace.core.ProbabilitySampler; +import org.apache.htrace.core.Sampler; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import org.codehaus.jackson.map.ObjectMapper; -import org.apache.htrace.Sampler; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; -import org.apache.htrace.impl.ProbabilitySampler; import com.google.common.base.Objects; import com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -983,8 +983,7 @@ public class PerformanceEvaluation extends Configured implements Tool { protected final TestOptions opts; private final Status status; - private final Sampler traceSampler; - private final SpanReceiverHost receiverHost; + private final Tracer tracer; protected Connection connection; // protected Table table; @@ -1000,18 +999,21 @@ public class PerformanceEvaluation extends Configured implements Tool { Test(final Connection con, final TestOptions options, final Status status) { this.connection = con; this.conf = con == null ? HBaseConfiguration.create() : this.connection.getConfiguration(); - this.receiverHost = this.conf == null? null: SpanReceiverHost.getInstance(conf); this.opts = options; this.status = status; this.testName = this.getClass().getSimpleName(); + HTraceConfiguration htraceConfiguration = new HBaseHTraceConfiguration(conf); + this.tracer = new Tracer.Builder("PE").conf(htraceConfiguration).build(); + Sampler sampler; if (options.traceRate >= 1.0) { - this.traceSampler = Sampler.ALWAYS; + sampler = Sampler.ALWAYS; } else if (options.traceRate > 0.0) { conf.setDouble("hbase.sampler.fraction", options.traceRate); - this.traceSampler = new ProbabilitySampler(new HBaseHTraceConfiguration(conf)); + sampler = new ProbabilitySampler(new HBaseHTraceConfiguration(conf)); } else { - this.traceSampler = Sampler.NEVER; + sampler = Sampler.NEVER; } + this.tracer.addSampler(sampler); everyN = (int) (opts.totalRows / (opts.totalRows * opts.sampleRate)); if (options.isValueZipf()) { this.zipf = new RandomDistribution.Zipf(this.rand, 1, options.getValueSize(), 1.2); @@ -1094,7 +1096,7 @@ public class PerformanceEvaluation extends Configured implements Tool { if (!opts.oneCon) { connection.close(); } - receiverHost.closeReceivers(); + tracer.close(); } abstract void onTakedown() throws IOException; @@ -1136,11 +1138,8 @@ public class PerformanceEvaluation extends Configured implements Tool { for (int i = startRow; i < lastRow; i++) { if (i % everyN != 0) continue; long startTime = System.nanoTime(); - TraceScope scope = Trace.startSpan("test row", traceSampler); - try { + try (TraceScope scope = this.tracer.newScope("test row");) { testRow(i); - } finally { - scope.close(); } if ( (i - startRow) > opts.measureAfter) { // If multiget is enabled, say set to 10, testRow() returns immediately first 9 times diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java index a79aa0a..b388657 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestReplicaWithCluster.java @@ -20,6 +20,15 @@ package org.apache.hadoop.hbase.client; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -51,15 +60,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - @Category({MediumTests.class, ClientTests.class}) public class TestReplicaWithCluster { private static final Log LOG = LogFactory.getLog(TestReplicaWithCluster.class); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java index cbcc166..9bd724d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/procedure/TestProcedureManager.java @@ -68,4 +68,4 @@ public class TestProcedureManager { assertArrayEquals("Incorrect return data from execProcedure", SimpleMasterProcedureManager.SIMPLE_DATA.getBytes(), result); } -} +} \ No newline at end of file diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java deleted file mode 100644 index 205f1d8..0000000 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/trace/TestHTraceHooks.java +++ /dev/null @@ -1,142 +0,0 @@ -/** - * 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.hadoop.hbase.trace; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; - -import java.lang.reflect.Method; -import java.util.Collection; - -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.TableName; -import org.apache.hadoop.hbase.Waiter; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Table; -import org.apache.hadoop.hbase.testclassification.MediumTests; -import org.apache.hadoop.hbase.testclassification.MiscTests; -import org.apache.htrace.Sampler; -import org.apache.htrace.Span; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; -import org.apache.htrace.TraceTree; -import org.apache.htrace.impl.POJOSpanReceiver; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category({MiscTests.class, MediumTests.class}) -public class TestHTraceHooks { - - private static final byte[] FAMILY_BYTES = "family".getBytes(); - private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static POJOSpanReceiver rcvr; - private static long ROOT_SPAN_ID = 0; - - @BeforeClass - public static void before() throws Exception { - - // Find out what the right value to use fo SPAN_ROOT_ID after HTRACE-111. We use HTRACE-32 - // to find out to detect if we are using HTrace 3.2 or not. - try { - Method m = Span.class.getMethod("addKVAnnotation", String.class, String.class); - } catch (NoSuchMethodException e) { - ROOT_SPAN_ID = 0x74aceL; // Span.SPAN_ROOT_ID pre HTrace-3.2 - } - - TEST_UTIL.startMiniCluster(2, 3); - rcvr = new POJOSpanReceiver(new HBaseHTraceConfiguration(TEST_UTIL.getConfiguration())); - Trace.addReceiver(rcvr); - } - - @AfterClass - public static void after() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - Trace.removeReceiver(rcvr); - rcvr = null; - } - - @Test - public void testTraceCreateTable() throws Exception { - TraceScope tableCreationSpan = Trace.startSpan("creating table", Sampler.ALWAYS); - Table table; - try { - - table = TEST_UTIL.createTable(TableName.valueOf("table"), - FAMILY_BYTES); - } finally { - tableCreationSpan.close(); - } - - // Some table creation is async. Need to make sure that everything is full in before - // checking to see if the spans are there. - TEST_UTIL.waitFor(1000, new Waiter.Predicate() { - @Override - public boolean evaluate() throws Exception { - return rcvr.getSpans().size() >= 5; - } - }); - - Collection spans = rcvr.getSpans(); - TraceTree traceTree = new TraceTree(spans); - Collection roots = traceTree.getSpansByParent().find(ROOT_SPAN_ID); - - assertEquals(1, roots.size()); - Span createTableRoot = roots.iterator().next(); - - assertEquals("creating table", createTableRoot.getDescription()); - - int createTableCount = 0; - - for (Span s : traceTree.getSpansByParent().find(createTableRoot.getSpanId())) { - if (s.getDescription().startsWith("MasterService.CreateTable")) { - createTableCount++; - } - } - - assertTrue(createTableCount >= 1); - assertTrue(traceTree.getSpansByParent().find(createTableRoot.getSpanId()).size() > 3); - assertTrue(spans.size() > 5); - - Put put = new Put("row".getBytes()); - put.addColumn(FAMILY_BYTES, "col".getBytes(), "value".getBytes()); - - TraceScope putSpan = Trace.startSpan("doing put", Sampler.ALWAYS); - try { - table.put(put); - } finally { - putSpan.close(); - } - - spans = rcvr.getSpans(); - traceTree = new TraceTree(spans); - roots = traceTree.getSpansByParent().find(ROOT_SPAN_ID); - - assertEquals(2, roots.size()); - Span putRoot = null; - for (Span root : roots) { - if (root.getDescription().equals("doing put")) { - putRoot = root; - } - } - - assertNotNull(putRoot); - } -} diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java index 7ce03b6..567a44f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/wal/WALPerformanceEvaluation.java @@ -51,18 +51,16 @@ import org.apache.hadoop.hbase.io.crypto.KeyProviderForTesting; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.LogRoller; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; -import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; -import org.apache.hadoop.hbase.trace.SpanReceiverHost; import org.apache.hadoop.hbase.wal.WALProvider.Writer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.hbase.util.Threads; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; -import org.apache.htrace.Sampler; -import org.apache.htrace.Trace; -import org.apache.htrace.TraceScope; -import org.apache.htrace.impl.ProbabilitySampler; +import org.apache.htrace.core.ProbabilitySampler; +import org.apache.htrace.core.Sampler; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; import com.codahale.metrics.ConsoleReporter; import com.codahale.metrics.Histogram; @@ -75,6 +73,7 @@ import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogReader; import org.apache.hadoop.hbase.regionserver.wal.SecureProtobufLogWriter; import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; +import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration; import static com.codahale.metrics.MetricRegistry.name; @@ -111,6 +110,8 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { private int valueSize = 512; private int keySize = 16; + private Tracer tracer; + @Override public void setConf(Configuration conf) { super.setConf(conf); @@ -173,14 +174,15 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { Random rand = new Random(Thread.currentThread().getId()); WAL wal = region.getWAL(); - TraceScope threadScope = - Trace.startSpan("WALPerfEval." + Thread.currentThread().getName()); + Tracer tracer = Tracer.curThreadTracer(); + TraceScope threadScope = tracer == null? null: + tracer.newScope("WALPerfEval." + Thread.currentThread().getName()); try { long startTime = System.currentTimeMillis(); int lastSync = 0; for (int i = 0; i < numIterations; ++i) { - assert Trace.currentSpan() == threadScope.getSpan() : "Span leak detected."; - TraceScope loopScope = Trace.startSpan("runLoopIter" + i, loopSampler); + TraceScope loopScope = tracer == null? null: + tracer.newScope("runLoopIter" + i); try { long now = System.nanoTime(); Put put = setupPut(rand, key, value, numFamilies); @@ -315,9 +317,9 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { FileSystem fs = FileSystem.get(getConf()); LOG.info("FileSystem: " + fs); - SpanReceiverHost receiverHost = trace ? SpanReceiverHost.getInstance(getConf()) : null; - final Sampler sampler = trace ? Sampler.ALWAYS : Sampler.NEVER; - TraceScope scope = Trace.startSpan("WALPerfEval", sampler); + this.tracer = new Tracer.Builder("WALPerfEval"). + conf(new HBaseHTraceConfiguration(getConf())).build(); + this.tracer.addSampler(Sampler.ALWAYS); try { if (rootRegionDir == null) { @@ -339,8 +341,9 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { // a table per desired region means we can avoid carving up the key space final HTableDescriptor htd = createHTableDescriptor(i, numFamilies); regions[i] = openRegion(fs, rootRegionDir, htd, wals, roll, roller); - benchmarks[i] = Trace.wrap(new WALPutBenchmark(regions[i], htd, numIterations, noSync, - syncInterval, traceFreq)); + benchmarks[i] = + new WALPutBenchmark(regions[i], htd, numIterations, noSync, + syncInterval, traceFreq); } ConsoleReporter reporter = ConsoleReporter.forRegistry(metrics). outputTo(System.out).convertRatesTo(TimeUnit.SECONDS).filter(MetricFilter.ALL).build(); @@ -391,8 +394,7 @@ public final class WALPerformanceEvaluation extends Configured implements Tool { } finally { // We may be called inside a test that wants to keep on using the fs. if (!noclosefs) fs.close(); - scope.close(); - if (receiverHost != null) receiverHost.closeReceivers(); + this.tracer.close(); } return(0); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java index e71210d..839343b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/zookeeper/TestRecoverableZooKeeper.java @@ -74,7 +74,7 @@ public class TestRecoverableZooKeeper { ZooKeeperWatcher zkw = new ZooKeeperWatcher(conf, "testSetDataVersionMismatchInLoop", abortable, true); String ensemble = ZKConfig.getZKQuorumServersString(conf); - RecoverableZooKeeper rzk = ZKUtil.connect(conf, ensemble, zkw); + RecoverableZooKeeper rzk = ZKUtil.connect(conf, ensemble, zkw, null); rzk.create(znode, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT); rzk.setData(znode, "OPENING".getBytes(), 0); Field zkField = RecoverableZooKeeper.class.getDeclaredField("zk"); diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml index cf63e94..9d40691 100644 --- a/hbase-shell/pom.xml +++ b/hbase-shell/pom.xml @@ -249,7 +249,7 @@ org.apache.htrace - htrace-core + htrace-core4 diff --git a/hbase-shell/src/main/ruby/shell/commands/trace.rb b/hbase-shell/src/main/ruby/shell/commands/trace.rb index 5e00930..8017215 100644 --- a/hbase-shell/src/main/ruby/shell/commands/trace.rb +++ b/hbase-shell/src/main/ruby/shell/commands/trace.rb @@ -17,8 +17,9 @@ # limitations under the License. # HTrace = org.apache.htrace.Trace -java_import org.apache.htrace.Sampler -java_import org.apache.hadoop.hbase.trace.SpanReceiverHost +java_import org.apache.hadoop.hbase.HBaseConfiguration +java_import org.apache.htrace.core.Tracer +java_import org.apache.hadoop.hbase.trace.HBaseHTraceConfiguration module Shell module Commands @@ -47,28 +48,32 @@ Examples: EOF end - def command(startstop="status", spanname="HBaseShell") + def command(startstop="status", spanName="HBaseShell") format_and_return_simple_command do - trace(startstop, spanname) + trace(startstop, spanName) end end - def trace(startstop, spanname) - @@receiver ||= SpanReceiverHost.getInstance(@shell.hbase.configuration) + def trace(startstop, spanName) if startstop == "start" if not tracing? - @@tracescope = HTrace.startSpan(spanname, Sampler.ALWAYS) + hbc = HBaseConfiguration.create(); + hbc.set("hbase.htrace.sampler.classes".to_java, + "org.apache.htrace.core.AlwaysSampler".to_java) + hhc = HBaseHTraceConfiguration.new(hbc); + tracer = Tracer::Builder.new("Shell".to_java).conf(hhc).build(); + @@tracescope = tracer.newScope(spanName.to_java) end elsif startstop == "stop" if tracing? - @@tracescope.close() + @@tracescope.close() if @@tracescope end end tracing? end def tracing?() - HTrace.isTracing() + not Tracer.curThreadTracer().nil? end end diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml index ff999dc..64f0f24 100644 --- a/hbase-thrift/pom.xml +++ b/hbase-thrift/pom.xml @@ -294,7 +294,7 @@ org.apache.htrace - htrace-core + htrace-core4 org.apache.thrift diff --git a/pom.xml b/pom.xml index cf6501c..a27b423 100644 --- a/pom.xml +++ b/pom.xml @@ -1226,7 +1226,7 @@ 1.6.8 4.12 1.3 - 3.1.0-incubating + 4.1.0-incubating-SNAPSHOT 1.2.17 1.10.8 2.5.0 @@ -1768,7 +1768,7 @@ org.apache.htrace - htrace-core + htrace-core4 ${htrace.version} diff --git a/src/main/asciidoc/_chapters/tracing.adoc b/src/main/asciidoc/_chapters/tracing.adoc index 0cddd8a..590a20a 100644 --- a/src/main/asciidoc/_chapters/tracing.adoc +++ b/src/main/asciidoc/_chapters/tracing.adoc @@ -123,26 +123,18 @@ into: [source,java] ---- -TraceScope ts = Trace.startSpan("Gets", Sampler.ALWAYS); +Tracer tracer = Tracer.curThreadTracer(); +TraceScope ts = tracer == null? null: tracer.newScope("Gets"); try { Table table = connection.getTable(TableName.valueOf("t1")); Get get = new Get(Bytes.toBytes("r1")); Result res = table.get(get); } finally { - ts.close(); + if (ts != null) ts.close(); } ---- -If you wanted to trace half of your 'get' operations, you would pass in: - -[source,java] ----- - -new ProbabilitySampler(0.5) ----- - -in lieu of `Sampler.ALWAYS` to `Trace.startSpan()`. -See the HTrace _README_ for more information on Samplers. +See the HTrace documentation for more information on Samplers. [[tracing.client.shell]] == Tracing from HBase Shell -- 2.6.1 From 3258fa6d2a22503751fa810bb4feb5956e9cca22 Mon Sep 17 00:00:00 2001 From: stack Date: Tue, 3 May 2016 09:48:46 -0700 Subject: [PATCH 2/2] Add nocluster tests for the HTable traces. Had to fix nocluster so it supported Scans, increments, etc. Some redo of client trace points and bug fixes so our trace profile looks better. --- .../apache/hadoop/hbase/client/AsyncProcess.java | 4 +- .../hbase/client/ConnectionImplementation.java | 12 +- .../apache/hadoop/hbase/client/HRegionLocator.java | 11 +- .../org/apache/hadoop/hbase/client/HTable.java | 190 +++++----- .../client/ResultBoundedCompletionService.java | 2 +- .../hbase/client/SimpleNoZookeeperRegistry.java | 53 +++ .../hadoop/hbase/client/TestClientNoCluster.java | 140 +++++--- .../hbase/client/TestClientTracingNoCluster.java | 388 +++++++++++++++++++++ 8 files changed, 625 insertions(+), 175 deletions(-) create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/client/SimpleNoZookeeperRegistry.java create mode 100644 hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientTracingNoCluster.java diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java index 5f34aa3..bf939f3 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncProcess.java @@ -1036,7 +1036,7 @@ class AsyncProcess { runnable.run(); } else { try { - pool.submit(connection.getTracer().wrap(runnable, "sendMultiAction")); + pool.submit(connection.getTracer().wrap(runnable, "AsyncProcess.sendMultiAction")); } catch (Throwable t) { if (t instanceof RejectedExecutionException) { // This should never happen. But as the pool is provided by the end user, @@ -1072,7 +1072,7 @@ class AsyncProcess { Runnable runnable = new SingleServerRequestRunnable(multiAction, numAttempt, server, callsInProgress); return Collections.singletonList( - connection.getTracer().wrap(runnable, "AsyncProcess.sendMultiAction")); + connection.getTracer().wrap(runnable, "AsyncProcess.multiAction")); } // group the actions by the amount of delay diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java index e7df3ab..f2dc604 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ConnectionImplementation.java @@ -203,9 +203,9 @@ class ConnectionImplementation implements ClusterConnection, Closeable { this.batchPool = pool; // Set up tracer. Add Connection hash to the TraceId so it comes out as "HConnection,hash". - conf.set(HBaseHTraceConfiguration.KEY_PREFIX + TracerId.TRACER_ID_KEY, toString()); + conf.set(HBaseHTraceConfiguration.KEY_PREFIX + TracerId.TRACER_ID_KEY, toString() + "/%{ip}"); HTraceConfiguration htraceConfiguration = new HBaseHTraceConfiguration(conf); - this.tracer = new Tracer.Builder("").conf(htraceConfiguration).build(); + this.tracer = new Tracer.Builder("hbase.connection").conf(htraceConfiguration).build(); this.connectionConfig = new ConnectionConfiguration(conf); this.closed = false; @@ -742,12 +742,13 @@ class ConnectionImplementation implements ClusterConnection, Closeable { final byte [] row, boolean useCache, boolean retry, int replicaId) throws IOException { if (this.closed) throw new IOException(toString() + " closed"); - if (tableName== null || tableName.getName().length == 0) { + if (tableName == null || tableName.getName().length == 0) { throw new IllegalArgumentException( "table name cannot be null or zero length"); } if (tableName.equals(TableName.META_TABLE_NAME)) { - try (TraceScope traceScope = getTracer().newScope("locateMeta")) { + try (TraceScope traceScope = + getTracer().newScope("locateMeta=" + Bytes.toStringBinary(row))) { return locateMeta(tableName, useCache, replicaId); } } else { @@ -2134,6 +2135,7 @@ class ConnectionImplementation implements ClusterConnection, Closeable { if (rpcClient != null) { rpcClient.close(); } + this.tracer.close(); } /** @@ -2323,4 +2325,4 @@ class ConnectionImplementation implements ClusterConnection, Closeable { public Tracer getTracer() { return this.tracer; } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java index 4d2311d..bc4ae30 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HRegionLocator.java @@ -32,7 +32,9 @@ import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.classification.InterfaceStability; +import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Pair; +import org.apache.htrace.core.TraceScope; /** * An implementation of {@link RegionLocator}. Used to view region location information for a single @@ -45,7 +47,6 @@ import org.apache.hadoop.hbase.util.Pair; @InterfaceAudience.Private @InterfaceStability.Stable public class HRegionLocator implements RegionLocator { - private final TableName tableName; private final ClusterConnection connection; @@ -78,7 +79,10 @@ public class HRegionLocator implements RegionLocator { @Override public HRegionLocation getRegionLocation(final byte [] row, boolean reload) throws IOException { - return connection.getRegionLocation(tableName, row, reload); + try (TraceScope traceScope = + this.connection.getTracer().newScope(Bytes.toString(row) + ", reload=" + reload)) { + return connection.getRegionLocation(tableName, row, reload); + } } @Override @@ -89,7 +93,6 @@ public class HRegionLocator implements RegionLocator { ArrayList regions = new ArrayList<>(locations.size()); for (Pair entry : locations) { regions.add(new HRegionLocation(entry.getFirst(), entry.getSecond())); - } if (regions.size() > 0) { connection.cacheLocation(tableName, new RegionLocations(regions)); @@ -159,4 +162,4 @@ public class HRegionLocator implements RegionLocator { public Configuration getConfiguration() { return connection.getConfiguration(); } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java index 0160c34..7bfd9e2 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/HTable.java @@ -74,6 +74,7 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.Message; import com.google.protobuf.Service; import com.google.protobuf.ServiceException; +import com.google.protobuf.TextFormat; /** * An implementation of {@link Table}. Used to communicate with a single HBase table. @@ -116,10 +117,9 @@ public class HTable implements HTableInterface { private int operationTimeout; // global timeout for each blocking method with retrying rpc private int rpcTimeout; // timeout for each rpc request private final boolean cleanupPoolOnClose; // shutdown the pool in close() - private final boolean cleanupConnectionOnClose; // close the connection in close() private Consistency defaultConsistency = Consistency.STRONG; private HRegionLocator locator; - private TraceScope tableTracerScope; + private final TraceScope tableTracerScope; /** The Async process for batch */ protected AsyncProcess multiAp; @@ -161,27 +161,16 @@ public class HTable implements HTableInterface { final RpcRetryingCallerFactory rpcCallerFactory, final RpcControllerFactory rpcControllerFactory, final ExecutorService pool) throws IOException { - if (connection == null || connection.isClosed()) { - throw new IllegalArgumentException("Connection is null or closed."); - } - this.tableName = tableName; - connection.getTracer().newScope(getName().toString()); - this.cleanupConnectionOnClose = false; - this.connection = connection; + this(connection, tableName, pool == null); this.configuration = connection.getConfiguration(); this.connConfiguration = tableConfig; this.pool = pool; if (pool == null) { this.pool = getDefaultExecutor(this.configuration); - this.cleanupPoolOnClose = true; - } else { - this.cleanupPoolOnClose = false; } - this.rpcCallerFactory = rpcCallerFactory; this.rpcControllerFactory = rpcControllerFactory; - - this.finishSetup(); + finishSetup(); } /** @@ -189,15 +178,25 @@ public class HTable implements HTableInterface { * @throws IOException */ @VisibleForTesting - protected HTable(ClusterConnection conn, BufferedMutatorParams params) throws IOException { - connection = conn; - tableName = params.getTableName(); - conn.getTracer().newScope(getName().toString()); - connConfiguration = new ConnectionConfiguration(connection.getConfiguration()); - cleanupPoolOnClose = false; - cleanupConnectionOnClose = false; + protected HTable(ClusterConnection connection, BufferedMutatorParams params) throws IOException { + this(connection, params.getTableName(), false); + this.connConfiguration = new ConnectionConfiguration(connection.getConfiguration()); // used from tests, don't trust the connection is real - this.mutator = new BufferedMutatorImpl(conn, null, null, params); + this.mutator = new BufferedMutatorImpl(connection, null, null, params); + } + + /** + * Private, partial constructor, just so we don't have to duplicate assignments. + */ + private HTable(final ClusterConnection connection, final TableName tableName, + final boolean cleanupPoolOnClose) { + if (connection == null || connection.isClosed()) { + throw new IllegalArgumentException("Connection is null or closed."); + } + this.connection = connection; + this.tableName = tableName; + this.tableTracerScope = connection.getTracer().newScope("New HTable=" + getName().toString()); + this.cleanupPoolOnClose = cleanupPoolOnClose; } /** @@ -421,37 +420,37 @@ public class HTable implements HTableInterface { } private Result get(Get get, final boolean checkExistenceOnly) throws IOException { - try (TraceScope traceScope = this.connection.getTracer().newScope("get")) { - // if we are changing settings to the get, clone it. - if (get.isCheckExistenceOnly() != checkExistenceOnly || get.getConsistency() == null) { - get = ReflectionUtils.newInstance(get.getClass(), get); - get.setCheckExistenceOnly(checkExistenceOnly); - if (get.getConsistency() == null){ - get.setConsistency(defaultConsistency); - } + // if we are changing settings to the get, clone it. + if (get.isCheckExistenceOnly() != checkExistenceOnly || get.getConsistency() == null) { + get = ReflectionUtils.newInstance(get.getClass(), get); + get.setCheckExistenceOnly(checkExistenceOnly); + if (get.getConsistency() == null){ + get.setConsistency(defaultConsistency); } + } - if (get.getConsistency() == Consistency.STRONG) { - // Good old call. - final Get getReq = get; - RegionServerCallable callable = new RegionServerCallable(this.connection, - getName(), get.getRow()) { - @Override - public Result call(int callTimeout) throws IOException { - ClientProtos.GetRequest request = - RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq); - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); - controller.setPriority(tableName); - controller.setCallTimeout(callTimeout); - try { - ClientProtos.GetResponse response = getStub().get(controller, request); - if (response == null) return null; - return ProtobufUtil.toResult(response.getResult(), controller.cellScanner()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } + if (get.getConsistency() == Consistency.STRONG) { + // Good old call. + final Get getReq = get; + RegionServerCallable callable = new RegionServerCallable(this.connection, + getName(), get.getRow()) { + @Override + public Result call(int callTimeout) throws IOException { + ClientProtos.GetRequest request = + RequestConverter.buildGetRequest(getLocation().getRegionInfo().getRegionName(), getReq); + PayloadCarryingRpcController controller = rpcControllerFactory.newController(); + controller.setPriority(tableName); + controller.setCallTimeout(callTimeout); + try { + ClientProtos.GetResponse response = getStub().get(controller, request); + if (response == null) return null; + return ProtobufUtil.toResult(response.getResult(), controller.cellScanner()); + } catch (ServiceException se) { + throw ProtobufUtil.getRemoteException(se); } - }; + } + }; + try (TraceScope traceScope = this.connection.getTracer().newScope("get")) { return rpcCallerFactory.newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); } @@ -463,7 +462,9 @@ public class HTable implements HTableInterface { connConfiguration.getRetriesNumber(), operationTimeout, connConfiguration.getPrimaryCallTimeoutMicroSecond()); - return callable.call(); + try (TraceScope traceScope = this.connection.getTracer().newScope("getReadReplicas")) { + return callable.call(); + } } @@ -477,7 +478,7 @@ public class HTable implements HTableInterface { } try (TraceScope traceScope = this.connection.getTracer().newScope("gets")) { Object[] r1 = new Object[gets.size()]; - batch((List) gets, r1); + batch((List) gets, r1); // translate. Result [] results = new Result[r1.length]; @@ -710,8 +711,10 @@ public class HTable implements HTableInterface { } } }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, + try (TraceScope traceScope = this.connection.getTracer().newScope("increment")) { + return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); + } } /** @@ -760,14 +763,22 @@ public class HTable implements HTableInterface { MutateResponse response = getStub().mutate(controller, request); Result result = ProtobufUtil.toResult(response.getResult(), controller.cellScanner()); + if (result == null || result.size() <= 0) { + // If no result, just return zero. Should never be no result other than in tests + // where server is mocked up. + LOG.warn("Null result: " + TextFormat.shortDebugString(request)); + return Long.valueOf(0); + } return Long.valueOf(Bytes.toLong(result.getValue(family, qualifier))); } catch (ServiceException se) { throw ProtobufUtil.getRemoteException(se); } } }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, + try (TraceScope traceScope = this.connection.getTracer().newScope("incrementColumnValue")) { + return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); + } } /** @@ -778,26 +789,7 @@ public class HTable implements HTableInterface { final byte [] family, final byte [] qualifier, final byte [] value, final Put put) throws IOException { - RegionServerCallable callable = - new RegionServerCallable(connection, getName(), row) { - @Override - public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); - controller.setPriority(tableName); - controller.setCallTimeout(callTimeout); - try { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), row, family, qualifier, - new BinaryComparator(value), CompareType.EQUAL, put); - MutateResponse response = getStub().mutate(controller, request); - return Boolean.valueOf(response.getProcessed()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, - this.operationTimeout); + return checkAndPut(row, family, qualifier, CompareOp.EQUAL, value, put); } /** @@ -827,8 +819,10 @@ public class HTable implements HTableInterface { } } }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, + try (TraceScope traceScope = this.connection.getTracer().newScope("checkAndPut")) { + return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); + } } /** @@ -839,26 +833,7 @@ public class HTable implements HTableInterface { final byte [] family, final byte [] qualifier, final byte [] value, final Delete delete) throws IOException { - RegionServerCallable callable = - new RegionServerCallable(connection, getName(), row) { - @Override - public Boolean call(int callTimeout) throws IOException { - PayloadCarryingRpcController controller = rpcControllerFactory.newController(); - controller.setPriority(tableName); - controller.setCallTimeout(callTimeout); - try { - MutateRequest request = RequestConverter.buildMutateRequest( - getLocation().getRegionInfo().getRegionName(), row, family, qualifier, - new BinaryComparator(value), CompareType.EQUAL, delete); - MutateResponse response = getStub().mutate(controller, request); - return Boolean.valueOf(response.getProcessed()); - } catch (ServiceException se) { - throw ProtobufUtil.getRemoteException(se); - } - } - }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, - this.operationTimeout); + return checkAndDelete(row, family, qualifier, CompareOp.EQUAL, value, delete); } /** @@ -888,8 +863,10 @@ public class HTable implements HTableInterface { } } }; - return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, + try (TraceScope traceScope = this.connection.getTracer().newScope("checkAndDelete")) { + return rpcCallerFactory. newCaller(rpcTimeout).callWithRetries(callable, this.operationTimeout); + } } /** @@ -944,8 +921,8 @@ public class HTable implements HTableInterface { if (ars.hasError()) { throw ars.getErrors(); } - - return ((Result)results[0]).getExists(); + Result result = results != null && results.length > 0? (Result)results[0]: null; + return result != null && result.size() > 0? result.getExists(): false; } /** @@ -1051,12 +1028,8 @@ public class HTable implements HTableInterface { LOG.warn("waitForTermination interrupted"); } } + this.getRegionLocator().close(); if (this.tableTracerScope != null) this.tableTracerScope.close(); - if (cleanupConnectionOnClose) { - if (this.connection != null) { - this.connection.close(); - } - } this.closed = true; } @@ -1148,7 +1121,10 @@ public class HTable implements HTableInterface { * This is a power user function: avoid unless you know the ramifications. */ public void clearRegionCache() { - this.connection.clearRegionCache(); + try (TraceScope traceScope = + this.connection.getTracer().newScope("ClearRegionCache=" + getName())) { + this.connection.clearRegionCache(); + } } /** @@ -1257,7 +1233,7 @@ public class HTable implements HTableInterface { @Override public String toString() { - return tableName + ";" + connection; + return "tableName=" + tableName + ", connection=" + connection; } /** @@ -1385,4 +1361,4 @@ public class HTable implements HTableInterface { } return mutator; } -} +} \ No newline at end of file diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java index 6ef451b..ebce035 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/ResultBoundedCompletionService.java @@ -144,7 +144,7 @@ public class ResultBoundedCompletionService { public void submit(RetryingCallable task, int callTimeout, int id) { QueueingFuture newFuture = new QueueingFuture(task, callTimeout); Tracer tracer = Tracer.curThreadTracer(); - executor.execute(tracer == null? newFuture: tracer.wrap(newFuture, "submit")); + executor.execute(tracer == null? newFuture: tracer.wrap(newFuture, "submit.rbcs")); tasks[id] = newFuture; } diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/SimpleNoZookeeperRegistry.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/SimpleNoZookeeperRegistry.java new file mode 100644 index 0000000..1073610 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/SimpleNoZookeeperRegistry.java @@ -0,0 +1,53 @@ +/** + * 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.hadoop.hbase.client; + +import java.io.IOException; + +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HRegionInfo; +import org.apache.hadoop.hbase.HRegionLocation; +import org.apache.hadoop.hbase.RegionLocations; +import org.apache.hadoop.hbase.ServerName; + +/** + * Simple cluster registry inserted in place of our usual zookeeper based one. + */ +class SimpleNoZookeeperRegistry implements Registry { + final ServerName META_HOST = ServerName.valueOf("meta.example.org", 16010, 12345); + + @Override + public void init(Connection connection) { + } + + @Override + public RegionLocations getMetaRegionLocation() throws IOException { + return new RegionLocations( + new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, META_HOST)); + } + + @Override + public String getClusterId() { + return HConstants.CLUSTER_ID_DEFAULT; + } + + @Override + public int getCurrentNrHRS() throws IOException { + return 1; + } +} \ No newline at end of file diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java index 0a5a37f..0a6095b 100644 --- a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientNoCluster.java @@ -25,6 +25,7 @@ import java.net.SocketTimeoutException; import java.util.Comparator; import java.util.HashMap; import java.util.Map; +import java.util.NavigableMap; import java.util.Random; import java.util.SortedMap; import java.util.concurrent.ConcurrentSkipListMap; @@ -32,23 +33,23 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hbase.CellComparator; import org.apache.commons.lang.NotImplementedException; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.hbase.Cell; +import org.apache.hadoop.hbase.CellComparator; +import org.apache.hadoop.hbase.CellUtil; import org.apache.hadoop.hbase.HBaseConfiguration; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.HRegionInfo; -import org.apache.hadoop.hbase.HRegionLocation; -import org.apache.hadoop.hbase.KeyValue; import org.apache.hadoop.hbase.MetaTableAccessor; -import org.apache.hadoop.hbase.RegionLocations; import org.apache.hadoop.hbase.RegionTooBusyException; import org.apache.hadoop.hbase.ServerName; import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.CellProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.BulkLoadHFileRequest; @@ -89,10 +90,11 @@ import com.google.common.base.Stopwatch; import com.google.protobuf.ByteString; import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; +import com.google.protobuf.TextFormat; /** * Test client behavior w/o setting up a cluster. - * Mock up cluster emissions. + * Mock up cluster emissions. Mocked Types from this test are used by others in this package. */ @Category({ClientTests.class, SmallTests.class}) public class TestClientNoCluster extends Configured implements Tool { @@ -107,34 +109,7 @@ public class TestClientNoCluster extends Configured implements Tool { // Run my HConnection overrides. Use my little ConnectionImplementation below which // allows me insert mocks and also use my Registry below rather than the default zk based // one so tests run faster and don't have zk dependency. - this.conf.set("hbase.client.registry.impl", SimpleRegistry.class.getName()); - } - - /** - * Simple cluster registry inserted in place of our usual zookeeper based one. - */ - static class SimpleRegistry implements Registry { - final ServerName META_HOST = META_SERVERNAME; - - @Override - public void init(Connection connection) { - } - - @Override - public RegionLocations getMetaRegionLocation() throws IOException { - return new RegionLocations( - new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, META_HOST)); - } - - @Override - public String getClusterId() { - return HConstants.CLUSTER_ID_DEFAULT; - } - - @Override - public int getCurrentNrHRS() throws IOException { - return 1; - } + this.conf.set("hbase.client.registry.impl", SimpleNoZookeeperRegistry.class.getName()); } /** @@ -180,7 +155,7 @@ public class TestClientNoCluster extends Configured implements Tool { localConfig.set("hbase.client.connection.impl", RpcTimeoutConnection.class.getName()); int pause = 10; localConfig.setInt("hbase.client.pause", pause); - localConfig.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 10); + localConfig.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3); // Set the operation timeout to be < the pause. Expectation is that after first pause, we will // fail out of the rpc because the rpc timeout will have been set to the operation tiemout // and it has expired. Otherwise, if this functionality is broke, all retries will be run -- @@ -321,15 +296,14 @@ public class TestClientNoCluster extends Configured implements Tool { /** * Fake many regionservers and many regions on a connection implementation. */ - static class ManyServersManyRegionsConnection - extends ConnectionImplementation { + static class ManyServersManyRegionsConnection extends ConnectionImplementation { // All access should be synchronized final Map serversByClient; /** * Map of faked-up rows of a 'meta table'. */ - final SortedMap> meta; + final NavigableMap> meta; final AtomicLong sequenceids = new AtomicLong(0); private final Configuration conf; @@ -393,12 +367,15 @@ public class TestClientNoCluster extends Configured implements Tool { */ static class FakeServer implements ClientService.BlockingInterface { private AtomicInteger multiInvocationsCount = new AtomicInteger(0); - private final SortedMap> meta; + private final NavigableMap> meta; private final AtomicLong sequenceids; private final long multiPause; private final int tooManyMultiRequests; + // Only one Scanner supported at the moment in spite of this being a Map. + private Map scanPosition = new HashMap(); - FakeServer(final Configuration c, final SortedMap> meta, + FakeServer(final Configuration c, + final NavigableMap> meta, final AtomicLong sequenceids) { this.meta = meta; this.sequenceids = sequenceids; @@ -430,17 +407,60 @@ public class TestClientNoCluster extends Configured implements Tool { } @Override - public MutateResponse mutate(RpcController controller, - MutateRequest request) throws ServiceException { - throw new NotImplementedException(); + public MutateResponse mutate(RpcController controller, MutateRequest request) + throws ServiceException { + return MutateResponse.newBuilder().build(); } @Override - public ScanResponse scan(RpcController controller, - ScanRequest request) throws ServiceException { - // Presume it is a scan of meta for now. Not all scans provide a region spec expecting - // the server to keep reference by scannerid. TODO. - return doMetaScanResponse(meta, sequenceids, request); + public ScanResponse scan(RpcController controller, ScanRequest request) + throws ServiceException { + if (LOG.isTraceEnabled()) { + LOG.trace("Scan " + TextFormat.shortDebugString(request)); + } + if (request.getRegion().hasType()) { + // We are in here if it is a Scan setup type of operation. + String regionName = Bytes.toStringBinary(request.getRegion().getValue().toByteArray()); + String firstMetaRegion = HRegionInfo.FIRST_META_REGIONINFO.getRegionNameAsString(); + if (firstMetaRegion.startsWith(regionName)) { + // Its opening a scanner on Meta. Presume the hbase:meta scan a short scan w/ all results + // returned here in one shot.. Scanner is done. + return doMetaScanResponse(meta, sequenceids, request); + } + // This is open scanner on the non-meta table. Just return empty response.. all is good. + // It'll come back to get data. + return ScanResponse.newBuilder().build(); + } + // Now we are scanning the non-meta table. Return START_ROW from each meta entry for we know + // for sure these rows are 'safe/known' for a particular region. + ScanResponse.Builder builder = ScanResponse.newBuilder(); + // Presume one scanner only at the moment. + long scanId = 0; + byte [] key = this.scanPosition.get(scanId); + if (key == null) { + key = this.meta.firstKey(); + } else { + NavigableMap> tail = this.meta.tailMap(key, false); + key = tail.size() <= 0? null/*We are at the end*/: tail.firstKey(); + } + if (key == null) { + // We are done. + builder.setMoreResults(false); + } else { + // Return the start key + Pair value = this.meta.get(key); + byte [] startKey = value.getFirst().getStartKey(); + if (startKey == null || startKey.length <= 0) { + startKey = Bytes.toBytes(""); + } + builder.addResults(ClientProtos.Result.newBuilder(). + addCell(ProtobufUtil. + toCell(CellUtil.createCell(startKey, startKey, startKey, 0, (byte)0, startKey))). + build()); + } + // Save off our current position. + this.scanPosition.put(scanId, key); + return builder.build(); } @Override @@ -480,13 +500,18 @@ public class TestClientNoCluster extends Configured implements Tool { } } - static ScanResponse doMetaScanResponse(final SortedMap> meta, + static ScanResponse doMetaScanResponse( + final NavigableMap> meta, final AtomicLong sequenceids, final ScanRequest request) { ScanResponse.Builder builder = ScanResponse.newBuilder(); int max = request.getNumberOfRows(); int count = 0; - Map> tail = - request.hasScan()? meta.tailMap(request.getScan().getStartRow().toByteArray()): meta; + // If Scan, check if it reversed or not. If a reverse Scan, then give a descending view on + // the passed in meta NavigableMap If NOT a Scan, return the Map as is. + Map> tail = request.hasScan()? + (request.getScan().hasReversed()? + meta.descendingMap().tailMap(request.getScan().getStartRow().toByteArray()): + meta.tailMap(request.getScan().getStartRow().toByteArray())): meta; ClientProtos.Result.Builder resultBuilder = ClientProtos.Result.newBuilder(); for (Map.Entry> e: tail.entrySet()) { // Can be 0 on open of a scanner -- i.e. rpc to setup scannerid only. @@ -576,7 +601,7 @@ public class TestClientNoCluster extends Configured implements Tool { return cellBuilder.build(); } - private static final byte [] BIG_USER_TABLE = Bytes.toBytes("t"); + static final byte [] BIG_USER_TABLE = Bytes.toBytes("t"); /** * Format passed integer. Zero-pad. @@ -638,7 +663,10 @@ public class TestClientNoCluster extends Configured implements Tool { private final CellComparator delegate = CellComparator.META_COMPARATOR; @Override public int compare(byte[] left, byte[] right) { - return delegate.compareRows(new KeyValue.KeyOnlyKeyValue(left), right, 0, right.length); + Cell l = CellUtil.createCell(left); + Cell r = CellUtil.createCell(right); + // LOG.info("Left=" + l + ", right=" + r); + return this.delegate.compareRows(l, r); } } @@ -647,10 +675,10 @@ public class TestClientNoCluster extends Configured implements Tool { * ServerName to return for this row. * @return Map with faked hbase:meta content in it. */ - static SortedMap> makeMeta(final byte [] tableName, + static NavigableMap> makeMeta(final byte [] tableName, final int regionCount, final long namespaceSpan, final int serverCount) { // I need a comparator for meta rows so we sort properly. - SortedMap> meta = + NavigableMap> meta = new ConcurrentSkipListMap>(new MetaRowsComparator()); HRegionInfo [] hris = makeHRegionInfos(tableName, regionCount, namespaceSpan); ServerName [] serverNames = makeServerNames(serverCount); @@ -740,7 +768,7 @@ public class TestClientNoCluster extends Configured implements Tool { getConf().set("hbase.client.connection.impl", ManyServersManyRegionsConnection.class.getName()); // Use simple kv registry rather than zk - getConf().set("hbase.client.registry.impl", SimpleRegistry.class.getName()); + getConf().set("hbase.client.registry.impl", SimpleNoZookeeperRegistry.class.getName()); // When to report fails. Default is we report the 10th. This means we'll see log everytime // an exception is thrown -- usually RegionTooBusyException when we have more than // hbase.test.multi.too.many requests outstanding at any time. @@ -797,4 +825,4 @@ public class TestClientNoCluster extends Configured implements Tool { public static void main(String[] args) throws Exception { System.exit(ToolRunner.run(HBaseConfiguration.create(), new TestClientNoCluster(), args)); } -} +} \ No newline at end of file diff --git a/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientTracingNoCluster.java b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientTracingNoCluster.java new file mode 100644 index 0000000..fd2ef57 --- /dev/null +++ b/hbase-client/src/test/java/org/apache/hadoop/hbase/client/TestClientTracingNoCluster.java @@ -0,0 +1,388 @@ +package org.apache.hadoop.hbase.client; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.CategoryBasedTimeout; +import org.apache.hadoop.hbase.HBaseConfiguration; +import org.apache.hadoop.hbase.TableName; +import org.apache.hadoop.hbase.client.TestClientNoCluster.ManyServersManyRegionsConnection; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.testclassification.ClientTests; +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Threads; +import org.apache.htrace.core.AlwaysSampler; +import org.apache.htrace.core.HTraceConfiguration; +import org.apache.htrace.core.POJOSpanReceiver; +import org.apache.htrace.core.Span; +import org.apache.htrace.core.TraceScope; +import org.apache.htrace.core.Tracer; +import org.apache.htrace.core.TracerPool; +import org.apache.log4j.Level; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TestName; +import org.junit.rules.TestRule; + +/** + * Some basic test of tracing the hbase client. Does only subset of Client functions. Does not + * do Admin. We only have a mocked ReglonServer in here. We do not have a mocked Master. + * + * DOC: To enable a sampler on construction of a Tracer, pass it in the configuration on the + * Tracer.SAMPLER_CLASSES_KEY config. + * DOC: What happens when two tracers in a context. + * TODO: Need a nice formatter for POJO receiver that outputs in order. + * TODO: Admin functions need mocking of a Master + * TODO: Three meta lookups? Even on real cluster? Looks like we are caching locations at least. + * See the Get test in the below. + * TODO: Two flushCommits when we call Puts with a List? + */ +@Category({ClientTests.class, SmallTests.class}) +public class TestClientTracingNoCluster { + private static final Log LOG = LogFactory.getLog(TestClientNoCluster.class); + static { + // Enable to track tracing better. + // ((Log4JLogger)LogFactory.getLog("org.apache.htrace.core")).getLogger().setLevel(Level.ALL); + }; + @Rule public final TestName testName = new TestName(); + @Rule public final TestRule timeout = CategoryBasedTimeout.builder(). + withTimeout(this.getClass()).withLookingForStuckThread(true).build(); + private Configuration conf; + private Connection connection; + private final TableName TABLENAME = TableName.valueOf(TestClientNoCluster.BIG_USER_TABLE); + private final int REGION_COUNT = 100; + /** + * Tracer for the test itself. Runs with Tracer.SAMPLER_CLASSES_KEY = AlwaysSampler. + */ + private Tracer testTracer; + + @Before + public void setUp() throws Exception { + this.conf = HBaseConfiguration.create(); + // Create a test tracer that is Always sampling and that uses the POJOSpanReceiver. See the + // below 'get' method for where we return appropriate classes for sampling and receiving. + this.testTracer = new Tracer.Builder("test").conf(new HTraceConfiguration() { + @Override + public String get(String key, String defaultValue) { + LOG.info("key=" + key + ", value=" + defaultValue); + if (key.equals(Tracer.SAMPLER_CLASSES_KEY)) { + String sampler = AlwaysSampler.class.getCanonicalName(); + LOG.info("For key=" + key + ", returning " + sampler); + return sampler; + } + if (key.equals(Tracer.SPAN_RECEIVER_CLASSES_KEY)) { + String receiver = POJOSpanReceiver.class.getCanonicalName(); + LOG.info("For key=" + key + ", returning " + receiver); + return receiver; + } + return defaultValue; + } + + @Override + public String get(String key) { + LOG.info("key=" + key); + return null; + } + }).build(); + + // Put in place mocked Connection and simple registry so can do w/o a cluster. + this.conf.set("hbase.client.registry.impl", SimpleNoZookeeperRegistry.class.getName()); + this.conf.set("hbase.client.connection.impl", ManyServersManyRegionsConnection.class.getName()); + // When to report fails. Default is we report the 10th. This means we'll see log everytime + // an exception is thrown -- usually RegionTooBusyException when we have more than + // hbase.test.multi.too.many requests outstanding at any time. + this.conf.setInt("hbase.client.start.log.errors.counter", 0); + // Ugly but this is only way to pass in configs.into ManyServersManyRegionsConnection class. + this.conf.setInt("hbase.test.regions", REGION_COUNT); + this.conf.setLong("hbase.test.namespace.span", 50000000); + this.conf.setLong("hbase.test.servers", 1); + this.conf.set("hbase.test.tablename", TABLENAME.toString()); + this.conf.setLong("hbase.test.multi.pause.when.done", 0); + // Let there be ten outstanding requests at a time before we throw RegionBusyException. + this.conf.setInt("hbase.test.multi.too.many", 10); + + // Have them all share the same connection so they all share the same instance of + // ManyServersManyRegionsConnection so I can keep an eye on how many requests by server. + final ExecutorService pool = Executors.newCachedThreadPool(Threads.getNamedThreadFactory("p")); + // Executors.newFixedThreadPool(servers * 10, Threads.getNamedThreadFactory("p")); + // Share a connection so I can keep counts in the 'server' on concurrency. + this.connection = ConnectionFactory.createConnection(this.conf, pool); + } + + @After + public void tearDown() throws Exception { + // Dump all spans. Presume only the one receiver of type POJOSpanReceiver. + POJOSpanReceiver spanReceiver = + (POJOSpanReceiver)TracerPool.getGlobalTracerPool().getReceivers()[0]; + // Sort by start time before we print. + Collection spans = spanReceiver.getSpans(); + List spanList = new ArrayList(spans); + Collections.sort(spanList, new Comparator() { + @Override + public int compare(Span left, Span right) { + return (int)(left.getStartTimeMillis() - right.getStartTimeMillis()); + } + }); + for (Span span: spanList) { + LOG.info(span); + } + this.connection.close(); + this.testTracer.close(); + } + + private void expectedSpanCount(final int expected) { + POJOSpanReceiver spanReceiver = + (POJOSpanReceiver)TracerPool.getGlobalTracerPool().getReceivers()[0]; + assertEquals("Number of traces expected", expected, spanReceiver.getSpans().size()); + } + + @Test + public void testConnectionTracer() { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + Tracer tracer = ((ClusterConnection)this.connection).getTracer(); + assertNotNull(tracer); + assertNotNull(tracer.toString()); + } + // One for the test scope. + expectedSpanCount(1); + } + + /** + * Make sure the handoff across two tracers happens; the test tracer to the connection tracer. + * @throws IllegalArgumentException + * @throws IOException + */ + @Test + public void testTable() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + LOG.info(table); + } + } + // One for the test and another for the creation of the table instance and its region locator + expectedSpanCount(2); + } + + @Test + public void testTableGet() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + table.get(new Get(Bytes.toBytes("x"))); + expectedSpanCount(5); + // See if we cache meta locations or not. + // Sleep one millisecond so next Span is 'different' (has different start time) else + // the simple POJOSpanReceiver will overwrite. + Threads.sleep(1); + table.get(new Get(Bytes.toBytes("y"))); + expectedSpanCount(6); + } + } + expectedSpanCount(8); + } + + @Test + public void testTableGets() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + List gets = new ArrayList(); + // Add more than one else it goes the 'get' route. + gets.add(new Get(Bytes.toBytes("x"))); + gets.add(new Get(Bytes.toBytes("y"))); + table.get(gets); + } + } + expectedSpanCount(10); + } + + @Test + public void testTableDelete() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + table.delete(new Delete(Bytes.toBytes("x"))); + } + } + expectedSpanCount(7); + } + + @Test + public void testTableDeletes() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + List deletes = new ArrayList(); + deletes.add(new Delete(Bytes.toBytes("x"))); + deletes.add(new Delete(Bytes.toBytes("y"))); + table.delete(deletes); + } + } + expectedSpanCount(10); + } + + @Test + public void testTablePut() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + table.put(makePut(Bytes.toBytes("x"))); + } + } + expectedSpanCount(10); + } + + @Test + public void testTablePuts() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + List puts = new ArrayList(); + puts.add(makePut(Bytes.toBytes("x"))); + puts.add(makePut(Bytes.toBytes("y"))); + table.put(puts); + } + } + expectedSpanCount(10); + } + + private static Put makePut(final byte [] bytes) { + Put put = new Put(bytes); + put.addColumn(bytes, bytes, bytes); + return put; + } + + @Test + public void testBufferedMutator() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + try (BufferedMutator bufferedMutator = + this.connection.getBufferedMutator(table.getName())) { + bufferedMutator.mutate(makePut(Bytes.toBytes("x"))); + bufferedMutator.mutate(makePut(Bytes.toBytes("y"))); + // See if we cache meta locations or not. + // Sleep one millisecond so next Span is 'different' (has different start time) else + // the simple POJOSpanReceiver will overwrite. + Threads.sleep(1); + bufferedMutator.flush(); + bufferedMutator.mutate(makePut(Bytes.toBytes("x"))); + bufferedMutator.mutate(makePut(Bytes.toBytes("y"))); + // See if we cache meta locations or not. + // Sleep one millisecond so next Span is 'different' (has different start time) else + // the simple POJOSpanReceiver will overwrite. + Threads.sleep(1); + } + } + } + expectedSpanCount(12); + } + + @Test + public void testTableScan() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + Scan scan = new Scan(); + // Set caching to 1 so we get one Cell at a time. + scan.setCaching(1); + try (ResultScanner scanner = table.getScanner(scan)) { + Result result = null; + while ((result = scanner.next()) != null) { + LOG.info(result); + } + } + } + } + expectedSpanCount(REGION_COUNT + 9); + } + + @Test + public void testTableAppend() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + table.append(makeAppend(Bytes.toBytes("x"))); + } + } + expectedSpanCount(7); + } + + private static Append makeAppend(final byte [] bytes) { + Append append = new Append(bytes); + append.add(bytes, bytes, bytes); + return append; + } + + @Test + public void testTableIncrement() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + table.increment(makeIncrement(Bytes.toBytes("x"))); + } + } + expectedSpanCount(7); + } + + private static Increment makeIncrement(final byte [] bytes) { + Increment increment = new Increment(bytes); + increment.addColumn(bytes, bytes, 1); + return increment; + } + + @Test + public void testTableIncrementColumnValue() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + byte [] bytes = Bytes.toBytes("x"); + table.incrementColumnValue(bytes, bytes, bytes, 1); + } + } + expectedSpanCount(7); + } + + @Test + public void testCheckAndPut() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + byte [] bytes = Bytes.toBytes("x"); + Put put = makePut(Bytes.toBytes("y")); + table.checkAndPut(bytes, bytes, bytes, bytes, put); + } + } + expectedSpanCount(7); + } + + @Test + public void testCheckAndDelete() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + byte [] bytes = Bytes.toBytes("x"); + table.checkAndDelete(bytes, bytes, bytes, bytes, new Delete(Bytes.toBytes("y"))); + } + } + expectedSpanCount(7); + } + + @Test + public void testCheckAndMutate() throws IllegalArgumentException, IOException { + try (TraceScope traceScope = this.testTracer.newScope(testName.getMethodName())) { + try (Table table = this.connection.getTable(TABLENAME)) { + byte [] bytes = Bytes.toBytes("x"); + RowMutations rms = new RowMutations(bytes); + rms.add(makePut(bytes)); + // Don't add another RM here; it will hang the test. Server only returns one result. Fix. + table.checkAndMutate(bytes, bytes, bytes, CompareOp.EQUAL, bytes, rms); + } + } + expectedSpanCount(8); + } +} \ No newline at end of file -- 2.6.1