Also, if you do end up using dynamic loading, you'll need a way to version your filters because the RS will not reload a JAR if it changes.
On Tue, Oct 21, 2014 at 9:46 PM, Kevin <kevin.macksa...@gmail.com> wrote: > I haven't tried dynamic loading of filters on RS, but I know it does > exist. See https://issues.apache.org/jira/browse/HBASE-9301. > > If you still can't get it to work, then I suggest distributing your > filters to the RS and restart them. Let us know how everything works out. > > On Tue, Oct 21, 2014 at 9:02 PM, Matt K <matvey1...@gmail.com> wrote: > >> Thanks Kevin! >> >> I was under impression, probably mistakingly, that as of 0.96 placing >> the filter on hdfs under hbase lib directory is sufficient and RS should >> load the filter dynamically from hdfs. Is that not the case? >> >> On Tuesday, October 21, 2014, Kevin <kevin.macksa...@gmail.com> wrote: >> >> > BTW, the error looks like you didn't distribute your custom filter to >> your >> > region servers. >> > >> > On Tue, Oct 21, 2014 at 1:34 PM, Kevin <kevin.macksa...@gmail.com >> > <javascript:;>> wrote: >> > >> > > Matt, >> > > >> > > You should create your own proto file and compile that with the Google >> > > Protocol Buffer compiler. Take a look at the SingleColumnValueFilter's >> > > code: >> > > >> > >> https://github.com/apache/hbase/blob/master/hbase-client/src/main/java/org/apache/hadoop/hbase/filter/SingleColumnValueFilter.java#L327 >> > > >> > > You will need to override `public byte[] toByteArray()` and `public >> > static >> > > Filter parseFrom(byte[] pbBytes)`. The output of toByteArray() should >> be >> > > the byte[] from serializing with the protocol buffer. This will also >> be >> > the >> > > input to parseFrom(byte[]), which is called using reflection on the >> > > server-side to instantiate your custom filter and use it. >> > > >> > > On Sun, Oct 19, 2014 at 11:31 AM, Matt K <matvey1...@gmail.com >> > <javascript:;>> wrote: >> > > >> > >> Anyone? >> > >> >> > >> On Thursday, October 16, 2014, Matt K <matvey1...@gmail.com >> > <javascript:;>> wrote: >> > >> >> > >> > Hi, can anyone help with above? Feels like I'm missing something >> > >> obvious. >> > >> > >> > >> > On Wednesday, October 15, 2014, Nishanth S < >> nishanth.2...@gmail.com >> > <javascript:;> >> > >> > <javascript:_e(%7B%7D,'cvml','nishanth.2...@gmail.com >> <javascript:;>');>> >> > wrote: >> > >> > >> > >> >> Thanks Ted .I will take a look. >> > >> >> >> > >> >> -Nishanth >> > >> >> >> > >> >> On Wed, Oct 15, 2014 at 3:43 PM, Ted Yu <yuzhih...@gmail.com >> > <javascript:;>> wrote: >> > >> >> >> > >> >> > Nishanth: >> > >> >> > Good question. >> > >> >> > >> > >> >> > As a general coding guide, writing unit test is always a good >> > start. >> > >> >> Using >> > >> >> > Matt's case as an example, take a look at TestPrefixFilter. >> > >> >> > >> > >> >> > There're various unit tests for Filters in hbase code. >> > >> >> > >> > >> >> > Cheers >> > >> >> > >> > >> >> > On Wed, Oct 15, 2014 at 2:30 PM, Nishanth S < >> > nishanth.2...@gmail.com <javascript:;> >> > >> > >> > >> >> > wrote: >> > >> >> > >> > >> >> > > Hi Ted , >> > >> >> > > Since I am also working on similar thing is there a way we >> can >> > >> first >> > >> >> > test >> > >> >> > > the filter on client side?.You know what I mean without >> > disrupting >> > >> >> > others >> > >> >> > > who are using the same cluster for other work? >> > >> >> > > >> > >> >> > > Thanks, >> > >> >> > > Nishanth >> > >> >> > > >> > >> >> > > On Wed, Oct 15, 2014 at 3:17 PM, Ted Yu <yuzhih...@gmail.com >> > <javascript:;>> >> > >> wrote: >> > >> >> > > >> > >> >> > > > bq. Or create a new file, compile it into ... >> > >> >> > > > >> > >> >> > > > You should go with the above approach. >> > >> >> > > > >> > >> >> > > > On Wed, Oct 15, 2014 at 2:08 PM, Matt K < >> matvey1...@gmail.com >> > <javascript:;>> >> > >> >> wrote: >> > >> >> > > > >> > >> >> > > > > Hi all, >> > >> >> > > > > >> > >> >> > > > > I'm trying to get a custom filter to work on HBase 0.96. >> > After >> > >> >> some >> > >> >> > > > > searching, I found that starting from 0.96, the >> implementer >> > is >> > >> >> > required >> > >> >> > > > to >> > >> >> > > > > implement "toByteArray" and "parseFrom" methods, using >> > Protocol >> > >> >> > > Buffers. >> > >> >> > > > > But I'm having trouble with the "how". >> > >> >> > > > > >> > >> >> > > > > The "proto" file for the existing filters is located here: >> > >> >> > > > > >> > >> >> > > > > >> > >> >> > > > >> > >> >> > > >> > >> >> > >> > >> >> >> > >> >> > >> https://github.com/apache/hbase/blob/master/hbase-protocol/src/main/protobuf/Filter.proto >> > >> >> > > > > >> > >> >> > > > > Am I supposed to modify that file? Or create a new file, >> > >> compile >> > >> >> it >> > >> >> > > into >> > >> >> > > > > Java, and package it up with the filter? >> > >> >> > > > > >> > >> >> > > > > In the meantime, I've taken a shortcut that's not working. >> > >> Here's >> > >> >> my >> > >> >> > > > code: >> > >> >> > > > > http://pastebin.com/iHFKu9Xz >> > >> >> > > > > >> > >> >> > > > > I'm using "PrefixFilter", which comes with HBase, since >> I'm >> > >> also >> > >> >> > > > filtering >> > >> >> > > > > by "prefix". However, that errors out with the following: >> > >> >> > > > > http://pastebin.com/zBg47p6Z >> > >> >> > > > > >> > >> >> > > > > Thanks in advance for helping! >> > >> >> > > > > >> > >> >> > > > > -Matt >> > >> >> > > > > >> > >> >> > > > >> > >> >> > > >> > >> >> > >> > >> >> >> > >> > >> > >> > >> > >> > -- >> > >> > www.calcmachine.com - easy online calculator. >> > >> > >> > >> >> > >> >> > >> -- >> > >> www.calcmachine.com - easy online calculator. >> > >> >> > > >> > > >> > >> >> >> -- >> www.calcmachine.com - easy online calculator. >> > >