hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jean-Marc Spaggiari <jean-m...@spaggiari.org>
Subject Re: scan filter: how to ignore everything with a prefix except one particular column with that prefix
Date Thu, 05 Jun 2014 23:52:37 GMT
He want to excluse everything starting with "c!" and keep c!someName.

So. First filter is a NOT, second is a include.

If we test c!notThis first will give false, second too. We rejest.
If we test d!this first will give true, second false. We take it.
If we test c!someName first will give false, second will give true. We take
it.

Do I miss something? It's possible because it's confusing ;) But I think it
might work.

JM


2014-06-05 19:47 GMT-04:00 Ted Yu <yuzhihong@gmail.com>:

> MUST_PASS_ONE represents boolean OR operator.
>
> According to Vrushali's description, "c!someName" should be excluded.
>
> Would MUST_PASS_ONE achieve what Vrushali wanted ?
>
> Cheers
>
>
> On Thu, Jun 5, 2014 at 4:33 PM, Jean-Marc Spaggiari <
> jean-marc@spaggiari.org
> > wrote:
>
> > I will still give a try to the 2 filters options.
> >
> > RegEx are nice and powerful but very expensive. It's non trivial. While
> the
> > prefix comparator is pretty simple and fast. So I'm not sure which of
> the 2
> > options will be faster.
> >
> > My opinion: Code wise, RegEx will be simpler, 2 filters will be faster.
> >
> >
> > 2014-06-05 18:55 GMT-04:00 Ted Yu <yuzhihong@gmail.com>:
> >
> > > You're welcome.
> > >
> > > Filters / comparators shipped with HBase are pretty powerful.
> > >
> > >
> > > On Thu, Jun 5, 2014 at 3:04 PM, Vrushali C <vrushali@ymail.com> wrote:
> > >
> > > > Thanks Ted! Using that regex comparator helped me resolve this.
> > > Appreciate
> > > > it very much!
> > > >
> > > >
> > > >   On Thursday, June 5, 2014 2:23 PM, Ted Yu <yuzhihong@gmail.com>
> > wrote:
> > > >
> > > >
> > > > Or, you can use RegexStringComparator.
> > > >
> > > > Here is a regex string, in Java, that matches columns with prefix c!
> > > except
> > > > column called c!someName :
> > > >
> > > > "^c\\!((?!someName).)*$"
> > > >
> > > > Cheers
> > > >
> > > >
> > > > On Thu, Jun 5, 2014 at 1:26 PM, Ted Yu <yuzhihong@gmail.com> wrote:
> > > >
> > > > > One option is to write your own Comparator (similar to
> > > > BinaryPrefixComparator
> > > > > in essence) that treats the known column name specially.
> > > > >
> > > > > Cheers
> > > > >
> > > > >
> > > > > On Thu, Jun 5, 2014 at 12:52 PM, Vrushali C <vrushali@ymail.com>
> > > wrote:
> > > > >
> > > > >>
> > > > >>
> > > > >> Hi
> > > > >> Is there a way to do this kind of filtering : In my scan, I want
> to
> > > > >> retrieve all columns except for columns starting with a certain
> > > prefix.
> > > > But
> > > > >> within that set of columns being ignored, I have one known column
> > name
> > > > that
> > > > >> I want to retrieve but ignore the rest. The reason is that columns
> > > with
> > > > >> this prefix have a lot of data and I am not interested in
> everything
> > > > EXCEPT
> > > > >> one of those.
> > > > >>
> > > > >> So for ignoring the columns with a certain prefix in the scan,
I
> am
> > > > doing
> > > > >> something like
> > > > >> filters.addFilter(
> > > > >>      new QualifierFilter(CompareFilter.CompareOp.NOT_EQUAL,
> > > > >>        new BinaryPrefixComparator(
> > > > >>                Bytes.add(Bytes.toBytes("c!"),
> > Constants.SEP_BYTES))))
> > > > >>
> > > > >> Which works. But what I also want to add, is something like this
> > > > >>
> > > > >> filters.addFilter(
> > > > >>      new QualifierFilter(CompareFilter.CompareOp.EQUAL,
> > > > >>        new BinaryPrefixComparator(
> > > > >>                Bytes.add(Bytes.toBytes("c!someName"),
> > > > >> Constants.SEP_BYTES))))
> > > > >>
> > > > >> I realize both filters are contradictory to each other, so how
do
> I
> > > > >> achieve this?
> > > > >>
> > > > >> thanks
> > > > >> Vrushali
> > > > >>
> > > > >
> > > > >
> > > >
> > > >
> > > >
> > >
> >
>

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