HBase
  1. HBase
  2. HBASE-8755

A new write thread model for HLog to improve the overall HBase write throughput

    Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Critical Critical
    • Resolution: Fixed
    • Affects Version/s: None
    • Fix Version/s: 0.98.0, 0.99.0
    • Component/s: Performance, wal
    • Labels:
      None
    • Hadoop Flags:
      Reviewed
    • Release Note:
      Redo of thread model writing edits to the WAL; slower when few clients but as concurrency rises, it makes for better throughput.

      Description

      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)

      1. 8755v9.txt
        28 kB
        stack
      2. 8755v8.txt
        29 kB
        stack
      3. thread.out
        434 kB
        stack
      4. HBASE-8755-trunk-v7.patch
        28 kB
        Honghua Feng
      5. HBASE-8755-trunk-v6.patch
        26 kB
        Honghua Feng
      6. HBASE-8755-v5.patch
        27 kB
        Himanshu Vashishtha
      7. 8755-syncer.patch
        16 kB
        Himanshu Vashishtha
      8. HBASE-8755-trunk-v4.patch
        26 kB
        Honghua Feng
      9. HBASE-8755-0.96-v0.patch
        26 kB
        Honghua Feng
      10. 8755trunkV2.txt
        25 kB
        stack
      11. HBASE-8755-trunk-V1.patch
        25 kB
        Honghua Feng
      12. HBASE-8755-0.94-V1.patch
        27 kB
        Honghua Feng
      13. HBASE-8755-trunk-V0.patch
        26 kB
        Honghua Feng
      14. HBASE-8755-0.94-V0.patch
        26 kB
        Honghua Feng

        Issue Links

        There are no Sub-Tasks for this issue.

          Activity

          Hide
          Honghua Feng added a comment -
          Show
          Honghua Feng added a comment - the patch HBASE-8755 -0.94-V0.patch is based on http://svn.apache.org/repos/asf/hbase/branches/0.94
          Hide
          Ted Yu added a comment -
          +  private final AtomicLong failedTxid = new AtomicLong(0);
          

          Add some comment w.r.t. the purpose of above variable.

          -    // When optionalFlushInterval is set as 0, don't start a thread for deferred log sync.
          -    if (this.optionalFlushInterval > 0) {
          

          Is deferred log sync still supported ?

          +    } catch (InterruptedException e) {
          +      LOG.error("Exception while waiting for syncer thread to die", e);
          

          The error message needs to be updated.

          +      synchronized (bufferLock) {
          +        doWrite(regionInfo, logKey, logEdit, htd);
          +        txid = this.unflushedEntries.incrementAndGet();
          +      }
          +      this.asyncWriter.setPendingTxid(txid);
          

          Should setPendingTxid() call be protected by the bufferLock ?
          In setPendingTxid():

          +    public void setPendingTxid(long txid) {
          +      if (txid <= this.pendingTxid)
          +        return;
          

          What if two simultaneous calls with different txid take place ? Both txid are greater than this.pendingTxid, the smaller txid may be written last.

          In several comments, replace 'local buffered writes' with 'buffered writes'

          +          assert this.pendingTxid > this.lastWrittenTxid :
          +            "pendingTxid not greater than lastWrittenTxid when wake-up!";
          

          Include the two Txid's in the message above.

          +            asyncIOE = e;
          +            failedTxid.set(this.pendingTxid);
          +         }
          +
          +          // 4. update 'lastWrittenTxid' and notify AsyncFlusher to do 'sync'
          +          this.lastWrittenTxid = this.pendingTxid;
          

          In error condition, failedTxid is still assigned to this.lastWrittenTxid. Is that safe ?
          Since AsyncFlusher does 'sync', should it be named AsyncSyncer ?

          +      } catch (InterruptedException e) {
          +        LOG.debug(getName() + " interrupted while waiting for " +
          

          Please restore interrupt status.

          +  // to make durability of those WALEdits on HDFS side
          

          Should be 'to make those WALEdits durable on HDFS side'

          +          long now = System.currentTimeMillis();
          

          Use EnvironmentEdge.

                          requestLogRoll();
          +              }
          +            } catch (IOException e) {
          +              LOG.debug("writer.getLength() failed");
          

          Should the log level be at warn or error ? Is writer.getLength() relevant here ?

          AsyncNotifier does notification by calling syncedTillHere.notifyAll(). Can this part be folded into AsyncFlusher ?

          +  void addPendingWrites(Entry e) throws IOException {
          

          Rename the above method addPendingWrite() since there is only one Entry involved.

          +  // it's caller's responsibility to hold updateLock before call this method
          +  List<Entry> getPendingWrites() {
          

          bufferLock is held before calling the above method. Update comment accordingly.

          +        } catch (InterruptedException e) {
          +          LOG.debug("interrupted while waiting for notification from AsyncNotifier");
          

          Restore interrupt status.

          Show
          Ted Yu added a comment - + private final AtomicLong failedTxid = new AtomicLong(0); Add some comment w.r.t. the purpose of above variable. - // When optionalFlushInterval is set as 0, don't start a thread for deferred log sync. - if ( this .optionalFlushInterval > 0) { Is deferred log sync still supported ? + } catch (InterruptedException e) { + LOG.error( "Exception while waiting for syncer thread to die" , e); The error message needs to be updated. + synchronized (bufferLock) { + doWrite(regionInfo, logKey, logEdit, htd); + txid = this .unflushedEntries.incrementAndGet(); + } + this .asyncWriter.setPendingTxid(txid); Should setPendingTxid() call be protected by the bufferLock ? In setPendingTxid(): + public void setPendingTxid( long txid) { + if (txid <= this .pendingTxid) + return ; What if two simultaneous calls with different txid take place ? Both txid are greater than this.pendingTxid, the smaller txid may be written last. In several comments, replace 'local buffered writes' with 'buffered writes' + assert this .pendingTxid > this .lastWrittenTxid : + "pendingTxid not greater than lastWrittenTxid when wake-up!" ; Include the two Txid's in the message above. + asyncIOE = e; + failedTxid.set( this .pendingTxid); + } + + // 4. update 'lastWrittenTxid' and notify AsyncFlusher to do 'sync' + this .lastWrittenTxid = this .pendingTxid; In error condition, failedTxid is still assigned to this.lastWrittenTxid. Is that safe ? Since AsyncFlusher does 'sync', should it be named AsyncSyncer ? + } catch (InterruptedException e) { + LOG.debug(getName() + " interrupted while waiting for " + Please restore interrupt status. + // to make durability of those WALEdits on HDFS side Should be 'to make those WALEdits durable on HDFS side' + long now = System .currentTimeMillis(); Use EnvironmentEdge. requestLogRoll(); + } + } catch (IOException e) { + LOG.debug( "writer.getLength() failed" ); Should the log level be at warn or error ? Is writer.getLength() relevant here ? AsyncNotifier does notification by calling syncedTillHere.notifyAll(). Can this part be folded into AsyncFlusher ? + void addPendingWrites(Entry e) throws IOException { Rename the above method addPendingWrite() since there is only one Entry involved. + // it's caller's responsibility to hold updateLock before call this method + List<Entry> getPendingWrites() { bufferLock is held before calling the above method. Update comment accordingly. + } catch (InterruptedException e) { + LOG.debug( "interrupted while waiting for notification from AsyncNotifier" ); Restore interrupt status.
          Hide
          Honghua Feng added a comment -

          Thanks Ted Yu for the detailed code review, answers to some important questions as below:

          1) Deferred log sync is still supported, in which case the write handler thread in Region don't pend on HLog.syncer() waiting for its txid to be sync-ed. This behaviour keeps intact.

          2) Good catch, "if (txid <= this.pendingTxid) return;" in setPendingTxid() moved into below "synchronized (this.writeLock)

          {...}

          " is OK.
          bufferLock is used only to guarantee the access to HLog's local pending buffer and unflushedEntries can't be interleaved by multiple threads.

          3) "failedTxid is still assigned to this.lastWrittenTxid. Is that safe ?"
          Yes. all write pending on syncer() with txid <= failedTxid will get failure. The lastWrittenTxid can safely proceed without incorrect behaviour, and it need to proceed to eventually wake-up pending write handler threads.

          I'll update the patch per your review tomorrow.

          Show
          Honghua Feng added a comment - Thanks Ted Yu for the detailed code review, answers to some important questions as below: 1) Deferred log sync is still supported, in which case the write handler thread in Region don't pend on HLog.syncer() waiting for its txid to be sync-ed. This behaviour keeps intact. 2) Good catch, "if (txid <= this.pendingTxid) return;" in setPendingTxid() moved into below "synchronized (this.writeLock) {...} " is OK. bufferLock is used only to guarantee the access to HLog's local pending buffer and unflushedEntries can't be interleaved by multiple threads. 3) "failedTxid is still assigned to this.lastWrittenTxid. Is that safe ?" Yes. all write pending on syncer() with txid <= failedTxid will get failure. The lastWrittenTxid can safely proceed without incorrect behaviour, and it need to proceed to eventually wake-up pending write handler threads. I'll update the patch per your review tomorrow.
          Hide
          stack added a comment -

          Honghua Feng Your approach looks good. Thanks for working on this. I like the numbers you are getting for throughput. Any idea of its effect on general latencies? Does HLogPerformanceEvaluation help evaluating this approach? Did you deploy this code to production?

          Can I still (if only optionally) sync every write as it comes in? (For the paranoid).

          In general, remove rather than comment stuff out:

          -    assertTrue("Should have an outstanding WAL edit", log.hasDeferredEntries());
          +    //assertTrue("Should have an outstanding WAL edit", log.hasDeferredEntries());
          

          Regards the above, the test is no longer valid given the indirection around sync/flush?

          To be clear, when we call doWrite, we just append the log edit to a linked list? (We call it a bufferLock but we just doing append to the linked list?)

          Patch is looking good. Thanks Honghua Feng

          How does deferred log flush still work when you remove stuff like optionalFlushInterval? You say '...don't pend on HLog.syncer() waiting for its txid to be sync-ed' but that is another behavior than what we had here previously.

          If first thread throws IE, then other threads will still be running (in below):

          +    try {
          +      asyncNotifier.interrupt();
          +      asyncNotifier.join();
          +
          +      asyncFlusher.interrupt();
          +      asyncFlusher.join();
          +
          +      asyncWriter.interrupt();
          +      asyncWriter.join();
          +    } catch (InterruptedException e) {
          +      LOG.error("Exception while waiting for syncer thread to die", e);
               }
          

          Thread names usually follow a pattern where the prefix is the name of the host (RS:SHORT_SERVERNAME-AsyncWriter).. we can fix around commit.

          This new thread does not implement HasThread:

          + private class AsyncNotifier extends Thread {

          Show
          stack added a comment - Honghua Feng Your approach looks good. Thanks for working on this. I like the numbers you are getting for throughput. Any idea of its effect on general latencies? Does HLogPerformanceEvaluation help evaluating this approach? Did you deploy this code to production? Can I still (if only optionally) sync every write as it comes in? (For the paranoid). In general, remove rather than comment stuff out: - assertTrue( "Should have an outstanding WAL edit" , log.hasDeferredEntries()); + //assertTrue( "Should have an outstanding WAL edit" , log.hasDeferredEntries()); Regards the above, the test is no longer valid given the indirection around sync/flush? To be clear, when we call doWrite, we just append the log edit to a linked list? (We call it a bufferLock but we just doing append to the linked list?) Patch is looking good. Thanks Honghua Feng How does deferred log flush still work when you remove stuff like optionalFlushInterval? You say '...don't pend on HLog.syncer() waiting for its txid to be sync-ed' but that is another behavior than what we had here previously. If first thread throws IE, then other threads will still be running (in below): + try { + asyncNotifier.interrupt(); + asyncNotifier.join(); + + asyncFlusher.interrupt(); + asyncFlusher.join(); + + asyncWriter.interrupt(); + asyncWriter.join(); + } catch (InterruptedException e) { + LOG.error( "Exception while waiting for syncer thread to die" , e); } Thread names usually follow a pattern where the prefix is the name of the host (RS:SHORT_SERVERNAME-AsyncWriter).. we can fix around commit. This new thread does not implement HasThread: + private class AsyncNotifier extends Thread {
          Hide
          Honghua Feng added a comment -

          Thanks Ted Yu and stack for the detailed review. I make and attach a update patch based on trunk according to your reviews.

          Below are answers to some important questions Ted/stack raised in the reviews (I have already answered some from Ted in above comment):

          [Ted] AsyncNotifier does notification by calling syncedTillHere.notifyAll(). Can this part be folded into AsyncFlusher ?

          ===> AsyncNotifier will compete syncedTillHere with all the write handler threads(which may finish the appendNoSync but not pend on syncer()). The performance is better by separating AsyncSyncer(which just get notified, do 'sync' and then notify AsyncNotifier) and AsyncNotifier(get notified by AsyncSyncer and wake-up all pending write handler threads)

          [stack] Any idea of its effect on general latencies? Does HLogPerformanceEvaluation help evaluating this approach? Did you deploy this code to production?

          ===> I don't run HLogPerformanceEvaluation for performance comparison. instead I used 5 YCSB clients to concurrently press on a single RS with a 5 data-node underlying HDFS. Everything are the same for test with Old/New write thread models except the RS bits are different. We are testing it in the test cluster for a month, but not deployed to production yet. Below is the detailed performance comparison for your reference.

          a> 5 YCSB clients, each with 80 concurrent write theads (auto-flush = true)
          b> each YCSB writes 5000,000 rows
          c> all 20 regions of the target table are moved to a single RS

          Old write thread model:

          row size(bytes) latency(ms) QPS
          ------------------------------------------
          2000 37.3 10715
          1000 32.8 12149
          500 30.9 12891
          200 26.9 14803
          10 24.5 16288

          New write thread model:

          row size(bytes) latency(ms) QPS
          -------------------------------------------
          2000 17.3 23024
          1000 12.6 31523
          500 11.7 33893
          200 11.4 34876
          10 11.1 35804

          [stack] Can I still (if only optionally) sync every write as it comes in? (For the paranoid).

          ===> can't for now, I'll consider how to make it configurable later on.

          [stack] Regards the above, the test is no longer valid given the indirection around sync/flush?

          ===> Yes, that test is not valid by new write thread modeldeferred log flush

          [stack] To be clear, when we call doWrite, we just append the log edit to a linked list? (We call it a bufferLock but we just doing append to the linked list?)

          ===> Yes, in both old and new write thread models what doWrite does is just appending log edit to a linked list which plays a role as a 'local' buffer for log edits what don't hit hdfs deferred log flushyet.

          [stack] How does deferred log flush still work when you remove stuff like optionalFlushInterval? You say '...don't pend on HLog.syncer() waiting for its txid to be sync-ed' but that is another behavior than what we had here previously.

          ===> When say 'still support deferred log flush' I mean for 'deferred log flush' it can still response write success to client without wait/pend on syncer(txid), in this sense, the AsyncWriter/AsyncSyncer do what the previous LogSyncer does from the point view of the write handler threads: clients don't wait for the write persist before get reponse success.

          Show
          Honghua Feng added a comment - Thanks Ted Yu and stack for the detailed review. I make and attach a update patch based on trunk according to your reviews. Below are answers to some important questions Ted/stack raised in the reviews (I have already answered some from Ted in above comment): [Ted] AsyncNotifier does notification by calling syncedTillHere.notifyAll(). Can this part be folded into AsyncFlusher ? ===> AsyncNotifier will compete syncedTillHere with all the write handler threads(which may finish the appendNoSync but not pend on syncer()). The performance is better by separating AsyncSyncer(which just get notified, do 'sync' and then notify AsyncNotifier) and AsyncNotifier(get notified by AsyncSyncer and wake-up all pending write handler threads) [stack] Any idea of its effect on general latencies? Does HLogPerformanceEvaluation help evaluating this approach? Did you deploy this code to production? ===> I don't run HLogPerformanceEvaluation for performance comparison. instead I used 5 YCSB clients to concurrently press on a single RS with a 5 data-node underlying HDFS. Everything are the same for test with Old/New write thread models except the RS bits are different. We are testing it in the test cluster for a month, but not deployed to production yet. Below is the detailed performance comparison for your reference. a> 5 YCSB clients, each with 80 concurrent write theads (auto-flush = true) b> each YCSB writes 5000,000 rows c> all 20 regions of the target table are moved to a single RS Old write thread model: row size(bytes) latency(ms) QPS ------------------------------------------ 2000 37.3 10715 1000 32.8 12149 500 30.9 12891 200 26.9 14803 10 24.5 16288 New write thread model: row size(bytes) latency(ms) QPS ------------------------------------------- 2000 17.3 23024 1000 12.6 31523 500 11.7 33893 200 11.4 34876 10 11.1 35804 [stack] Can I still (if only optionally) sync every write as it comes in? (For the paranoid). ===> can't for now, I'll consider how to make it configurable later on. [stack] Regards the above, the test is no longer valid given the indirection around sync/flush? ===> Yes, that test is not valid by new write thread modeldeferred log flush [stack] To be clear, when we call doWrite, we just append the log edit to a linked list? (We call it a bufferLock but we just doing append to the linked list?) ===> Yes, in both old and new write thread models what doWrite does is just appending log edit to a linked list which plays a role as a 'local' buffer for log edits what don't hit hdfs deferred log flushyet. [stack] How does deferred log flush still work when you remove stuff like optionalFlushInterval? You say '...don't pend on HLog.syncer() waiting for its txid to be sync-ed' but that is another behavior than what we had here previously. ===> When say 'still support deferred log flush' I mean for 'deferred log flush' it can still response write success to client without wait/pend on syncer(txid), in this sense, the AsyncWriter/AsyncSyncer do what the previous LogSyncer does from the point view of the write handler threads: clients don't wait for the write persist before get reponse success.
          Hide
          Honghua Feng added a comment -

          new write thread model patch based on trunk

          Show
          Honghua Feng added a comment - new write thread model patch based on trunk
          Hide
          Honghua Feng added a comment -

          an update patch based on 0.94 according to Ted/stack's review attached

          Show
          Honghua Feng added a comment - an update patch based on 0.94 according to Ted/stack's review attached
          Hide
          Honghua Feng added a comment -

          HBASE-8755-trunk-V0.patch also includes changes according to the review comment from Ted/stack. Thanks again for Ted/stack for the detailed review

          Show
          Honghua Feng added a comment - HBASE-8755 -trunk-V0.patch also includes changes according to the review comment from Ted/stack. Thanks again for Ted/stack for the detailed review
          Hide
          Otis Gospodnetic added a comment -

          Feng Hua - this is about improving writes, but your table shows QPS (Queries Per Second). Is that QPS really Writes Per Second? Thanks.

          Show
          Otis Gospodnetic added a comment - Feng Hua - this is about improving writes , but your table shows QPS (Queries Per Second). Is that QPS really Writes Per Second? Thanks.
          Hide
          Honghua Feng added a comment -

          Otis Gospodnetic Yes, QPS really means Writes Per Second here. A typo.

          btw: My name is Feng Honghua, not Feng Hua

          Show
          Honghua Feng added a comment - Otis Gospodnetic Yes, QPS really means Writes Per Second here. A typo. btw: My name is Feng Honghua, not Feng Hua
          Hide
          Otis Gospodnetic added a comment -

          Thanks and I'm sorry about the name messup. Feel free to mess up mine - you've got 15 characters to play with. And thanks for this patch. Crazy improvement!

          Show
          Otis Gospodnetic added a comment - Thanks and I'm sorry about the name messup. Feel free to mess up mine - you've got 15 characters to play with. And thanks for this patch. Crazy improvement!
          Hide
          Lars Hofhansl added a comment -

          The numbers are great, the patch looks good. I'm fine even for 0.94 (but need to study the patch a bit better).

          Show
          Lars Hofhansl added a comment - The numbers are great, the patch looks good. I'm fine even for 0.94 (but need to study the patch a bit better).
          Hide
          Elliott Clark added a comment -

          For 0.94 this would remove a feature that people are currently using. So for me that would make 0.94 inclusion problematic.

          For 0.95/0.96 this looks awesome. I'll do a more thorough review now.

          Show
          Elliott Clark added a comment - For 0.94 this would remove a feature that people are currently using. So for me that would make 0.94 inclusion problematic. For 0.95/0.96 this looks awesome. I'll do a more thorough review now.
          Hide
          Sergey Shelukhin added a comment -
          +      synchronized (this.writeLock) {
          +        if (txid <= this.pendingTxid)
          +          return;
          

          pendingTxid can only go up, right? If so it may make sense to speculatively check it outside the lock; same with other places.
          How often does this condition happen?

          upateLock

          typo

          LOG.warn("writer.getLength() failed,this failure won't block here");
          

          This message is not very clear, also exception should be logged.

          +      synchronized (this.notifyLock) {
          +        this.flushedTxid = txid;
          +        this.notifyLock.notify();
                 }
          

          Here check is done outside lock but not inside, could this race?

          if (txid <= this.failedTxid.get()) {
          

          I don't quite understand the logic here. If 2 batches go thru the writer-syncer pipeline, 1st one succeeds and
          the 2nd one fails, before notifier thread wakes up, wouldn't it report the first batch also as failed?

          The same interaction I wonder about in writer and syncer.
          I am not sure how HDFS write and sync interact, is the following possible or not?
          Writer writes the first batch and wakes up syncer. Before syncer wakes up writer starts the 2nd batch.
          Syncer wakes up and syncs, invisibly to HBase code, to the middle of the 2nd batch that is being written (sync has no upper bound) and succeeds.
          Then finishing to write the 2nd batch, or sync after, fails, so now we wrote to WAL but reported failure.

          Also can you please put comment somewhere with regard to thread safety of log rolling... I am assuming it will be thread safe
          because if we write to one file, roll in the middle and sync a different file it will just be extra sync call, so harmless.

          +        addPendingWrite(new HLog.Entry(logKey, logEdit));
          

          addPendingWrite is called without bufferLock in some places, with in others.

          Can you please add comment to bufferLock to elaborate what it locks. And that updateLock cannot be taken inside bufferLock.
          It seems that right now this holds.

          Also, I understand the need for writer and sync thread, but is separate notifier thread necessary? It doesn't do any blocking operations other than interacting with flusher thread, or taking syncedTillHere lock, which looks like it should be uncontested most of the time.
          Couldn't flusher thread have the 4~ lines that set syncedTillHere?

          Show
          Sergey Shelukhin added a comment - + synchronized ( this .writeLock) { + if (txid <= this .pendingTxid) + return ; pendingTxid can only go up, right? If so it may make sense to speculatively check it outside the lock; same with other places. How often does this condition happen? upateLock typo LOG.warn( "writer.getLength() failed, this failure won't block here" ); This message is not very clear, also exception should be logged. + synchronized ( this .notifyLock) { + this .flushedTxid = txid; + this .notifyLock.notify(); } Here check is done outside lock but not inside, could this race? if (txid <= this .failedTxid.get()) { I don't quite understand the logic here. If 2 batches go thru the writer-syncer pipeline, 1st one succeeds and the 2nd one fails, before notifier thread wakes up, wouldn't it report the first batch also as failed? The same interaction I wonder about in writer and syncer. I am not sure how HDFS write and sync interact, is the following possible or not? Writer writes the first batch and wakes up syncer. Before syncer wakes up writer starts the 2nd batch. Syncer wakes up and syncs, invisibly to HBase code, to the middle of the 2nd batch that is being written (sync has no upper bound) and succeeds. Then finishing to write the 2nd batch, or sync after, fails, so now we wrote to WAL but reported failure. Also can you please put comment somewhere with regard to thread safety of log rolling... I am assuming it will be thread safe because if we write to one file, roll in the middle and sync a different file it will just be extra sync call, so harmless. + addPendingWrite( new HLog.Entry(logKey, logEdit)); addPendingWrite is called without bufferLock in some places, with in others. Can you please add comment to bufferLock to elaborate what it locks. And that updateLock cannot be taken inside bufferLock. It seems that right now this holds. Also, I understand the need for writer and sync thread, but is separate notifier thread necessary? It doesn't do any blocking operations other than interacting with flusher thread, or taking syncedTillHere lock, which looks like it should be uncontested most of the time. Couldn't flusher thread have the 4~ lines that set syncedTillHere?
          Hide
          Elliott Clark added a comment -

          The AsyncSyncer can throw an error on data that actually got written to the log. If the edit actually made it to the wal but the syncer errors out, then data will be reported as not written but in fact it could be written. I don't think there's anything that can be done about it but maybe a comment about that possibility around 1167. It's not something new but I expect since we'll be syncing larger chunks that it's more possible.

          As a follow up jira, I think that we can get even better through put by reducing contention on update and buffer locks. If we have the AsyncWriter sort wal edits then we can use a read write lock rather than two different locks (both of which are needed for append).

          As another follow up jira I think that we can do away with txid in FSHLog and just use seqnum.

          As I see it, I'm +1 but with such a critical piece it would be nice if there were others.

          Show
          Elliott Clark added a comment - The AsyncSyncer can throw an error on data that actually got written to the log. If the edit actually made it to the wal but the syncer errors out, then data will be reported as not written but in fact it could be written. I don't think there's anything that can be done about it but maybe a comment about that possibility around 1167. It's not something new but I expect since we'll be syncing larger chunks that it's more possible. As a follow up jira, I think that we can get even better through put by reducing contention on update and buffer locks. If we have the AsyncWriter sort wal edits then we can use a read write lock rather than two different locks (both of which are needed for append). As another follow up jira I think that we can do away with txid in FSHLog and just use seqnum. As I see it, I'm +1 but with such a critical piece it would be nice if there were others.
          Hide
          Lars Hofhansl added a comment -

          Which feature is removed?
          You mean this:

          [stack] Can I still (if only optionally) sync every write as it comes in? (For the paranoid).

          ===> can't for now, I'll consider how to make it configurable later on.

          I think this is the same as now. Previously all threads append to the log (without sync'ing), then they all try to sync as far as they can. The only difference is that now is single thread is sync'ing as much as it can while the writer thread is waiting. Is that the concern?

          Or you mean the deferred log flush interval? I think there the main gain was to do sync asynchronous to the writer thread not the wait interval, which is more of an implementation detail.

          Either way for 0.95/0.96 this is a great feature. Can decide on 0.94 after weighing the details.

          Show
          Lars Hofhansl added a comment - Which feature is removed? You mean this: [stack] Can I still (if only optionally) sync every write as it comes in? (For the paranoid). ===> can't for now, I'll consider how to make it configurable later on. I think this is the same as now. Previously all threads append to the log (without sync'ing), then they all try to sync as far as they can. The only difference is that now is single thread is sync'ing as much as it can while the writer thread is waiting. Is that the concern? Or you mean the deferred log flush interval? I think there the main gain was to do sync asynchronous to the writer thread not the wait interval, which is more of an implementation detail. Either way for 0.95/0.96 this is a great feature. Can decide on 0.94 after weighing the details.
          Hide
          Elliott Clark added a comment -

          For 0.94 users got one wal edit means one sync. Meaning if you failed to sync it was for your edit only. Not really a huge deal, but that was my convern.

          Show
          Elliott Clark added a comment - For 0.94 users got one wal edit means one sync. Meaning if you failed to sync it was for your edit only. Not really a huge deal, but that was my convern.
          Hide
          Elliott Clark added a comment -

          I don't quite understand the logic here. If 2 batches go thru the writer-syncer pipeline, 1st one succeeds and

          the 2nd one fails, before notifier thread wakes up, wouldn't it report the first batch also as failed?

          Yeah I think you are correct. The check should check that the txid is greater than last successful txid and less than the failed.

          Show
          Elliott Clark added a comment - I don't quite understand the logic here. If 2 batches go thru the writer-syncer pipeline, 1st one succeeds and the 2nd one fails, before notifier thread wakes up, wouldn't it report the first batch also as failed? Yeah I think you are correct. The check should check that the txid is greater than last successful txid and less than the failed.
          Hide
          stack added a comment -

          Lars Hofhansl I was concerned that you could not write sync write sync anymore w/ this patch in place. Chatting w/ Elliott, I think it fine letting this go so it is more write write sync write write write sync, etc. Also, the way deferred flush works is different w/ this patch. In past a background thread would sync on a period. I think it also fine that this behavior changes a little (after talking w/ Elliott) so the write and sync are done after we have let the client go ('deferred'). Just need to add it up in release note.

          Show
          stack added a comment - Lars Hofhansl I was concerned that you could not write sync write sync anymore w/ this patch in place. Chatting w/ Elliott, I think it fine letting this go so it is more write write sync write write write sync, etc. Also, the way deferred flush works is different w/ this patch. In past a background thread would sync on a period. I think it also fine that this behavior changes a little (after talking w/ Elliott) so the write and sync are done after we have let the client go ('deferred'). Just need to add it up in release note.
          Hide
          Lars Hofhansl added a comment -

          For 0.94 users got one wal edit means one sync. Meaning if you failed to sync it was for your edit only. Not really a huge deal, but that was my convern.

          Is that true, though? (say) Two services threads call doMiniBatchMutation on behalf of two clients at roughly the same time, both threads would write to the WAL without sync'ing, and they would sync. Depending on the interleaving the first could sync all edits and the 2nd could be a no-op (the txid <= this.syncedTillHere test) in the the write-write-sync-sync case.

          It is true that both threads would execute a sync in the context of their own thread, and that would no longer be the case. In fact that is the main concern with this patch: It is harder to trace the code to verify an edit is truly sync'ed in all cases; I am also concerned about possible new/different deadlock scenarios with which the existing code is riddled.
          Also I wonder if some folks do rely on changing deferred-sync-interval.

          If I weren't in Germany right with no VPN access I'd offer to load 0.94 with the patch onto one of our test clusters and run some workload on it...

          Show
          Lars Hofhansl added a comment - For 0.94 users got one wal edit means one sync. Meaning if you failed to sync it was for your edit only. Not really a huge deal, but that was my convern. Is that true, though? (say) Two services threads call doMiniBatchMutation on behalf of two clients at roughly the same time, both threads would write to the WAL without sync'ing, and they would sync. Depending on the interleaving the first could sync all edits and the 2nd could be a no-op (the txid <= this.syncedTillHere test) in the the write-write-sync-sync case. It is true that both threads would execute a sync in the context of their own thread, and that would no longer be the case. In fact that is the main concern with this patch: It is harder to trace the code to verify an edit is truly sync'ed in all cases; I am also concerned about possible new/different deadlock scenarios with which the existing code is riddled. Also I wonder if some folks do rely on changing deferred-sync-interval. If I weren't in Germany right with no VPN access I'd offer to load 0.94 with the patch onto one of our test clusters and run some workload on it...
          Hide
          Sergey Shelukhin added a comment -

          Yeah I think you are correct. The check should check that the txid is greater than last successful txid and less than the failed.

          The successful id is currently set to the same as failed id, to solve this, tracking each batch is necessary (because otherwise one could come up with scenario with 3 batches ).
          Maybe, as other comment say, this is ok.

          Show
          Sergey Shelukhin added a comment - Yeah I think you are correct. The check should check that the txid is greater than last successful txid and less than the failed. The successful id is currently set to the same as failed id, to solve this, tracking each batch is necessary (because otherwise one could come up with scenario with 3 batches ). Maybe, as other comment say, this is ok.
          Hide
          Elliott Clark added a comment -

          tracking each batch is necessary (because otherwise one could come up with scenario with 3 batches ).

          I don't think that's true. If you fail a sync. Then you can't append to the write pipeline and there can be no more batches that can succeed either step.

          Show
          Elliott Clark added a comment - tracking each batch is necessary (because otherwise one could come up with scenario with 3 batches ). I don't think that's true. If you fail a sync. Then you can't append to the write pipeline and there can be no more batches that can succeed either step.
          Hide
          Lars Hofhansl added a comment -

          Should I make a backport jira for 0.94 so that we can discuss the merits of this change and 0.94 considerations separately?

          Show
          Lars Hofhansl added a comment - Should I make a backport jira for 0.94 so that we can discuss the merits of this change and 0.94 considerations separately?
          Hide
          Liang Xie added a comment -

          I totally agreed with u,Lars Hofhansl

          Show
          Liang Xie added a comment - I totally agreed with u, Lars Hofhansl
          Hide
          chunhui shen added a comment -

          Attractd by the crazy improvement, I have tried a quick performance test, seems not same as my initial think.

          Test Data:
          1.0.94 version with this patch
          2.One client putting data to one regionserver(autoflush=true)
          3.300 RPC handler for regionserver

          a.client using 5 concurrent thread

          Without patch:

          Write Threads: 5 Write Rows: 200000 Consume Time: 42s
          Avg TPS: 4651

          With patch:

          Write Threads: 5 Write Rows: 200000 Consume Time: 43s
          Avg TPS: 4545

          b.client using 50 concurrent thread

          Without patch:

          Write Threads: 50 Write Rows: 2000000 Consume Time: 110s
          Avg TPS: 18018

          With patch:

          Write Threads: 50 Write Rows: 2000000 Consume Time: 118s
          Avg TPS: 16806

          c.client using 200concurrent thread

          Without patch:

          Write Threads: 200 Write Rows: 2000000 Consume Time: 80s
          Avg TPS: 24691

          With patch:

          Write Threads: 200 Write Rows: 2000000 Consume Time: 64s
          Avg TPS: 30769

          {format}
          a> 5 YCSB clients, each with 80 concurrent write theads (auto-flush = true)
          b> each YCSB writes 5000,000 rows
          c> all 20 regions of the target table are moved to a single RS{format}

          As the above test description, it means 400 concurrent theads writing data to one RS.

          I personally think this patch will work if regionserver is under very high pressure,
          for general pressure, it will degrade a little.

          I just take a quick test, maybe there's something wrong.
          About the improvement scenario, more tests would be better.

          Show
          chunhui shen added a comment - Attractd by the crazy improvement, I have tried a quick performance test, seems not same as my initial think. Test Data: 1.0.94 version with this patch 2.One client putting data to one regionserver(autoflush=true) 3.300 RPC handler for regionserver a.client using 5 concurrent thread Without patch: Write Threads: 5 Write Rows: 200000 Consume Time: 42s Avg TPS: 4651 With patch: Write Threads: 5 Write Rows: 200000 Consume Time: 43s Avg TPS: 4545 b.client using 50 concurrent thread Without patch: Write Threads: 50 Write Rows: 2000000 Consume Time: 110s Avg TPS: 18018 With patch: Write Threads: 50 Write Rows: 2000000 Consume Time: 118s Avg TPS: 16806 c.client using 200concurrent thread Without patch: Write Threads: 200 Write Rows: 2000000 Consume Time: 80s Avg TPS: 24691 With patch: Write Threads: 200 Write Rows: 2000000 Consume Time: 64s Avg TPS: 30769 {format} a> 5 YCSB clients, each with 80 concurrent write theads (auto-flush = true) b> each YCSB writes 5000,000 rows c> all 20 regions of the target table are moved to a single RS{format} As the above test description, it means 400 concurrent theads writing data to one RS. I personally think this patch will work if regionserver is under very high pressure, for general pressure, it will degrade a little. I just take a quick test, maybe there's something wrong. About the improvement scenario, more tests would be better.
          Hide
          Lars Hofhansl added a comment -

          Hmm... Let's try to find out why the discrepancy between Feng's and Chunhui's tests.

          Show
          Lars Hofhansl added a comment - Hmm... Let's try to find out why the discrepancy between Feng's and Chunhui's tests.
          Hide
          chunhui shen added a comment -

          Let's try to find out why the discrepancy between Feng's and Chunhui's tests.

          The discrepancy should come from the diffrrent pressure, one client in my tests vs 5 clients * 80 concurrent threads in Feng's test.
          Thus I personally presume that this patch just takes effect if RS is under very high pressure.
          For the general scenarios, seems no improvement, even be worse, it needs more test to verify.

          Show
          chunhui shen added a comment - Let's try to find out why the discrepancy between Feng's and Chunhui's tests. The discrepancy should come from the diffrrent pressure, one client in my tests vs 5 clients * 80 concurrent threads in Feng's test. Thus I personally presume that this patch just takes effect if RS is under very high pressure. For the general scenarios, seems no improvement, even be worse, it needs more test to verify.
          Hide
          Lars Hofhansl added a comment - - edited

          I guess that makes sense. More handoff required between threads, which would impact latency, but better throughput in high pressure cases because of fewer threads contenting for locks.

          Any chance to measure context switches during the test with/without the patch? I wonder if those increase due to thread handoffs, until the ctx switches from contended locks outweigh those.

          Show
          Lars Hofhansl added a comment - - edited I guess that makes sense. More handoff required between threads, which would impact latency, but better throughput in high pressure cases because of fewer threads contenting for locks. Any chance to measure context switches during the test with/without the patch? I wonder if those increase due to thread handoffs, until the ctx switches from contended locks outweigh those.
          Hide
          Hangjun Ye added a comment -

          Chunhui, just want to confirm whether your regionserver is running against a HDFS cluster or the local disk? And the row size?

          I asked this because the discrepancy between Honghua's test and Chunhui's test is mainly the number about the original model (w/o patch), just want to make sure the basic environment is the same.

          Show
          Hangjun Ye added a comment - Chunhui, just want to confirm whether your regionserver is running against a HDFS cluster or the local disk? And the row size? I asked this because the discrepancy between Honghua's test and Chunhui's test is mainly the number about the original model (w/o patch), just want to make sure the basic environment is the same.
          Hide
          Honghua Feng added a comment -

          Thanks Chunhui for this verification test. We didn't test with small/medium write pressure, we'll do tests with small-medium write pressure soon and provide the numbers when done.

          A quick response on your test result: We never saw a such high throughput as 24691 for a single RS in cluster before we applied the new write thread model. We ever did a series of stress test for write throughput and the maximum we ever got is about 10000 using 1 YCSB client.

          "Without patch:
          Write Threads: 200 Write Rows: 2000000 Consume Time: 80s
          Avg TPS: 24691
          With patch:
          Write Threads: 200 Write Rows: 2000000 Consume Time: 64s
          Avg TPS: 30769"

          Show
          Honghua Feng added a comment - Thanks Chunhui for this verification test. We didn't test with small/medium write pressure, we'll do tests with small-medium write pressure soon and provide the numbers when done. A quick response on your test result: We never saw a such high throughput as 24691 for a single RS in cluster before we applied the new write thread model. We ever did a series of stress test for write throughput and the maximum we ever got is about 10000 using 1 YCSB client. "Without patch: Write Threads: 200 Write Rows: 2000000 Consume Time: 80s Avg TPS: 24691 With patch: Write Threads: 200 Write Rows: 2000000 Consume Time: 64s Avg TPS: 30769"
          Hide
          Honghua Feng added a comment -

          chunhui shen

          What's the row-size used in your test? You tested against a back-end HDFS, not local disk, right?

          And would you test using a bit more test data(such as 5,000,000 - 10,000,000 rows)? Thanks.

          Show
          Honghua Feng added a comment - chunhui shen What's the row-size used in your test? You tested against a back-end HDFS, not local disk, right? And would you test using a bit more test data(such as 5,000,000 - 10,000,000 rows)? Thanks.
          Hide
          chunhui shen added a comment -

          What's the row-size used in your test?

          Row:50 bytes, Value:100 bytes

          You tested against a back-end HDFS, not local disk, right?

          Using the real HDFS cluster

          the maximum we ever got is about 10000 using 1 YCSB client

          If the KV size is smaller than 200 and the client has used many concurrent threads, it should be abnormal.

          we'll do tests with small-medium write pressure soon and provide the numbers when done

          Thanks, it would be a good reference.

          Show
          chunhui shen added a comment - What's the row-size used in your test? Row:50 bytes, Value:100 bytes You tested against a back-end HDFS, not local disk, right? Using the real HDFS cluster the maximum we ever got is about 10000 using 1 YCSB client If the KV size is smaller than 200 and the client has used many concurrent threads, it should be abnormal. we'll do tests with small-medium write pressure soon and provide the numbers when done Thanks, it would be a good reference.
          Hide
          Honghua Feng added a comment -

          chunhui shen We run the same tests as yours, and below are the result:

          1). One YCSB client with 5/50/200 write threads respectively
          2). One RS with 300 RPC handlers, 20 regions (5 data-nodes back-end HDFS running CDH 4.1.1)
          3). row-size = 150 bytes

          threads row-count new-throughput new-latency old-throughput old-latency
          ---------------------------------------------------------------------------------------
          5 200000 3191 1.551(ms) 3172 1.561(ms)
          50 2000000 23215 2.131(ms) 7437 6.693(ms)
          200 2000000 35793 5.450(ms) 10816 18.312(ms)
          ---------------------------------------------------------------------------------------

          A). the difference is negligible when 5 threads of YCSB client
          B). new-model still has 3X+ improvement compared to old-model when threads are 50/200

          Anybody else can help do the similar tests using the same test configuration as Chunhui?

          Show
          Honghua Feng added a comment - chunhui shen We run the same tests as yours, and below are the result: 1). One YCSB client with 5/50/200 write threads respectively 2). One RS with 300 RPC handlers, 20 regions (5 data-nodes back-end HDFS running CDH 4.1.1) 3). row-size = 150 bytes threads row-count new-throughput new-latency old-throughput old-latency --------------------------------------------------------------------------------------- 5 200000 3191 1.551(ms) 3172 1.561(ms) 50 2000000 23215 2.131(ms) 7437 6.693(ms) 200 2000000 35793 5.450(ms) 10816 18.312(ms) --------------------------------------------------------------------------------------- A). the difference is negligible when 5 threads of YCSB client B). new-model still has 3X+ improvement compared to old-model when threads are 50/200 Anybody else can help do the similar tests using the same test configuration as Chunhui?
          Hide
          Honghua Feng added a comment -

          chunhui shen: We run the same tests as yours, and below are the result:
          1). One YCSB client with 5/50/200 write threads respectively
          2). One RS with 300 RPC handlers, 20 regions (5 data-nodes back-end HDFS running CDH 4.1.1)
          3). row-size = 150 bytes

          client-threads row-count new-model throughput new-model latency old-model throughput old-model latency
          5 200000 3191 1.551(ms) 3172 1.561(ms)
          50 2000000 23215 2.131(ms) 7437 6.693(ms)
          200 2000000 35793 5.450(ms) 10816 18.312(ms)

          A). the difference is negligible when 5 threads of YCSB client, this is because
          B). new-model still has 3X+ improvement compared to old-model when threads are 50/200
          Can anybody else help do the tests using the same configurations as Chunhui?

          Another guess is the HDFS used by chunhui has much better performance on HLog's write/sync, which makes the new model in HBase has less impact. Just guess.

          Show
          Honghua Feng added a comment - chunhui shen : We run the same tests as yours, and below are the result: 1). One YCSB client with 5/50/200 write threads respectively 2). One RS with 300 RPC handlers, 20 regions (5 data-nodes back-end HDFS running CDH 4.1.1) 3). row-size = 150 bytes client-threads row-count new-model throughput new-model latency old-model throughput old-model latency 5 200000 3191 1.551(ms) 3172 1.561(ms) 50 2000000 23215 2.131(ms) 7437 6.693(ms) 200 2000000 35793 5.450(ms) 10816 18.312(ms) A). the difference is negligible when 5 threads of YCSB client, this is because B). new-model still has 3X+ improvement compared to old-model when threads are 50/200 Can anybody else help do the tests using the same configurations as Chunhui? Another guess is the HDFS used by chunhui has much better performance on HLog's write/sync, which makes the new model in HBase has less impact. Just guess.
          Hide
          chunhui shen added a comment -

          As the above tests, try to find out why the old-throughput is so low.

          Do your client run on the regionserver or another separated server?

          Show
          chunhui shen added a comment - As the above tests, try to find out why the old-throughput is so low. Do your client run on the regionserver or another separated server?
          Hide
          Honghua Feng added a comment -

          Our comparison tests have only the RS bits different, and all others(client/HDFS/cluster/row-size...) remain the same.

          The client runs on a different machine other than the RS, we don't run client on RS because almost all our applications using HBase run their application in their own machines different from the HBase cluster.

          Actually we never saw a such high throughput as 18018/24691 for a single RS in our cluster. It's really weird .

          Show
          Honghua Feng added a comment - Our comparison tests have only the RS bits different, and all others(client/HDFS/cluster/row-size...) remain the same. The client runs on a different machine other than the RS, we don't run client on RS because almost all our applications using HBase run their application in their own machines different from the HBase cluster. Actually we never saw a such high throughput as 18018/24691 for a single RS in our cluster. It's really weird .
          Hide
          Honghua Feng added a comment -

          If possible, would anybody else help do the same comparison test as Chunhui/me? Thanks in advance. Lars Hofhansl Ted Yu Sergey Shelukhin stack

          Show
          Honghua Feng added a comment - If possible, would anybody else help do the same comparison test as Chunhui/me? Thanks in advance. Lars Hofhansl Ted Yu Sergey Shelukhin stack
          Hide
          stack added a comment -

          Jean-Marc Spaggiari You want to set up a rig to test this one?

          Show
          stack added a comment - Jean-Marc Spaggiari You want to set up a rig to test this one?
          Hide
          Jean-Marc Spaggiari added a comment -

          Sure! Let me prepare that. I will read this JIRA from the beginning and try to start the tests today.

          Show
          Jean-Marc Spaggiari added a comment - Sure! Let me prepare that. I will read this JIRA from the beginning and try to start the tests today.
          Hide
          Lars Hofhansl added a comment -

          I ran some local tests, and even with a single threaded client I did not see a performance degradation with this patch (0.94).
          I inserted 10m small rows. The cluster consists of a single RS/DN, HDFS is backed by a SSD. So the HDFS cost of writes should be low, and that in turns should bring out any new inefficiencies introduced by this patch.

          So from my angle this patch is good.

          Show
          Lars Hofhansl added a comment - I ran some local tests, and even with a single threaded client I did not see a performance degradation with this patch (0.94). I inserted 10m small rows. The cluster consists of a single RS/DN, HDFS is backed by a SSD. So the HDFS cost of writes should be low, and that in turns should bring out any new inefficiencies introduced by this patch. So from my angle this patch is good.
          Hide
          Jean-Marc Spaggiari added a comment -

          Here are my results:

          Test Trunk 8755
          org.apache.hadoop.hbase.PerformanceEvaluation$FilteredScanTest 437101.8 431561.7
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomReadTest 774306.8 768559
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomScanWithRange100Test 21999.7 22277.6
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomSeekScanTest 134429.3 134370.9
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomWriteTest 112814.7 125324.2
          org.apache.hadoop.hbase.PerformanceEvaluation$SequentialWriteTest 78574.42 80064.5

          It's time. so the smaller, the better. So overall, it's pretty the same results, except for the random write which are 11% slower.

          org.apache.hadoop.hbase.PerformanceEvaluation$FilteredScanTest 1.27%
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomReadTest 0.74%
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomScanWithRange100Test -1.26%
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomSeekScanTest 0.04%
          org.apache.hadoop.hbase.PerformanceEvaluation$RandomWriteTest -11.09%
          org.apache.hadoop.hbase.PerformanceEvaluation$SequentialWriteTest -1.90%

          I will redo the RandomWrite tests and try them with 1, 10 and 100 thread to see if only mono-threaded access is impacted...

          Show
          Jean-Marc Spaggiari added a comment - Here are my results: Test Trunk 8755 org.apache.hadoop.hbase.PerformanceEvaluation$FilteredScanTest 437101.8 431561.7 org.apache.hadoop.hbase.PerformanceEvaluation$RandomReadTest 774306.8 768559 org.apache.hadoop.hbase.PerformanceEvaluation$RandomScanWithRange100Test 21999.7 22277.6 org.apache.hadoop.hbase.PerformanceEvaluation$RandomSeekScanTest 134429.3 134370.9 org.apache.hadoop.hbase.PerformanceEvaluation$RandomWriteTest 112814.7 125324.2 org.apache.hadoop.hbase.PerformanceEvaluation$SequentialWriteTest 78574.42 80064.5 It's time. so the smaller, the better. So overall, it's pretty the same results, except for the random write which are 11% slower. org.apache.hadoop.hbase.PerformanceEvaluation$FilteredScanTest 1.27% org.apache.hadoop.hbase.PerformanceEvaluation$RandomReadTest 0.74% org.apache.hadoop.hbase.PerformanceEvaluation$RandomScanWithRange100Test -1.26% org.apache.hadoop.hbase.PerformanceEvaluation$RandomSeekScanTest 0.04% org.apache.hadoop.hbase.PerformanceEvaluation$RandomWriteTest -11.09% org.apache.hadoop.hbase.PerformanceEvaluation$SequentialWriteTest -1.90% I will redo the RandomWrite tests and try them with 1, 10 and 100 thread to see if only mono-threaded access is impacted...
          Hide
          Lars Hofhansl added a comment -

          Interesting. For writing the WAL it should not make any difference whether we write sequentially or at random.
          A possible reason is that the batching there is less efficient, maybe that brings out the extra thread hand off needed here.
          (my test was the equivalent of a sequential write, but using a handcoded test)

          Show
          Lars Hofhansl added a comment - Interesting. For writing the WAL it should not make any difference whether we write sequentially or at random. A possible reason is that the batching there is less efficient, maybe that brings out the extra thread hand off needed here. (my test was the equivalent of a sequential write, but using a handcoded test)
          Hide
          stack added a comment -

          Jean-Marc Spaggiari Thanks boss. What test is this? YCSB w/ how many clients? Thank you sir.

          Show
          stack added a comment - Jean-Marc Spaggiari Thanks boss. What test is this? YCSB w/ how many clients? Thank you sir.
          Hide
          Jean-Marc Spaggiari added a comment -

          It was PerformanceEvaluation and not YCSB. And I ran it with only one thread. It's now running with 10 threads. Will have the results tomorrow I think, or maybe by end of day if it's going fast enought. I will run YCSB after that, but not sure it's relevant in standalone mode. I will still try. And very soon add another dedicated test node...

          Show
          Jean-Marc Spaggiari added a comment - It was PerformanceEvaluation and not YCSB. And I ran it with only one thread. It's now running with 10 threads. Will have the results tomorrow I think, or maybe by end of day if it's going fast enought. I will run YCSB after that, but not sure it's relevant in standalone mode. I will still try. And very soon add another dedicated test node...
          Hide
          Hangjun Ye added a comment -

          It possibly doesn't have significant difference under standalone mode.
          The point of this patch is to avoid heavy race conditions of multiple handler threads, we found many race conditions happened in HDFS client (when many threads called write/hflush concurrently)

          We tested on a single RS with a 5 data-node underlying HDFS cluster using YCSB, would you like to have a try on a similar environment?

          Show
          Hangjun Ye added a comment - It possibly doesn't have significant difference under standalone mode. The point of this patch is to avoid heavy race conditions of multiple handler threads, we found many race conditions happened in HDFS client (when many threads called write/hflush concurrently) We tested on a single RS with a 5 data-node underlying HDFS cluster using YCSB, would you like to have a try on a similar environment?
          Hide
          Honghua Feng added a comment -

          Jean-Marc Spaggiari, what's your result of running YCSB against real cluster environment?

          Show
          Honghua Feng added a comment - Jean-Marc Spaggiari , what's your result of running YCSB against real cluster environment?
          Hide
          Jean-Marc Spaggiari added a comment -

          Hi Honghua Feng, the serves dedicated for those tests were to much different so I think it will have not been a good idea to run on them (difficult to interpret the results). So I just bought 3 absolutely identical nodes (the same as one I already have). By the end of the week I will have 4 servers with the same MB, same CPU, same memory (branch, PN, etc.) and same hard drive! Master will have a 1xSSD+1xSATA, others will have 2xSATA. To start.

          I will run the YCSB on that with and without this patch as soon as I get the hardware and I install the OS. Should be sometime later this week.

          I will also see if I can run PE.

          More to come.

          Show
          Jean-Marc Spaggiari added a comment - Hi Honghua Feng , the serves dedicated for those tests were to much different so I think it will have not been a good idea to run on them (difficult to interpret the results). So I just bought 3 absolutely identical nodes (the same as one I already have). By the end of the week I will have 4 servers with the same MB, same CPU, same memory (branch, PN, etc.) and same hard drive! Master will have a 1xSSD+1xSATA, others will have 2xSATA. To start. I will run the YCSB on that with and without this patch as soon as I get the hardware and I install the OS. Should be sometime later this week. I will also see if I can run PE. More to come.
          Hide
          Honghua Feng added a comment -

          Thanks a lot Jean-Marc Spaggiari, looking forward to your result.

          Show
          Honghua Feng added a comment - Thanks a lot Jean-Marc Spaggiari , looking forward to your result.
          Hide
          Tianying Chang added a comment -

          chunhui shen In your test, you use 300 RPC handler. Since default is 10, and we use 50 in our production cluster, when do we really need that many RPC handler? Will that be more of overhead than benefit? Do you see the write throughput up when increase the # of handler? Any particular scenario where high # of handler help?

          Show
          Tianying Chang added a comment - chunhui shen In your test, you use 300 RPC handler. Since default is 10, and we use 50 in our production cluster, when do we really need that many RPC handler? Will that be more of overhead than benefit? Do you see the write throughput up when increase the # of handler? Any particular scenario where high # of handler help?
          Hide
          Honghua Feng added a comment -

          updated patch rebased on latest trunk code base

          Show
          Honghua Feng added a comment - updated patch rebased on latest trunk code base
          Hide
          Jean-Marc Spaggiari added a comment -

          Hi Honghua Feng,

          I was in a process to give it a try, is there a new version for 0.94 coming? Or the one attached in the JIRA is already the good one?

          Show
          Jean-Marc Spaggiari added a comment - Hi Honghua Feng , I was in a process to give it a try, is there a new version for 0.94 coming? Or the one attached in the JIRA is already the good one?
          Hide
          Honghua Feng added a comment -

          Jean-Marc Spaggiari HBASE-8755-0.94-V1.patch is good. Let me know if any problem. Thanks Jean-Marc

          Show
          Honghua Feng added a comment - Jean-Marc Spaggiari HBASE-8755 -0.94-V1.patch is good. Let me know if any problem. Thanks Jean-Marc
          Hide
          Jean-Marc Spaggiari added a comment -

          Ok. I did some other tries and here are the results.

          jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.1.2.txt
          421428.8
          jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.1.2-8755.txt
          427172.1
          jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.2.0.txt
          419673.3
          jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.2.0-8755.txt
          432413.9

          This is elapse time. Between each iteration I totally delete (rm -rf) the hadoop directories, stop all the java processes, etc. Test is 10M randomWrite.

          So unfortunately I have not been able to see any real improvement. For YCSB, any specific load I should run to be able to see something better that without 8755? I guess it's a write intensive load that we want? Also, I have tested this on a pseudo-distributed instance (no more a standalone one), but I can dedicate 4 nodes to a test if required...

          Show
          Jean-Marc Spaggiari added a comment - Ok. I did some other tries and here are the results. jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.1.2.txt 421428.8 jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.1.2-8755.txt 427172.1 jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.2.0.txt 419673.3 jmspaggi@hbasetest:~/hbase/hbase-$ cat output-1.2.0-8755.txt 432413.9 This is elapse time. Between each iteration I totally delete (rm -rf) the hadoop directories, stop all the java processes, etc. Test is 10M randomWrite. So unfortunately I have not been able to see any real improvement. For YCSB, any specific load I should run to be able to see something better that without 8755? I guess it's a write intensive load that we want? Also, I have tested this on a pseudo-distributed instance (no more a standalone one), but I can dedicate 4 nodes to a test if required...
          Hide
          Honghua Feng added a comment -

          Jean-Marc Spaggiari, thanks for your test, some questions about your test: Is it against real HDFS? how many data-nodes and RS? what's the write pressure(client number, write thread number)? what's the total throughput you get?

          Yes this jira aims for throughput improvement under write intensive load. It should be tested and verified under write intensive load against real cluster / HDFS environment. And as you can see this jira only refactors the write thread model rather than tuning any write sub-phase along the whole write path for any individual write request, no obvious improvement is expected for low/ordinary write pressure.

          If you have a real cluster environment with 4 data-nodes, it would be better to re-do the test chunhui/I did with the similar test configuration/load which are listed in detail in above comments. 1 client with 200 write threads is OK for pressing a single RS and 4 clients each with 200 write threads for pressing 4 RS.

          Thanks again.

          Show
          Honghua Feng added a comment - Jean-Marc Spaggiari , thanks for your test, some questions about your test: Is it against real HDFS? how many data-nodes and RS? what's the write pressure(client number, write thread number)? what's the total throughput you get? Yes this jira aims for throughput improvement under write intensive load. It should be tested and verified under write intensive load against real cluster / HDFS environment. And as you can see this jira only refactors the write thread model rather than tuning any write sub-phase along the whole write path for any individual write request, no obvious improvement is expected for low/ordinary write pressure. If you have a real cluster environment with 4 data-nodes, it would be better to re-do the test chunhui/I did with the similar test configuration/load which are listed in detail in above comments. 1 client with 200 write threads is OK for pressing a single RS and 4 clients each with 200 write threads for pressing 4 RS. Thanks again.
          Hide
          Jean-Marc Spaggiari added a comment -

          Hi Honghua Feng, I ran in pseudo-distributed for all the tests, which mean HBase again a real HDFS. Tests with 1.1.2 and 1.2.0. The test was a 10 million randomWrite test. Ran 10 of them.

          I will re-run some tests again a single node (still in the process to install the OS on the other one) and will but more load against it. Like you said, 200 threads doing 100 000 writes eaches... More to come...

          Show
          Jean-Marc Spaggiari added a comment - Hi Honghua Feng , I ran in pseudo-distributed for all the tests, which mean HBase again a real HDFS. Tests with 1.1.2 and 1.2.0. The test was a 10 million randomWrite test. Ran 10 of them. I will re-run some tests again a single node (still in the process to install the OS on the other one) and will but more load against it. Like you said, 200 threads doing 100 000 writes eaches... More to come...
          Hide
          stack added a comment -

          Let me try this.

          Show
          stack added a comment - Let me try this.
          Hide
          stack added a comment -

          Rebase of trunk.

          Show
          stack added a comment - Rebase of trunk.
          Hide
          stack added a comment -

          Staring in on taking a look at this. I tried HLogPerformanceEvaluation. Going via ycsb would seem to add noise and indirection. I ran with 1, 5, and 50 threads w/ sizes that are like Honghua's (key of 50 and value of 150).

          I ran the test like this on trunk using localfs:

          $ for i in 1 5 50; do  for j in 1 2 3; do ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation  -verify -threads "${i}" -iterations 1000000 -nocleanup -verbose -keySize 50 -valueSize 100 &> /tmp/log-patch"${i}"."${j}".txt; done; done
          

          Needs fixes over in HBASE-9460 for above to work on localfs.

          With localfs, the patch is twice as slow. localfs does not support sync so this is probably what makes the difference – the extra threads do better w/ the dfsclient's stutter around sync. Let me try up on hdfs next.

          Below are numbers. I ran each test three times: i.e. three times without patch with one thread, then three times with patch and one thread, etc.

          Each thread did 1M puts. The table is times for the test to complete in seconds so less is better.

          Thread Count WithoutPatch WithPatch
          1 4.895 29.088
          1 4.856 29.544
          1 4.901 29.326
          5 24.173 53.974
          5 24.013 55.976
          5 23.390 55.858
          50 253.773 454.147
          50 247.095 443.215
          50 254.044 449.043
          Show
          stack added a comment - Staring in on taking a look at this. I tried HLogPerformanceEvaluation. Going via ycsb would seem to add noise and indirection. I ran with 1, 5, and 50 threads w/ sizes that are like Honghua's (key of 50 and value of 150). I ran the test like this on trunk using localfs: $ for i in 1 5 50; do for j in 1 2 3; do ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -verify -threads "${i}" -iterations 1000000 -nocleanup -verbose -keySize 50 -valueSize 100 &> /tmp/log-patch "${i}" . "${j}" .txt; done; done Needs fixes over in HBASE-9460 for above to work on localfs. With localfs, the patch is twice as slow. localfs does not support sync so this is probably what makes the difference – the extra threads do better w/ the dfsclient's stutter around sync. Let me try up on hdfs next. Below are numbers. I ran each test three times: i.e. three times without patch with one thread, then three times with patch and one thread, etc. Each thread did 1M puts. The table is times for the test to complete in seconds so less is better. Thread Count WithoutPatch WithPatch 1 4.895 29.088 1 4.856 29.544 1 4.901 29.326 5 24.173 53.974 5 24.013 55.976 5 23.390 55.858 50 253.773 454.147 50 247.095 443.215 50 254.044 449.043
          Hide
          Honghua Feng added a comment -

          Thanks stack. Looking forward to your test result on hdfs

          Show
          Honghua Feng added a comment - Thanks stack . Looking forward to your test result on hdfs
          Hide
          stack added a comment -

          Trying this on hdfs. It takes WAY longer. Threads stuck here:

          "t1" prio=10 tid=0x00007f49fd4fc800 nid=0xfc4 in Object.wait() [0x00007f49d0d99000]
             java.lang.Thread.State: TIMED_WAITING (on object monitor)
                  at java.lang.Object.wait(Native Method)
                  at org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1795)
                  - locked <0x0000000423dd1cc8> (a java.util.LinkedList)
                  at org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1689)
                  at org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1582)
                  at org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1567)
                  at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116)
                  at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.sync(ProtobufLogWriter.java:135)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.syncer(FSHLog.java:1072)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1195)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:910)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:844)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:838)
                  at org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation$HLogPutBenchmark.run(HLogPerformanceEvaluation.java:110)
                  at java.lang.Thread.run(Thread.java:662)
          

          Here are numbers I have so far for WITHOUT patch:

          /tmp/log-patch1.1.txt:2013-09-17 21:19:31,495 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 991.258s 1008.819ops/s
          /tmp/log-patch1.2.txt:2013-09-17 21:35:04,715 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 924.881s 1081.220ops/s
          /tmp/log-patch1.3.txt:2013-09-17 21:51:32,416 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 979.312s 1021.125ops/s
          /tmp/log-patch5.1.txt:2013-09-17 22:07:31,712 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 950.968s 5257.800ops/s
          /tmp/log-patch5.2.txt:2013-09-17 22:23:39,680 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 939.312s 5323.045ops/s

          Will clean up later but write rate is constant whether 1 or 5 threads. Will see when 50. Looks like I need to mode the HLogPE. It calls FSHLog#doWrite directory which is not as interesting since is by-passes locks in FSHLog when it does not call append.

          Will be back.

          Show
          stack added a comment - Trying this on hdfs. It takes WAY longer. Threads stuck here: "t1" prio=10 tid=0x00007f49fd4fc800 nid=0xfc4 in Object .wait() [0x00007f49d0d99000] java.lang. Thread .State: TIMED_WAITING (on object monitor) at java.lang. Object .wait(Native Method) at org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:1795) - locked <0x0000000423dd1cc8> (a java.util.LinkedList) at org.apache.hadoop.hdfs.DFSOutputStream.flushOrSync(DFSOutputStream.java:1689) at org.apache.hadoop.hdfs.DFSOutputStream.hflush(DFSOutputStream.java:1582) at org.apache.hadoop.hdfs.DFSOutputStream.sync(DFSOutputStream.java:1567) at org.apache.hadoop.fs.FSDataOutputStream.sync(FSDataOutputStream.java:116) at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogWriter.sync(ProtobufLogWriter.java:135) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.syncer(FSHLog.java:1072) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1195) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:910) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:844) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:838) at org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation$HLogPutBenchmark.run(HLogPerformanceEvaluation.java:110) at java.lang. Thread .run( Thread .java:662) Here are numbers I have so far for WITHOUT patch: /tmp/log-patch1.1.txt:2013-09-17 21:19:31,495 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 991.258s 1008.819ops/s /tmp/log-patch1.2.txt:2013-09-17 21:35:04,715 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 924.881s 1081.220ops/s /tmp/log-patch1.3.txt:2013-09-17 21:51:32,416 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 979.312s 1021.125ops/s /tmp/log-patch5.1.txt:2013-09-17 22:07:31,712 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 950.968s 5257.800ops/s /tmp/log-patch5.2.txt:2013-09-17 22:23:39,680 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 939.312s 5323.045ops/s Will clean up later but write rate is constant whether 1 or 5 threads. Will see when 50. Looks like I need to mode the HLogPE. It calls FSHLog#doWrite directory which is not as interesting since is by-passes locks in FSHLog when it does not call append. Will be back.
          Hide
          stack added a comment -

          Just parking the numbers here. Takes same time to write 5x as much (5M entries by 5 threads) as 1M entries by 1 thread. 50x takes 6x longer (50 threads writing 50M entries)

          /tmp/log-patch1.1.txt:2013-09-17 21:19:31,495 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 991.258s 1008.819ops/s
          /tmp/log-patch1.2.txt:2013-09-17 21:35:04,715 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 924.881s 1081.220ops/s
          /tmp/log-patch1.3.txt:2013-09-17 21:51:32,416 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 979.312s 1021.125ops/s
          /tmp/log-patch50.1.txt:2013-09-17 23:29:45,188 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=50, iterations=1000000 took 2960.095s 16891.350ops/s
          /tmp/log-patch50.2.txt:2013-09-18 00:22:48,849 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=50, iterations=1000000 took 2924.844s 17094.930ops/s
          /tmp/log-patch50.3.txt:2013-09-18 01:15:58,646 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=50, iterations=1000000 took 2927.617s 17078.736ops/s
          /tmp/log-patch5.1.txt:2013-09-17 22:07:31,712 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 950.968s 5257.800ops/s
          /tmp/log-patch5.2.txt:2013-09-17 22:23:39,680 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 939.312s 5323.045ops/s
          /tmp/log-patch5.3.txt:2013-09-17 22:39:56,011 INFO  [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 947.183s 5278.811ops/s
          
          Show
          stack added a comment - Just parking the numbers here. Takes same time to write 5x as much (5M entries by 5 threads) as 1M entries by 1 thread. 50x takes 6x longer (50 threads writing 50M entries) /tmp/log-patch1.1.txt:2013-09-17 21:19:31,495 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 991.258s 1008.819ops/s /tmp/log-patch1.2.txt:2013-09-17 21:35:04,715 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 924.881s 1081.220ops/s /tmp/log-patch1.3.txt:2013-09-17 21:51:32,416 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=1, iterations=1000000 took 979.312s 1021.125ops/s /tmp/log-patch50.1.txt:2013-09-17 23:29:45,188 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=50, iterations=1000000 took 2960.095s 16891.350ops/s /tmp/log-patch50.2.txt:2013-09-18 00:22:48,849 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=50, iterations=1000000 took 2924.844s 17094.930ops/s /tmp/log-patch50.3.txt:2013-09-18 01:15:58,646 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=50, iterations=1000000 took 2927.617s 17078.736ops/s /tmp/log-patch5.1.txt:2013-09-17 22:07:31,712 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 950.968s 5257.800ops/s /tmp/log-patch5.2.txt:2013-09-17 22:23:39,680 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 939.312s 5323.045ops/s /tmp/log-patch5.3.txt:2013-09-17 22:39:56,011 INFO [main] wal.HLogPerformanceEvaluation: Summary: threads=5, iterations=1000000 took 947.183s 5278.811ops/s
          Hide
          stack added a comment -

          I was wrong that HLogPE called doWrite. It calls append as an HRegionServer would.

          Here are the numbers. They confirm what chunhui shen found way back up top of this issue.

          I was running HLogPE against a five node HDFS cluster. The DataNodes were persisting to a fusionio drive so little friction at the drive.

          Below are seconds elapsed running following:

          $ for i in 1 5 50; do  for j in 1 2 3; do ./bin/hbase --config /home/stack/conf_hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation  -verify -threads "${i}" -iterations 1000000 -nocleanup  -keySize 50 -valueSize 100 &> /tmp/log-patch"${i}"."${j}".txt; done; done
          
          Thread Count WithoutPatch WithPatch %diff
          1 991.258 1125.208 -11.90
          1 924.881 1137.754 -18.70
          1 979.312 1142.959 -14.31
          5 950.968 1914.448 -50.32
          5 939.312 1918.188 -51.03
          5 947.183 1939.806 -51.17
          50 2960.095 1918.808 54.26
          50 2924.844 1933.020 51.30
          50 2927.617 1955.358 49.72

          So, about 20% slower when single threaded writes. About 50% slower when five threads writing concurrently BUT 50% faster when 50 concurrent threads (our current default).

          Can we have the best of both worlds somehow where we switch to this new model when high contention?

          Show
          stack added a comment - I was wrong that HLogPE called doWrite. It calls append as an HRegionServer would. Here are the numbers. They confirm what chunhui shen found way back up top of this issue. I was running HLogPE against a five node HDFS cluster. The DataNodes were persisting to a fusionio drive so little friction at the drive. Below are seconds elapsed running following: $ for i in 1 5 50; do for j in 1 2 3; do ./bin/hbase --config /home/stack/conf_hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -verify -threads "${i}" -iterations 1000000 -nocleanup -keySize 50 -valueSize 100 &> /tmp/log-patch "${i}" . "${j}" .txt; done; done Thread Count WithoutPatch WithPatch %diff 1 991.258 1125.208 -11.90 1 924.881 1137.754 -18.70 1 979.312 1142.959 -14.31 5 950.968 1914.448 -50.32 5 939.312 1918.188 -51.03 5 947.183 1939.806 -51.17 50 2960.095 1918.808 54.26 50 2924.844 1933.020 51.30 50 2927.617 1955.358 49.72 So, about 20% slower when single threaded writes. About 50% slower when five threads writing concurrently BUT 50% faster when 50 concurrent threads (our current default). Can we have the best of both worlds somehow where we switch to this new model when high contention?
          Hide
          Hangjun Ye added a comment -

          Thanks stack for doing this experiment! The result looks positive in some way.
          China is in holiday (Mid-Autumn Festival) until this Saturday, we would check your suggestion after the holiday.

          Show
          Hangjun Ye added a comment - Thanks stack for doing this experiment! The result looks positive in some way. China is in holiday (Mid-Autumn Festival) until this Saturday, we would check your suggestion after the holiday.
          Hide
          stack added a comment - - edited

          Hangjun Ye Happy holidays! I didn't profile to see where time is spent. That would be next I'd say.

          Show
          stack added a comment - - edited Hangjun Ye Happy holidays! I didn't profile to see where time is spent. That would be next I'd say.
          Hide
          Honghua Feng added a comment -

          stack thanks so much for the detailed comparison test!

          1. what are the respective OPS with/without patch? – I wonder if the test stresses out the maximum throughput
          2. is the write load against a single node, or five nodes? – to confirm the throughput is per-node or per-cluster(with five nodes)

          We re-did a series of comparison tests in bed and use different HDFS/HBase versions, below are our conclusion (single node):
          1. for trunk, the (maximum) throughout improvement is about 150% (2.5 times)
          2. for 0.94.3(our internal branch), the (maximum) throughput improvement is about 200% (3 times)
          3. for trunk against hdfs-3u3(chunhui shen used), the (maximum) throughput improvement is 40% (28000 vs. 20000) – we wonder hdfs performance has some downgrade from 3u3 since current hdfs can't see a high hlog throughput such as 20000

          We'll redo the comparison test as you using HLogPE(we always use YCSB, not use HLogPE) to see if there is -50% downgrade when the thread number is 5.

          Show
          Honghua Feng added a comment - stack thanks so much for the detailed comparison test! 1. what are the respective OPS with/without patch? – I wonder if the test stresses out the maximum throughput 2. is the write load against a single node, or five nodes? – to confirm the throughput is per-node or per-cluster(with five nodes) We re-did a series of comparison tests in bed and use different HDFS/HBase versions, below are our conclusion (single node): 1. for trunk, the (maximum) throughout improvement is about 150% (2.5 times) 2. for 0.94.3(our internal branch), the (maximum) throughput improvement is about 200% (3 times) 3. for trunk against hdfs-3u3( chunhui shen used), the (maximum) throughput improvement is 40% (28000 vs. 20000) – we wonder hdfs performance has some downgrade from 3u3 since current hdfs can't see a high hlog throughput such as 20000 We'll redo the comparison test as you using HLogPE(we always use YCSB, not use HLogPE ) to see if there is -50% downgrade when the thread number is 5.
          Hide
          Honghua Feng added a comment -

          If I weren't in Germany right with no VPN access I'd offer to load 0.94 with the patch onto one of our test clusters and run some workload on it...

          Lars Hofhansl Wonder if you now have access to your test clusters, if possible would you please also help do the performance comparison test on your test clusters? (I'm confused why no one else get comparable improvement test result as ours, while we get test results with almost same improvement for multi-tests)

          Show
          Honghua Feng added a comment - If I weren't in Germany right with no VPN access I'd offer to load 0.94 with the patch onto one of our test clusters and run some workload on it... Lars Hofhansl Wonder if you now have access to your test clusters, if possible would you please also help do the performance comparison test on your test clusters? (I'm confused why no one else get comparable improvement test result as ours, while we get test results with almost same improvement for multi-tests)
          Hide
          stack added a comment -

          What was implication of not shutting this? Were tests failing or is this just make-work?

          I did not save avg ops/s. It was a set amount of work.

          2. is the write load against a single node, or five nodes? – to confirm the throughput is per-node or per-cluster(with five nodes)

          I had a client writing to a WAL hosted in hdfs on a 5-node HDFS cluster.

          Thanks Honghua Feng

          Show
          stack added a comment - What was implication of not shutting this? Were tests failing or is this just make-work? I did not save avg ops/s. It was a set amount of work. 2. is the write load against a single node, or five nodes? – to confirm the throughput is per-node or per-cluster(with five nodes) I had a client writing to a WAL hosted in hdfs on a 5-node HDFS cluster. Thanks Honghua Feng
          Hide
          Liu Shaohui added a comment -

          stack Honghua Feng
          We redo the comparision test using HogPE. Here are the results:

          Test env:

          hdfs: cdh 4.1.0, five datanode, each node has 12 sata disks.
          hbase: 0.94.3
          HLogPE is run on one of these datanodes, so one replica of hlog's block will be at local datanode.

          The params of HLogPE are: -iterations 1000000 -keySize 50 -valueSize 100, which are same as stack's tests.

           
          for i in 1 5 50 75 100; do 
              for j in 1 2 3; do
                  ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -verify -threads "${i}" -iterations 1000000 -keySize 50 -valueSize 100 &> log-patch"${i}"."${j}".txt; 
                  grep "Summary: " log-patch"${i}"."${j}".txt
              done; 
          done
          
          Thread Count WithoutPatch WithPatch
          1 579.380 625.937 -8.03
          1 580.307 630.346 -8.62
          1 577.853 654.20 -13.21
          5 799.579 785.696 1.73
          5 795.013 780.642 1.80
          5 826.270 781.909 5.36
          50 3290.482 1165.773 64.57
          50 3298.387 1167.992 64.58
          50 3224.495 1154.921 64.18
          75 4450.760 1253.448 71.83
          75 4506.143 1269.806 71.82
          75 4516.453 1245.954 72.41
          100 5561.074 1493.102 73.15
          100 5616.810 1496.263 73.36
          100 5612.268 1468.500 73.83

          a, When thread number is 1, we see that the performance of our test is about 40% better than that of stack's test, both in old thread mode and new thread mode. stack what's the hdfs version in your test or are there special configs?
          b, When thread number is 5, we do not see -50% downgrade.

          Show
          Liu Shaohui added a comment - stack Honghua Feng We redo the comparision test using HogPE. Here are the results: Test env: hdfs: cdh 4.1.0, five datanode, each node has 12 sata disks. hbase: 0.94.3 HLogPE is run on one of these datanodes, so one replica of hlog's block will be at local datanode. The params of HLogPE are: -iterations 1000000 -keySize 50 -valueSize 100, which are same as stack's tests. for i in 1 5 50 75 100; do for j in 1 2 3; do ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -verify -threads "${i}" -iterations 1000000 -keySize 50 -valueSize 100 &> log-patch "${i}" . "${j}" .txt; grep "Summary: " log-patch "${i}" . "${j}" .txt done; done Thread Count WithoutPatch WithPatch 1 579.380 625.937 -8.03 1 580.307 630.346 -8.62 1 577.853 654.20 -13.21 5 799.579 785.696 1.73 5 795.013 780.642 1.80 5 826.270 781.909 5.36 50 3290.482 1165.773 64.57 50 3298.387 1167.992 64.58 50 3224.495 1154.921 64.18 75 4450.760 1253.448 71.83 75 4506.143 1269.806 71.82 75 4516.453 1245.954 72.41 100 5561.074 1493.102 73.15 100 5616.810 1496.263 73.36 100 5612.268 1468.500 73.83 a, When thread number is 1, we see that the performance of our test is about 40% better than that of stack's test, both in old thread mode and new thread mode. stack what's the hdfs version in your test or are there special configs? b, When thread number is 5, we do not see -50% downgrade.
          Hide
          Liu Shaohui added a comment -

          update the result table and add ops diff

          Thread number Time without Patch Ops without Patch Time with Patch Ops with Patch Time diff % Ops diff %
          1 579.38 1725.983 625.937 1597.605 -8.04 -7.44
          1 580.307 1723.226 630.346 1586.43 -8.62 -7.94
          1 577.853 1730.544 654.205 1528.573 -13.21 -11.67
          5 799.579 6253.291 785.696 6363.785 1.74 1.77
          5 795.013 6289.206 780.642 6404.984 1.81 1.84
          5 826.27 6051.291 781.909 6394.606 5.37 5.67
          50 3290.482 15195.343 1165.773 42890 64.57 182.26
          50 3298.387 15158.925 1167.992 42808.516 64.59 182.40
          50 3224.495 15506.304 1154.921 43293.004 64.18 179.20
          75 4450.76 16851.055 1253.448 59834.953 71.84 255.08
          75 4506.143 16643.945 1269.806 59064.141 71.82 254.87
          75 4516.453 16605.951 1245.954 60194.84 72.41 262.49
          100 5561.074 17982.137 1493.102 66974.656 73.15 272.45
          100 5616.81 17803.699 1496.263 66833.172 73.36 275.39
          100 5612.268 17818.107 1468.5 68096.695 73.83 282.18

          Time diff = (Time without Patch - Time with Patch) / Time without Patch * 100
          Ops diff = (Ops with Patch - Ops without Patch) / Ops without Patch * 100

          stack What are the hdfs and hbase version of your test? We may rebo the tests in cluster with same hdfs and hbase versions as yours.

          Show
          Liu Shaohui added a comment - update the result table and add ops diff Thread number Time without Patch Ops without Patch Time with Patch Ops with Patch Time diff % Ops diff % 1 579.38 1725.983 625.937 1597.605 -8.04 -7.44 1 580.307 1723.226 630.346 1586.43 -8.62 -7.94 1 577.853 1730.544 654.205 1528.573 -13.21 -11.67 5 799.579 6253.291 785.696 6363.785 1.74 1.77 5 795.013 6289.206 780.642 6404.984 1.81 1.84 5 826.27 6051.291 781.909 6394.606 5.37 5.67 50 3290.482 15195.343 1165.773 42890 64.57 182.26 50 3298.387 15158.925 1167.992 42808.516 64.59 182.40 50 3224.495 15506.304 1154.921 43293.004 64.18 179.20 75 4450.76 16851.055 1253.448 59834.953 71.84 255.08 75 4506.143 16643.945 1269.806 59064.141 71.82 254.87 75 4516.453 16605.951 1245.954 60194.84 72.41 262.49 100 5561.074 17982.137 1493.102 66974.656 73.15 272.45 100 5616.81 17803.699 1496.263 66833.172 73.36 275.39 100 5612.268 17818.107 1468.5 68096.695 73.83 282.18 Time diff = (Time without Patch - Time with Patch) / Time without Patch * 100 Ops diff = (Ops with Patch - Ops without Patch) / Ops without Patch * 100 stack What are the hdfs and hbase version of your test? We may rebo the tests in cluster with same hdfs and hbase versions as yours.
          Hide
          Liu Shaohui added a comment -

          stack Honghua FengHangjun Ye
          We redo the same HogPE comparision test using hbase trunk. Results are followed.

          Test env:
          hdfs: cdh 4.1.0, five datanode, each node has 12 sata disks.
          hbase: hbase trunk 0.97 r1516083 (for in r1516084, trunk update the protobuf to 2.5 which is incompatible with protobuf 2.4 used in cdh 4.1.0)
          HLogPE is run on one of these datanodes, so one replica of hlog's block will be at local datanode.

          Thread number Time without Patch(s) Ops without Patch Time with Patch(s) Ops with Patch Time diff % Ops diff %
          1 580.309 1723.22 624.709 1600.745 -7.65 -7.11
          1 591.177 1691.541 631.34 1583.932 -6.79 -6.36
          1 591.948 1689.338 634.518 1575.999 -7.19 -6.71
          5 794.034 6296.959 1201.563 4161.247 -51.32 -33.92
          5 781.033 6401.778 1191.776 4195.419 -52.59 -34.46
          5 805.597 6206.577 1187.179 4211.665 -47.37 -32.14
          50 3222.659 15515.139 1815.586 27539.316 43.66 77.50
          50 3191.131 15668.426 1821.956 27443.033 42.91 75.15
          50 3222.407 15516.352 1817.754 27506.473 43.59 77.27
          75 4517.149 16603.393 2024.359 37048.766 55.19 123.14
          75 4498.987 16670.42 2016.899 37185.797 55.17 123.06
          75 4554.122 16468.598 2037.155 36816.051 55.27 123.55
          100 5186.292 19281.598 2147.581 46564.016 58.59 141.49
          100 5181.344 19300.012 2135.768 46821.563 58.78 142.60
          100 5189.396 19270.064 2143.529 46652.039 58.69 142.10
          Show
          Liu Shaohui added a comment - stack Honghua Feng Hangjun Ye We redo the same HogPE comparision test using hbase trunk. Results are followed. Test env: hdfs: cdh 4.1.0, five datanode, each node has 12 sata disks. hbase: hbase trunk 0.97 r1516083 (for in r1516084, trunk update the protobuf to 2.5 which is incompatible with protobuf 2.4 used in cdh 4.1.0) HLogPE is run on one of these datanodes, so one replica of hlog's block will be at local datanode. Thread number Time without Patch(s) Ops without Patch Time with Patch(s) Ops with Patch Time diff % Ops diff % 1 580.309 1723.22 624.709 1600.745 -7.65 -7.11 1 591.177 1691.541 631.34 1583.932 -6.79 -6.36 1 591.948 1689.338 634.518 1575.999 -7.19 -6.71 5 794.034 6296.959 1201.563 4161.247 -51.32 -33.92 5 781.033 6401.778 1191.776 4195.419 -52.59 -34.46 5 805.597 6206.577 1187.179 4211.665 -47.37 -32.14 50 3222.659 15515.139 1815.586 27539.316 43.66 77.50 50 3191.131 15668.426 1821.956 27443.033 42.91 75.15 50 3222.407 15516.352 1817.754 27506.473 43.59 77.27 75 4517.149 16603.393 2024.359 37048.766 55.19 123.14 75 4498.987 16670.42 2016.899 37185.797 55.17 123.06 75 4554.122 16468.598 2037.155 36816.051 55.27 123.55 100 5186.292 19281.598 2147.581 46564.016 58.59 141.49 100 5181.344 19300.012 2135.768 46821.563 58.78 142.60 100 5189.396 19270.064 2143.529 46652.039 58.69 142.10
          Hide
          stack added a comment -

          Honghua Feng Our results are the same?

          Show
          stack added a comment - Honghua Feng Our results are the same?
          Hide
          Honghua Feng added a comment -

          stack: Yes, seems this patch against trunk has obvious downgrade compared to against 0.94.3(our internal branch) for:
          1) 5 threads: withPatch has worse perf than withoutPatch (33% ops downgrade, 4.2K vs. 6.3K)
          2) 100 threads: withpatch's perf is about 2.5X of withoutPatch (46.6K vs. 19.2K)

          A short summary:
          1) withoutPatch, the max ops of HLog is less than 20K (19K for trunk and 17K for 0.94.3)
          2) withPatch, the max ops of HLog is more than 45K (46K for trunk and 68K for 0.94.3)
          3) for trunk, withPatch can have even worse perf than withoutPatch (about 33% downgrade)

          We'll try to figure out why withPatch performs worse than withoutPatch for trunk, and try to ensure the performance is about equal when stress is low and still keep obvious upgrade when stress is high.

          stack : would you please redo the test using 75/100 threads to re-confirm whether the ops upgrade matches our tests? (we see 37K vs 16K for 75 threads and 46K vs 19K for 100 threads)

          chunhui shen : what version of HBase do you apply the patch to? trunk or 0.94? I wonder if the reason is the same for our difference and stack's

          Show
          Honghua Feng added a comment - stack : Yes, seems this patch against trunk has obvious downgrade compared to against 0.94.3(our internal branch) for: 1) 5 threads: withPatch has worse perf than withoutPatch (33% ops downgrade, 4.2K vs. 6.3K) 2) 100 threads: withpatch's perf is about 2.5X of withoutPatch (46.6K vs. 19.2K) A short summary: 1) withoutPatch, the max ops of HLog is less than 20K (19K for trunk and 17K for 0.94.3) 2) withPatch, the max ops of HLog is more than 45K (46K for trunk and 68K for 0.94.3) 3) for trunk, withPatch can have even worse perf than withoutPatch (about 33% downgrade) We'll try to figure out why withPatch performs worse than withoutPatch for trunk, and try to ensure the performance is about equal when stress is low and still keep obvious upgrade when stress is high. stack : would you please redo the test using 75/100 threads to re-confirm whether the ops upgrade matches our tests? (we see 37K vs 16K for 75 threads and 46K vs 19K for 100 threads) chunhui shen : what version of HBase do you apply the patch to? trunk or 0.94? I wonder if the reason is the same for our difference and stack's
          Hide
          stack added a comment -

          Honghua Feng will do. Sorry. Meant to do it earlier. Will try to do some profiling too myself to see why worse wen low thread count.

          Show
          stack added a comment - Honghua Feng will do. Sorry. Meant to do it earlier. Will try to do some profiling too myself to see why worse wen low thread count.
          Hide
          Honghua Feng added a comment -

          stack thanks a lot

          btw: the HLogPE test result withoutPatch/withPatch against 0.94.3(our internal branch) done by Liu Shaohui matches the one against hdfs sequence file, described in the 'Description' of this JIRA as below (17K vs. 68K):

          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+).

          Show
          Honghua Feng added a comment - stack thanks a lot btw: the HLogPE test result withoutPatch/withPatch against 0.94.3(our internal branch) done by Liu Shaohui matches the one against hdfs sequence file, described in the 'Description' of this JIRA as below (17K vs. 68K): 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+).
          Hide
          stack added a comment -

          Does that mean that we can just use HLogPE going forward evaluating this patch and not have to set up a cluster? Because it gives same results as a cluster does? Thanks Honghua Feng

          Show
          stack added a comment - Does that mean that we can just use HLogPE going forward evaluating this patch and not have to set up a cluster? Because it gives same results as a cluster does? Thanks Honghua Feng
          Hide
          Honghua Feng added a comment -

          stack: Agree. Seems HLogPE is enough for profiling/evaluating, let us run YCSB on a cluster for later double verification?

          Show
          Honghua Feng added a comment - stack : Agree. Seems HLogPE is enough for profiling/evaluating, let us run YCSB on a cluster for later double verification?
          Hide
          Hangjun Ye added a comment -

          To clarify, a hdfs cluster is still needed to store the sequence file.

          Show
          Hangjun Ye added a comment - To clarify, a hdfs cluster is still needed to store the sequence file.
          Hide
          stack added a comment -

          Five datanodes with fusionio. hbase tip of 0.96 branch and hadoop-2.1.0-beta. HLogPE on master node.

          Threads w/o patch time w/o patch ops w/ patch time w/ patch ops
          1 1048.033s 954.168ops/s 1100.423s 908.741ops/s
          1 1042.126s 959.577ops/s 1156.557s 864.635ops/s
          1 1052.601s 950.028ops/s 1143.271s 874.683ops/s
          5 904.176s 5529.896ops/s 1916.229s 2609.292ops/s
          5 910.469s 5491.675ops/s 1911.841s 2615.280ops/s
          5 925.778s 5400.863ops/s 1970.565s 2537.344ops/s
          50 2699.752s 18520.221ops/s 1889.877s 26456.748ops/s
          50 2689.678s 18589.586ops/s 1922.716s 26004.881ops/s
          50 2711.144s 18442.398ops/s 1893.439s 26406.977ops/s
          75 4945.563s 15165.108ops/s 1997.553s 37545.938ops/s
          75 4852.779s 15455.063ops/s 1992.425s 37642.570ops/s
          75 4921.685s 15238.684ops/s - -
          100 6224.527s 16065.479ops/s 2086.691s 47922.766ops/s
          100 6195.727s 16140.156ops/s 2091.869s 47804.145ops/s

          Diffs are small when 1 thread only. Its bad at 5 threads but thereafter the patch starts to shine. If we could make the 5 threads better, we could commit this patch.

          Show
          stack added a comment - Five datanodes with fusionio. hbase tip of 0.96 branch and hadoop-2.1.0-beta. HLogPE on master node. Threads w/o patch time w/o patch ops w/ patch time w/ patch ops 1 1048.033s 954.168ops/s 1100.423s 908.741ops/s 1 1042.126s 959.577ops/s 1156.557s 864.635ops/s 1 1052.601s 950.028ops/s 1143.271s 874.683ops/s 5 904.176s 5529.896ops/s 1916.229s 2609.292ops/s 5 910.469s 5491.675ops/s 1911.841s 2615.280ops/s 5 925.778s 5400.863ops/s 1970.565s 2537.344ops/s 50 2699.752s 18520.221ops/s 1889.877s 26456.748ops/s 50 2689.678s 18589.586ops/s 1922.716s 26004.881ops/s 50 2711.144s 18442.398ops/s 1893.439s 26406.977ops/s 75 4945.563s 15165.108ops/s 1997.553s 37545.938ops/s 75 4852.779s 15455.063ops/s 1992.425s 37642.570ops/s 75 4921.685s 15238.684ops/s - - 100 6224.527s 16065.479ops/s 2086.691s 47922.766ops/s 100 6195.727s 16140.156ops/s 2091.869s 47804.145ops/s Diffs are small when 1 thread only. Its bad at 5 threads but thereafter the patch starts to shine. If we could make the 5 threads better, we could commit this patch.
          Hide
          Honghua Feng added a comment -

          stack thanks a lot for the detailed perf test. agree with you. we'll try to make the 5 threads better

          Show
          Honghua Feng added a comment - stack thanks a lot for the detailed perf test. agree with you. we'll try to make the 5 threads better
          Hide
          Sergey Shelukhin added a comment -

          Question from above:

          ...is separate notifier thread necessary? It doesn't do any blocking operations other than interacting with flusher thread, or taking syncedTillHere lock, which looks like it should be uncontested most of the time.
          Couldn't flusher thread have the 4~ lines that set syncedTillHere?

          Show
          Sergey Shelukhin added a comment - Question from above: ...is separate notifier thread necessary? It doesn't do any blocking operations other than interacting with flusher thread, or taking syncedTillHere lock, which looks like it should be uncontested most of the time. Couldn't flusher thread have the 4~ lines that set syncedTillHere?
          Hide
          Hangjun Ye added a comment -

          Sergey Shelukhin, that's a very good question.
          We found syncedTillHere.notifyAll() was a blocking operation and it might take a long time if many threads were waiting on it (which also surprised us). So we put this logic in a separate thread.

          Show
          Hangjun Ye added a comment - Sergey Shelukhin , that's a very good question. We found syncedTillHere.notifyAll() was a blocking operation and it might take a long time if many threads were waiting on it (which also surprised us). So we put this logic in a separate thread.
          Hide
          Sergey Shelukhin added a comment -

          nice. Thanks!

          Show
          Sergey Shelukhin added a comment - nice. Thanks!
          Hide
          stack added a comment -

          Moving out of 0.96.1. We'll get it when we get it.

          Show
          stack added a comment - Moving out of 0.96.1. We'll get it when we get it.
          Hide
          Jean-Marc Spaggiari added a comment -

          Honghua Feng, any chance to rebase on the last 0.96 and optimize for less than 5 clients? If so I can give it a spin on a small cluster and run the PerfEval suite for 24 h to see the results?

          Show
          Jean-Marc Spaggiari added a comment - Honghua Feng , any chance to rebase on the last 0.96 and optimize for less than 5 clients? If so I can give it a spin on a small cluster and run the PerfEval suite for 24 h to see the results?
          Hide
          Honghua Feng added a comment -

          Jean-Marc Spaggiari sure, I'm tuning for less than 5 threads, and will provide patch based on the last 0.96 when the bottleneck is found and the tuning is done.

          Show
          Honghua Feng added a comment - Jean-Marc Spaggiari sure, I'm tuning for less than 5 threads, and will provide patch based on the last 0.96 when the bottleneck is found and the tuning is done.
          Hide
          Honghua Feng added a comment -

          stack : I fixed the downgrade for 5 threads and below are the test result, the write throughput capacity after tuning is about 3.5 times of the one before tuning while keeping the perf is almost equal when thread number is <= 5. (for protobuf/test compatibility reason I based on hbase trunk 0.97 r1516083)

            for i in 1 3 5 10 25 50 100 200; do
              for j in 1; do
                  ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -path /user/h_fenghonghua/new-thread-v2/ -verify -threads "${i}" -iterations 1000000 -keySize 50 -valueSize 100 &> log-patch"${i}"."${j}".txt;
                  grep "Summary: " log-patch"${i}"."${j}".txt
              done;
          done
          
          threads time-without-patch ops-without-patch time-with-patch ops-with-patch ops-diff
          1 582 1716 630 1586 -7.5%
          3 943 3179 951 3153 -0.8%
          5 820 6091 847 5899 -3.1%
          10 1141 8760 983 10166 +16%
          25 1920 13019 1286 19426 +49.2%
          50 3334 14995 1627 30715 +104.8%
          100 5312 18824 1925 51943 +185%
          200 11022 18144 3229 61922 +241.2%

          Jean-Marc Spaggiari : I attached patches for latest trunk (HBASE-8755-trunk-v4.patch) and for the last 0.96 (HBASE-8755-0.96-v0.patch). Thanks very much if you can run similar comparison perf tests, and any issue please feel free to raise

          Show
          Honghua Feng added a comment - stack : I fixed the downgrade for 5 threads and below are the test result, the write throughput capacity after tuning is about 3.5 times of the one before tuning while keeping the perf is almost equal when thread number is <= 5. (for protobuf/test compatibility reason I based on hbase trunk 0.97 r1516083) for i in 1 3 5 10 25 50 100 200; do for j in 1; do ./bin/hbase org.apache.hadoop.hbase.regionserver.wal.HLogPerformanceEvaluation -path /user/h_fenghonghua/ new -thread-v2/ -verify -threads "${i}" -iterations 1000000 -keySize 50 -valueSize 100 &> log-patch "${i}" . "${j}" .txt; grep "Summary: " log-patch "${i}" . "${j}" .txt done; done threads time-without-patch ops-without-patch time-with-patch ops-with-patch ops-diff 1 582 1716 630 1586 -7.5% 3 943 3179 951 3153 -0.8% 5 820 6091 847 5899 -3.1% 10 1141 8760 983 10166 +16% 25 1920 13019 1286 19426 +49.2% 50 3334 14995 1627 30715 +104.8% 100 5312 18824 1925 51943 +185% 200 11022 18144 3229 61922 +241.2% Jean-Marc Spaggiari : I attached patches for latest trunk ( HBASE-8755 -trunk-v4.patch) and for the last 0.96 ( HBASE-8755 -0.96-v0.patch). Thanks very much if you can run similar comparison perf tests, and any issue please feel free to raise
          Hide
          stack added a comment -

          I'm starting tests running now.

          Show
          stack added a comment - I'm starting tests running now.
          Hide
          stack added a comment -

          I ran Feng's script except I left out trying 200 threads.

          Threads time-wo-patch ops-wo-patch time-w-patch ops-w-patch ops-diff
          1 973.673 1027.039 1119.825 892.997 -15%
          3 1303.891 2300.806 1400.848 2141.560 -7%
          5 855.775 5842.657 873.990 5720.889 -2%
          10 1093.330 9146.370 1090.158 9172.982 0%
          25 1632.263 15316.160 1215.196 20572.813 +25%
          50 2432.653 20553.691 1341.847 37262.070 +45%
          100 4058.650 24638.734 1725.729 57946.527 57%

          This was stock hadoop 2.2 and tip of the hbase trunk.

          Those are pretty big improvements once we pass out ten concurrent threads. Some small down when one writer only is acceptable I'd say. Let me look again at the patch.

          Show
          stack added a comment - I ran Feng's script except I left out trying 200 threads. Threads time-wo-patch ops-wo-patch time-w-patch ops-w-patch ops-diff 1 973.673 1027.039 1119.825 892.997 -15% 3 1303.891 2300.806 1400.848 2141.560 -7% 5 855.775 5842.657 873.990 5720.889 -2% 10 1093.330 9146.370 1090.158 9172.982 0% 25 1632.263 15316.160 1215.196 20572.813 +25% 50 2432.653 20553.691 1341.847 37262.070 +45% 100 4058.650 24638.734 1725.729 57946.527 57% This was stock hadoop 2.2 and tip of the hbase trunk. Those are pretty big improvements once we pass out ten concurrent threads. Some small down when one writer only is acceptable I'd say. Let me look again at the patch.
          Hide
          stack added a comment -

          We looked at using disruptor here? MPSC seems to be what we have going on here to which disruptor seems pretty well suited? Looking at the patch now...

          Show
          stack added a comment - We looked at using disruptor here? MPSC seems to be what we have going on here to which disruptor seems pretty well suited? Looking at the patch now...
          Hide
          Honghua Feng added a comment -

          Thanks stack

          A small clarification of why we got so different ops-diff: I used ops-diff = (new - old) / old and you used ops-diff = (new - old) / new. for example, the 100 threads result, old one is 24638 and new one is 57946, the ops-diff is 135.2% (new one is 2.35 times of old one), while you got 57%

          Show
          Honghua Feng added a comment - Thanks stack A small clarification of why we got so different ops-diff: I used ops-diff = (new - old) / old and you used ops-diff = (new - old) / new . for example, the 100 threads result, old one is 24638 and new one is 57946, the ops-diff is 135.2% (new one is 2.35 times of old one), while you got 57%
          Hide
          stack added a comment -

          Pardon my bad math Honghua Feng Below is correction (numbers are better now).

          Threads time-wo-patch ops-wo-patch time-w-patch ops-w-patch ops-diff
          1 973.673 1027.039 1119.825 892.997 -15%
          3 1303.891 2300.806 1400.848 2141.560 -7%
          5 855.775 5842.657 873.990 5720.889 -2%
          10 1093.330 9146.370 1090.158 9172.982 0%
          25 1632.263 15316.160 1215.196 20572.813 +34%
          50 2432.653 20553.691 1341.847 37262.070 +81%
          100 4058.650 24638.734 1725.729 57946.527 +135%
          Show
          stack added a comment - Pardon my bad math Honghua Feng Below is correction (numbers are better now). Threads time-wo-patch ops-wo-patch time-w-patch ops-w-patch ops-diff 1 973.673 1027.039 1119.825 892.997 -15% 3 1303.891 2300.806 1400.848 2141.560 -7% 5 855.775 5842.657 873.990 5720.889 -2% 10 1093.330 9146.370 1090.158 9172.982 0% 25 1632.263 15316.160 1215.196 20572.813 +34% 50 2432.653 20553.691 1341.847 37262.070 +81% 100 4058.650 24638.734 1725.729 57946.527 +135%
          Hide
          stack added a comment -

          Looking at patch...

          Do these no longer pass?

          • assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries());
            + //assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries());

          We have hard-coded 5 asyncSyncers? Why 5?

          We are starting 7 threads to run this new write model : the 5 syncers, a notifier, and a writer (I suppose we are also removing one, the old LogSyncer – so 6 new threads).

          We set the below outside of a sync block:

          + this.asyncWriter.setPendingTxid(txid);

          Could we set the txid out of order here? If so, will that be a problem? Hmm... it seems not. If we get a lower txid, we just return and prevail w/ the highest we've seen.

          If we fail to find a free syncer, i don't follow what is going on w/ choosing a random syncer and setting txid as in below (Generally comments in patch are good – this looks like a section that could do w/ some):

          + this.lastWrittenTxid = this.txidToWrite;
          + boolean hasIdleSyncer = false;
          + for (int i = 0; i < asyncSyncers.length; ++i) {
          + if (!asyncSyncers[i].isSyncing())

          { + hasIdleSyncer = true; + asyncSyncers[i].setWrittenTxid(this.lastWrittenTxid); + break; + }

          + }
          + if (!hasIdleSyncer)

          { + int idx = rd.nextInt(asyncSyncers.length); + asyncSyncers[idx].setWrittenTxid(this.lastWrittenTxid); + }

          Thanks.

          Show
          stack added a comment - Looking at patch... Do these no longer pass? assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries()); + //assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries()); We have hard-coded 5 asyncSyncers? Why 5? We are starting 7 threads to run this new write model : the 5 syncers, a notifier, and a writer (I suppose we are also removing one, the old LogSyncer – so 6 new threads). We set the below outside of a sync block: + this.asyncWriter.setPendingTxid(txid); Could we set the txid out of order here? If so, will that be a problem? Hmm... it seems not. If we get a lower txid, we just return and prevail w/ the highest we've seen. If we fail to find a free syncer, i don't follow what is going on w/ choosing a random syncer and setting txid as in below (Generally comments in patch are good – this looks like a section that could do w/ some): + this.lastWrittenTxid = this.txidToWrite; + boolean hasIdleSyncer = false; + for (int i = 0; i < asyncSyncers.length; ++i) { + if (!asyncSyncers [i] .isSyncing()) { + hasIdleSyncer = true; + asyncSyncers[i].setWrittenTxid(this.lastWrittenTxid); + break; + } + } + if (!hasIdleSyncer) { + int idx = rd.nextInt(asyncSyncers.length); + asyncSyncers[idx].setWrittenTxid(this.lastWrittenTxid); + } Thanks.
          Hide
          Honghua Feng added a comment -

          Do these no longer pass?

          => yes, under new thread model, no explicit method to do the sync and can't tell if there is outstanding deferred entries (the AsyncWriter/AsyncSyncer threads do write/sync in a best-effort way)

          We have hard-coded 5 asyncSyncers? Why 5?

          => yes, I tried 2/3/5/10 and found 5 is the best number (2/3 have worse perf, 10 has equal perf but introduces too many extra threads)

          If we fail to find a free syncer, i don't follow what is going on w/ choosing a random syncer and setting txid as in below

          => when fail to find a idle syncer(which is doing sync), choosing a random syncer and setting txid that way fall into the same way before introducing extra asyncSyncer threads: when asyncWriter pushes new entries to hdfs before asyncSyncer sync the previously pushed ones, asyncSyncer gets notified the newly pushed txid, but these txid will be synced by next time after asyncSyncer is done with the current ones, notice we use txidToFlush to record txid each sync is for, and it can't change during each sync, while writtenTxid can change during each sync)

          To summary: the sync operation is the most time-consuming phase, under old write model every write handler issues a separate sync directly for itself(if not return early by syncedTillHere). and under new write model, though separate threads significantly reduce the lock race, but if concurrent write threads is few, the benefit by reducing lock race(fewer write threads, fewer benefit) can't offset the inefficiency by using a single asyncSyncer threads(each time asyncSyncer thread can only sync for a portion of the writes, but the write handlers which already have their entries in buffer or pushed to hdfs also need to wait for its completeness, and can't proceed until its next sync phase is done)
          By introducing extra asyncSyncer threads, the correctness of this model is the same as before: still a single asyncWriter thread which push buffered entries to hdfs sequentially(txid increases sequentially), and when each asyncSyncer is done, it's guaranteed all txids smaller are pushed to hdfs and successfully sync-ed.

          Show
          Honghua Feng added a comment - Do these no longer pass? => yes, under new thread model, no explicit method to do the sync and can't tell if there is outstanding deferred entries (the AsyncWriter/AsyncSyncer threads do write/sync in a best-effort way) We have hard-coded 5 asyncSyncers? Why 5? => yes, I tried 2/3/5/10 and found 5 is the best number (2/3 have worse perf, 10 has equal perf but introduces too many extra threads) If we fail to find a free syncer, i don't follow what is going on w/ choosing a random syncer and setting txid as in below => when fail to find a idle syncer(which is doing sync), choosing a random syncer and setting txid that way fall into the same way before introducing extra asyncSyncer threads: when asyncWriter pushes new entries to hdfs before asyncSyncer sync the previously pushed ones, asyncSyncer gets notified the newly pushed txid, but these txid will be synced by next time after asyncSyncer is done with the current ones, notice we use txidToFlush to record txid each sync is for, and it can't change during each sync, while writtenTxid can change during each sync) To summary: the sync operation is the most time-consuming phase, under old write model every write handler issues a separate sync directly for itself(if not return early by syncedTillHere). and under new write model, though separate threads significantly reduce the lock race, but if concurrent write threads is few, the benefit by reducing lock race(fewer write threads, fewer benefit) can't offset the inefficiency by using a single asyncSyncer threads(each time asyncSyncer thread can only sync for a portion of the writes, but the write handlers which already have their entries in buffer or pushed to hdfs also need to wait for its completeness, and can't proceed until its next sync phase is done) By introducing extra asyncSyncer threads, the correctness of this model is the same as before: still a single asyncWriter thread which push buffered entries to hdfs sequentially(txid increases sequentially), and when each asyncSyncer is done, it's guaranteed all txids smaller are pushed to hdfs and successfully sync-ed.
          Hide
          Honghua Feng added a comment -

          => yes, under new thread model, no explicit method to do the sync and can't tell if there is outstanding deferred entries (the AsyncWriter/AsyncSyncer threads do write/sync in a best-effort way)

          I meant calling 'sync' can guarantee no outstanding deferred entries, but no calling 'sync' after writew can't guarantee there must be some outstanding deferred entries since they can be sync-ed by asyncWriter/asyncSyncer threads. This is not the same behavior as under old write model.

          Show
          Honghua Feng added a comment - => yes, under new thread model, no explicit method to do the sync and can't tell if there is outstanding deferred entries (the AsyncWriter/AsyncSyncer threads do write/sync in a best-effort way) I meant calling 'sync' can guarantee no outstanding deferred entries, but no calling 'sync' after writew can't guarantee there must be some outstanding deferred entries since they can be sync-ed by asyncWriter/asyncSyncer threads. This is not the same behavior as under old write model.
          Hide
          Ted Yu added a comment -

          2/3 have worse perf, 10 has equal perf but introduces too many extra threads

          Do you remember how many extra threads were introduced ?

          Show
          Ted Yu added a comment - 2/3 have worse perf, 10 has equal perf but introduces too many extra threads Do you remember how many extra threads were introduced ?
          Hide
          stack added a comment -

          Do you remember how many extra threads were introduced ?

          This question is answered above in my last review. Why do you ask?

          Show
          stack added a comment - Do you remember how many extra threads were introduced ? This question is answered above in my last review. Why do you ask?
          Hide
          Honghua Feng added a comment -

          tried 4 asyncSyncer threads, below are the results. a bit worse than 5 threads but looks like acceptable?

          threads ops-wo-patch ops-w-patch ops-diff
          1 1716 1572 -8.4%
          3 3179 3189 +0.3%
          5 6091 5593 -8.1%
          10 8760 9450 +7.8%
          25 13019 18055 +38.7%
          50 14995 26597 +77.3%
          100 18824 51441 +173.2%
          200 18144 61531 +239.1%

          additional explanation on correctness when introducing extra asyncSyncer threads:

          • when a txid(t0) is notified, all txid smaller than t0 must already be written to hdfs and by sync-ed: before t0 is notified, t0 must be sync-ed by an asyncSyncer thread; before t0 is sync-ed, t0 must be written to hdfs by asyncWriter thread; before t0 is written to hdfs, all txid smaller than t0 must be written to hdfs, so the sync of t0 can guarantee all txid smaller than t0 must be sync-ed (either before the sync of t0, or by the sync of t0)
          • when a txid(t0) can't find free(idle) asyncSyncer thread and added to a random one, it won't be sync-ed until its asyncSyncer thread is done with the txid at hand. but its entries already have been written to hdfs, and if any bigger txid than t0 (say t1) is successfully sync-ed by another parallel asyncSyncer thread, that sync can guarantee t0 also successfully sync-ed, hence when t1 is notified, t0 can also be correctly notified.

          any further comments?

          Show
          Honghua Feng added a comment - tried 4 asyncSyncer threads, below are the results. a bit worse than 5 threads but looks like acceptable? threads ops-wo-patch ops-w-patch ops-diff 1 1716 1572 -8.4% 3 3179 3189 +0.3% 5 6091 5593 -8.1% 10 8760 9450 +7.8% 25 13019 18055 +38.7% 50 14995 26597 +77.3% 100 18824 51441 +173.2% 200 18144 61531 +239.1% additional explanation on correctness when introducing extra asyncSyncer threads: when a txid(t0) is notified, all txid smaller than t0 must already be written to hdfs and by sync-ed: before t0 is notified, t0 must be sync-ed by an asyncSyncer thread; before t0 is sync-ed, t0 must be written to hdfs by asyncWriter thread; before t0 is written to hdfs, all txid smaller than t0 must be written to hdfs, so the sync of t0 can guarantee all txid smaller than t0 must be sync-ed (either before the sync of t0, or by the sync of t0) when a txid(t0) can't find free(idle) asyncSyncer thread and added to a random one, it won't be sync-ed until its asyncSyncer thread is done with the txid at hand. but its entries already have been written to hdfs, and if any bigger txid than t0 (say t1) is successfully sync-ed by another parallel asyncSyncer thread, that sync can guarantee t0 also successfully sync-ed, hence when t1 is notified, t0 can also be correctly notified. any further comments?
          Hide
          stack added a comment -

          Sorry for the delay getting back to this Honghua Feng and thanks for the explanation.

          I am having trouble reviewing the patch because I am trying to understand what is going on here in FSHLog. It is hard to follow (not your patch necessarily but what is there currently) in spite of multiple reviews. I keep trying to grok what is going on because this is critical code.

          The numbers are hard to argue with and it does some nice cleanup of FSHLog which makes it easier to understand. We could commit this patch and then work on undoing the complexity that is rife here; your patch adds yet more because it adds interacting threads w/ new synchronizations, notifications, AtomicBoolean states, etc., which cost performance-wise but at least it is clearer what is going on and we have tools for comparing approaches now. We could work on simplication and removal of sync points in a follow-on (See below for a note on one approach).

          I now get why the need for multiple syncers. It is a little counter-intuitiive given we want to batch up edits more to get more performance on the one hand, but then on the other, we have to sync more often because sync'ing is outstanding for too much time, so much time it holds up handlers too long.

          + I am trying to understand why we keep aside the edits in a linked-list. This was there before your time. You just continue the practice. The original comment says "We keep them cached here instead of writing them to HDFS piecemeal, because the HDFS write-method is pretty heavyweight as far as locking is concerned." Yet, when we eventually flush the edits, we don't do anything special; we just call write on the dfsoutputstream. We are not avoiding locking in hdfs. It must be the hbase flush/update locking that is being referred to here.
          + AsyncSyncer is a confounding name for a class – but it makes sense in this context. The flush object in this thread is a syncer synchronization object not for memstore flushes... as I thought it was (there is use of flush in here when it probably should be sync to be consistent).

          Off-list, a few other lads are interested in reviewing this patch (it is a popular patch!)... our Jonathan Hsieh and possible Himanshu Vashishtha because they are getting stuck in this area. If they don't get to it soon, I'll commit unless objection.

          Show
          stack added a comment - Sorry for the delay getting back to this Honghua Feng and thanks for the explanation. I am having trouble reviewing the patch because I am trying to understand what is going on here in FSHLog. It is hard to follow (not your patch necessarily but what is there currently) in spite of multiple reviews. I keep trying to grok what is going on because this is critical code. The numbers are hard to argue with and it does some nice cleanup of FSHLog which makes it easier to understand. We could commit this patch and then work on undoing the complexity that is rife here; your patch adds yet more because it adds interacting threads w/ new synchronizations, notifications, AtomicBoolean states, etc., which cost performance-wise but at least it is clearer what is going on and we have tools for comparing approaches now. We could work on simplication and removal of sync points in a follow-on (See below for a note on one approach). I now get why the need for multiple syncers. It is a little counter-intuitiive given we want to batch up edits more to get more performance on the one hand, but then on the other, we have to sync more often because sync'ing is outstanding for too much time, so much time it holds up handlers too long. + I am trying to understand why we keep aside the edits in a linked-list. This was there before your time. You just continue the practice. The original comment says "We keep them cached here instead of writing them to HDFS piecemeal, because the HDFS write-method is pretty heavyweight as far as locking is concerned." Yet, when we eventually flush the edits, we don't do anything special; we just call write on the dfsoutputstream. We are not avoiding locking in hdfs. It must be the hbase flush/update locking that is being referred to here. + AsyncSyncer is a confounding name for a class – but it makes sense in this context. The flush object in this thread is a syncer synchronization object not for memstore flushes... as I thought it was (there is use of flush in here when it probably should be sync to be consistent). Off-list, a few other lads are interested in reviewing this patch (it is a popular patch!)... our Jonathan Hsieh and possible Himanshu Vashishtha because they are getting stuck in this area. If they don't get to it soon, I'll commit unless objection.
          Hide
          Honghua Feng added a comment -

          Thanks stack, and also the review from Jonathan Hsieh and Himanshu Vashishtha, and other experienced guys is welcome, it's better to have this patch reviewed by as many guys as possible.

          Any question on this patch is welcome

          Show
          Honghua Feng added a comment - Thanks stack , and also the review from Jonathan Hsieh and Himanshu Vashishtha , and other experienced guys is welcome, it's better to have this patch reviewed by as many guys as possible. Any question on this patch is welcome
          Hide
          stack added a comment -

          Here is more review on the patch. Make the changes suggested below and I'll +1 it.

          (Discussion off-line w/ Feng on this issue helped me better understand this patch and put to rest any notion that there is an easier 'fix' than the one proposed here. That said. There is much room for improvement but this can be done in a follow-on)

          Remove these asserts rather than comment them out given they depended on a facility this patch removes. Leaving them in will only make the next reader of the code – very likely lacking the context you have – feel uneasy thinking someone removed asserts just to get tests to pass.

          8 - assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries());
          9 + //assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries());

          On the below...

          +import java.util.Random;

          ... using a Random for choosing an arbitrary thread for a list of 4 is heavyweight. Can you not take last digit of timestamp or nano timestamp or some attribute of the edit instead? Something more lightweight?

          Please remove all mentions of AsyncFlush since it no longer exists:

          // all writes pending on AsyncWrite/AsyncFlush thread with

          Leaving it in will confuse readers when they can't find any such thread class.

          Is this comment right?

          // txid <= failedTxid will fail by throwing asyncIOE

          Should it be >= failedTxid?

          This should be volatile since it is set by AsyncSync and then used by the main FSHLog thread (you have an assert to check it not null – maybe you ran into an issue here already?):

          + private IOException asyncIOE = null;

          + private final Object bufferLock = new Object();

          'bufferLock' if a very generic name. Could it be more descriptive? It is a lock held for a short while while AsyncWriter moves queued edits off the globally seen queue to a local queue just before we send the edits to the WAL. You add a method named getPendingWrites that requires this lock be held. Could we tie the method and the lock together better? Name it pendingWritesLock? (The name of the list to hold the pending writes is pendingWrites).

          ...because the HDFS write-method is pretty heavyweight as far as locking is concerned.

          I think the heavyweight referred to in the above is hbase locking, not hdfs locking as the comment would imply. If you agree (you know this code better than I), please adjust the comment.

          Comments on what these threads do will help the next code reader. AsyncWriter does adding of edits to HDFS. AsyncSyncer needs a comment because it is oxymoronic (though it makes sense in this context). In particular, a comment would draw out why we need so many instances of a syncer thread because everyone's first thought here is going to be why do we need this? Ditto on the AsyncNotifier. In the reviews above, folks have asked why we need this thread at all and a code reader will likely think similar on a first pass. Bottom-line, your patch raised questions from reviewers; it would be cool if the questions were answered in code comments where possible so the questions do not come up again.

          4 + private final AsyncWriter asyncWriter;
          5 + private final AsyncSyncer[] asyncSyncers = new AsyncSyncer[5];
          6 + private final AsyncNotifier asyncNotifier;

          You remove the LogSyncer facility in this patch. That is good (need to note this in release notes). Your patch should remove the optional flush config from hbase-default.xml too since it no longer is relevant.

          3 - this.optionalFlushInterval =
          4 - conf.getLong("hbase.regionserver.optionallogflushinterval", 1 * 1000);

          I see it here...

          hbase-common/src/main/resources/hbase-default.xml: <name>hbase.regionserver.optionallogflushinterval</name>

          A small nit is you might look at other threads in hbase and see how they are named...

          3 + asyncWriter = new AsyncWriter("AsyncHLogWriter");

          Ditto here:

          + asyncSyncers[i] = new AsyncSyncer("AsyncHLogSyncer" + i);

          Probably make the number of asyncsyncers a configuration (you don't have to put the option out in hbase-default.xml.. just make it so that if someone is reading the code and trips over this issue, they can change it by adding to hbase-site.xml w/o having to change code – lets not reproduce the hard-coded '80' that is in the head of dfsclient we discussed yesterday – smile).

          ... and here: asyncNotifier = new AsyncNotifier("AsyncHLogNotifier");

          Not important but check out how other threads are named in hbase. It might be good if these better align.

          Maybe make a method for shutting down all these thread or use the Threads#shutdown method in Threads.java?

          LOG.error("Exception while waiting for AsyncNotifier threads to die", e);

          Do LOG.error("Exception while waiting for" + t.getName() + " threads to die", e); instead?

          Call this method per AsyncSyncer rather than do as you have here:

          6 + try {
          7 + for (int i = 0; i < asyncSyncers.length; ++i)

          { 8 + asyncSyncers[i].interrupt(); 9 + asyncSyncers[i].join(); 10 }

          11 + } catch (InterruptedException e)

          { 12 + LOG.error("Exception while waiting for AsyncSyncer threads to die", e); 13 }

          where if we fail to join on the first asyncsync, we will not shut down the rest. This is a nit. Not important since we are closing down anyways and if an exception here, it is probably fatal.

          These threads are NOT daemon threads though so a fail to shut them down will hold up the JVM's going down. Maybe they should all be daemon threads just in case?

          I see this in close:

          4 + synchronized (bufferLock)

          { 5 + doWrite(info, logKey, edits, htd); 6 + txid = this.unflushedEntries.incrementAndGet(); 7 + }

          ... but we do not seem to be doing it on the other call to doWrite at around line #969 inside in append. Should we be holding the lock at this location (maybe we do and I am just not seeing it because I am just looking at FSHLog and the patch side-by-side)?

          What is going on here?

            1 +    // wake up (called by (write) handler thread) AsyncWriter thread
            0 +    // to write buffered writes to HDFS
            1 +    public void setPendingTxid(long txid) {
          

          This method is only called at close time if I read the patch right. It is not called by '...(write) handler thread'? I am having trouble understanding how this signaling works.

          Is this 'fatal'? Or is it an 'error' since we are going into recovery... trying to role the log?

          2 + LOG.fatal("Error while AsyncWriter write, request close of hlog ", e);

          Here we failed a write:

           11 +          } catch(IOException e) {
           10 +            LOG.fatal("Error while AsyncWriter write, request close of hlog ", e);
            9              requestLogRoll();
            8 -            Threads.sleep(this.optionalFlushInterval);
            7 +
            6 +            asyncIOE = e;
            5 +            failedTxid.set(this.txidToWrite);
            4            }
          

          .. and request a log roll yet we carry on to try and sync, an op that will likely fail? We are ok here? We updated the write txid but not the sync txid so that should be fine.

          Do we need this: if (!asyncSyncers[i].isSyncing()) DFSClient will allow us call sync concurrently?

          Yeah... could use the last digit of the txid rather than random? That should be enough?

          3 + int idx = rd.nextInt(asyncSyncers.length);
          2 + asyncSyncers[idx].setWrittenTxid(this.lastWrittenTxid);

          In the above, what if the writtentxid is > than what we are to sync? Will that cause a problem? What will the sync do in this case? (nvm... I asked this question already. It is answered later in the code. The sync threads are running all the time. I thought they fired once and completed. My misunderstanding. Ignore the above).

          Can these be static classes or do they need context form the hosting FSHLog?

          private class AsyncSyncer extends HasThread {

          This class definetly needs more comment either here or above when it is set up as a data member:

          3 + // thread to request HDFS to sync the WALEdits written by AsyncWriter
          2 + // to make those WALEdits durable on HDFS side

          These method names should not talk about 'flush'. They should be named 'sync' instead:

          3 + private long txidToFlush = 0;
          2 + private long lastFlushedTxid = 0;

          Same for the flushlock.

          Why atomic boolean and not just a volatile here?

          private AtomicBoolean isSyncing = new AtomicBoolean(false);

          You are not using any feature of Atomic other than get and set so volatile should work.

          + LOG.info(getName() + " exiting");

          The above is very important. All your threads do this? Nothing worse than a silently exiting thread (smile).

          Here is that comment again:

            1 +  // appends new writes to the pendingWrites. It is better to keep it in
            0 +  // our own queue rather than writing it to the HDFS output stream because
            1 +  // HDFSOutputStream.writeChunk is not lightweight at all.
            2 +  // it's caller's responsibility to hold updateLock before call this method
          

          It talks about writeChunk being expensive but we are not doing anything to ameliorate dfsclient writes if I dig down into our log writer... we just write as we go. So, the comment is either correct and we are ignoring it in implementation or the comment is wrong and we should be removed. This is out of scope of your patch. You are just moving this comment I'd say. If so, update the comment to add a TODO saying needs investigation.

          Patch is great. Lets get it in. The above remarks are mostly minor just looking for clarification. Thanks Honghua Feng

          Show
          stack added a comment - Here is more review on the patch. Make the changes suggested below and I'll +1 it. (Discussion off-line w/ Feng on this issue helped me better understand this patch and put to rest any notion that there is an easier 'fix' than the one proposed here. That said. There is much room for improvement but this can be done in a follow-on) Remove these asserts rather than comment them out given they depended on a facility this patch removes. Leaving them in will only make the next reader of the code – very likely lacking the context you have – feel uneasy thinking someone removed asserts just to get tests to pass. 8 - assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries()); 9 + //assertTrue("Should have an outstanding WAL edit", ((FSHLog) log).hasDeferredEntries()); On the below... +import java.util.Random; ... using a Random for choosing an arbitrary thread for a list of 4 is heavyweight. Can you not take last digit of timestamp or nano timestamp or some attribute of the edit instead? Something more lightweight? Please remove all mentions of AsyncFlush since it no longer exists: // all writes pending on AsyncWrite/AsyncFlush thread with Leaving it in will confuse readers when they can't find any such thread class. Is this comment right? // txid <= failedTxid will fail by throwing asyncIOE Should it be >= failedTxid? This should be volatile since it is set by AsyncSync and then used by the main FSHLog thread (you have an assert to check it not null – maybe you ran into an issue here already?): + private IOException asyncIOE = null; + private final Object bufferLock = new Object(); 'bufferLock' if a very generic name. Could it be more descriptive? It is a lock held for a short while while AsyncWriter moves queued edits off the globally seen queue to a local queue just before we send the edits to the WAL. You add a method named getPendingWrites that requires this lock be held. Could we tie the method and the lock together better? Name it pendingWritesLock? (The name of the list to hold the pending writes is pendingWrites). ...because the HDFS write-method is pretty heavyweight as far as locking is concerned. I think the heavyweight referred to in the above is hbase locking, not hdfs locking as the comment would imply. If you agree (you know this code better than I), please adjust the comment. Comments on what these threads do will help the next code reader. AsyncWriter does adding of edits to HDFS. AsyncSyncer needs a comment because it is oxymoronic (though it makes sense in this context). In particular, a comment would draw out why we need so many instances of a syncer thread because everyone's first thought here is going to be why do we need this? Ditto on the AsyncNotifier. In the reviews above, folks have asked why we need this thread at all and a code reader will likely think similar on a first pass. Bottom-line, your patch raised questions from reviewers; it would be cool if the questions were answered in code comments where possible so the questions do not come up again. 4 + private final AsyncWriter asyncWriter; 5 + private final AsyncSyncer[] asyncSyncers = new AsyncSyncer [5] ; 6 + private final AsyncNotifier asyncNotifier; You remove the LogSyncer facility in this patch. That is good (need to note this in release notes). Your patch should remove the optional flush config from hbase-default.xml too since it no longer is relevant. 3 - this.optionalFlushInterval = 4 - conf.getLong("hbase.regionserver.optionallogflushinterval", 1 * 1000); I see it here... hbase-common/src/main/resources/hbase-default.xml: <name>hbase.regionserver.optionallogflushinterval</name> A small nit is you might look at other threads in hbase and see how they are named... 3 + asyncWriter = new AsyncWriter("AsyncHLogWriter"); Ditto here: + asyncSyncers [i] = new AsyncSyncer("AsyncHLogSyncer" + i); Probably make the number of asyncsyncers a configuration (you don't have to put the option out in hbase-default.xml.. just make it so that if someone is reading the code and trips over this issue, they can change it by adding to hbase-site.xml w/o having to change code – lets not reproduce the hard-coded '80' that is in the head of dfsclient we discussed yesterday – smile). ... and here: asyncNotifier = new AsyncNotifier("AsyncHLogNotifier"); Not important but check out how other threads are named in hbase. It might be good if these better align. Maybe make a method for shutting down all these thread or use the Threads#shutdown method in Threads.java? LOG.error("Exception while waiting for AsyncNotifier threads to die", e); Do LOG.error("Exception while waiting for" + t.getName() + " threads to die", e); instead? Call this method per AsyncSyncer rather than do as you have here: 6 + try { 7 + for (int i = 0; i < asyncSyncers.length; ++i) { 8 + asyncSyncers[i].interrupt(); 9 + asyncSyncers[i].join(); 10 } 11 + } catch (InterruptedException e) { 12 + LOG.error("Exception while waiting for AsyncSyncer threads to die", e); 13 } where if we fail to join on the first asyncsync, we will not shut down the rest. This is a nit. Not important since we are closing down anyways and if an exception here, it is probably fatal. These threads are NOT daemon threads though so a fail to shut them down will hold up the JVM's going down. Maybe they should all be daemon threads just in case? I see this in close: 4 + synchronized (bufferLock) { 5 + doWrite(info, logKey, edits, htd); 6 + txid = this.unflushedEntries.incrementAndGet(); 7 + } ... but we do not seem to be doing it on the other call to doWrite at around line #969 inside in append. Should we be holding the lock at this location (maybe we do and I am just not seeing it because I am just looking at FSHLog and the patch side-by-side)? What is going on here? 1 + // wake up (called by (write) handler thread) AsyncWriter thread 0 + // to write buffered writes to HDFS 1 + public void setPendingTxid( long txid) { This method is only called at close time if I read the patch right. It is not called by '...(write) handler thread'? I am having trouble understanding how this signaling works. Is this 'fatal'? Or is it an 'error' since we are going into recovery... trying to role the log? 2 + LOG.fatal("Error while AsyncWriter write, request close of hlog ", e); Here we failed a write: 11 + } catch (IOException e) { 10 + LOG.fatal( "Error while AsyncWriter write, request close of hlog " , e); 9 requestLogRoll(); 8 - Threads.sleep( this .optionalFlushInterval); 7 + 6 + asyncIOE = e; 5 + failedTxid.set( this .txidToWrite); 4 } .. and request a log roll yet we carry on to try and sync, an op that will likely fail? We are ok here? We updated the write txid but not the sync txid so that should be fine. Do we need this: if (!asyncSyncers [i] .isSyncing()) DFSClient will allow us call sync concurrently? Yeah... could use the last digit of the txid rather than random? That should be enough? 3 + int idx = rd.nextInt(asyncSyncers.length); 2 + asyncSyncers [idx] .setWrittenTxid(this.lastWrittenTxid); In the above, what if the writtentxid is > than what we are to sync? Will that cause a problem? What will the sync do in this case? (nvm... I asked this question already. It is answered later in the code. The sync threads are running all the time. I thought they fired once and completed. My misunderstanding. Ignore the above). Can these be static classes or do they need context form the hosting FSHLog? private class AsyncSyncer extends HasThread { This class definetly needs more comment either here or above when it is set up as a data member: 3 + // thread to request HDFS to sync the WALEdits written by AsyncWriter 2 + // to make those WALEdits durable on HDFS side These method names should not talk about 'flush'. They should be named 'sync' instead: 3 + private long txidToFlush = 0; 2 + private long lastFlushedTxid = 0; Same for the flushlock. Why atomic boolean and not just a volatile here? private AtomicBoolean isSyncing = new AtomicBoolean(false); You are not using any feature of Atomic other than get and set so volatile should work. + LOG.info(getName() + " exiting"); The above is very important. All your threads do this? Nothing worse than a silently exiting thread (smile). Here is that comment again: 1 + // appends new writes to the pendingWrites. It is better to keep it in 0 + // our own queue rather than writing it to the HDFS output stream because 1 + // HDFSOutputStream.writeChunk is not lightweight at all. 2 + // it's caller's responsibility to hold updateLock before call this method It talks about writeChunk being expensive but we are not doing anything to ameliorate dfsclient writes if I dig down into our log writer... we just write as we go. So, the comment is either correct and we are ignoring it in implementation or the comment is wrong and we should be removed. This is out of scope of your patch. You are just moving this comment I'd say. If so, update the comment to add a TODO saying needs investigation. Patch is great. Lets get it in. The above remarks are mostly minor just looking for clarification. Thanks Honghua Feng
          Hide
          Himanshu Vashishtha added a comment -

          This is some awesome stuff happening here.

          I have few comments apart from what Stack already mentioned.

          1) log rolling thread safety: Log rolling happens in parallel with flush/sync. Currently, in FSHLog, sync call grabs the updateLock to ensure it has a non-null writer (because of parallel log rolling). How does this patch address the non-null writer? Or is it not needed anymore? Also, if you go for the updatelock in sync, that might result in deadlock.

          2) Error handling: It is not very clear how is flush/sync failures are being handled? For example, if a write fails for txid 10, it notifies AsyncSyncer that writer is done with txid 10. And, AsyncSyncer notifies the notifier thread, which finally notifies the blocked handler using notifyAll. The handler checks for the failedTxid here:

          +          if (txid <= this.failedTxid.get()) {
          

          Let's say there are two handlers waiting for sync, t1 on txid 8 and t2 on txid 10. And, t1 wakes up on notification. Would t1 also get this exception? Wouldn't it be wrong, because txid 8 may have succeeded? Please correct me if I missed anything.

          3) I think the current HLogPE doesn't do justice to the real use case. Almost all HLog calls are appendNoSync, followed by a sync call.
          In the current HLogPE, we are calling append calls, which also does the sync. When I changed it to represent the above common case,
          the performance numbers of current FSHLog using HLogPE improves quite a bit. I still need to figure out the reason, but the HLogPE change affects the perf numbers considerably IMO.
          For example, on a 5 node cluster, I see this difference on trunk:
          Earlier:

          Summary: threads=3, iterations=100000 took 218.590s 1372.432ops/s
          

          Post HlogPE change:

          Summary: threads=3, iterations=100000 took 172.648s 1737.640ops/s
          

          I think it would be great if we can test this with the correct HLogPE. What do you think Fenghua?

          4) Perf numbers are super impressive. It would have been wonderful to have such numbers for lesser number of handler threads also (e.g., 5-10 threads). IMHO, that represents most common case scenario, but I could be wrong. I know this has been beaten to death in the discussions above, but just echoing my thoughts here.

          5) I should also mention that while working on a different use case, I was trying to bring a layer of indirection b/w regionserver handlers and sync operation (Sync is the most costly affair in all HLog story). What resulted is a separate set of syncer threads, which does the work of flushing the edits and syncing to HDFS. This is what it looks like:
          a) The handlers append their entries to the FSHLog buffer as they do currently.
          b) They invoke sync API. There, they wait on the Syncer threads to do the work for them and notify.
          c) It results in batched sync effort but without much extra locking/threads.
          Basically, it is similar to what you did here but minus Writer & Notifier threads and minus the bufferlock.
          I mentioned it here because with that approach, I see some perf improvement even with lesser number of handler threads. And, it also keeps the current deferredLogFlush behavior. This work is still in progress and is still a prototype. It would be great to know your take on it.

          Here are some numbers on a 5 node cluster; hdfs 2.1.0; hbase is trunk; client on a different node. I haven't tested it with larger number of threads but would be good to compare I think (its 2am here... ). It uses 3 syncers at the moment (and varying it would be a good thing to experiment too).
          Also, without patch in the below table means trunk + HLogPE patch.

          Threads w/o patch time w/o patch ops w/ patch time w/ patch ops
          3 172.648s 1737.640ops/s 170.332s 1761.266ops/s
          3 170.977s 1754.622ops/s 174.568s 1718.528ops/s
          5 213.738s 2339.313ops/s 191.119s 2616.171ops/s
          5 211.072s 2368.860ops/s 189.671s 2636.144ops/s
          10 254.641s 3926.419ops/s 216.494s 4619.319ops/s
          10 251.503s 3978.564ops/s 215.333s 4643.266ops/s
          10 251.692s 3970.579ops/s 217.151s 4605.854ops/s
          20 648.943s 6163.870ops/s 646.279s 6189.277ops/s
          20 658.654s 6072.991ops/s 656.277s 6094.987ops/s
          25 282.654s 8861.991ops/s 249.277s 10033.987ops/s
          Show
          Himanshu Vashishtha added a comment - This is some awesome stuff happening here. I have few comments apart from what Stack already mentioned. 1) log rolling thread safety: Log rolling happens in parallel with flush/sync. Currently, in FSHLog, sync call grabs the updateLock to ensure it has a non-null writer (because of parallel log rolling). How does this patch address the non-null writer? Or is it not needed anymore? Also, if you go for the updatelock in sync, that might result in deadlock. 2) Error handling: It is not very clear how is flush/sync failures are being handled? For example, if a write fails for txid 10, it notifies AsyncSyncer that writer is done with txid 10. And, AsyncSyncer notifies the notifier thread, which finally notifies the blocked handler using notifyAll. The handler checks for the failedTxid here: + if (txid <= this .failedTxid.get()) { Let's say there are two handlers waiting for sync, t1 on txid 8 and t2 on txid 10. And, t1 wakes up on notification. Would t1 also get this exception? Wouldn't it be wrong, because txid 8 may have succeeded? Please correct me if I missed anything. 3) I think the current HLogPE doesn't do justice to the real use case. Almost all HLog calls are appendNoSync, followed by a sync call. In the current HLogPE, we are calling append calls, which also does the sync. When I changed it to represent the above common case, the performance numbers of current FSHLog using HLogPE improves quite a bit. I still need to figure out the reason, but the HLogPE change affects the perf numbers considerably IMO. For example, on a 5 node cluster, I see this difference on trunk: Earlier: Summary: threads=3, iterations=100000 took 218.590s 1372.432ops/s Post HlogPE change: Summary: threads=3, iterations=100000 took 172.648s 1737.640ops/s I think it would be great if we can test this with the correct HLogPE. What do you think Fenghua? 4) Perf numbers are super impressive. It would have been wonderful to have such numbers for lesser number of handler threads also (e.g., 5-10 threads). IMHO, that represents most common case scenario, but I could be wrong. I know this has been beaten to death in the discussions above, but just echoing my thoughts here. 5) I should also mention that while working on a different use case, I was trying to bring a layer of indirection b/w regionserver handlers and sync operation (Sync is the most costly affair in all HLog story). What resulted is a separate set of syncer threads, which does the work of flushing the edits and syncing to HDFS. This is what it looks like: a) The handlers append their entries to the FSHLog buffer as they do currently. b) They invoke sync API. There, they wait on the Syncer threads to do the work for them and notify. c) It results in batched sync effort but without much extra locking/threads. Basically, it is similar to what you did here but minus Writer & Notifier threads and minus the bufferlock. I mentioned it here because with that approach, I see some perf improvement even with lesser number of handler threads. And, it also keeps the current deferredLogFlush behavior. This work is still in progress and is still a prototype. It would be great to know your take on it. Here are some numbers on a 5 node cluster; hdfs 2.1.0; hbase is trunk; client on a different node. I haven't tested it with larger number of threads but would be good to compare I think (its 2am here... ). It uses 3 syncers at the moment (and varying it would be a good thing to experiment too). Also, without patch in the below table means trunk + HLogPE patch. Threads w/o patch time w/o patch ops w/ patch time w/ patch ops 3 172.648s 1737.640ops/s 170.332s 1761.266ops/s 3 170.977s 1754.622ops/s 174.568s 1718.528ops/s 5 213.738s 2339.313ops/s 191.119s 2616.171ops/s 5 211.072s 2368.860ops/s 189.671s 2636.144ops/s 10 254.641s 3926.419ops/s 216.494s 4619.319ops/s 10 251.503s 3978.564ops/s 215.333s 4643.266ops/s 10 251.692s 3970.579ops/s 217.151s 4605.854ops/s 20 648.943s 6163.870ops/s 646.279s 6189.277ops/s 20 658.654s 6072.991ops/s 656.277s 6094.987ops/s 25 282.654s 8861.991ops/s 249.277s 10033.987ops/s
          Hide
          Himanshu Vashishtha added a comment -

          Attached is the prototype patch (HLogPE change inclusive). Thanks.

          Show
          Himanshu Vashishtha added a comment - Attached is the prototype patch (HLogPE change inclusive). Thanks.
          Hide
          stack added a comment -

          Nice review Himanshu Vashishtha

          On 3., above, yes, that is true. HLogPE does not seem to be representative as you suggest. But does your change below....

          -            hlog.append(hri, hri.getTable(), walEdit, now, htd, region.getSequenceId());
          +            // this is how almost all users of HLog use it (all but compaction calls).
          +            long txid = hlog.appendNoSync(hri, hri.getTable(), walEdit, clusters, now, htd,
          +              region.getSequenceId(), true, nonce, nonce);
          +            hlog.sync(txid);
          +
          

          ... bring it closer to a 'real' use case? I see over in HRegion that we do a bunch of appendNoSync in minibatch or even in put before we call sync. Should we append more than just one set of edits before we call the sync?

          I suppose on a regionserver with a load of regions loaded up on it, all these syncs can come crashing in on top of each other on to the underlying WAL in an arbitary manner – something Feng Honghua's patch mitigates some by making it so syncs are done when FSHLog thinks it appropriate rather than when some arbitrary HRegion call thinks it right ... and this is probably part of the reason for the perf improvement.

          Could we better regulate the sync calls so they are even less arbitrary? Even them out? It could make for better performance if there was a mechanism against syncs clumping together.

          Looking at your patch, the syncer is very much like Feng Honghua's – it is interesting that you two independently came up w/ similar multithreaded syncing mechanism. That would seem to 'prove' this is a good approach. Feng's patch is much further along with a bunch of cleanup of FSHLog. Will wait on his comments on what he thinks of doing without AsyncWriter and AsyncNotifier.

          Looks like your patch is far enough long for us to do tests comparing the approaches?

          Show
          stack added a comment - Nice review Himanshu Vashishtha On 3., above, yes, that is true. HLogPE does not seem to be representative as you suggest. But does your change below.... - hlog.append(hri, hri.getTable(), walEdit, now, htd, region.getSequenceId()); + // this is how almost all users of HLog use it (all but compaction calls). + long txid = hlog.appendNoSync(hri, hri.getTable(), walEdit, clusters, now, htd, + region.getSequenceId(), true , nonce, nonce); + hlog.sync(txid); + ... bring it closer to a 'real' use case? I see over in HRegion that we do a bunch of appendNoSync in minibatch or even in put before we call sync. Should we append more than just one set of edits before we call the sync? I suppose on a regionserver with a load of regions loaded up on it, all these syncs can come crashing in on top of each other on to the underlying WAL in an arbitary manner – something Feng Honghua's patch mitigates some by making it so syncs are done when FSHLog thinks it appropriate rather than when some arbitrary HRegion call thinks it right ... and this is probably part of the reason for the perf improvement. Could we better regulate the sync calls so they are even less arbitrary? Even them out? It could make for better performance if there was a mechanism against syncs clumping together. Looking at your patch, the syncer is very much like Feng Honghua's – it is interesting that you two independently came up w/ similar multithreaded syncing mechanism. That would seem to 'prove' this is a good approach. Feng's patch is much further along with a bunch of cleanup of FSHLog. Will wait on his comments on what he thinks of doing without AsyncWriter and AsyncNotifier. Looks like your patch is far enough long for us to do tests comparing the approaches?
          Hide
          Jonathan Hsieh added a comment -

          I posted v4 for trunk on reviewboard and will reviewing there.

          https://reviews.apache.org/r/16052/

          Show
          Jonathan Hsieh added a comment - I posted v4 for trunk on reviewboard and will reviewing there. https://reviews.apache.org/r/16052/
          Hide
          stack added a comment -

          Let me look into changing HLogPE after Himanshu's suggestion above. I'll add an option to batch up edits some. My sense is that it will make the difference between current code and Honghua's patch even larger when the count of threads is low but that it will not change the numbers when thread count is high.

          Show
          stack added a comment - Let me look into changing HLogPE after Himanshu's suggestion above. I'll add an option to batch up edits some. My sense is that it will make the difference between current code and Honghua's patch even larger when the count of threads is low but that it will not change the numbers when thread count is high.
          Hide
          Himanshu Vashishtha added a comment -

          I agree on HLogPE comment. Yes, grouping appendNoSync before calling sync is the right way to go. The existing one is way-off the real use.

          I figured that the w/o patch col in the above table contains Feng's patch too. I am re-running the experiments at the moment with three versions: Trunk, Trunk + Feng's patch, Trunk + Syncer's approach. And, all versions have that HLogPe fix on it; will report back the numbers once done.

          Going by what we are seeing here, batching sync calls is definitely the right way IMO.
          I agree that Feng Honghua's patch has been tested well enough, and I really like the radical cleanup it does. The code reads pretty clean now, though it involves far more number of threads and synchronization stuff (which makes it more interesting to debug too ), I just wanted to ensure that it is safe.

          The reason I mentioned this different approach is it adds lesser number of threads (while keeping the current behaviour), and also shows improvement with smaller number of handlers, which to me looks like a nice win over current FSHLog. This is still in a prototype stage, and I absolutely don't want to block Feng's superb piece of work here. It would be good to know his thoughts on this. Thanks.

          Show
          Himanshu Vashishtha added a comment - I agree on HLogPE comment. Yes, grouping appendNoSync before calling sync is the right way to go. The existing one is way-off the real use. I figured that the w/o patch col in the above table contains Feng's patch too. I am re-running the experiments at the moment with three versions: Trunk, Trunk + Feng's patch, Trunk + Syncer's approach. And, all versions have that HLogPe fix on it; will report back the numbers once done. Going by what we are seeing here, batching sync calls is definitely the right way IMO. I agree that Feng Honghua's patch has been tested well enough, and I really like the radical cleanup it does. The code reads pretty clean now, though it involves far more number of threads and synchronization stuff (which makes it more interesting to debug too ), I just wanted to ensure that it is safe. The reason I mentioned this different approach is it adds lesser number of threads (while keeping the current behaviour), and also shows improvement with smaller number of handlers, which to me looks like a nice win over current FSHLog. This is still in a prototype stage, and I absolutely don't want to block Feng's superb piece of work here. It would be good to know his thoughts on this. Thanks.
          Hide
          Honghua Feng added a comment -

          Thanks very much for Himanshu Vashishtha and stack's review and further improvement suggestion. Currently I have some other stuff on hand to handle... I'll try best to come back soon and read through above comments. Thanks again.

          Show
          Honghua Feng added a comment - Thanks very much for Himanshu Vashishtha and stack 's review and further improvement suggestion. Currently I have some other stuff on hand to handle... I'll try best to come back soon and read through above comments. Thanks again.
          Hide
          Andrew Purtell added a comment -

          This work looks pretty far along. If we can get it in soon I would be willing to try putting this in to 0.98 so this major improvement can manifest in a release. Exciting results.

          Show
          Andrew Purtell added a comment - This work looks pretty far along. If we can get it in soon I would be willing to try putting this in to 0.98 so this major improvement can manifest in a release. Exciting results.
          Hide
          Himanshu Vashishtha added a comment -

          Patch is remarkably stale after 10010 and 10048 went in; attaching a rebased patch of the original version.

          Show
          Himanshu Vashishtha added a comment - Patch is remarkably stale after 10010 and 10048 went in; attaching a rebased patch of the original version.
          Hide
          Honghua Feng added a comment -

          stack : thanks for the comment, below are the comments after corresponding change. a new patch based on Himanshu Vashishtha's latest v5 patch is attached(thanks Himanshu Vashishtha)

          Remove these asserts rather than comment them out given they depended on a facility this patch removes.

          ==> done

          using a Random for choosing an arbitrary thread for a list of 4 is heavyweight

          ==> done

          Please remove all mentions of AsyncFlush since it no longer exists

          ==> done

          Is this comment right? // txid <= failedTxid will fail by throwing asyncIOE; Should it be >= failedTxid?

          ==> this comment is right: txid larger than failedTxid isn't sync-ed by the one that notifies failedTxid. but txid smaller than or equal to failedTxid is (not must be, but since we don't maintain a txid range to syncer mapping, so we fail all txid smaller than or equal to failedTxid, this aligns with HBase's write semantic of 'failed write may succeed in fact'. this is a point we can refine later on by adding txid range to sync operation mapping to precisely indicate failure)

          This should be volatile since it is set by AsyncSync and then used by the main FSHLog thread (you have an assert to check it not null – maybe you ran into an issue here already?): + private IOException asyncIOE = null;

          ==> done

          'bufferLock' if a very generic name. Could it be more descriptive? It is a lock held for a short while while AsyncWriter moves queued edits off the globally seen queue to a local queue just before we send the edits to the WAL. You add a method named getPendingWrites that requires this lock be held. Could we tie the method and the lock together better? Name it pendingWritesLock? (The name of the list to hold the pending writes is pendingWrites).

          ==> done

          (because the HDFS write-method is pretty heavyweight as far as locking is concerned.) I think the heavyweight referred to in the above is hbase locking...please adjust the comment

          ==> done

          Comments on what these threads do will help the next code reader

          ==> done

          Your patch should remove the optional flush config from hbase-default.xml too since it no longer is relevant

          ==> done

          A small nit is you might look at other threads in hbase and see how they are named...It might be good if these better align

          ==> done

          Probably make the number of asyncsyncers a configuration

          ==> done

          but we do not seem to be doing it on the other call to doWrite at around line #969 inside in append

          ==> doWrite is called inside append, and the bufferLock(now renamed to pendingWritesLock) is held there

          This method(setPendingTxid) is only called at close time if I read the patch right

          ==> it's called inside append() once doWrite() is done to notify AsyncWriter there are new pendingWrites to write to HDFS, it's not called at close time. you can double check it

          Is this 'fatal'? Or is it an 'error'

          ==> done

          and request a log roll yet we carry on to try and sync, an op that will likely fail? We are ok here? We updated the write txid but not the sync txid so that should be fine.

          ==> we can't retry to sync after a log roll since we can't sync to a new hlog while the writes were written to the old hlog. we failed all the transactions with txid <= write txid, it's ok here.

          Do we need this: if (!asyncSyncers[ i ].isSyncing()) DFSClient will allow us call sync concurrently. I think DFSClient allow us call sync concurrently...HDFS will handle(synchronize) concurrent sync?

          Can these be static classes or do they need context form the hosting FSHLog?

          ==> they all need context from hosting FSHLog (such as writer/asyncWriter/asyncSyncer/asyncNotifier)

          These method names should not talk about 'flush'. They should be named 'sync' instead. Same for the flushlock.

          ==> done

          Why atomic boolean and not just a volatile here? private AtomicBoolean isSyncing = new AtomicBoolean(false);

          ==> done

          The above is very important. All your threads do this?

          ==> yes

          It talks about writeChunk being expensive but we are not doing anything to ameliorate dfsclient writes if I dig down into our log writer

          ==> done (remove it)

          Show
          Honghua Feng added a comment - stack : thanks for the comment, below are the comments after corresponding change. a new patch based on Himanshu Vashishtha 's latest v5 patch is attached(thanks Himanshu Vashishtha ) Remove these asserts rather than comment them out given they depended on a facility this patch removes. ==> done using a Random for choosing an arbitrary thread for a list of 4 is heavyweight ==> done Please remove all mentions of AsyncFlush since it no longer exists ==> done Is this comment right? // txid <= failedTxid will fail by throwing asyncIOE; Should it be >= failedTxid? ==> this comment is right: txid larger than failedTxid isn't sync-ed by the one that notifies failedTxid. but txid smaller than or equal to failedTxid is (not must be, but since we don't maintain a txid range to syncer mapping, so we fail all txid smaller than or equal to failedTxid, this aligns with HBase's write semantic of 'failed write may succeed in fact'. this is a point we can refine later on by adding txid range to sync operation mapping to precisely indicate failure) This should be volatile since it is set by AsyncSync and then used by the main FSHLog thread (you have an assert to check it not null – maybe you ran into an issue here already?): + private IOException asyncIOE = null; ==> done 'bufferLock' if a very generic name. Could it be more descriptive? It is a lock held for a short while while AsyncWriter moves queued edits off the globally seen queue to a local queue just before we send the edits to the WAL. You add a method named getPendingWrites that requires this lock be held. Could we tie the method and the lock together better? Name it pendingWritesLock? (The name of the list to hold the pending writes is pendingWrites). ==> done (because the HDFS write-method is pretty heavyweight as far as locking is concerned.) I think the heavyweight referred to in the above is hbase locking...please adjust the comment ==> done Comments on what these threads do will help the next code reader ==> done Your patch should remove the optional flush config from hbase-default.xml too since it no longer is relevant ==> done A small nit is you might look at other threads in hbase and see how they are named...It might be good if these better align ==> done Probably make the number of asyncsyncers a configuration ==> done but we do not seem to be doing it on the other call to doWrite at around line #969 inside in append ==> doWrite is called inside append, and the bufferLock(now renamed to pendingWritesLock) is held there This method(setPendingTxid) is only called at close time if I read the patch right ==> it's called inside append() once doWrite() is done to notify AsyncWriter there are new pendingWrites to write to HDFS, it's not called at close time. you can double check it Is this 'fatal'? Or is it an 'error' ==> done and request a log roll yet we carry on to try and sync, an op that will likely fail? We are ok here? We updated the write txid but not the sync txid so that should be fine. ==> we can't retry to sync after a log roll since we can't sync to a new hlog while the writes were written to the old hlog. we failed all the transactions with txid <= write txid, it's ok here. Do we need this: if (!asyncSyncers[ i ].isSyncing()) DFSClient will allow us call sync concurrently. I think DFSClient allow us call sync concurrently...HDFS will handle(synchronize) concurrent sync? Can these be static classes or do they need context form the hosting FSHLog? ==> they all need context from hosting FSHLog (such as writer/asyncWriter/asyncSyncer/asyncNotifier) These method names should not talk about 'flush'. They should be named 'sync' instead. Same for the flushlock. ==> done Why atomic boolean and not just a volatile here? private AtomicBoolean isSyncing = new AtomicBoolean(false); ==> done The above is very important. All your threads do this? ==> yes It talks about writeChunk being expensive but we are not doing anything to ameliorate dfsclient writes if I dig down into our log writer ==> done (remove it)
          Hide
          Honghua Feng added a comment -

          I'll read and update according to Himanshu Vashishtha's comment tomorrow. thanks.

          Show
          Honghua Feng added a comment - I'll read and update according to Himanshu Vashishtha 's comment tomorrow. thanks.
          Hide
          Jean-Marc Spaggiari added a comment -

          For the random, you can also use something like System.currentTimeMillis() % asyncSyncers.length; Not saying that yours is not correct

          Show
          Jean-Marc Spaggiari added a comment - For the random, you can also use something like System.currentTimeMillis() % asyncSyncers.length; Not saying that yours is not correct
          Hide
          Honghua Feng added a comment -

          Himanshu Vashishtha:

          1) log rolling thread safety: Log rolling happens in parallel with flush/sync. Currently, in FSHLog, sync call grabs the updateLock to ensure it has a non-null writer (because of parallel log rolling). How does this patch address the non-null writer? Or is it not needed anymore? Also, if you go for the updatelock in sync, that might result in deadlock.

          ==> Good question. in rollWriter(), before switching this.writer to the newly created writer, updateLock is held and cleanupCurrentWriter() is called. updateLock is held guarantees no new edits enters pendingWrites, and cleanupCurrentWriter() is called guarantees all edits in current pendingWrites must be written to hdfs and sync-ed(inside this method 'sync()' is called to provide this guarantee). This means when switching this.writer to newly HLog writer, no new edits enter and all current edits have already been sync-ed to hdfs, all AsyncWriter/AsyncSyncer threads have nothing to do and are idle, so no log rolling thread safety issue here

          2) Error handling: It is not very clear how is flush/sync failures are being handled?... Let's say there are two handlers waiting for sync, t1 on txid 8 and t2 on txid 10. And, t1 wakes up on notification. Would t1 also get this exception? Wouldn't it be wrong, because txid 8 may have succeeded? Please correct me if I missed anything.

          ==> your understanding here is correct, but the write semantic of HBase is 'successful write response means a successful write, but failed write response can mean either a successful write or a failed write', right? and I have already mentioned this in above comment. this behavior can be improved by adding a txid-range to txid mapping, this mapping can help exactly indicate a failed txid fail which pending txid-range

          3) I think the current HLogPE doesn't do justice to the real use case...Almost all HLog calls are appendNoSync, followed by a sync call. In the current HLogPE, we are calling append calls, which also does the sync

          ==> you're right, but from an whole view of write process, these two have no big difference concerning performance: append() calls sync() inside, and in real case of HRegion.java, appendNoSync and sync is called, since sync now is a pending on notification, the difference of these two behavior is sooner/later to pend on notification, no impact on overall write performance(after put edits to pendingWrites, write handler thread can just wait for write/sync to hdfs to finish and can't help/influence write performance)...right?

          4) Perf numbers are super impressive. It would have been wonderful to have such numbers for lesser number of handler threads also (e.g., 5-10 threads). IMHO, that represents most common case scenario, but I could be wrong. I know this has been beaten to death in the discussions above, but just echoing my thoughts here

          ==> I think maybe many guys hold a same point of view as yours here, but I also have my personal thought on this Throughput is different from latency: latency represents how quickly a system perform user requests, the quicker the better; while throughput represents how many requests a system perform user requests(concurrently, or to be more accurate, within a given time frame), the more the better. these two are both indicate a system's capability for performing user requests, but in different angles. certainly for application with low-medium write stress less write threads within client to issue requests is OK, but for application with high write stress, users/clients may feel bad if system just can't serve/reach their real-world throughput, no matter how many client threads are configured/added. with the improvement of this patch, at least we can satisfy users' such high throughput requirement by adding client threads. And without improving individual write request's latency(as this patch does, it does nothing for individual write's latency), it's hard to improve throughput for synchronous client write thread(it can issue next request only after it's done with the current one), that's why this patch has less effect for single/less write threads. If HBase supports asynchronous client write thread, I think this patch can also provide big improvement even with less client write threads.

          5) I should also mention that while working on a different use case, I was trying to bring a layer of indirection b/w regionserver handlers and sync operation (Sync is the most costly affair in all HLog story). ...

          ===> glad to see similar approach. wonder if have same improvement under heavy write stress(such as 50/100/200 client write threads). Looking forward to seeing your final test results

          Finally, Himanshu Vashishtha, thank you very much, you raised some really good questions.

          Show
          Honghua Feng added a comment - Himanshu Vashishtha : 1) log rolling thread safety: Log rolling happens in parallel with flush/sync. Currently, in FSHLog, sync call grabs the updateLock to ensure it has a non-null writer (because of parallel log rolling). How does this patch address the non-null writer? Or is it not needed anymore? Also, if you go for the updatelock in sync, that might result in deadlock. ==> Good question . in rollWriter(), before switching this.writer to the newly created writer, updateLock is held and cleanupCurrentWriter() is called . updateLock is held guarantees no new edits enters pendingWrites, and cleanupCurrentWriter() is called guarantees all edits in current pendingWrites must be written to hdfs and sync-ed(inside this method 'sync()' is called to provide this guarantee). This means when switching this.writer to newly HLog writer, no new edits enter and all current edits have already been sync-ed to hdfs, all AsyncWriter/AsyncSyncer threads have nothing to do and are idle, so no log rolling thread safety issue here 2) Error handling: It is not very clear how is flush/sync failures are being handled?... Let's say there are two handlers waiting for sync, t1 on txid 8 and t2 on txid 10. And, t1 wakes up on notification. Would t1 also get this exception? Wouldn't it be wrong, because txid 8 may have succeeded? Please correct me if I missed anything. ==> your understanding here is correct, but the write semantic of HBase is 'successful write response means a successful write, but failed write response can mean either a successful write or a failed write', right? and I have already mentioned this in above comment. this behavior can be improved by adding a txid-range to txid mapping, this mapping can help exactly indicate a failed txid fail which pending txid-range 3) I think the current HLogPE doesn't do justice to the real use case...Almost all HLog calls are appendNoSync, followed by a sync call. In the current HLogPE, we are calling append calls, which also does the sync ==> you're right, but from an whole view of write process, these two have no big difference concerning performance: append() calls sync() inside, and in real case of HRegion.java, appendNoSync and sync is called, since sync now is a pending on notification, the difference of these two behavior is sooner/later to pend on notification, no impact on overall write performance(after put edits to pendingWrites, write handler thread can just wait for write/sync to hdfs to finish and can't help/influence write performance)...right? 4) Perf numbers are super impressive. It would have been wonderful to have such numbers for lesser number of handler threads also (e.g., 5-10 threads). IMHO, that represents most common case scenario, but I could be wrong. I know this has been beaten to death in the discussions above, but just echoing my thoughts here ==> I think maybe many guys hold a same point of view as yours here, but I also have my personal thought on this Throughput is different from latency: latency represents how quickly a system perform user requests, the quicker the better; while throughput represents how many requests a system perform user requests(concurrently, or to be more accurate, within a given time frame), the more the better. these two are both indicate a system's capability for performing user requests, but in different angles. certainly for application with low-medium write stress less write threads within client to issue requests is OK, but for application with high write stress, users/clients may feel bad if system just can't serve/reach their real-world throughput, no matter how many client threads are configured/added. with the improvement of this patch, at least we can satisfy users' such high throughput requirement by adding client threads. And without improving individual write request's latency(as this patch does, it does nothing for individual write's latency), it's hard to improve throughput for synchronous client write thread(it can issue next request only after it's done with the current one), that's why this patch has less effect for single/less write threads. If HBase supports asynchronous client write thread, I think this patch can also provide big improvement even with less client write threads. 5) I should also mention that while working on a different use case, I was trying to bring a layer of indirection b/w regionserver handlers and sync operation (Sync is the most costly affair in all HLog story). ... ===> glad to see similar approach . wonder if have same improvement under heavy write stress(such as 50/100/200 client write threads). Looking forward to seeing your final test results Finally, Himanshu Vashishtha , thank you very much, you raised some really good questions.
          Hide
          Honghua Feng added a comment -

          Jean-Marc Spaggiari

          For the random, you can also use something like System.currentTimeMillis() % asyncSyncers.length

          ==> yeah. System.currentTimeMillis() is a good candidate. txid can do the same thing as well. thanks

          Show
          Honghua Feng added a comment - Jean-Marc Spaggiari For the random, you can also use something like System.currentTimeMillis() % asyncSyncers.length ==> yeah. System.currentTimeMillis() is a good candidate. txid can do the same thing as well. thanks
          Hide
          stack added a comment -

          See what hadoopqa thinks of Honghua Feng 's patch

          Show
          stack added a comment - See what hadoopqa thinks of Honghua Feng 's patch
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12617842/HBASE-8755-trunk-v6.patch
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 6 new or modified tests.

          +1 hadoop1.0. The patch compiles against the hadoop 1.0 profile.

          +1 hadoop1.1. The patch compiles against the hadoop 1.1 profile.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 2 new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 lineLengths. The patch does not introduce lines longer than 100

          -1 site. The patch appears to cause mvn site goal to fail.

          +1 core tests. The patch passed unit tests in .

          Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
          Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//console

          This message is automatically generated.

          Show
          Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12617842/HBASE-8755-trunk-v6.patch against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 6 new or modified tests. +1 hadoop1.0 . The patch compiles against the hadoop 1.0 profile. +1 hadoop1.1 . The patch compiles against the hadoop 1.1 profile. +1 javadoc . The javadoc tool did not generate any warning messages. +1 javac . The applied patch does not increase the total number of javac compiler warnings. -1 findbugs . The patch appears to introduce 2 new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 lineLengths . The patch does not introduce lines longer than 100 -1 site . The patch appears to cause mvn site goal to fail. +1 core tests . The patch passed unit tests in . Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8118//console This message is automatically generated.
          Hide
          stack added a comment -

          All sounds good above Honghua Feng I'm running a few tests here on cluster to see it basically works. Any other comments by anyone else? Otherwise, was planning on committing. We can work on further speedup in new issues; e.g. see if we can do less threads as per Himanshu Vashishtha

          Show
          stack added a comment - All sounds good above Honghua Feng I'm running a few tests here on cluster to see it basically works. Any other comments by anyone else? Otherwise, was planning on committing. We can work on further speedup in new issues; e.g. see if we can do less threads as per Himanshu Vashishtha
          Hide
          Ted Yu added a comment -

          +1 from me.

          Show
          Ted Yu added a comment - +1 from me.
          Hide
          Jonathan Hsieh added a comment - - edited

          I did most of a review of v4 last week – here are a few nits:


          nit: (fix on commit)

          +          //    up and holds the lock
          +          // NOTE! can't hold 'upateLock' here since rollWriter will pend
          +          // on 'sync()' with 'updateLock', but 'sync()' will wait for
          +          // AsyncWriter/AsyncSyncer/AsyncNotifier series. without upateLock
          +          // can leads to pendWrites more than pendingTxid, but not problem
          

          spelling 2x: upate -> update


          This can go in a follow up issue – and please add a description of the threads / queues / invariants and how a wal writes happens in the class javadoc. An updated version of the 1-6 list in the description would be great.


          Good stuff Honghua Feng!

          Show
          Jonathan Hsieh added a comment - - edited I did most of a review of v4 last week – here are a few nits: nit: (fix on commit) + // up and holds the lock + // NOTE! can't hold 'upateLock' here since rollWriter will pend + // on 'sync()' with 'updateLock', but 'sync()' will wait for + // AsyncWriter/AsyncSyncer/AsyncNotifier series. without upateLock + // can leads to pendWrites more than pendingTxid, but not problem spelling 2x: upate -> update This can go in a follow up issue – and please add a description of the threads / queues / invariants and how a wal writes happens in the class javadoc. An updated version of the 1-6 list in the description would be great. Good stuff Honghua Feng !
          Hide
          Himanshu Vashishtha added a comment -

          Thanks for the explanation Honghua Feng; and it pretty much answers all my questions. Also, looking more, getting rid of LogSyncer thread eases out the locking semantics of rolling.

          Yes, I reran the above experiments on a more standard environment (4 DNs with HLogPE running on a DN, and log level set to INFO instead of Debug), and got mixed results this time. Varied threads from 2 to 100 and didn't get a clear winner. Given the current state of this patch and the cleanup it does, I am +1 for committing this.

          Looking forward to it. Thanks.

          Show
          Himanshu Vashishtha added a comment - Thanks for the explanation Honghua Feng ; and it pretty much answers all my questions. Also, looking more, getting rid of LogSyncer thread eases out the locking semantics of rolling. Yes, I reran the above experiments on a more standard environment (4 DNs with HLogPE running on a DN, and log level set to INFO instead of Debug), and got mixed results this time. Varied threads from 2 to 100 and didn't get a clear winner. Given the current state of this patch and the cleanup it does, I am +1 for committing this. Looking forward to it. Thanks.
          Hide
          Honghua Feng added a comment -

          Thanks Jonathan Hsieh, I've made and attached a new patch based on your comment.
          Thanks everyone again for your valuable comment and feedback.

          Show
          Honghua Feng added a comment - Thanks Jonathan Hsieh , I've made and attached a new patch based on your comment. Thanks everyone again for your valuable comment and feedback.
          Hide
          Honghua Feng added a comment -

          Himanshu Vashishtha:

          Looking forward to your performance comparison test result (trunk, with-this-patch, with-syncer-only) using latest HLogPE with new 'appendWithoutSync + sync' logic. And I'll also try to do the same test for double comparison/confirm.

          Performance improvement is the first and foremost goal of this patch, code cleanup is a just by-the-way side-effect, so we want to see this patch accepted/checked-in because of the performance improvement it brings, but because of the code cleanup it does

          Show
          Honghua Feng added a comment - Himanshu Vashishtha : Looking forward to your performance comparison test result (trunk, with-this-patch, with-syncer-only) using latest HLogPE with new 'appendWithoutSync + sync' logic. And I'll also try to do the same test for double comparison/confirm. Performance improvement is the first and foremost goal of this patch, code cleanup is a just by-the-way side-effect, so we want to see this patch accepted/checked-in because of the performance improvement it brings, but because of the code cleanup it does
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12618171/HBASE-8755-trunk-v7.patch
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 6 new or modified tests.

          +1 hadoop1.0. The patch compiles against the hadoop 1.0 profile.

          +1 hadoop1.1. The patch compiles against the hadoop 1.1 profile.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          -1 findbugs. The patch appears to introduce 3 new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 lineLengths. The patch does not introduce lines longer than 100

          -1 site. The patch appears to cause mvn site goal to fail.

          +1 core tests. The patch passed unit tests in .

          Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
          Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//console

          This message is automatically generated.

          Show
          Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12618171/HBASE-8755-trunk-v7.patch against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 6 new or modified tests. +1 hadoop1.0 . The patch compiles against the hadoop 1.0 profile. +1 hadoop1.1 . The patch compiles against the hadoop 1.1 profile. +1 javadoc . The javadoc tool did not generate any warning messages. +1 javac . The applied patch does not increase the total number of javac compiler warnings. -1 findbugs . The patch appears to introduce 3 new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 lineLengths . The patch does not introduce lines longer than 100 -1 site . The patch appears to cause mvn site goal to fail. +1 core tests . The patch passed unit tests in . Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8132//console This message is automatically generated.
          Hide
          stack added a comment -

          I'm running some tests local just to make sure. Will report back...

          Show
          stack added a comment - I'm running some tests local just to make sure. Will report back...
          Hide
          stack added a comment -

          Pardon me. This is taking a while; hardware issues and now I trunk seems to have issue where it hangs syncing, pre-patch I believe... investigating.

          Here is what I see. Lots of threads BLOCKED here:

          "RpcServer.handler=0,port=60020" daemon prio=10 tid=0x00000000012f1800 nid=0x3cb5 waiting for monitor entry [0x00007fdb0eb55000]
             java.lang.Thread.State: BLOCKED (on object monitor)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:1006)
                  - waiting to lock <0x0000000456c00390> (a java.lang.Object)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.appendNoSync(FSHLog.java:1054)
                  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)
          

          Then the fella w/ the lock is doing this:

          "regionserver60020.logRoller" daemon prio=10 tid=0x0000000001159800 nid=0x3ca7 in Object.wait() [0x00007fdb0f964000]
             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:1307)
                  - locked <0x0000000456bf37a8> (a java.util.concurrent.atomic.AtomicLong)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.syncer(FSHLog.java:1299)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1412)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.cleanupCurrentWriter(FSHLog.java:760)
                  at org.apache.hadoop.hbase.regionserver.wal.FSHLog.rollWriter(FSHLog.java:566)
                  - locked <0x0000000456c00390> (a java.lang.Object)
                  - locked <0x0000000456c00330> (a java.lang.Object)
                  at org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:96)
                  at java.lang.Thread.run(Thread.java:744)
          

          Server is bound up.

          Show
          stack added a comment - Pardon me. This is taking a while; hardware issues and now I trunk seems to have issue where it hangs syncing, pre-patch I believe... investigating. Here is what I see. Lots of threads BLOCKED here: "RpcServer.handler=0,port=60020" daemon prio=10 tid=0x00000000012f1800 nid=0x3cb5 waiting for monitor entry [0x00007fdb0eb55000] java.lang. Thread .State: BLOCKED (on object monitor) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.append(FSHLog.java:1006) - waiting to lock <0x0000000456c00390> (a java.lang. Object ) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.appendNoSync(FSHLog.java:1054) 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) Then the fella w/ the lock is doing this: "regionserver60020.logRoller" daemon prio=10 tid=0x0000000001159800 nid=0x3ca7 in Object .wait() [0x00007fdb0f964000] 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:1307) - locked <0x0000000456bf37a8> (a java.util.concurrent.atomic.AtomicLong) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.syncer(FSHLog.java:1299) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.sync(FSHLog.java:1412) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.cleanupCurrentWriter(FSHLog.java:760) at org.apache.hadoop.hbase.regionserver.wal.FSHLog.rollWriter(FSHLog.java:566) - locked <0x0000000456c00390> (a java.lang. Object ) - locked <0x0000000456c00330> (a java.lang. Object ) at org.apache.hadoop.hbase.regionserver.LogRoller.run(LogRoller.java:96) at java.lang. Thread .run( Thread .java:744) Server is bound up.
          Hide
          stack added a comment -

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

          "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)
          

          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:

          "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)
          

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

           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();
          ...
          

          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 }

          Show
          stack added a comment - Hmm... Happens when this patch is in place. Stuck here: "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) 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: "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) ... but can't make progress blocked on updateLock. 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(); ... 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 }
          Hide
          stack added a comment -

          Will provide more on this tomorrow.

          Show
          stack added a comment - Will provide more on this tomorrow.
          Hide
          Honghua Feng added a comment -

          The stack-traces are ok: when log-roll occurs, it holds the updateLock to prevent any subsequent writer handler put edits to pendingWrites (that's why all writer handler threads pend on updateLock), and then it calls sync to wait for Async* threads to sync all edits currently in pendingWrites (that's why logroller pend on sync())...
          Why no progress, we need to see why Async* threads don't finish the sync of current pendingWrites, would you please provide the Async* threads' stack traces?
          Only AsyncWriter from Async* threads needs the pendingWritesLock to grab the edits from pendingWrites, and AsyncNotifier needs syncTillHere to update it and notifyAll, these two both are hold-able under current situation: write handler threads can't hold pendingWritesLock before holding updateLock(within append()), logroller doesn't hold pendingWritesLock at all, and logroller is waiting for syncTillHere's change...

          Show
          Honghua Feng added a comment - The stack-traces are ok: when log-roll occurs, it holds the updateLock to prevent any subsequent writer handler put edits to pendingWrites (that's why all writer handler threads pend on updateLock), and then it calls sync to wait for Async* threads to sync all edits currently in pendingWrites (that's why logroller pend on sync())... Why no progress, we need to see why Async* threads don't finish the sync of current pendingWrites, would you please provide the Async* threads' stack traces? Only AsyncWriter from Async* threads needs the pendingWritesLock to grab the edits from pendingWrites, and AsyncNotifier needs syncTillHere to update it and notifyAll, these two both are hold-able under current situation: write handler threads can't hold pendingWritesLock before holding updateLock(within append()), logroller doesn't hold pendingWritesLock at all, and logroller is waiting for syncTillHere's change...
          Hide
          stack added a comment -

          A couple of thread dumps w/ WAL hang in them.

          Sorry, should have attached this last night Honghua Feng

          Show
          stack added a comment - A couple of thread dumps w/ WAL hang in them. Sorry, should have attached this last night Honghua Feng
          Hide
          stack added a comment -

          All the Async* are just hanging out waiting. Missed notification or no notification?

          Show
          stack added a comment - All the Async* are just hanging out waiting. Missed notification or no notification?
          Hide
          stack added a comment -

          I notice a syncer exited (it is missing from the thread dump).

          2013-12-11 22:40:28,887 INFO [regionserver60020-AsyncHLogSyncer3-1386830380159] wal.FSHLog: regionserver60020-AsyncHLogSyncer3-1386830380159 exiting

          On a new run, again a thread exits:

          2013-12-12 10:34:34,620 DEBUG [regionserver60020.logRoller] regionserver.LogRoller: HLog roll requested
          2013-12-12 10:34:35,526 DEBUG [regionserver60020.logRoller] wal.FSHLog: cleanupCurrentWriter waiting for transactions to get synced total 37240 synced till here 37210
          2013-12-12 10:34:36,560 INFO [regionserver60020-AsyncHLogSyncer1-1386873205908] wal.FSHLog: regionserver60020-AsyncHLogSyncer1-1386873205908 exiting

          Let me try and figure the why.

          Show
          stack added a comment - I notice a syncer exited (it is missing from the thread dump). 2013-12-11 22:40:28,887 INFO [regionserver60020-AsyncHLogSyncer3-1386830380159] wal.FSHLog: regionserver60020-AsyncHLogSyncer3-1386830380159 exiting On a new run, again a thread exits: 2013-12-12 10:34:34,620 DEBUG [regionserver60020.logRoller] regionserver.LogRoller: HLog roll requested 2013-12-12 10:34:35,526 DEBUG [regionserver60020.logRoller] wal.FSHLog: cleanupCurrentWriter waiting for transactions to get synced total 37240 synced till here 37210 2013-12-12 10:34:36,560 INFO [regionserver60020-AsyncHLogSyncer1-1386873205908] wal.FSHLog: regionserver60020-AsyncHLogSyncer1-1386873205908 exiting Let me try and figure the why.
          Hide
          stack added a comment -

          Catching all exceptions, I got a NPE.

          2013-12-12 11:03:43,870 INFO [Thread-14] regionserver.DefaultStoreFlusher: Flushed, sequenceid=2680455, memsize=129.3 M, hasBloomFilter=true, into tmp file hdfs://c2020.halxg.cloudera.com:8020/hbase/data/default/usertable/8dcf9c17c090f476346c8a31e4c9eddb/.tmp/64aaf14b38224f4fbce0a999f92dd8f4
          2013-12-12 11:03:43,879 INFO [regionserver60020-AsyncHLogSyncer2-1386874930310] wal.FSHLog: UNEXPECTED
          java.lang.NullPointerException
          at org.apache.hadoop.hbase.regionserver.wal.FSHLog$AsyncSyncer.run(FSHLog.java:1205)
          at java.lang.Thread.run(Thread.java:744)

          Writer is null.

          Show
          stack added a comment - Catching all exceptions, I got a NPE. 2013-12-12 11:03:43,870 INFO [Thread-14] regionserver.DefaultStoreFlusher: Flushed, sequenceid=2680455, memsize=129.3 M, hasBloomFilter=true, into tmp file hdfs://c2020.halxg.cloudera.com:8020/hbase/data/default/usertable/8dcf9c17c090f476346c8a31e4c9eddb/.tmp/64aaf14b38224f4fbce0a999f92dd8f4 2013-12-12 11:03:43,879 INFO [regionserver60020-AsyncHLogSyncer2-1386874930310] wal.FSHLog: UNEXPECTED java.lang.NullPointerException at org.apache.hadoop.hbase.regionserver.wal.FSHLog$AsyncSyncer.run(FSHLog.java:1205) at java.lang.Thread.run(Thread.java:744) Writer is null.
          Hide
          stack added a comment -

          Same as Honghua Feng's patch only it checks for null writer before using it – this is currently in the code and seems to make this patch work again (i'm testing) – and adds this on tail of each Async* thread:

          + } catch (Exception e)

          { + LOG.error("UNEXPECTED", e); }

          finally {

          Also renames the threads from AsyncHLog* to WAL.Async. Minor.

          Show
          stack added a comment - Same as Honghua Feng 's patch only it checks for null writer before using it – this is currently in the code and seems to make this patch work again (i'm testing) – and adds this on tail of each Async* thread: + } catch (Exception e) { + LOG.error("UNEXPECTED", e); } finally { Also renames the threads from AsyncHLog* to WAL.Async. Minor.
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12618499/8755v8.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 6 new or modified tests.

          -1 hadoop1.0. The patch failed to compile against the hadoop 1.0 profile.
          Here is snippet of errors:

          [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:2.5.1:compile (default-compile) on project hbase-server: Compilation failure: Compilation failure:
          [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[436,42] unclosed string literal
          [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[436,63] ';' expected
          [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[437,17] illegal start of expression
          [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[437,23] ';' expected
          [ERROR] -> [Help 1]
          org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:2.5.1:compile (default-compile) on project hbase-server: Compilation failure
          	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:213)
          	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153)
          	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145)
          	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:84)
          	at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:59)
          --
          Caused by: org.apache.maven.plugin.CompilationFailureException: Compilation failure
          	at org.apache.maven.plugin.AbstractCompilerMojo.execute(AbstractCompilerMojo.java:729)
          	at org.apache.maven.plugin.CompilerMojo.execute(CompilerMojo.java:128)
          	at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:101)
          	at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:209)
          	... 19 more

          Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8148//console

          This message is automatically generated.

          Show
          Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12618499/8755v8.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 6 new or modified tests. -1 hadoop1.0 . The patch failed to compile against the hadoop 1.0 profile. Here is snippet of errors: [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:2.5.1:compile ( default -compile) on project hbase-server: Compilation failure: Compilation failure: [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[436,42] unclosed string literal [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[436,63] ';' expected [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[437,17] illegal start of expression [ERROR] /home/jenkins/jenkins-slave/workspace/PreCommit-HBASE-Build/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java:[437,23] ';' expected [ERROR] -> [Help 1] org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:2.5.1:compile ( default -compile) on project hbase-server: Compilation failure at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:213) at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153) at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145) at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:84) at org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:59) -- Caused by: org.apache.maven.plugin.CompilationFailureException: Compilation failure at org.apache.maven.plugin.AbstractCompilerMojo.execute(AbstractCompilerMojo.java:729) at org.apache.maven.plugin.CompilerMojo.execute(CompilerMojo.java:128) at org.apache.maven.plugin.DefaultBuildPluginManager.executeMojo(DefaultBuildPluginManager.java:101) at org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:209) ... 19 more Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8148//console This message is automatically generated.
          Hide
          stack added a comment -

          Fix compile issue.

          Show
          stack added a comment - Fix compile issue.
          Hide
          stack added a comment -

          This is what I'll commit. I've been running it on small cluster this afternoon and after fixing hardware, it seems to run fine at about the same speed as what we have currently (ycsb read/write loading).

          Show
          stack added a comment - This is what I'll commit. I've been running it on small cluster this afternoon and after fixing hardware, it seems to run fine at about the same speed as what we have currently (ycsb read/write loading).
          Hide
          stack added a comment -

          32 threads on one node writing a cluster of 4 nodes (~8 threads per server which according to our tests to date shows this model running slower than what we have). It does 10% less throughput after ~25minutes. We need to get the other speedups in after this goes in.

          Show
          stack added a comment - 32 threads on one node writing a cluster of 4 nodes (~8 threads per server which according to our tests to date shows this model running slower than what we have). It does 10% less throughput after ~25minutes. We need to get the other speedups in after this goes in.
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12618509/8755v9.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          +1 tests included. The patch appears to include 6 new or modified tests.

          +1 hadoop1.0. The patch compiles against the hadoop 1.0 profile.

          +1 hadoop1.1. The patch compiles against the hadoop 1.1 profile.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 lineLengths. The patch does not introduce lines longer than 100

          -1 site. The patch appears to cause mvn site goal to fail.

          +1 core tests. The patch passed unit tests in .

          Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//testReport/
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html
          Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html
          Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//console

          This message is automatically generated.

          Show
          Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12618509/8755v9.txt against trunk revision . +1 @author . The patch does not contain any @author tags. +1 tests included . The patch appears to include 6 new or modified tests. +1 hadoop1.0 . The patch compiles against the hadoop 1.0 profile. +1 hadoop1.1 . The patch compiles against the hadoop 1.1 profile. +1 javadoc . The javadoc tool did not generate any warning messages. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 lineLengths . The patch does not introduce lines longer than 100 -1 site . The patch appears to cause mvn site goal to fail. +1 core tests . The patch passed unit tests in . Test results: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//testReport/ Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop2-compat.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-prefix-tree.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-client.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-common.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-protocol.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-server.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-examples.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-thrift.html Findbugs warnings: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//artifact/trunk/patchprocess/newPatchFindbugsWarningshbase-hadoop-compat.html Console output: https://builds.apache.org/job/PreCommit-HBASE-Build/8149//console This message is automatically generated.
          Hide
          Honghua Feng added a comment -

          stack thanks. seems no further blocking issue?

          Show
          Honghua Feng added a comment - stack thanks. seems no further blocking issue?
          Hide
          Honghua Feng added a comment -

          It does 10% less throughput after ~25minutes.

          ==> it's normal when flush/compact occurs after ~25 minutes write, we saw such level downgrade when doing long time tests for both with/without patch.

          Show
          Honghua Feng added a comment - It does 10% less throughput after ~25minutes. ==> it's normal when flush/compact occurs after ~25 minutes write, we saw such level downgrade when doing long time tests for both with/without patch.
          Hide
          stack added a comment -

          I did a compare over three hours and throughput flips in favor of this patch the longer we run (15% more writes after three hours)

          Let me commit. This patch makes for better throughput when under high contention, cleans up the code, and lays groundwork for multiwal with its detaching syncing from handlers but I don't like the slowdown at low numbers. Let us fix that promptly. I created a subtask to address this and assigned myself.

          Show
          stack added a comment - I did a compare over three hours and throughput flips in favor of this patch the longer we run (15% more writes after three hours) Let me commit. This patch makes for better throughput when under high contention, cleans up the code, and lays groundwork for multiwal with its detaching syncing from handlers but I don't like the slowdown at low numbers. Let us fix that promptly. I created a subtask to address this and assigned myself.
          Hide
          stack added a comment -

          Committed to trunk. Andrew Purtell Do you want this in 0.98?

          Thanks for the hard work Honghua Feng and persistence getting this in.

          Show
          stack added a comment - Committed to trunk. Andrew Purtell Do you want this in 0.98? Thanks for the hard work Honghua Feng and persistence getting this in.
          Hide
          Andrew Purtell added a comment -

          Yes stack, let's get this out into a near release so we can see how it holds up. If we see perf problems during RC evaluation, we can revert if necessary.

          Show
          Andrew Purtell added a comment - Yes stack , let's get this out into a near release so we can see how it holds up. If we see perf problems during RC evaluation, we can revert if necessary.
          Hide
          stack added a comment -

          Committed to 0.98 also.

          Show
          stack added a comment - Committed to 0.98 also.
          Hide
          Hudson added a comment -

          SUCCESS: Integrated in HBase-TRUNK #4722 (See https://builds.apache.org/job/HBase-TRUNK/4722/)
          HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550778)

          • /hbase/trunk/hbase-common/src/main/resources/hbase-default.xml
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Show
          Hudson added a comment - SUCCESS: Integrated in HBase-TRUNK #4722 (See https://builds.apache.org/job/HBase-TRUNK/4722/ ) HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550778) /hbase/trunk/hbase-common/src/main/resources/hbase-default.xml /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Hide
          Hudson added a comment -

          SUCCESS: Integrated in HBase-0.98 #11 (See https://builds.apache.org/job/HBase-0.98/11/)
          HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550782)

          • /hbase/branches/0.98/hbase-common/src/main/resources/hbase-default.xml
          • /hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
          • /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
          • /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Show
          Hudson added a comment - SUCCESS: Integrated in HBase-0.98 #11 (See https://builds.apache.org/job/HBase-0.98/11/ ) HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550782) /hbase/branches/0.98/hbase-common/src/main/resources/hbase-default.xml /hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Hide
          Hudson added a comment -

          SUCCESS: Integrated in HBase-0.98-on-Hadoop-1.1 #8 (See https://builds.apache.org/job/HBase-0.98-on-Hadoop-1.1/8/)
          HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550782)

          • /hbase/branches/0.98/hbase-common/src/main/resources/hbase-default.xml
          • /hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
          • /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
          • /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Show
          Hudson added a comment - SUCCESS: Integrated in HBase-0.98-on-Hadoop-1.1 #8 (See https://builds.apache.org/job/HBase-0.98-on-Hadoop-1.1/8/ ) HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550782) /hbase/branches/0.98/hbase-common/src/main/resources/hbase-default.xml /hbase/branches/0.98/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java /hbase/branches/0.98/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Hide
          Hudson added a comment -

          SUCCESS: Integrated in HBase-TRUNK-on-Hadoop-1.1 #5 (See https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-1.1/5/)
          HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550778)

          • /hbase/trunk/hbase-common/src/main/resources/hbase-default.xml
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Show
          Hudson added a comment - SUCCESS: Integrated in HBase-TRUNK-on-Hadoop-1.1 #5 (See https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-1.1/5/ ) HBASE-8755 A new write thread model for HLog to improve the overall HBase write throughput (stack: rev 1550778) /hbase/trunk/hbase-common/src/main/resources/hbase-default.xml /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          Hide
          ramkrishna.s.vasudevan added a comment -

          Nice one.

          Show
          ramkrishna.s.vasudevan added a comment - Nice one.
          Hide
          Hudson added a comment -

          SUCCESS: Integrated in HBase-TRUNK #4856 (See https://builds.apache.org/job/HBase-TRUNK/4856/)
          HBASE-10156 FSHLog Refactor (WAS -> Fix up the HBASE-8755 slowdown when low contention) (stack: rev 1561450)

          • /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
          • /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
          • /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java
          • /hbase/trunk/hbase-server/pom.xml
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
          • /hbase/trunk/pom.xml
          Show
          Hudson added a comment - SUCCESS: Integrated in HBase-TRUNK #4856 (See https://builds.apache.org/job/HBase-TRUNK/4856/ ) HBASE-10156 FSHLog Refactor (WAS -> Fix up the HBASE-8755 slowdown when low contention) (stack: rev 1561450) /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java /hbase/trunk/hbase-server/pom.xml /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java /hbase/trunk/pom.xml
          Hide
          Hudson added a comment -

          FAILURE: Integrated in HBase-TRUNK-on-Hadoop-1.1 #65 (See https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-1.1/65/)
          HBASE-10156 FSHLog Refactor (WAS -> Fix up the HBASE-8755 slowdown when low contention) (stack: rev 1561450)

          • /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java
          • /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java
          • /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java
          • /hbase/trunk/hbase-server/pom.xml
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java
          • /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java
          • /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java
          • /hbase/trunk/pom.xml
          Show
          Hudson added a comment - FAILURE: Integrated in HBase-TRUNK-on-Hadoop-1.1 #65 (See https://builds.apache.org/job/HBase-TRUNK-on-Hadoop-1.1/65/ ) HBASE-10156 FSHLog Refactor (WAS -> Fix up the HBASE-8755 slowdown when low contention) (stack: rev 1561450) /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedLogCloseException.java /hbase/trunk/hbase-client/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FailedSyncBeforeLogCloseException.java /hbase/trunk/hbase-server/pom.xml /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSHLog.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLog.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogFactory.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogKey.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/HLogUtil.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/ProtobufLogWriter.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/RingBufferTruck.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/SyncFuture.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALCoprocessorHost.java /hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/WALEdit.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/mapreduce/TestHLogRecordReader.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestDistributedLogSplitting.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestParallelPut.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/HLogPerformanceEvaluation.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestDurability.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLogSplit.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollAbort.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRolling.java /hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestLogRollingNoCluster.java /hbase/trunk/pom.xml

            People

            • Assignee:
              Honghua Feng
              Reporter:
              Honghua Feng
            • Votes:
              1 Vote for this issue
              Watchers:
              47 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development