hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jimmy Xiang <jxi...@cloudera.com>
Subject Re: Coprocessor Aggregation supposed to be ~20x slower than Scans?
Date Wed, 16 May 2012 17:28:20 GMT
Yes, it is fixed in CDH4.  It will be in the coming release.

Thanks,
Jimmy

On Tue, May 15, 2012 at 5:34 PM, Ted Yu <yuzhihong@gmail.com> wrote:

> Hopefully this gets fixed in
>
> https://repository.cloudera.com/artifactory/public/org/apache/hbase/hbase/0.92.0-cdh4b2-SNAPSHOT/
>
> A developer from Cloudera would be able to better help you.
>
> On Tue, May 15, 2012 at 5:30 PM, anil gupta <anilgupta84@gmail.com> wrote:
>
> > Hi Ted,
> >
> > I looked into hbase-0.92.0-cdh4b1-20120206.193413-23-sources.jar and it
> > also doesn't have it.
> >
> > On Tue, May 15, 2012 at 5:07 PM, Ted Yu <yuzhihong@gmail.com> wrote:
> >
> > > Why did you need to decompile ?
> > > Here is the source code:
> > >
> > >
> > >
> >
> https://repository.cloudera.com/artifactory/public/org/apache/hbase/hbase/0.92.0-cdh4b1-SNAPSHOT/
> > >
> > > On Tue, May 15, 2012 at 4:58 PM, anil gupta <anilgupta84@gmail.com>
> > wrote:
> > >
> > > > Hi Ted,
> > > >
> > > > I decompiled the hbase-0.92.0-cdh4b1.jar using JD-GUI and in
> > > > validateParameter method i don't find that condition.
> > > >
> > > > Thanks,
> > > > Anil
> > > >
> > > > On Tue, May 15, 2012 at 1:37 PM, Ted Yu <yuzhihong@gmail.com> wrote:
> > > >
> > > > > I checked the code in Apache HBase 0.92 and trunk. I see the
> > following
> > > > line
> > > > > in validateParameters():
> > > > >            !Bytes.equals(scan.getStopRow(),
> > > HConstants.EMPTY_END_ROW))) {
> > > > >
> > > > > Can you confirm that the bug is in cdh4b1 only ?
> > > > >
> > > > > Sorry for not doing the validation earlier.
> > > > >
> > > > > On Tue, May 15, 2012 at 12:09 PM, anil gupta <
> anilgupta84@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > Oh i c.. Now if i look closely at your gmail id then i can see
> your
> > > > > name. I
> > > > > > was totally confused.
> > > > > >
> > > > > > So, you want to force the user to specify stopRow if the filter
> is
> > > not
> > > > > > used? What if the user just wants to scan the table from startRow
> > > till
> > > > > the
> > > > > > end of table? In your solution user will have explicitly set
the
> > > > stopRow
> > > > > as
> > > > > > HConstants.EMPTY_END_ROW. Do we really want to force this?
> > > > > >
> > > > > > As per your solution the code would look like this:
> > > > > >      if(scan.hasFilter())
> > > > > >      {  if (scan == null || (Bytes.equals(scan.getStartRow(),
> > > > > > scan.getStopRow()) && !Bytes.equals(scan.getStartRow(),
> > > > > > HConstants.EMPTY_START_ROW)) ||
> > (Bytes.compareTo(scan.getStartRow(),
> > > > > > scan.getStopRow()) > 0 &&
> > > > > >             !Bytes.equals(scan.getStopRow(),
> > > HConstants.EMPTY_END_ROW)
> > > > > )) {
> > > > > >      throw new IOException(
> > > > > >          "Agg client Exception: Startrow should be smaller than
> > > > > Stoprow");
> > > > > >    } else if (scan.getFamilyMap().size() != 1) {
> > > > > >      throw new IOException("There must be only one family.");
> > > > > >    }
> > > > > >      }
> > > > > >      else
> > > > > >      {  if (scan == null || (Bytes.equals(scan.getStartRow(),
> > > > > > scan.getStopRow()) && !Bytes.equals(scan.getStartRow(),
> > > > > > HConstants.EMPTY_START_ROW)) ||
> Bytes.compareTo(scan.getStartRow(),
> > > > > > scan.getStopRow()) > 0) {
> > > > > >          throw new IOException(
> > > > > >               "Agg client Exception: Startrow should be smaller
> > than
> > > > > > Stoprow");
> > > > > >        } else if (scan.getFamilyMap().size() != 1) {
> > > > > >          throw new IOException("There must be only one family.");
> > > > > >        }
> > > > > >      }
> > > > > >
> > > > > > Let me know your thoughts.
> > > > > >
> > > > > > Thanks,
> > > > > > Anil
> > > > > >
> > > > > >
> > > > > > On Tue, May 15, 2012 at 11:46 AM, Ted Yu <yuzhihong@gmail.com>
> > > wrote:
> > > > > >
> > > > > > > Anil:
> > > > > > > I am having trouble accessing JIRA.
> > > > > > >
> > > > > > > Ted Yu and Zhihong Yu are the same person :-)
> > > > > > >
> > > > > > > I think it would be good to remind user of aggregation
client
> to
> > > > narrow
> > > > > > > range of scan. That's why I proposed adding check of
> hasFilter().
> > > > > > >
> > > > > > > Cheers
> > > > > > >
> > > > > > > On Tue, May 15, 2012 at 10:47 AM, Ted Yu <yuzhihong@gmail.com>
> > > > wrote:
> > > > > > >
> > > > > > > > Take your time.
> > > > > > > > Once you complete your first submission, subsequent
> > contributions
> > > > > would
> > > > > > > be
> > > > > > > > easier.
> > > > > > > >
> > > > > > > >
> > > > > > > > On Tue, May 15, 2012 at 10:34 AM, anil gupta <
> > > > anilgupta84@gmail.com
> > > > > > > >wrote:
> > > > > > > >
> > > > > > > >> Hi Ted,
> > > > > > > >>
> > > > > > > >> I created the jira:
> > > > > https://issues.apache.org/jira/browse/HBASE-5999for
> > > > > > > >> fixing this.
> > > > > > > >>
> > > > > > > >> Creating the patch might take me sometime(due
to learning
> > curve)
> > > > as
> > > > > > this
> > > > > > > >> is
> > > > > > > >> the first time i would be creating a patch.
> > > > > > > >>
> > > > > > > >> Thanks,
> > > > > > > >> Anil Gupta
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> On Mon, May 14, 2012 at 4:00 PM, Ted Yu <
> yuzhihong@gmail.com>
> > > > > wrote:
> > > > > > > >>
> > > > > > > >> > I was aware of the following change.
> > > > > > > >> >
> > > > > > > >> > Can you log a JIRA and attach the patch to
it ?
> > > > > > > >> >
> > > > > > > >> > Thanks for trying out and improving aggregation
client.
> > > > > > > >> >
> > > > > > > >> > On Mon, May 14, 2012 at 3:31 PM, anil gupta
<
> > > > > anilgupta84@gmail.com>
> > > > > > > >> wrote:
> > > > > > > >> >
> > > > > > > >> > > Hi Ted,
> > > > > > > >> > >
> > > > > > > >> > > If we change the if statement condition
in
> > > validateParameters
> > > > > > method
> > > > > > > >> in
> > > > > > > >> > > AggregationClient.java to:
> > > > > > > >> > > if (scan == null || (Bytes.equals(scan.getStartRow(),
> > > > > > > >> scan.getStopRow())
> > > > > > > >> > &&
> > > > > > > >> > > !Bytes.equals(scan.getStartRow(),
> > > HConstants.EMPTY_START_ROW))
> > > > > ||
> > > > > > > >> > > (Bytes.compareTo(scan.getStartRow(),
scan.getStopRow())
> >
> > 0
> > > &&
> > > > > > > >> > > *!Bytes.equals(scan.getStopRow(),
> > > > > > > >> > > HConstants.EMPTY_END_ROW)* ))
> > > > > > > >> > >
> > > > > > > >> > > Condition specified in the bold and
Italic will handle
> the
> > > > case
> > > > > > when
> > > > > > > >> the
> > > > > > > >> > > stopRow is not specified. IMHO, it's
not an error if we
> > are
> > > > not
> > > > > > > >> > specifying
> > > > > > > >> > > the stopRow. This is what is was looking
for because in
> my
> > > > case
> > > > > i
> > > > > > > >> didnt
> > > > > > > >> > > wanted to set the stop row as I am using
a prefix
> filter.
> > I
> > > > have
> > > > > > > >> tested
> > > > > > > >> > the
> > > > > > > >> > > above specified code and it works fine
when i only
> specify
> > > the
> > > > > > > >> startRow.
> > > > > > > >> > Is
> > > > > > > >> > > this a desirable functionality? If yes,
should this be
> > added
> > > > to
> > > > > > > trunk?
> > > > > > > >> > >
> > > > > > > >> > > Here is the link for source of AggregationClient:
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> http://grepcode.com/file_/repo1.maven.org/maven2/org.apache.hbase/hbase/0.92.0/org/apache/hadoop/hbase/client/coprocessor/AggregationClient.java/?v=source
> > > > > > > >> > >
> > > > > > > >> > > Thanks,
> > > > > > > >> > > Anil Gupta
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > On Mon, May 14, 2012 at 1:58 PM, Ted
Yu <
> > > yuzhihong@gmail.com>
> > > > > > wrote
> > > > > > > >> > >
> > > > > > > >> > > > Anil:
> > > > > > > >> > > > As code #3 shows, having stopRow
helps narrow the
> range
> > of
> > > > > rows
> > > > > > > >> > > > participating in aggregation.
> > > > > > > >> > > >
> > > > > > > >> > > > Do you have suggestion on how this
process can be made
> > > more
> > > > > > > >> > > user-friendly ?
> > > > > > > >> > > >
> > > > > > > >> > > > Thanks
> > > > > > > >> > > >
> > > > > > > >> > > > On Mon, May 14, 2012 at 1:47 PM,
anil gupta <
> > > > > > > anilgupta84@gmail.com>
> > > > > > > >> > > wrote:
> > > > > > > >> > > >
> > > > > > > >> > > > > HI Ted,
> > > > > > > >> > > > >
> > > > > > > >> > > > > My bad, i missed out a big
difference between the
> Scan
> > > > > object
> > > > > > i
> > > > > > > am
> > > > > > > >> > > using
> > > > > > > >> > > > in
> > > > > > > >> > > > > my filter and Scan object
used in coprocessors. So,
> > scan
> > > > > > object
> > > > > > > is
> > > > > > > >> > not
> > > > > > > >> > > > > same.
> > > > > > > >> > > > > Basically, i am doing filtering
on the basis of a
> > prefix
> > > > of
> > > > > > > >> RowKey.
> > > > > > > >> > > > >
> > > > > > > >> > > > > So, in my filter i do this
to build scanner:
> > > > > > > >> > > > > Code 1:
> > > > > > > >> > > > >  Filter filter = new
> > > > PrefixFilter(Bytes.toBytes(strPrefix));
> > > > > > > >> > > > >            Scan scan = new
Scan();
> > > > > > > >> > > > >            scan.setFilter(filter);
> > > > > > > >> > > > >
>  scan.setStartRow(Bytes.toBytes(strPrefix));
> > > //
> > > > I
> > > > > > dont
> > > > > > > >> set
> > > > > > > >> > > any
> > > > > > > >> > > > > stopRow in this scanner.
> > > > > > > >> > > > >
> > > > > > > >> > > > > In coprocessor, i do the following
for scanner:
> > > > > > > >> > > > > Code 2:
> > > > > > > >> > > > >  Scan scan = new Scan();
> > > > > > > >> > > > > scan.setFilter(new
> > PrefixFilter(Bytes.toBytes(prefix)));
> > > > > > > >> > > > >
> > > > > > > >> > > > >  I dont have startRow in above
code because if i
> only
> > > use
> > > > > only
> > > > > > > the
> > > > > > > >> > > > startRow
> > > > > > > >> > > > > in coprocessor scanner then
i get the following
> > > > > exception(due
> > > > > > to
> > > > > > > >> > this I
> > > > > > > >> > > > > removed the startRow from
CP scan object code):
> > > > > > > >> > > > > java.io.IOException: Agg client
Exception: Startrow
> > > should
> > > > > be
> > > > > > > >> smaller
> > > > > > > >> > > > than
> > > > > > > >> > > > > Stoprow
> > > > > > > >> > > > >    at
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hbase.client.coprocessor.AggregationClient.validateParameters(AggregationClient.java:116)
> > > > > > > >> > > > >    at
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hbase.client.coprocessor.AggregationClient.max(AggregationClient.java:85)
> > > > > > > >> > > > >    at
> > > > > > > >> > > > >
> > > > > > > >> >
> > > > > >
> > > com.intuit.ihub.hbase.poc.DummyClass.doAggregation(DummyClass.java:81)
> > > > > > > >> > > > >    at
> > > sun.reflect.NativeMethodAccessorImpl.invoke0(Native
> > > > > > > Method)
> > > > > > > >> > > > >    at
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > I modified the above code#2
to add the stopRow also:
> > > > > > > >> > > > > Code 3:
> > > > > > > >> > > > > Scan scan = new Scan();
> > > > > > > >> > > > >        scan.setStartRow(Bytes.toBytes(prefix));
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > >
> > > > > > > >>
> > > > > > >
> > > > >
> > >
> scan.setStopRow(Bytes.toBytes(String.valueOf(Long.parseLong(prefix)+1)));
> > > > > > > >> > > > >        scan.setFilter(new
> > > > > > PrefixFilter(Bytes.toBytes(prefix)));
> > > > > > > >> > > > >
> > > > > > > >> > > > > When, i run the coprocessor
with Code #3, its
> blazing
> > > > fast.
> > > > > I
> > > > > > > >> gives
> > > > > > > >> > the
> > > > > > > >> > > > > result in around 200 millisecond.
:)
> > > > > > > >> > > > > Since, this was just testing
a coprocessors i added
> > the
> > > > > logic
> > > > > > to
> > > > > > > >> add
> > > > > > > >> > > the
> > > > > > > >> > > > > stopRow manually. What is
the reason that Scan
> object
> > in
> > > > > > > >> coprocessor
> > > > > > > >> > > > always
> > > > > > > >> > > > > requires stopRow along with
startRow?(code #1 works
> > fine
> > > > > even
> > > > > > > >> when i
> > > > > > > >> > > dont
> > > > > > > >> > > > > use stopRow)  Can this restriction
be relaxed?
> > > > > > > >> > > > >
> > > > > > > >> > > > > Thanks,
> > > > > > > >> > > > > Anil Gupta
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > On Mon, May 14, 2012 at 12:55
PM, Ted Yu <
> > > > > yuzhihong@gmail.com
> > > > > > >
> > > > > > > >> > wrote:
> > > > > > > >> > > > >
> > > > > > > >> > > > > > Anil:
> > > > > > > >> > > > > > I think the performance
was related to your custom
> > > > filter.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Please tell us more about
the filter next time.
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > Thanks
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > On Mon, May 14, 2012
at 12:31 PM, anil gupta <
> > > > > > > >> > anilgupta84@gmail.com>
> > > > > > > >> > > > > > wrote:
> > > > > > > >> > > > > >
> > > > > > > >> > > > > > > HI Stack,
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > I'll look into Gary
Helming post and try to do
> > > > profiling
> > > > > > of
> > > > > > > >> > > > coprocessor
> > > > > > > >> > > > > > and
> > > > > > > >> > > > > > > share the results.
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > Thanks,
> > > > > > > >> > > > > > > Anil Gupta
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > On Mon, May 14,
2012 at 12:08 PM, Stack <
> > > > > stack@duboce.net
> > > > > > >
> > > > > > > >> > wrote:
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > > On Mon, May
14, 2012 at 12:02 PM, anil gupta <
> > > > > > > >> > > > anilgupta84@gmail.com>
> > > > > > > >> > > > > > > > wrote:
> > > > > > > >> > > > > > > > > I loaded
around 70 thousand 1-2KB records in
> > > > HBase.
> > > > > > For
> > > > > > > >> > scans,
> > > > > > > >> > > > with
> > > > > > > >> > > > > > my
> > > > > > > >> > > > > > > > > custom
filter i am able to get 97 rows in
> 500
> > > > > > > milliseconds
> > > > > > > >> > and
> > > > > > > >> > > > for
> > > > > > > >> > > > > > > doing
> > > > > > > >> > > > > > > > > sum, max,
min(in built aggregations of
> HBase)
> > on
> > > > the
> > > > > > > same
> > > > > > > >> > > custom
> > > > > > > >> > > > > > filter
> > > > > > > >> > > > > > > > its
> > > > > > > >> > > > > > > > > taking
11000 milliseconds. Does this mean
> that
> > > > > > > >> coprocessors
> > > > > > > >> > > > > > aggregation
> > > > > > > >> > > > > > > > is
> > > > > > > >> > > > > > > > > supposed
to be around ~20x slower than
> scans?
> > > Am i
> > > > > > > missing
> > > > > > > >> > any
> > > > > > > >> > > > > trick
> > > > > > > >> > > > > > > over
> > > > > > > >> > > > > > > > > here?
> > > > > > > >> > > > > > > > >
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > > > That seems
like a high tax to pay for running
> > CPs.
> > > > >  Can
> > > > > > > you
> > > > > > > >> dig
> > > > > > > >> > > in
> > > > > > > >> > > > on
> > > > > > > >> > > > > > > > where the time
is being spent?  (See another
> > > recent
> > > > > note
> > > > > > > on
> > > > > > > >> > this
> > > > > > > >> > > > list
> > > > > > > >> > > > > > > > or on dev where
Gary Helmling talks about how
> he
> > > did
> > > > > > basic
> > > > > > > >> > > > profiling
> > > > > > > >> > > > > > > > of CPs).
> > > > > > > >> > > > > > > > St.Ack
> > > > > > > >> > > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > > > --
> > > > > > > >> > > > > > > Thanks & Regards,
> > > > > > > >> > > > > > > Anil Gupta
> > > > > > > >> > > > > > >
> > > > > > > >> > > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > >
> > > > > > > >> > > > > --
> > > > > > > >> > > > > Thanks & Regards,
> > > > > > > >> > > > > Anil Gupta
> > > > > > > >> > > > >
> > > > > > > >> > > >
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > >
> > > > > > > >> > > --
> > > > > > > >> > > Thanks & Regards,
> > > > > > > >> > > Anil Gupta
> > > > > > > >> > >
> > > > > > > >> >
> > > > > > > >>
> > > > > > > >>
> > > > > > > >>
> > > > > > > >> --
> > > > > > > >> Thanks & Regards,
> > > > > > > >> Anil Gupta
> > > > > > > >>
> > > > > > > >
> > > > > > > >
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > Thanks & Regards,
> > > > > > Anil Gupta
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > Thanks & Regards,
> > > > Anil Gupta
> > > >
> > >
> >
> >
> >
> > --
> > Thanks & Regards,
> > Anil Gupta
> >
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message