hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Vikram Singh Chandel <vikramsinghchan...@gmail.com>
Subject Re: Why Custom Filters are more resource exhaustive then normal single column value filter
Date Wed, 18 Jun 2014 11:01:24 GMT
Hi Anoop here is the code for Custom Filter implementation

public class MyCustomFilter extends FilterBase {

/**
 * @param args
 * @author vikram singh chandel
 */

private byte[] value = null;
private boolean filterRow = true;
public MyCustomFilter() {
super();
}

public MyCustomFilter(byte[] value) {
this.value = value;
}

@Override
public ReturnCode filterKeyValue(KeyValue kv) {
if (Bytes.compareTo(value, kv.getValue()) == 0) {
filterRow = false;
}
return ReturnCode.INCLUDE;
}

@Override
public void reset() {
this.filterRow = true;
}

@Override
public boolean filterRow() {
return filterRow;
}

public void readFields(DataInput dataInput) throws IOException {
this.value = Bytes.readByteArray(dataInput);
}

public void write(DataOutput dataOutput) {
try {

Bytes.writeByteArray(dataOutput, this.value);

} catch (IOException e) {

e.printStackTrace();
}
}

}

and here is the *client code*

               ResultScanner scanner=null;
rowKeys=new ArrayList<String>();
List<Filter> filters = new ArrayList<Filter>();
System.out.println("1");
Filter filter1 = new
MyCustomFilter(Bytes.toBytes("dd9ab79b5973cd3fee04ae195a1e5940"));
filters.add(filter1);
FilterList filterList = new FilterList(
FilterList.Operator.MUST_PASS_ONE, filters);
Scan scan = new Scan();
scan.addFamily(Bytes.toBytes("Attributes"));
scan.setFilter(filterList);
System.out.println("2");
try {
 scanner = table.getScanner(scan);
 for (Result result : scanner) {
 rowKeys.add(Bytes.toString(result.getRow()));
 //System.out.println(Bytes.toString(result.getRow()));
 }


*Yes, problem number 1,2,4 and 5 i faced during cluster testing*




On Wed, Jun 18, 2014 at 3:10 PM, Anoop John <anoop.hbase@gmail.com> wrote:

> A normal scan is not executed parallely on regions. It is serial way.   And
> a Filter is a way to filter out some data during the scan.   So Filter
> works along with scan. There is nothing like Filter execute parellel or so.
>
> What was working paralle is the Endpoints.  So if u impl one and call a
> method of EP, it will execute parallel.
>
> So if a custom filter seems to be more resource exhaustive, that means the
> code in the filter is having some issue.   Can u tell u what u wrote in ur
> custom filter?  Mind attaching the code.
>
> The problems u mention with CP
> Are these u faced in ur real cluster testing?
>
> -Anoop-
>
> On Wed, Jun 18, 2014 at 11:46 AM, Vikram Singh Chandel <
> vikramsinghchandel@gmail.com> wrote:
>
> > Hi Ted
> >
> > There is no problem in execution of Custom filters the issue is why are
> > they more resource exhaustive.
> > So i have two questions
> >
> > 1. Are custom filter work in similar fashion as CoPro works? i.e working
> > parallel on all RS.
> > If yes why are they more resource exhaustive
> > 2. What exactly we mean when we say CoPro works parallel on all RS ?
> isn't
> > a normal scan does the same, as data is distributed on RS
> > so each RS is performing a scan at its level and then returning a
> > aggregated result
> >
> >
> > On Tue, Jun 17, 2014 at 7:59 PM, Ted Yu <yuzhihong@gmail.com> wrote:
> >
> > > Vikram:
> > > Discussion on coprocessor may get very long.
> > >
> > > Let's focus on making your custom filter work.
> > >
> > > Do you have unit test for the custom filter ?
> > > If you limit the scan (with this filter) to one region, you should be
> > able
> > > to obtain jstack which would give us more clue.
> > >
> > > Cheers
> > >
> > >
> > > On Tue, Jun 17, 2014 at 6:55 AM, Vikram Singh Chandel <
> > > vikramsinghchandel@gmail.com> wrote:
> > >
> > > > Hi Ramkrishna
> > > >
> > > > issues with CoPro there are many
> > > > 1: Garbage Collection
> > > > 2: JVM Freeze
> > > > 3: Removal of CoPro from RS unless you do a rolling restart
> > > > 4: Class unloading
> > > > 5:Class Collision Issues
> > > > 6:Shared JVM between CoPro and RS
> > > >
> > > > there may be some more that i am not aware of :-)
> > > >
> > > >
> > > > On Tue, Jun 17, 2014 at 5:38 PM, ramkrishna vasudevan <
> > > > ramkrishna.s.vasudevan@gmail.com> wrote:
> > > >
> > > > > >> If they don't
> > > > > process records parallel on RS why not use HDFS to distribute the
> > jar?
> > > > > See https://issues.apache.org/jira/browse/HBASE-1936 for dynamic
> > class
> > > > > loading of jars.
> > > > >
> > > > > In the first mail you had mentioned that there are various problems
> > in
> > > > CP?
> > > > > What are those problems?  Filters are mainly used to work on the
> > > > Rows/cells
> > > > > that are retrieved in every region.  Whereas CPs are used to add
> some
> > > > user
> > > > > extensions to the way Servers works in write path or read path.
> > > > >
> > > > > Not sure if I understand your problem correctly.
> > > > >
> > > > >
> > > > > On Tue, Jun 17, 2014 at 5:20 PM, Vikram Singh Chandel <
> > > > > vikramsinghchandel@gmail.com> wrote:
> > > > >
> > > > > > Hi
> > > > > > ya Anoop the comparison was between SCVF and Custom Filter.
> Seeing
> > > the
> > > > > > deployment process of Custom Filter, i.e building a jar deploying
> > it
> > > on
> > > > > to
> > > > > > all RS
> > > > > > *i thought that the work in similar fashion as that of CP*.
If
> they
> > > > don't
> > > > > > process records parallel on RS why not use HDFS to distribute
the
> > > jar?
> > > > > >
> > > > > >
> > > > > > On Mon, Jun 16, 2014 at 6:52 PM, Ted Yu <yuzhihong@gmail.com>
> > wrote:
> > > > > >
> > > > > > > Can you format the table for better reading ?
> > > > > > > You can put picture on third party website and post link
here.
> > > > > > >
> > > > > > > Does your custom filter maintain state ?
> > > > > > >
> > > > > > > What hbase version are you using ?
> > > > > > >
> > > > > > > Have you checked region server logs during the 15 minutes
?
> > > > > > >
> > > > > > > Cheers
> > > > > > >
> > > > > > > On Jun 16, 2014, at 3:25 AM, Vikram Singh Chandel <
> > > > > > > vikramsinghchandel@gmail.com> wrote:
> > > > > > >
> > > > > > > > Hi
> > > > > > > > I was working on way to find a substitute for
> > coprocessor(because
> > > > of
> > > > > > > > various issue with them)
> > > > > > > > so tried Custom Filters and found that the custom
filter are
> > much
> > > > > more
> > > > > > > > resource exhaustive (CPU) then coprocessor having
single
> column
> > > > value
> > > > > > > > filter.
> > > > > > > >
> > > > > > > > *Query: Get List of Publications for UDANDA (CF:Attributes
> > > > > > Col:Country) -
> > > > > > > > Result: 509 Records. *
> > > > > > > >
> > > > > > > > *Heap 3 Gb *
> > > > > > > >
> > > > > > > >
> > > > > > > > With Custom Filter
> > > > > > > > No of Scan Req
> > > > > > > > Max Heap(Mb)
> > > > > > > > Max CPU(%)
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > 1
> > > > > > > > 86.74
> > > > > > > > 7.7
> > > > > > > > 5
> > > > > > > > 412
> > > > > > > > 63.37
> > > > > > > > 10
> > > > > > > > 926
> > > > > > > > 99
> > > > > > > > *With Single Column Value Filter*
> > > > > > > > No of Scan Req
> > > > > > > > Max Heap
> > > > > > > > Max CPU
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > 10
> > > > > > > > 453.66
> > > > > > > > 17.7
> > > > > > > > Query ran for 15 minutes didn't get results. Terminated
> > execution
> > > > > > > >
> > > > > > > >
> > > > > > > >
> > > > > > > > *Can anyone shed some light on this.*
> > > > > > > > --
> > > > > > > >
> > > > > > > > *Regards*
> > > > > > > >
> > > > > > > > *VIKRAM SINGH CHANDEL*
> > > > > > > >
> > > > > > > > Please do not print this email unless it is absolutely
> > > > > > necessary,Reduce.
> > > > > > > > Reuse. Recycle. Save our planet.
> > > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > --
> > > > > > *Regards*
> > > > > >
> > > > > > *VIKRAM SINGH CHANDEL*
> > > > > >
> > > > > > Please do not print this email unless it is absolutely
> > > > necessary,Reduce.
> > > > > > Reuse. Recycle. Save our planet.
> > > > > >
> > > > >
> > > >
> > > >
> > > >
> > > > --
> > > > *Regards*
> > > >
> > > > *VIKRAM SINGH CHANDEL*
> > > >
> > > > Please do not print this email unless it is absolutely
> > necessary,Reduce.
> > > > Reuse. Recycle. Save our planet.
> > > >
> > >
> >
> >
> >
> > --
> > *Regards*
> >
> > *VIKRAM SINGH CHANDEL*
> >
> > Please do not print this email unless it is absolutely necessary,Reduce.
> > Reuse. Recycle. Save our planet.
> >
>



-- 
*Regards*

*VIKRAM SINGH CHANDEL*

Please do not print this email unless it is absolutely necessary,Reduce.
Reuse. Recycle. Save our planet.

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