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

Jonathan Gray commented on HBASE-1935:
--------------------------------------

Went over the patch twice.  Looks pretty good.

There is some cross-over with work done in Multi operations (MultiGet, 
MultiDelete, etc..).  I think the first thing to decide is if we want to create 
some unified threading system or take passed-in ExecutorServices as is done 
with the patch.  And do we need a special ParallelHTable, or should the normal 
HTable support threading?  I believe the latter.

At either the HCM or HTable level, I think we should have a local, bounded 
ExecutorService pool.  You would be able to modify its size through the 
constructor, but default settings would come from something in the conf like 
hbase.client.threads.

One thing I do like (at least for early versions of threaded clients) is just 
failing immediately when encountering a problem like a split.  Properly 
handling this is one of the hardest parts about this (and other things like 
stateful filters), and retries are tricky and imperfect.  With batched/parallel 
reads (get or scan) we should just fail-fast and throw exceptions to let the 
client deal.  With batched/parallel writes (put or delete) we should process 
what we can and return back to the client what was not completed.

Another thing I'm a little confused about... this seems to be designed for 
completely out-of-order receipt of results.  Rather than aggregating up a list 
of Futures, and then waiting for them to complete in order, this uses a 
ExecutorCompletionService which returns things as they finish.  I can see in 
certain use cases this would make sense, but is a bit more limited.  However, I 
don't see why we can't support both using two different task completion-waiting 
paths and with very small changes to the constructor APIs.

> Scan in parallel
> ----------------
>
>                 Key: HBASE-1935
>                 URL: https://issues.apache.org/jira/browse/HBASE-1935
>             Project: Hadoop HBase
>          Issue Type: New Feature
>            Reporter: stack
>         Attachments: pscanner.patch
>
>
> A scanner that rather than scan in series, instead scanned multiple regions 
> in parallell would be more involved but could complete much faster 
> partiularly if results are sparse.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.

Reply via email to