bq. no built-in filter intelligently determines which column family is
essential, except for SingleColumnValueFilter

Mostly right - don't forget about SingleColumnValueExcludeFilter which
extends SingleColumnValueFilter.

Cheers


On Wed, Aug 6, 2014 at 9:34 PM, innowireless TaeYun Kim <
taeyun....@innowireless.co.kr> wrote:

> Thank you Ted.
>
> But RowFilter class has no method that can be uses to set which column
> family is essential. (Actually no built-in filter class provides such a
> method)
>
> So, if I (ever) want to apply the 'dummy' column family technique(?), it
> seems that I must do as follows:
>
> - Write my own filter that's a subclass of the RowFilter.
> - In that filter class, override isFamilyEssential() method to return true
> only when the name of the 'dummy' column family is passed as an argument.
>
> Now, HBase calls isFamilyEssential() method of my filter object for all
> the column families including the 'dummy' column family, and in result only
> loads the 'dummy' column family and happily filters rowkey using the
> KeyValue objects from the 'dummy' column family HFile(s).
>
> Am I right?
>
> BTW, it would be nice to have a method like
> 'setEssentialColumnFamilies(byte[][] names)' to set the essential families
> manually, since no built-in filter intelligently determines which column
> family is essential, except for SingleColumnValueFilter.
>
> Thanks.
>
> -----Original Message-----
> From: Ted Yu [mailto:yuzhih...@gmail.com]
> Sent: Thursday, August 07, 2014 12:38 PM
> To: user@hbase.apache.org
> Subject: Re: Question on the number of column families
>
> bq. While scanning, an entire row will be read even for a rowkey filtering
>
> If you specify essential column family in your filter, the above would not
> be true - only the essential column family would be loaded into memory
> first. Once the filter passes, the other family would be loaded.
>
> Cheers
>
>
> On Wed, Aug 6, 2014 at 4:00 AM, innowireless TaeYun Kim <
> taeyun....@innowireless.co.kr> wrote:
>
> > Hi Ted,
> >
> > Now I finished reading the filtering section and the source code of
> > TestJoinedScanners(0.94).
> >
> > Facts learned:
> >
> > - While scanning, an entire row will be read even for a rowkey filtering.
> > (Since a rowkey is not a physically separate entity and stored in
> > KeyValue object, it's natural. Am I right?)
> > - The key API for the essential column family support is
> > setLoadColumnFamiliesOnDemand().
> >
> > So, now I have questions:
> >
> > On rowkey filtering, which column family's KeyValue object is read?
> > If HBase just reads a KeyValue from a randomly selected (or just the
> > first) column family, how is setLoadColumnFamiliesOnDemand() affected?
> > Can HBase select a smaller column family intelligently?
> >
> > If setLoadColumnFamiliesOnDemand() can be applied to a rowkey
> > filtering, a 'dummy' column family can be used to minimize the scan cost.
> >
> > Thank you.
> >
> >
> > -----Original Message-----
> > From: innowireless TaeYun Kim [mailto:taeyun....@innowireless.co.kr]
> > Sent: Wednesday, August 06, 2014 1:48 PM
> > To: user@hbase.apache.org
> > Subject: RE: Question on the number of column families
> >
> > Thank you.
> >
> > The 'dummy' column will always hold the value '1' (or even an empty
> > string), that only signifies that this row exists. (And the real value
> > is in the other 'big' column family) The value is irrelevant since
> > with current schema the filtering will be done by rowkey components
> > alone. No column value is needed. (I will begin reading the filtering
> > section shortly
> > - it is only 6 pages ahead. So sorry for my premature thoughts)
> >
> >
> > -----Original Message-----
> > From: Ted Yu [mailto:yuzhih...@gmail.com]
> > Sent: Wednesday, August 06, 2014 1:38 PM
> > To: user@hbase.apache.org
> > Subject: Re: Question on the number of column families
> >
> > bq. add a 'dummy' column family and apply HBASE-5416 technique
> >
> > Adding dummy column family is not the way to utilize essential column
> > family support - what would this dummy column family hold ?
> >
> > bq. since I have not read the filtering section of the book I'm
> > reading yet
> >
> > Once you finish reading, you can look at the unit test
> > (TestJoinedScanners) from HBASE-5416. You would understand this
> > feature better.
> >
> > Cheers
> >
> >
> > On Tue, Aug 5, 2014 at 9:21 PM, innowireless TaeYun Kim <
> > taeyun....@innowireless.co.kr> wrote:
> >
> > > Thank you all.
> > >
> > > Facts learned:
> > >
> > > - Having 130 column families is too much. Don't do that.
> > > - While scanning, an entire row will be read for filtering, unless
> > > HBASE-5416 technique is applied which makes only relevant column
> > > family is loaded. (But it seems that still one can't load just a
> > > column needed while
> > > scanning)
> > > - Big row size is maybe not good.
> > >
> > > Currently it seems appropriate to follow the one-column solution
> > > that Alok Singh suggested, in part since currently there is no
> > > reasonable grouping of the fields.
> > >
> > > Here is my current thinking:
> > >
> > > - One column family, one column. Field name will be included in rowkey.
> > > - Eliminate filtering altogether (in most case) by properly ordering
> > > rowkey components.
> > > - If a filtering is absolutely needed, add a 'dummy' column family
> > > and apply HBASE-5416 technique to minimize disk read, since the
> > > field value can be large(~5MB). (This dummy column thing may not be
> > > right, I'm not sure, since I have not read the filtering section of
> > > the book I'm reading yet)
> > >
> > > Hope that I am not missing or misunderstanding something...
> > > (I'm a total newbie. I've started to read a HBase book since last
> > > week...)
> > >
> > >
> > >
> > >
> > >
> > >
> >
> >
>
>

Reply via email to