Return-Path: X-Original-To: apmail-hbase-user-archive@www.apache.org Delivered-To: apmail-hbase-user-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7451517720 for ; Wed, 22 Oct 2014 01:53:34 +0000 (UTC) Received: (qmail 45172 invoked by uid 500); 22 Oct 2014 01:53:32 -0000 Delivered-To: apmail-hbase-user-archive@hbase.apache.org Received: (qmail 45095 invoked by uid 500); 22 Oct 2014 01:53:32 -0000 Mailing-List: contact user-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@hbase.apache.org Delivered-To: mailing list user@hbase.apache.org Received: (qmail 45081 invoked by uid 99); 22 Oct 2014 01:53:31 -0000 Received: from mx1-us-east.apache.org (HELO mx1-us-east.apache.org) (54.164.171.186) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 22 Oct 2014 01:53:31 +0000 Received: from mx1-us-east.apache.org (localhost [127.0.0.1]) by mx1-us-east.apache.org (ASF Mail Server at mx1-us-east.apache.org) with ESMTP id D92BC43C4F for ; Wed, 22 Oct 2014 01:53:51 +0000 (UTC) Received: by mx1-us-east.apache.org (ASF Mail Server at mx1-us-east.apache.org, from userid 111) id CE42F43C6B; Wed, 22 Oct 2014 01:53:51 +0000 (UTC) X-Spam-Checker-Version: SpamAssassin 3.4.0 (2014-02-07) on mx1-us-east.apache.org X-Spam-Level: * X-Spam-Status: No, score=1.3 required=10.0 tests=HTML_MESSAGE, RCVD_IN_DNSWL_LOW,RCVD_IN_MSPIKE_H2,SPF_PASS,T_DKIM_INVALID,URIBL_BLOCKED autolearn=disabled version=3.4.0 Received: from mail-yh0-f48.google.com (mail-yh0-f48.google.com [209.85.213.48]) by mx1-us-east.apache.org (ASF Mail Server at mx1-us-east.apache.org) with ESMTPS id 5227843C4F for ; Wed, 22 Oct 2014 01:53:51 +0000 (UTC) Received: by mail-yh0-f48.google.com with SMTP id v1so2570416yhn.7 for ; Tue, 21 Oct 2014 18:53:24 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=20120113; h=mime-version:in-reply-to:references:date:message-id:subject:from:to :content-type; bh=SI0BrWZMADF61sh7VxKNM1u9Ul7pXtpglk93fNuAN90=; b=hBl3Wi8lGMitMrVR1sSCeZ4b0d8yQdFhfhukQXmj3oqrbz7XAMq+cwCAi0TZbHn7gk jBmYZMT6LzqsUJc6HPD/udDMZYfe0pt8czmf2uCEgANGAyF0pyBu4A7qwO15BBHk446O E5uNikXy4mT3SF9lma9KZwhyXndK+zueqrfWStDzplGe6pc1jiEed0VbMUOif5sLDz1Z rO008o1SdftGllf4zZxtj8zKILcbTbyCdNYn4BtS8vBTBzSuVUCLkhYLChzEM5q3jS3t t/UiYv/zau4wkDN15Z3yhy1u4UGbRbIdMefgKmdZmhPqM5x3zGnOja6+6Tpld3X4ZOgt ewaA== MIME-Version: 1.0 X-Received: by 10.236.1.197 with SMTP id 45mr54343714yhd.88.1413942804246; Tue, 21 Oct 2014 18:53:24 -0700 (PDT) Received: by 10.170.180.7 with HTTP; Tue, 21 Oct 2014 18:53:24 -0700 (PDT) In-Reply-To: References: Date: Tue, 21 Oct 2014 18:53:24 -0700 Message-ID: Subject: Re: custom filter on hbase 0.96 From: Ted Yu To: "user@hbase.apache.org" Content-Type: multipart/alternative; boundary=089e013a15d09478d60505f93680 X-Virus-Scanned: ClamAV using ClamSMTP --089e013a15d09478d60505f93680 Content-Type: text/plain; charset=UTF-8 See this blog post: http://www.flurry.com/2012/12/06/exploring-dynamic-loading-of-custom-filters-i#.VEcNtNR4rZg Cheers On Tue, Oct 21, 2014 at 6:48 PM, Kevin wrote: > 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 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 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 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 >> > > 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 >> > > wrote: > >> > > > >> > >> Anyone? > >> > >> > >> > >> On Thursday, October 16, 2014, Matt K >> > > wrote: > >> > >> > >> > >> > Hi, can anyone help with above? Feels like I'm missing something > >> > >> obvious. > >> > >> > > >> > >> > On Wednesday, October 15, 2014, Nishanth S < > >> nishanth.2884@gmail.com > >> > > >> > >> > >> ');>> > >> > wrote: > >> > >> > > >> > >> >> Thanks Ted .I will take a look. > >> > >> >> > >> > >> >> -Nishanth > >> > >> >> > >> > >> >> On Wed, Oct 15, 2014 at 3:43 PM, Ted Yu >> > > 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.2884@gmail.com > >> > >> > > >> > >> >> > 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 < > yuzhihong@gmail.com > >> > > > >> > >> 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 < > >> matvey1414@gmail.com > >> > > > >> > >> >> 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. > >> > > > > > --089e013a15d09478d60505f93680--