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

Zhihong Ted Yu commented on HBASE-6466:
---------------------------------------

{code}
+    this.handlerCount = conf.getInt("hbase.regionserver.thread.flush", 1);
{code}
How about naming the new config param hbase.hstore.flusher.count ?
I would expect a default value greater than 1.
{code}
+      String threadName = "Cache Flush Handler " + this.instanceNumber;
{code}
Name the thread 'MemStore Flush Handler ' ?
{code}
+                // Wasn't able to flush any region, but we're above low water
+                // mark
{code}
There is no need to reformat comment.
{code}
+    this.cacheFlusher.setUncaughtExceptionHandler(handler);
+    this.cacheFlusher.start();
{code}
flushHandlers array is populated in start() method. Would the call to 
setUncaughtExceptionHandler() be effective ?
{code}
+      } catch (InterruptedException e) {
+        LOG.warn(flushHander.getName(), e);
+      }
{code}
Please restore interrupt status above.

If you can publish percentage of performance improvement, that would be great.
                
> Enable multi-thread for Flush
> -----------------------------
>
>                 Key: HBASE-6466
>                 URL: https://issues.apache.org/jira/browse/HBASE-6466
>             Project: HBase
>          Issue Type: Improvement
>            Reporter: chunhui shen
>            Assignee: chunhui shen
>         Attachments: HBASE-6466.patch
>
>
> If the KV is large or Hlog is closed with high-pressure putting, we found 
> memstore is often above the high water mark and block the putting.
> So should we enable multi-thread for Memstore Flush?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to