Thanks guys! We'll certainly need InclusiveStopFilter once we support range scans with closed endpoints.
JVS From: Jonathan Gray <jg...@facebook.com<mailto:jg...@facebook.com>> Date: September 13, 2010 10:34:23 AM PDT To: "user@hbase.apache.org<mailto:user@hbase.apache.org>" <user@hbase.apache.org<mailto:user@hbase.apache.org>> Subject: RE: question on Scan.setStopRow (a) Looks right to me. (b) Your expected behavior is to return just the single startRow, correct? Then all you need are start/stop rows, no other filtering necessary. JG -----Original Message----- From: John Sichi [mailto:jsi...@facebook.com] Sent: Monday, September 13, 2010 10:07 AM To: user@hbase.apache.org<mailto:user@hbase.apache.org> Subject: question on Scan.setStopRow Hey hbasers, I'm working on pushing predicates down from Hive, and starting with a very basic case (equality on a simple key). This ends up looking like a get, but for now it still uses TableInputFormatBase since later we'll be extending it to range scans as well. After converting the key to byte [] startRow, I do the following, since stopRow is exclusive rather than inclusive: byte [] stopRow = new byte[startRow.length + 1]; System.arraycopy(startRow, 0, stopRow, 0, startRow.length); scan.setStartRow(startRow); scan.setStopRow(stopRow); I just wanted to confirm (a) Is this the correct way to create a stopRow for a point lookup (appending a single zero byte to make the successor lexical value)? (b) Do I also need a WhileMatchFilter, or is setStartRow+setStopRow guaranteed to do all the filtering directly? Thanks, JVS From: Stack <st...@duboce.net<mailto:st...@duboce.net>> Date: September 13, 2010 10:37:22 AM PDT To: <user@hbase.apache.org<mailto:user@hbase.apache.org>> Subject: Re: question on Scan.setStopRow On Mon, Sep 13, 2010 at 10:07 AM, John Sichi <jsi...@facebook.com<mailto:jsi...@facebook.com>> wrote: Hey hbasers, I'm working on pushing predicates down from Hive, and starting with a very basic case (equality on a simple key). This ends up looking like a get, but for now it still uses TableInputFormatBase since later we'll be extending it to range scans as well. After converting the key to byte [] startRow, I do the following, since stopRow is exclusive rather than inclusive: byte [] stopRow = new byte[startRow.length + 1]; System.arraycopy(startRow, 0, stopRow, 0, startRow.length); scan.setStartRow(startRow); scan.setStopRow(stopRow); I just wanted to confirm (a) Is this the correct way to create a stopRow for a point lookup (appending a single zero byte to make the successor lexical value)? That looks like it should work. You've seen InclusiveStopFilter? (b) Do I also need a WhileMatchFilter, or is setStartRow+setStopRow guaranteed to do all the filtering directly? You don't need to add filters if you are using setStartRow, setStopRow. Internally they do the business themselves [1]. Let us know if it gives you trouble John, St.Ack 1. The 'business' is performed in the HRegion scanner. Here is where it is created http://hbase.apache.org/docs/r0.89.20100726/xref/org/apache/hadoop/hbase/regionserver/HRegion.html#2299 Here in the internal 'next' on the region scanner is where it acts on stop row: http://hbase.apache.org/docs/r0.89.20100726/xref/org/apache/hadoop/hbase/regionserver/HRegion.html#2394