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 <[email protected]> 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 <[email protected]> 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 <[email protected]> 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 <[email protected]> > > > 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 <[email protected]> wrote: > > > > > > > > > On Mon, May 14, 2012 at 12:02 PM, anil gupta < > [email protected]> > > > > > 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
