hbase-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Anoop John <anoop.hb...@gmail.com>
Subject Re: SingleColumnValueFilter with setFilterIfMissing=True behaviour
Date Fri, 11 Apr 2014 13:19:57 GMT
None of these fix version are actually released. If u can get the latest 98
code and build tar and use then u can get rid of this bug.
Else on 96 or 98  u can use
Scan#setLoadColumnFamiliesOnDemand(false).
This setting in Scan will solve this bug. (With a perf penalty)

-Anoop-

On Fri, Apr 11, 2014 at 5:30 PM, Nadya Privalova <nprivalova@mirantis.com>wrote:

> Ted, now the issue is reproduced with 0.98.1-hadoop1. I will try 0.98.2
> because the bug Anoop pointed contains "Fix Version/s: 0.99.0, 0.98.2,
> 0.96.3".
> Will keep you posted.
>
> Thanks for your help,
> Nadya
>
>
>
> On Fri, Apr 11, 2014 at 3:37 PM, Ted Yu <yuzhihong@gmail.com> wrote:
>
> > There're two column families: f and m
> >
> > Nadya:
> > Can you try running tip of 0.98 to see if the issue is resolved ?
> >
> > Thanks
> >
> > On Apr 11, 2014, at 2:56 AM, Anoop John <anoop.hbase@gmail.com> wrote:
> >
> > > Sorry I have not looked at the data what u have pasted.  Do u have more
> > > than one CF getting scanned?  If so u might be facing the bug
> HBASE-10850
> > >
> > > -Anoop-
> > >
> > > On Fri, Apr 11, 2014 at 3:22 PM, Nadya Privalova <
> > nprivalova@mirantis.com>wrote:
> > >
> > >> I tried 0.96 and 0.98 hadoop 1. Behaviour is the same. HBase
> > configuration
> > >> is the most simple, from QuickStartGuide.
> > >>
> > >> Nadya
> > >>
> > >>
> > >> On Fri, Apr 11, 2014 at 1:43 PM, Anoop John <anoop.hbase@gmail.com>
> > wrote:
> > >>
> > >>> Which version you are using?
> > >>>
> > >>> -Anoop-
> > >>>
> > >>> On Fri, Apr 11, 2014 at 2:42 PM, Nadya Privalova <
> > >> nprivalova@mirantis.com
> > >>>> wrote:
> > >>>
> > >>>> Hello folks,
> > >>>> I saw a lot of questions about SingleColumnValueFilter but I have
> one
> > >>> more.
> > >>>> The question is about setFilterIfMissing value and it's behaviour
> with
> > >>>> other column families.
> > >>>> I have the following in my db:
> > >>>> hbase(main):004:0> scan 'c59d09d425244b9bb216a229c2441819_resource'
> > >>>> ROW
> > >>>> COLUMN+CELL
> > >>>>
> > >>>> resource-id
> > >>> column=f:project_id,
> > >>>> timestamp=1397138905401,
> > >>>> value="project-id"
> > >>>>
> > >>>> resource-id
> > >> column=f:s_test-1,
> > >>>> timestamp=1397138905401,
> > >>>> value="1"
> > >>>>
> > >>>> resource-id
> > >>>> column=m:9222030811254775807+test-1+instance!cumulative!,
> > >>>> timestamp=1397138905401, value={"$date":
> > >>>> 1341225600000}
> > >>>> resource-id
> > >>>> column=m:9222030811314775807+test-1+instance!cumulative!,
> > >>>> timestamp=1397138905377, value={"$date":
> > >>>> 1341225540000}
> > >>>> resource-id-2
> > >>> column=f:project_id,
> > >>>> timestamp=1397138905422,
> > >>>> value="project-id-2"
> > >>>>
> > >>>> resource-id-2
> column=f:s_test,
> > >>>> timestamp=1397138905422,
> > >>>> value="1"
> > >>>>
> > >>>>
> > >>>> resource-id-2
> > >>>> column=m:9222030811134775807+test+instance!cumulative!,
> > >>>> timestamp=1397138905422, value={"$date": 1341225720000}
> > >>>>
> > >>>> After filter applying  I see the following:
> > >>>> hbase(main):005:0> scan 'c59d09d425244b9bb216a229c2441819_resource',
> > >>>> {FILTER => "(SingleColumnValueFilter ('f', 's_test-1', =,
> > >> 'binary:\"1\"',
> > >>>> true, false))"}
> > >>>> ROW
> > >>>> COLUMN+CELL
> > >>>>
> > >>>> resource-id
> > >>> column=f:project_id,
> > >>>> timestamp=1397138905401,
> > >>>> value="project-id"
> > >>>>
> > >>>> resource-id
> > >> column=f:s_test-1,
> > >>>> timestamp=1397138905401,
> > >>>> value="1"
> > >>>>
> > >>>>
> > >>>> resource-id
> > >>>> column=m:9222030811254775807+test-1+instance!cumulative!,
> > >>>> timestamp=1397138905401, value={"$date":
> > >>>> 1341225600000}
> > >>>> resource-id
> > >>>> column=m:9222030811314775807+test-1+instance!cumulative!,
> > >>>> timestamp=1397138905377, value={"$date":
> > >>>> 1341225540000}
> > >>>> resource-id-2
> > >>>> column=m:9222030811134775807+test+instance!cumulative!,
> > >>>> timestamp=1397138905422, value={"$date": 1341225720000}
> > >>>>
> > >>>> I wonder why I see  resource-id-2 in output even with
> > >>>> setFilterIfMissing==True? Row with id 'resource-id-2' doesn't
> contain
> > >>>> "f:s_test-1", it contains only "f:s_test". From docs about
> > >>>> setFilterIfMissing: "If true, the entire row will be skipped if
the
> > >>> column
> > >>>> is not found."
> > >>>> So column 's_test-1' is not found in resource-id-2. But I still
see
> > >> this
> > >>>> row (but only 'm' CF) in output.
> > >>>> Of course I can determine COLUMNS=['f'] and resource-id-2 will
not
> be
> > >>>> shown. But I can't because I need values from 'm'.
> > >>>> Could you please comment this behaviour?
> > >>>>
> > >>>> Thanks,
> > >>>> Nadya
> > >>>> (From OpenStack Ceilometer team)
> > >>
> >
>

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