If your cells are extremely small try setting the caching even higher than
10k.  You want to strike a balance between MBs of response size and number
of calls needed, leaning towards larger response sizes as far as your
system can handle (account for RS max memory, and memory available to your
mappers).

You could use the KeyOnlyFilter to further limit the sizes of responses
transferred.

Another thing that may help would be increasing your block size.  This
would speed up sequential read but slow down random access.  It would be a
matter of making the config change and then running a major compaction to
re-write the data.

We constantly run multiple MR jobs (often on the order of 10's) against the
same hbase cluster and don't often see issues.  They are not full table
scans, but they do often overlap.  I think it would be worth at least
attempting to run multiple jobs at once.




On Fri, Sep 20, 2013 at 8:09 PM, James Birchfield <
jbirchfi...@stumbleupon.com> wrote:

> 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 <vrodio...@carrieriq.com>
> 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: vrodio...@carrieriq.com
> >
> > ________________________________________
> > From: James Birchfield [jbirchfi...@stumbleupon.com]
> > Sent: Friday, September 20, 2013 3:50 PM
> > To: user@hbase.apache.org
> > 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 <yuzhih...@gmail.com> 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 <
> >> jbirchfi...@stumbleupon.com> 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 notificati...@carrieriq.com and
> delete or destroy any copy of this message and its attachments.
>
>

Reply via email to