hbase-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "stack (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (HBASE-8755) A new write thread model for HLog to improve the overall HBase write throughput
Date Thu, 12 Dec 2013 07:00:17 GMT

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

stack commented on HBASE-8755:
------------------------------

Hmm... Happens when this patch is in place.  Stuck here:

{code}
"regionserver60020.logRoller" daemon prio=10 tid=0x00007f6f08822800 nid=0x5b0a in Object.wait()
[0x00007f6eeccef000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        at java.lang.Object.wait(Object.java:503)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.syncer(FSHLog.java:1304)
        - locked <0x000000045756db98> (a java.util.concurrent.atomic.AtomicLong)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.syncer(FSHLog.java:1296)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1409)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.cleanupCurrentWriter(FSHLog.java:759)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.rollWriter(FSHLog.java:565)
        - locked <0x000000045756dc70> (a java.lang.Object)
        - locked <0x000000045756dc10> (a java.lang.Object)
        at org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:96)
        at java.lang.Thread.run(Thread.java:744)
{code}

Which is here:

1294   // sync all known transactions
1295   private void syncer() throws IOException {
1296     syncer(this.unflushedEntries.get()); // sync all pending items
1297   }
1298
1299   // sync all transactions upto the specified txid
1300   private void syncer(long txid) throws IOException {
1301     synchronized (this.syncedTillHere) {
1302       while (this.syncedTillHere.get() < txid) {
1303         try {
1304           this.syncedTillHere.wait();
1305
1306           if (txid <= this.failedTxid.get()) {
1307             assert asyncIOE != null :
1308               "current txid is among(under) failed txids, but asyncIOE is null!";
1309             throw asyncIOE;
1310           }
1311         } catch (InterruptedException e) {
1312           LOG.debug("interrupted while waiting for notification from AsyncNotifier");
1313         }
1314       }
1315     }
1316   }

All other threads are trying to do an appendnosync:

{code}
"RpcServer.handler=0,port=60020" daemon prio=10 tid=0x00007f6f08a26800 nid=0x5b1b waiting
for monitor entry [0x00007f6eebee1000]
   java.lang.Thread.State: BLOCKED (on object monitor)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:1005)
        - waiting to lock <0x000000045756dc70> (a java.lang.Object)
        at org.apache.hadoop.hbase.regionserver.wal.FSHLog.appendNoSync(FSHLog.java:1053)
        at org.apache.hadoop.hbase.regionserver.HRegion.doMiniBatchMutation(HRegion.java:2369)
        at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2087)
        at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2037)
        at org.apache.hadoop.hbase.regionserver.HRegion.batchMutate(HRegion.java:2041)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.doBatchOp(HRegionServer.java:4175)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.doNonAtomicRegionMutation(HRegionServer.java:3424)
        at org.apache.hadoop.hbase.regionserver.HRegionServer.multi(HRegionServer.java:3328)
        at org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:28460)
        at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2008)
        at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:92)
        at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.consumerLoop(SimpleRpcScheduler.java:160)
        at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler.access$000(SimpleRpcScheduler.java:38)
        at org.apache.hadoop.hbase.ipc.SimpleRpcScheduler$1.run(SimpleRpcScheduler.java:110)
        at java.lang.Thread.run(Thread.java:744)
{code}

... but can't make progress blocked on updateLock.

{code}
 995   private long append(HRegionInfo info, TableName tableName, WALEdit edits, List<UUID>
clusterIds,
 996       final long now, HTableDescriptor htd, boolean doSync, boolean isInMemstore,
 997       AtomicLong sequenceId, long nonceGroup, long nonce) throws IOException {
 998       if (edits.isEmpty()) return this.unflushedEntries.get();
 999       if (this.closed) {
1000         throw new IOException("Cannot append; log is closed");
1001       }
1002       TraceScope traceScope = Trace.startSpan("FSHlog.append");
1003       try {
1004         long txid = 0;
1005         synchronized (this.updateLock) {
1006           // get the sequence number from the passed Long. In normal flow, it is coming
from the
1007           // region.
1008           long seqNum = sequenceId.incrementAndGet();
...
{code}

The update lock is held when rolling log here:

 562         synchronized (updateLock) {
 563           // Clean up current writer.
 564           oldNumEntries = this.numEntries.get();
 565           oldFile = cleanupCurrentWriter(currentFilenum);
 566           this.writer = nextWriter;
 567           this.hdfs_out = nextHdfsOut;
 568           this.numEntries.set(0);
 569           if (oldFile != null) {
 570             this.hlogSequenceNums.put(oldFile, this.latestSequenceNums);
 571             this.latestSequenceNums = new HashMap<byte[], Long>();
 572           }
 573         }



> A new write thread model for HLog to improve the overall HBase write throughput
> -------------------------------------------------------------------------------
>
>                 Key: HBASE-8755
>                 URL: https://issues.apache.org/jira/browse/HBASE-8755
>             Project: HBase
>          Issue Type: Improvement
>          Components: Performance, wal
>            Reporter: Feng Honghua
>            Assignee: stack
>            Priority: Critical
>         Attachments: 8755-syncer.patch, 8755trunkV2.txt, HBASE-8755-0.94-V0.patch, HBASE-8755-0.94-V1.patch,
HBASE-8755-0.96-v0.patch, HBASE-8755-trunk-V0.patch, HBASE-8755-trunk-V1.patch, HBASE-8755-trunk-v4.patch,
HBASE-8755-trunk-v6.patch, HBASE-8755-trunk-v7.patch, HBASE-8755-v5.patch
>
>
> In current write model, each write handler thread (executing put()) will individually
go through a full 'append (hlog local buffer) => HLog writer append (write to hdfs) =>
HLog writer sync (sync hdfs)' cycle for each write, which incurs heavy race condition on updateLock
and flushLock.
> The only optimization where checking if current syncTillHere > txid in expectation
for other thread help write/sync its own txid to hdfs and omitting the write/sync actually
help much less than expectation.
> Three of my colleagues(Ye Hangjun / Wu Zesheng / Zhang Peng) at Xiaomi proposed a new
write thread model for writing hdfs sequence file and the prototype implementation shows a
4X improvement for throughput (from 17000 to 70000+). 
> I apply this new write thread model in HLog and the performance test in our test cluster
shows about 3X throughput improvement (from 12150 to 31520 for 1 RS, from 22000 to 70000 for
5 RS), the 1 RS write throughput (1K row-size) even beats the one of BigTable (Precolator
published in 2011 says Bigtable's write throughput then is 31002). I can provide the detailed
performance test results if anyone is interested.
> The change for new write thread model is as below:
>  1> All put handler threads append the edits to HLog's local pending buffer; (it notifies
AsyncWriter thread that there is new edits in local buffer)
>  2> All put handler threads wait in HLog.syncer() function for underlying threads
to finish the sync that contains its txid;
>  3> An single AsyncWriter thread is responsible for retrieve all the buffered edits
in HLog's local pending buffer and write to the hdfs (hlog.writer.append); (it notifies AsyncFlusher
thread that there is new writes to hdfs that needs a sync)
>  4> An single AsyncFlusher thread is responsible for issuing a sync to hdfs to persist
the writes by AsyncWriter; (it notifies the AsyncNotifier thread that sync watermark increases)
>  5> An single AsyncNotifier thread is responsible for notifying all pending put handler
threads which are waiting in the HLog.syncer() function
>  6> No LogSyncer thread any more (since there is always AsyncWriter/AsyncFlusher threads
do the same job it does)



--
This message was sent by Atlassian JIRA
(v6.1.4#6159)

Mime
View raw message