Hi Lars, Well the details are as follows:
table1 has the rowkey as some url, and 2 ColumnFamilies as described below: one columnFamily called content and one columnFamily called status [which takes the values ANALYSED, UNANALYSED] (all in upper case... i checked it, there is no issue with the spelling/case). Hope this helps, Thanks. Rakhi On Wed, Apr 8, 2009 at 1:59 PM, Lars George <[email protected]> wrote: > Hi Rakhi, > > Wow, same here. I copied your RowFilter line and when I press the dot key > and the fly up opens Eclipse hangs. Nice... NOT! > > Apart from that, you are also saying that the filter is not working as > expected? Do you use any column qualifiers for the "Status:" column? Are the > values in the correct casing, i.e. are the values stored in uppercase as you > have it in your example below? I assume the comparison is byte sensitive. > Please give us more details, maybe a small sample table dump so that we can > test this? > > Lars > > Rakhi Khatwani wrote: > >> Hi, >> I did try the filter... but using ColumnValueFilter. i declared >> a >> ColumnValueFilter as follows: >> >> public class TableInputFilter extends TableInputFormat >> implements JobConfigurable { >> >> public void configure(final JobConf jobConf) { >> >> setHtable(tablename); >> >> setInputColumns(columnName); >> >> >> final RowFilterInterface colFilter = >> new >> ColumnValueFilter("Status:".getBytes(), ColumnValueFilter.CompareOp.EQUAL, >> "UNCOLLECTED".getBytes()); >> setRowFilter(colFilter); >> } >> >> } >> >> and thn i use my class as the input format to my map function. >> >> >> in my map function, i set my log to display the value of my Status Column >> family. >> >> when i execute my map reduce function, it displays "Status:: Uncollected" >> for some rows >> and Status = "Collected" for rest of the rows. >> >> but what i want is to send only those records whose 'Status: is >> uncollected'. >> >> i even considered using the method filterRow described by the API as >> follows: >> boolean *filterRow< >> http://hadoop.apache.org/hbase/docs/current/api/org/apache/hadoop/hbase/filter/ColumnValueFilter.html#filterRow%28java.util.SortedMap%29 >> > >> *(SortedMap< >> http://java.sun.com/javase/6/docs/api/java/util/SortedMap.html?is-external=true >> > >> <byte[],Cell< >> http://hadoop.apache.org/hbase/docs/current/api/org/apache/hadoop/hbase/io/Cell.html >> > >> >> >>> columns) >>> >>> >> Filter on the fully assembled row. >> >> but as soon as i type colFilter followed by a '.', my eclipse hangs. >> its really weird... i have tried it on 3 different machines (2 machines on >> linux running eclipse gannymade 3.4 and one on windows using myEclipse). >> >> >> i dunno if i am going wrong somewhere >> >> Thanks, >> Raakhi >> >> >> On Tue, Apr 7, 2009 at 7:18 PM, Lars George <[email protected]> wrote: >> >> >> >>> Hi Rakhi, >>> >>> The way the filters work is that you either use the supplied filters or >>> create your own subclasses - but then you will have to deploy that class >>> to >>> all RegionServers while adding it to their respective hbase-env.sh (in >>> the >>> "export HBASE_CLASSPATH" variable). We are discussing currently if this >>> could be done dynamically ( >>> https://issues.apache.org/jira/browse/HBASE-1288). >>> >>> Once you have that done or use one of the supplied one then you can >>> assign >>> the filter by overriding the TableInputFormat's configure() method and >>> assign it like so: >>> >>> public void configure(JobConf job) { >>> RegExpRowFilter filter = new RegExpRowFilter("ABC.*"); >>> setRowFilter(filter); >>> } >>> >>> As Tim points out, setting the whole thing up is done in your main M/R >>> tool >>> based application, similar to: >>> >>> JobConf job = new JobConf(...); >>> TableMapReduceUtil.initTableMapJob("<table-name>", "<colums>", >>> IdentityTableMap.class, >>> ImmutableBytesWritable.class, RowResult.class, job); >>> job.setReducerClass(MyTableReduce.class); >>> job.setInputFormat(MyTableInputFormat.class); >>> job.setOutputFormat(MyTableOutputFormat.class); >>> >>> Of course depending on what classes you want to replace or if this is a >>> Reduce oriented job (means a default identity + filter map and all the >>> work >>> done in the Reduce phase) or the other way around. But the principles and >>> filtering are the same. >>> >>> HTH, >>> Lars >>> >>> >>> >>> Rakhi Khatwani wrote: >>> >>> >>> >>>> Thanks Ryan, i will try that >>>> >>>> On Tue, Apr 7, 2009 at 3:05 PM, Ryan Rawson <[email protected]> wrote: >>>> >>>> >>>> >>>> >>>> >>>>> there is a server-side mechanism to filter rows, it's found in the >>>>> org.apache.hadoop.hbase.filter package. im not sure how this interops >>>>> with >>>>> the TableInputFormat exactly. >>>>> >>>>> setting a filter to reduce the # of rows returned is pretty much >>>>> exactly >>>>> what you want. >>>>> >>>>> On Tue, Apr 7, 2009 at 2:26 AM, Rakhi Khatwani < >>>>> [email protected] >>>>> >>>>> >>>>> >>>>> >>>>>> wrote: >>>>>> Hi, >>>>>> i have a map reduce program with which i read from a hbase table. >>>>>> In my map program i check if the column value of a is xxx, if yes then >>>>>> continue with processing else skip it. >>>>>> however if my table is really big, most of my time in the map gets >>>>>> wasted >>>>>> for processing unwanted rows. >>>>>> is there any way through which we could send a subset of rows (based >>>>>> on >>>>>> >>>>>> >>>>>> >>>>>> >>>>> the >>>>> >>>>> >>>>> >>>>> >>>>>> value of a particular column family) to the map??? >>>>>> >>>>>> i have also gone through TableInputFormatBase but am not able to >>>>>> figure >>>>>> >>>>>> >>>>>> >>>>>> >>>>> out >>>>> >>>>> >>>>> >>>>> >>>>>> how do we set the input format if we are using TableMapReduceUtil >>>>>> class >>>>>> >>>>>> >>>>>> >>>>>> >>>>> to >>>>> >>>>> >>>>> >>>>> >>>>>> initialize table map jobs. or is there any other way i could use it. >>>>>> >>>>>> Thanks in Advance, >>>>>> Raakhi. >>>>>> >>>>>> >>>>>> >>>>>> >>>>>> >>>>> >>>> >>> >> >> >
