[ 
https://issues.apache.org/jira/browse/HBASE-20236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16418453#comment-16418453
 ] 

stack commented on HBASE-20236:
-------------------------------

I just tried a run where we skip the Semaphore and just go to the queue 
instead. Seems to run faster, about 78k/second vs 73k/second doing workloadc 
(100% reads). 1.2.7 does 87k ops/second.

Here is patch:
{code}
diff --git 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
 b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPat
index eaea34d..4a65ada 100644
--- 
a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
+++ 
b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/FastPathBalancedQueueRpcExecutor.java
@@ -80,7 +80,7 @@ public class FastPathBalancedQueueRpcExecutor extends 
BalancedQueueRpcExecutor {
     // if an empty queue of CallRunners so we are available for direct handoff 
when one comes in.
     final Deque<FastPathHandler> fastPathHandlerStack;
     // Semaphore to coordinate loading of fastpathed loadedTask and our 
running it.
-    private Semaphore semaphore = new Semaphore(0);
+    // private Semaphore semaphore = new Semaphore(0);
     // The task we get when fast-pathing.
     private CallRunner loadedCallRunner;

@@ -98,6 +98,7 @@ public class FastPathBalancedQueueRpcExecutor extends 
BalancedQueueRpcExecutor {
       if (cr == null) {
         // Else, if a fastPathHandlerStack present and no callrunner in Q, 
register ourselves for
         // the fastpath handoff done via fastPathHandlerStack.
+        /*
         if (this.fastPathHandlerStack != null) {
           this.fastPathHandlerStack.push(this);
           this.semaphore.acquire();
@@ -105,8 +106,11 @@ public class FastPathBalancedQueueRpcExecutor extends 
BalancedQueueRpcExecutor {
           this.loadedCallRunner = null;
         } else {
           // No fastpath available. Block until a task comes available.
+          // */
           cr = super.getCallRunner();
+/*
         }
+        */
       }
       return cr;
     }
@@ -116,9 +120,12 @@ public class FastPathBalancedQueueRpcExecutor extends 
BalancedQueueRpcExecutor {
      * @return True if we successfully loaded our task
      */
     boolean loadCallRunner(final CallRunner cr) {
+      /*
       this.loadedCallRunner = cr;
       this.semaphore.release();
       return true;
+      */
+      return false;
     }
   }
 }
{code}

YCSB figures:
{code}
 {
  "metric" : "OVERALL",
  "measurement" : "RunTime(ms)",
  "value" : 1200135
}, {
  "metric" : "OVERALL",
  "measurement" : "Throughput(ops/sec)",
  "value" : 78433.52456182013
},
{code}

So, Semaphore is broke or I imagined a speedup doing fastpath?

Here is what locking profile looks like w/ no Semaphore:

{code}
1 Started [lock] profiling
   2 --- Execution profile ---
   3 Total samples:         113365
   4
   5 Frame buffer usage:    0.173%
   6
   7 Total: 15130846044 (61.74%)  samples: 1
   8   [ 0] java.lang.Object
   9   [ 1] sun.nio.ch.ServerSocketChannelImpl.accept
  10   [ 2] org.eclipse.jetty.server.ServerConnector.accept
  11   [ 3] org.eclipse.jetty.server.AbstractConnector$Acceptor.run
  12   [ 4] org.eclipse.jetty.util.thread.QueuedThreadPool.runJob
  13   [ 5] org.eclipse.jetty.util.thread.QueuedThreadPool$2.run
  14   [ 6] java.lang.Thread.run
  15
  16 Total: 3752193080 (15.31%)  samples: 44234
  17   [ 0] java.util.concurrent.locks.ReentrantLock$NonfairSync
  18   [ 1] sun.misc.Unsafe.park
  19   [ 2] java.util.concurrent.locks.LockSupport.park
  20   [ 3] 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt
  21   [ 4] java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued
  22   [ 5] java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire
  23   [ 6] java.util.concurrent.locks.ReentrantLock$NonfairSync.lock
  24   [ 7] java.util.concurrent.locks.ReentrantLock.lock
  25   [ 8] java.util.concurrent.LinkedBlockingQueue.poll
  26   [ 9] 
org.apache.hadoop.hbase.ipc.FastPathBalancedQueueRpcExecutor$FastPathHandler.getCallRunner
  27   [10] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
  28
  29 Total: 1277795387 (5.21%)  samples: 4895
  30   [ 0] java.util.concurrent.ConcurrentHashMap
  31   [ 1] org.apache.hadoop.hbase.regionserver.HRegion.getScanner
  32   [ 2] org.apache.hadoop.hbase.regionserver.HRegion.getScanner
  33   [ 3] org.apache.hadoop.hbase.regionserver.HRegion.getScanner
  34   [ 4] org.apache.hadoop.hbase.regionserver.RSRpcServices.get
  35   [ 5] org.apache.hadoop.hbase.regionserver.RSRpcServices.get
  36   [ 6] 
org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod
  37   [ 7] org.apache.hadoop.hbase.ipc.RpcServer.call
  38   [ 8] org.apache.hadoop.hbase.ipc.CallRunner.run
  39   [ 9] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
  40   [10] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
  41
  42 Total: 1156160619 (4.72%)  samples: 15310
  43   [ 0] java.util.concurrent.locks.ReentrantLock$NonfairSync
  44   [ 1] sun.misc.Unsafe.park
  45   [ 2] java.util.concurrent.locks.LockSupport.park
  46   [ 3] 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt
  47   [ 4] 
java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireInterruptibly
  48   [ 5] 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly
  49   [ 6] java.util.concurrent.locks.ReentrantLock.lockInterruptibly
  50   [ 7] java.util.concurrent.LinkedBlockingQueue.take
  51   [ 8] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.getCallRunner
  52   [ 9] 
org.apache.hadoop.hbase.ipc.FastPathBalancedQueueRpcExecutor$FastPathHandler.getCallRunner
  53   [10] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
  54
  55 Total: 825752851 (3.37%)  samples: 9624
  56   [ 0] java.util.concurrent.locks.ReentrantLock$NonfairSync
  57   [ 1] sun.misc.Unsafe.park
  58   [ 2] java.util.concurrent.locks.LockSupport.park
  59   [ 3] 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt
  60   [ 4] java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued
  61   [ 5] java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire
  62   [ 6] java.util.concurrent.locks.ReentrantLock$NonfairSync.lock
  63   [ 7] java.util.concurrent.locks.ReentrantLock.lock
  64   [ 8] java.util.concurrent.LinkedBlockingQueue.signalNotEmpty
  65   [ 9] java.util.concurrent.LinkedBlockingQueue.offer
  66   [10] org.apache.hadoop.hbase.ipc.BalancedQueueRpcExecutor.dispatch
  67   [11] 
org.apache.hadoop.hbase.ipc.FastPathBalancedQueueRpcExecutor.dispatch
  68   [12] org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.dispatch
  69   [13] org.apache.hadoop.hbase.ipc.ServerRpcConnection.processRequest
  70   [14] org.apache.hadoop.hbase.ipc.ServerRpcConnection.processOneRpc
  71   [15] org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process
  72   [16] org.apache.hadoop.hbase.ipc.NettyServerRpcConnection.process
  73   [17] org.apache.hadoop.hbase.ipc.NettyRpcServerRequestDecoder.channelRead
  74   [18] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead
  75   [19] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead
  76   [20] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead
  77   [21] 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead
  78   [22] 
org.apache.hbase.thirdparty.io.netty.handler.codec.ByteToMessageDecoder.channelRead
  79   [23] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead
  80   [24] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead
  81   [25] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.fireChannelRead
  82   [26] 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline$HeadContext.channelRead
  83   [27] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead
  84   [28] 
org.apache.hbase.thirdparty.io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead
  85   [29] 
org.apache.hbase.thirdparty.io.netty.channel.DefaultChannelPipeline.fireChannelRead
  86   [30] 
org.apache.hbase.thirdparty.io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady
  87   [31] 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.processReady
  88   [32] 
org.apache.hbase.thirdparty.io.netty.channel.epoll.EpollEventLoop.run
  89   [33] 
org.apache.hbase.thirdparty.io.netty.util.concurrent.SingleThreadEventExecutor$5.run
  90   [34] 
org.apache.hbase.thirdparty.io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run
  91   [35] java.lang.Thread.run
  92
  93 Total: 393702335 (1.61%)  samples: 2545
  94   [ 0] org.apache.hadoop.hdfs.PeerCache
  95   [ 1] java.lang.String.hashCode
  96   [ 2] org.apache.hadoop.hdfs.protocol.DatanodeID.hashCode
  97   [ 3] org.apache.hadoop.hdfs.protocol.DatanodeInfo.hashCode
  98   [ 4] org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage.hashCode
  99   [ 5] org.apache.hadoop.hdfs.PeerCache$Key.hashCode
 100   [ 6] java.util.HashMap.hash
 101   [ 7] java.util.HashMap.put
 102   [ 8] com.google.common.collect.LinkedListMultimap.addNode
 103   [ 9] com.google.common.collect.LinkedListMultimap.put
 104   [10] org.apache.hadoop.hdfs.PeerCache.putInternal
 105   [11] org.apache.hadoop.hdfs.PeerCache.put
 106   [12] org.apache.hadoop.hdfs.RemoteBlockReader2.close
 107   [13] org.apache.hadoop.hdfs.DFSInputStream.actualGetFromOneDataNode
 108   [14] org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange
 109   [15] org.apache.hadoop.hdfs.DFSInputStream.pread
 110   [16] org.apache.hadoop.hdfs.DFSInputStream.read
 111   [17] org.apache.hadoop.fs.FSDataInputStream.read
 112   [18] org.apache.hadoop.hbase.io.hfile.HFileBlock.positionalReadWithExtra
 113   [19] 
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readAtOffset
 114   [20] 
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockDataInternal
 115   [21] 
org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderImpl.readBlockData
 116   [22] org.apache.hadoop.hbase.io.hfile.HFileReaderImpl.readBlock
....
{code}
Let me post a picture of No-Semaphore vs Semaphore (current branch-2.0 tip).

> [locking] Write-time worst offenders
> ------------------------------------
>
>                 Key: HBASE-20236
>                 URL: https://issues.apache.org/jira/browse/HBASE-20236
>             Project: HBase
>          Issue Type: Sub-task
>          Components: Performance
>    Affects Versions: 2.0.0-beta-2
>            Reporter: stack
>            Priority: Major
>
> Messing w/ my new toy, here are worst offenders locking; they must be bad if 
> they show up in this sampling profiler:
> {code}
>  7 Total: 769321884622 (99.24%)  samples: 2965
>   8   [ 0] java.util.concurrent.Semaphore$NonfairSync
>   9   [ 1] sun.misc.Unsafe.park
>  10   [ 2] java.util.concurrent.locks.LockSupport.park
>  11   [ 3] 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt
>  12   [ 4] 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly
>  13   [ 5] 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly
>  14   [ 6] java.util.concurrent.Semaphore.acquire
>  15   [ 7] 
> org.apache.hadoop.hbase.ipc.FastPathBalancedQueueRpcExecutor$FastPathHandler.getCallRunner
>  16   [ 8] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
>  17
>  18 Total: 4284274263 (0.55%)  samples: 23543
>  19   [ 0] org.apache.hadoop.hbase.regionserver.MutableSegment
>  20   [ 1] org.apache.hadoop.hbase.ByteBufferKeyValue.getSequenceId
>  21   [ 2] org.apache.hadoop.hbase.regionserver.Segment.updateMetaInfo
>  22   [ 3] org.apache.hadoop.hbase.regionserver.Segment.internalAdd
>  23   [ 4] org.apache.hadoop.hbase.regionserver.MutableSegment.add
>  24   [ 5] org.apache.hadoop.hbase.regionserver.AbstractMemStore.internalAdd
>  25   [ 6] org.apache.hadoop.hbase.regionserver.AbstractMemStore.add
>  26   [ 7] org.apache.hadoop.hbase.regionserver.AbstractMemStore.add
>  27   [ 8] org.apache.hadoop.hbase.regionserver.HStore.add
>  28   [ 9] org.apache.hadoop.hbase.regionserver.HRegion.applyToMemStore
>  29   [10] org.apache.hadoop.hbase.regionserver.HRegion.access$600
>  30   [11] 
> org.apache.hadoop.hbase.regionserver.HRegion$BatchOperation.applyFamilyMapToMemStore
>  31   [12] 
> org.apache.hadoop.hbase.regionserver.HRegion$BatchOperation.lambda$writeMiniBatchOperationsToMemStore$0
>  32   [13] 
> org.apache.hadoop.hbase.regionserver.HRegion$BatchOperation$$Lambda$442.1445825895.visit
>  33   [14] 
> org.apache.hadoop.hbase.regionserver.HRegion$BatchOperation.visitBatchOperations
>  34   [15] 
> org.apache.hadoop.hbase.regionserver.HRegion$BatchOperation.writeMiniBatchOperationsToMemStore
>  35   [16] 
> org.apache.hadoop.hbase.regionserver.HRegion$MutationBatchOperation.writeMiniBatchOperationsToMemStore
>  36   [17] org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutate
>  37   [18] org.apache.hadoop.hbase.regionserver.HRegion.batchMutate
>  38   [19] org.apache.hadoop.hbase.regionserver.HRegion.batchMutate
>  39   [20] org.apache.hadoop.hbase.regionserver.RSRpcServices.doBatchOp
>  40   [21] 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicBatchOp
>  41   [22] 
> org.apache.hadoop.hbase.regionserver.RSRpcServices.doNonAtomicRegionMutation
>  42   [23] org.apache.hadoop.hbase.regionserver.RSRpcServices.multi
>  43   [24] 
> org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod
>  44   [25] org.apache.hadoop.hbase.ipc.RpcServer.call
>  45   [26] org.apache.hadoop.hbase.ipc.CallRunner.run
>  46   [27] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
>  47   [28] org.apache.hadoop.hbase.ipc.RpcExecutor$Handler.run
>  48
>  49 Total: 717708856 (0.09%)  samples: 214
>  50   [ 0] java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync
>  51   [ 1] sun.misc.Unsafe.park
>  52   [ 2] java.util.concurrent.locks.LockSupport.park
>  53   [ 3] 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt
>  54   [ 4] java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued
>  55   [ 5] java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire
>  56   [ 6] java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock
>  57   [ 7] org.apache.hadoop.hbase.regionserver.HRegion.blockUpdates
>  58   [ 8] 
> org.apache.hadoop.hbase.regionserver.RegionServicesForStores.blockUpdates
>  59   [ 9] 
> org.apache.hadoop.hbase.regionserver.CompactingMemStore.flushInMemory
>  60   [10] 
> org.apache.hadoop.hbase.regionserver.CompactingMemStore$InMemoryFlushRunnable.run
>  61   [11] java.util.concurrent.ThreadPoolExecutor.runWorker
>  62   [12] java.util.concurrent.ThreadPoolExecutor$Worker.run
>  63   [13] java.lang.Thread.run
> ...
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to