Just wanted to follow up here with a little update.  We enabled the Aggregation 
coprocessor on our dev cluster.  Here are the quick timing stats.

Tables: 565
Total Rows: 2,749,015,957
Total Time (to count): 52m:33s

Will be interesting to see how this fairs against our production clusters with 
a lot more data.

Thanks again for all of your help!
Birch
On Sep 20, 2013, at 10:06 PM, lars hofhansl <[email protected]> wrote:

> Hey we all start somewhere. I did the "LocalJobRunner" thing many times and 
> wondered why it was so slow, until I realized I hadn't setup my client 
> correctly.
> The LocalJobRunner runs the M/R job on the client machine. This is really 
> just for testing and terribly slow.
> 
> From later emails in this I gather you managed to run this as an actual M/R 
> on the cluster? (by the way you do not need to start the job on a machine on 
> the cluster, but just configure your client correctly to ship the job to the 
> M/R cluster)
> 
> 
> Was that still too slow? I would love to get my hand on some numbers. If you 
> have trillions of rows and can run this job with a few mappers per machines, 
> those would be good numbers to publish here.
> In any case, let us know how it goes.
> 
> 
> -- Lars
> 
> 
> btw. my calculation were assuming that network IO is the bottleneck. For 
> larger jobs (such as yours) it's typically either that or disk IO. 
> ________________________________
> 
> From: James Birchfield <[email protected]>
> To: [email protected]; lars hofhansl <[email protected]> 
> Sent: Friday, September 20, 2013 6:21 PM
> Subject: Re: HBase Table Row Count Optimization - A Solicitation For Help
> 
> 
> Thanks Lars.  I like your time calculations much better than mine.
> 
> So this is where my inexperience is probably going to come glaring through.  
> And maybe the root of all this.  I am not running the MapReduce job on a node 
> in the cluster.  It is running on a development server that connects remotely 
> to the cluster.  Further more, I am not executing the MpReduce job from the 
> command line using the CLI as seen in many of the examples.  I am executing 
> them in process of a stand-alone Java process I have written.  It is simple 
> in nature, it simply creates an HBaseAdmin connection, list the tables and 
> looks up the column families, code the admin connection, then loops over the 
> table list, and runs the following code:
> 
> public class RowCounterRunner {
> 
>     public static long countRows(String tableName) throws Exception {
> 
>         Job job = RowCounter.createSubmittableJob(
>                 ConfigManager.getConfiguration(), new String[]{tableName});
>         boolean waitForCompletion = job.waitForCompletion(true);
>         Counters counters = job.getCounters();
>         Counter findCounter = 
> counters.findCounter(hbaseadminconnection.Counters.ROWS);
>         long value2 = findCounter.getValue();
>         return value2;
> 
>     }
> }
> 
> I sort of stumbled on to this implementation as a fairly easy way to automate 
> the process.  So based on your comments, and the fact that I see this in my 
> log:
> 
> 2013-09-20 23:41:05,556 INFO  [LocalJobRunner Map Task Executor #0] 
> LocalJobRunner                 : map
> 
> makes me think I am not taking advantage of the cluster effectively, if at 
> all.  I do not mind at all running the MapReduce job using the hbase/hadoop 
> CLI, I can script that as well.  I just thought this would work decently 
> enough.
> 
> It does seem like it will be possible to use the Agregation coprocessor as 
> suggested a little earlier in this thread.  It may speed things up as well.  
> But either way, I need to understand if I am losing significant performance 
> running in the manner I am.  Which at this point sounds like I probably am.
> 
> Birch
> On Sep 20, 2013, at 6:09 PM, lars hofhansl <[email protected]> wrote:
> 
>> From your numbers below you have about 26k regions, thus each region is 
>> about 545tb/26k = 20gb. Good.
>> 
>> How many mappers are you running?
>> And just to rule out the obvious, the M/R is running on the cluster and not 
>> locally, right? (it will default to a local runner when it cannot use the 
>> M/R cluster).
>> 
>> Some back of the envelope calculations tell me that assuming 1ge network 
>> cards, the best you can expect for 110 machines to map through this data is 
>> about 10h. (so way faster than what you see).
>> (545tb/(110*1/8gb/s) ~ 40ks ~11h)
>> 
>> 
>> We should really add a rowcounting coprocessor to HBase and allow using it 
>> via M/R.
>> 
>> -- Lars
>> 
>> 
>> 
>> ________________________________
>> From: James Birchfield <[email protected]>
>> To: [email protected] 
>> Sent: Friday, September 20, 2013 5:09 PM
>> Subject: Re: HBase Table Row Count Optimization - A Solicitation For Help
>> 
>> 
>> I did not implement accurate timing, but the current table being counted has 
>> been running for about 10 hours, and the log is estimating the map portion 
>> at 10%
>> 
>> 2013-09-20 23:40:24,099 INFO  [main] Job                            :  map 
>> 10% reduce 0%
>> 
>> So a loooong time.  Like I mentioned, we have billions, if not trillions of 
>> rows potentially.
>> 
>> Thanks for the feedback on the approaches I mentioned.  I was not sure if 
>> they would have any effect overall.
>> 
>> I will look further into coprocessors.
>> 
>> Thanks!
>> Birch
>> On Sep 20, 2013, at 4:58 PM, Vladimir Rodionov <[email protected]> 
>> wrote:
>> 
>>> How long does it take for RowCounter Job for largest table to finish on 
>>> your cluster?
>>> 
>>> Just curious.
>>> 
>>> On your options:
>>> 
>>> 1. Not worth it probably - you may overload your cluster
>>> 2. Not sure this one differs from 1. Looks the same to me but more complex.
>>> 3. The same as 1 and 2
>>> 
>>> Counting rows in efficient way can be done if you sacrifice some accuracy :
>>> 
>>> http://highscalability.com/blog/2012/4/5/big-data-counting-how-to-count-a-billion-distinct-objects-us.html
>>> 
>>> Yeah, you will need coprocessors for that.
>>> 
>>> Best regards,
>>> Vladimir Rodionov
>>> Principal Platform Engineer
>>> Carrier IQ, www.carrieriq.com
>>> e-mail: [email protected]
>>> 
>>> ________________________________________
>>> From: James Birchfield [[email protected]]
>>> Sent: Friday, September 20, 2013 3:50 PM
>>> To: [email protected]
>>> Subject: Re: HBase Table Row Count Optimization - A Solicitation For Help
>>> 
>>> Hadoop 2.0.0-cdh4.3.1
>>> 
>>> HBase 0.94.6-cdh4.3.1
>>> 
>>> 110 servers, 0 dead, 238.2364 average load
>>> 
>>> Some other info, not sure if it helps or not.
>>> 
>>> Configured Capacity: 1295277834158080 (1.15 PB)
>>> Present Capacity: 1224692609430678 (1.09 PB)
>>> DFS Remaining: 624376503857152 (567.87 TB)
>>> DFS Used: 600316105573526 (545.98 TB)
>>> DFS Used%: 49.02%
>>> Under replicated blocks: 0
>>> Blocks with corrupt replicas: 1
>>> Missing blocks: 0
>>> 
>>> It is hitting a production cluster, but I am not really sure how to 
>>> calculate the load placed on the cluster.
>>> On Sep 20, 2013, at 3:19 PM, Ted Yu <[email protected]> wrote:
>>> 
>>>> How many nodes do you have in your cluster ?
>>>> 
>>>> When counting rows, what other load would be placed on the cluster ?
>>>> 
>>>> What is the HBase version you're currently using / planning to use ?
>>>> 
>>>> Thanks
>>>> 
>>>> 
>>>> On Fri, Sep 20, 2013 at 2:47 PM, James Birchfield <
>>>> [email protected]> wrote:
>>>> 
>>>>>         After reading the documentation and scouring the mailing list
>>>>> archives, I understand there is no real support for fast row counting in
>>>>> HBase unless you build some sort of tracking logic into your code.  In our
>>>>> case, we do not have such logic, and have massive amounts of data already
>>>>> persisted.  I am running into the issue of very long execution of the
>>>>> RowCounter MapReduce job against very large tables (multi-billion for many
>>>>> is our estimate).  I understand why this issue exists and am slowly
>>>>> accepting it, but I am hoping I can solicit some possible ideas to help
>>>>> speed things up a little.
>>>>> 
>>>>>         My current task is to provide total row counts on about 600
>>>>> tables, some extremely large, some not so much.  Currently, I have a
>>>>> process that executes the MapRduce job in process like so:
>>>>> 
>>>>>                         Job job = RowCounter.createSubmittableJob(
>>>>>                                         ConfigManager.getConfiguration(),
>>>>> new String[]{tableName});
>>>>>                         boolean waitForCompletion =
>>>>> job.waitForCompletion(true);
>>>>>                         Counters counters = job.getCounters();
>>>>>                         Counter rowCounter =
>>>>> counters.findCounter(hbaseadminconnection.Counters.ROWS);
>>>>>                         return rowCounter.getValue();
>>>>> 
>>>>>         At the moment, each MapReduce job is executed in serial order, so
>>>>> counting one table at a time.  For the current implementation of this 
>>>>> whole
>>>>> process, as it stands right now, my rough timing calculations indicate 
>>>>> that
>>>>> fully counting all the rows of these 600 tables will take anywhere between
>>>>> 11 to 22 days.  This is not what I consider a desirable timeframe.
>>>>> 
>>>>>         I have considered three alternative approaches to speed things up.
>>>>> 
>>>>>         First, since the application is not heavily CPU bound, I could use
>>>>> a ThreadPool and execute multiple MapReduce jobs at the same time looking
>>>>> at different tables.  I have never done this, so I am unsure if this would
>>>>> cause any unanticipated side effects.
>>>>> 
>>>>>         Second, I could distribute the processes.  I could find as many
>>>>> machines that can successfully talk to the desired cluster properly, give
>>>>> them a subset of tables to work on, and then combine the results post
>>>>> process.
>>>>> 
>>>>>         Third, I could combine both the above approaches and run a
>>>>> distributed set of multithreaded process to execute the MapReduce jobs in
>>>>> parallel.
>>>>> 
>>>>>         Although it seems to have been asked and answered many times, I
>>>>> will ask once again.  Without the need to change our current 
>>>>> configurations
>>>>> or restart the clusters, is there a faster approach to obtain row counts?
>>>>> FYI, my cache size for the Scan is set to 1000.  I have experimented with
>>>>> different numbers, but nothing made a noticeable difference.  Any advice 
>>>>> or
>>>>> feedback would be greatly appreciated!
>>>>> 
>>>>> Thanks,
>>>>> Birch
>>> 
>>> 
>>> Confidentiality Notice:  The information contained in this message, 
>>> including any attachments hereto, may be confidential and is intended to be 
>>> read only by the individual or entity to whom this message is addressed. If 
>>> the reader of this message is not the intended recipient or an agent or 
>>> designee of the intended recipient, please note that any review, use, 
>>> disclosure or distribution of this message or its attachments, in any form, 
>>> is strictly prohibited.  If you have received this message in error, please 
>>> immediately notify the sender and/or [email protected] and delete 
>>> or destroy any copy of this message and its attachments.

Attachment: smime.p7s
Description: S/MIME cryptographic signature

Reply via email to