You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@hbase.apache.org by Tony Dean <To...@sas.com> on 2013/06/21 23:08:32 UTC

Scan performance

Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{
  ...
}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I'm filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I'm returning only 1 column in both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704






RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
My row key is relatively small so that's why RowFilter is not making any difference.  Thanks.

I am scanning a single table/cf for a particular qualifier.  The scan has a start row and stop row with a filter.  In my test case, only 1 result is returned.  I put this call in a 10 iteration loop to show how performance gets better after the data is primed.

638ms
61ms
4ms
3ms
3ms
4ms
3ms
4ms
3ms
4ms

All the time is spent in getting the next scan iterator.

Thanks.


-----Original Message-----
From: lars hofhansl [mailto:larsh@apache.org] 
Sent: Monday, June 24, 2013 5:05 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

RowFilter can help. It depends on the setup.
RowFilter skip all column of the row when the row key does not match.
That will help with IO *if* your rows are larger than the HFile block size (64k by default). Otherwise it still needs to touch each block.

An HTable does some priming when it is created. The region information for all tables could be substantial, so it does not make much sense to prime the cache for all tables.
How are you using the client. If you pre-create a reuse HTable and/or HConnection you should be OK.


-- Lars



________________________________
 From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Monday, June 24, 2013 1:48 PM
Subject: RE: Scan performance
 

Lars,
I'm waiting for some time to exchange out hbase jars in cluster (that support FuzzyRow filter) in order to try out.  In the meantime, I'm wondering why RowFilter regex is not more helpful.  I'm guessing that FuzzyRow filter helps in disk io while Row filter just filters after the disk io has completed.  Also, I turned on row level bloom filter which does not seem to help either.

On a different performance note, I'm wondering if there is a way to prime client connection information and such so that the first client query isn't miserably slow.  After the first query, response times do get considerably better due to caching necessary information.  Is there a way to get around this first initial hit?  I assume any such priming would have to be application specific.

Thanks.

-----Original Message-----
From: lars hofhansl [mailto:larsh@apache.org] 
Sent: Saturday, June 22, 2013 9:24 AM
To: user@hbase.apache.org
Subject: Re: Scan performance

"essential column families" help when you filter on one column but want to return *other* columns for the rows that matched the column.

Check out HBASE-5416.

-- Lars



________________________________
From: Vladimir Rodionov <vr...@carrieriq.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Friday, June 21, 2013 5:00 PM
Subject: RE: Scan performance


Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into
the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will
need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in
execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.

Re: Scan performance

Posted by Jean-Marc Spaggiari <je...@spaggiari.org>.
Should we expect same kind of performances difference with get between
addColumn(CF) and addColumn(CF, C)?

Might worse to give it a try...


2013/8/8 Viral Bajaria <vi...@gmail.com>

> Hi Tony,
>
> I know it's been a while and am not sure if you already figured out the
> issue but try taking at HBASE-9079 and see if it's similar to the problem
> that you are facing with FuzzyRowFilter. I have attached a patch to that
> ticket too and have verified that it fixed things for me in production.
>
> Thanks,
> Viral
>
> On Tue, Jul 16, 2013 at 8:07 PM, Tony Dean <To...@sas.com> wrote:
>
> > I was able to test scan performance with 0.94.9 with around 6000 rows X
> 40
> > columns and FuzzyRowFilter gave us 2-4 times better performance.  I was
> > able to test this offline without any problems.  However, once I turned
> it
> > on in our development cluster, we noticed that with some row keys that
> > should have matched were not matching.  After reverting back to
> > SingleColumnValueFilter the cases that were failing, began to work again.
> >  We thought that the anomaly was due to certain data in row key, but we
> > managed to create identical row keys in a different table and see the
> scan
> > work.  So, bottom line I can't explain this behavior.  Has anyone seen
> this
> > behavior and does anyone have debugging tips?
> >
> > Thanks.
> >
>

Re: Scan performance

Posted by Viral Bajaria <vi...@gmail.com>.
Hi Tony,

I know it's been a while and am not sure if you already figured out the
issue but try taking at HBASE-9079 and see if it's similar to the problem
that you are facing with FuzzyRowFilter. I have attached a patch to that
ticket too and have verified that it fixed things for me in production.

Thanks,
Viral

On Tue, Jul 16, 2013 at 8:07 PM, Tony Dean <To...@sas.com> wrote:

> I was able to test scan performance with 0.94.9 with around 6000 rows X 40
> columns and FuzzyRowFilter gave us 2-4 times better performance.  I was
> able to test this offline without any problems.  However, once I turned it
> on in our development cluster, we noticed that with some row keys that
> should have matched were not matching.  After reverting back to
> SingleColumnValueFilter the cases that were failing, began to work again.
>  We thought that the anomaly was due to certain data in row key, but we
> managed to create identical row keys in a different table and see the scan
> work.  So, bottom line I can't explain this behavior.  Has anyone seen this
> behavior and does anyone have debugging tips?
>
> Thanks.
>

RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
I should emphasize that that the exact same row is in both tables.  The fuzzyrowfilter scan works with table containing ~6000 rows.  The same fuzzyrowfilter scan fails with table containing millions of rows.  Thanks for any insight.

-----Original Message-----
From: Tony Dean 
Sent: Tuesday, July 16, 2013 9:29 PM
To: user@hbase.apache.org
Subject: RE: Scan performance

I was able to test scan performance with 0.94.9 with around 6000 rows X 40 columns and FuzzyRowFilter gave us 2-4 times better performance.  I was able to test this offline without any problems.  However, once I turned it on in our development cluster, we noticed that with some row keys that should have matched were not matching.  After reverting back to SingleColumnValueFilter the cases that were failing, began to work again.  We thought that the anomaly was due to certain data in row key, but we managed to create identical row keys in a different table and see the scan work.  So, bottom line I can't explain this behavior.  Has anyone seen this behavior and does anyone have debugging tips?

Thanks.

-----Original Message-----
From: Ted Yu [mailto:yuzhihong@gmail.com]
Sent: Tuesday, July 02, 2013 6:11 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

Tony:
Take a look at
http://blog.sematext.com/2012/08/09/consider-using-fuzzyrowfilter-when-in-need-for-secondary-indexes-in-hbase/

Cheers

On Tue, Jul 2, 2013 at 2:31 PM, Tony Dean <To...@sas.com> wrote:

> The following information is what I discovered from Scan performance 
> testing.
>
> Setup
> -------
> row key format:
> positiion1,position2,position3
> where position1 is a fixed literal, and position2 and position3 are 
> variable data.
>
> I have created data with 6000 rows with ~40 columns in each row.  The 
> table contains only 1 column family.
>
> The row that I want to query is:
> vid,sid-0,Logon    event:customer value=?
>
> -------
>
> Case 1:
> use fully qualified row specification in start/stop row key (e.g.,
> vid,sid-0,Logon) with a SingleColumnValueFilter in the Scan.
>
> avg response time to get Scan iterator and iterate the single result 
> is ~5ms.  This is expected.
>
>
> Case 2:
> This is the normal case where position2 in the row key is unknown at 
> the time of the query: vid,?,Logon.
> Using a SingleColumnValueFilter in the Scan, the avg response time to 
> get Scan iterator and iterate the single result is ~100ms.
> This is the use case that I'm trying to improve upon.
>
> Case 3:
> After upgrading to 0.94.8 I was able to change Case2 by using 
> FuzzyRowFilter instead of SingleColumnValueFilter.  It's a good 
> candidate since I know position1 and position3.
> The avg response time to get Scan iterator and iterate the single 
> result was ~5ms (pretty much the same response time as case 1 where I 
> knew the complete row key).
>
> I didn't expect such an improvement.  Can you explain how 
> FuzzyRowFilter optimizes scanning rows from disk?  In my case it needs 
> to scan rows
> (vid,?,xxxx) until xxxx is greater than "Logon".  Then it can just 
> stop after that; thereby optimizing the scan, correct?  So, 
> optimization using FuzzyRowFilter is very dependent upon the data that you are scanning.
>
> Thanks for any insight.
>
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Monday, June 24, 2013 5:05 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> RowFilter can help. It depends on the setup.
> RowFilter skip all column of the row when the row key does not match.
> That will help with IO *if* your rows are larger than the HFile block 
> size (64k by default). Otherwise it still needs to touch each block.
>
> An HTable does some priming when it is created. The region information 
> for all tables could be substantial, so it does not make much sense to 
> prime the cache for all tables.
> How are you using the client. If you pre-create a reuse HTable and/or 
> HConnection you should be OK.
>
>
> -- Lars
>
>
>
> ________________________________
>  From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl < 
> larsh@apache.org>
> Sent: Monday, June 24, 2013 1:48 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I'm waiting for some time to exchange out hbase jars in cluster (that 
> support FuzzyRow filter) in order to try out.  In the meantime, I'm 
> wondering why RowFilter regex is not more helpful.  I'm guessing that 
> FuzzyRow filter helps in disk io while Row filter just filters after 
> the disk io has completed.  Also, I turned on row level bloom filter 
> which does not seem to help either.
>
> On a different performance note, I'm wondering if there is a way to 
> prime client connection information and such so that the first client 
> query isn't miserably slow.  After the first query, response times do 
> get considerably better due to caching necessary information.  Is 
> there a way to get around this first initial hit?  I assume any such 
> priming would have to be application specific.
>
> Thanks.
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Saturday, June 22, 2013 9:24 AM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> "essential column families" help when you filter on one column but 
> want to return *other* columns for the rows that matched the column.
>
> Check out HBASE-5416.
>
> -- Lars
>
>
>
> ________________________________
> From: Vladimir Rodionov <vr...@carrieriq.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl < 
> larsh@apache.org>
> Sent: Friday, June 21, 2013 5:00 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I thought that column family is the locality group and placement 
> columns which are frequently accessed together into the same column 
> family (locality group) is the obvious performance improvement tip.
> What are the "essential column families" for in this context?
>
> As for original question..  Unless you place your column into a 
> separate column family in Table 2, you will need to scan (load from 
> disk if not cached) ~ 40x more data for the second table (because you 
> have 40 columns). This may explain why do  see such a difference in 
> execution time if all data needs to be loaded first from HDFS.
>
> Best regards,
> Vladimir Rodionov
> Principal Platform Engineer
> Carrier IQ, www.carrieriq.com
> e-mail: vrodionov@carrieriq.com
>
> ________________________________________
> From: lars hofhansl [larsh@apache.org]
> Sent: Friday, June 21, 2013 3:37 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> HBase is a key value (KV) store. Each column is stored in its own KV, 
> a row is just a set of KVs that happen to have the row key (which is 
> the first part of the key).
> I tried to summarize this here:
> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>
> In the StoreFiles all KVs are sorted in row/column order, but HBase 
> still needs to skip over many KVs in order to "reach" the next row. So 
> more disk and memory IO is needed.
>
> If you using 0.94 there is a new feature "essential column families". 
> If you always search by the same column you can place that one in its 
> own column family and all other column in another column family. In 
> that case your scan performance should be close identical.
>
>
> -- Lars
> ________________________________
>
> From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>
> Sent: Friday, June 21, 2013 2:08 PM
> Subject: Scan performance
>
>
>
>
> Hi,
>
> I hope that you can shed some light on these 2 scenarios below.
>
> I have 2 small tables of 6000 rows.
> Table 1 has only 1 column in each of its rows.
> Table 2 has 40 columns in each of its rows.
> Other than that the two tables are identical.
>
> In both tables there is only 1 row that contains a matching column that I
> am filtering on.   And the Scan performs correctly in both cases by
> returning only the single result.
>
> The code looks something like the following:
>
> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
> include all 6000 rows
> scan.addColumn(cf, qualifier); // only return the column that I am 
> interested in (should only be in 1 row and only 1 version)
>
> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new 
> SingleColumnValueFilter(cf, qualifier, CompareFilter.CompareOp.EQUALS, 
> value); scan.setFilter(new FilterList(f1, f2));
>
> scan .setTimeRange(0, MAX_LONG);
> scan.setMaxVersions(1);
>
> ResultScanner rs = t.getScanner(scan); for (Result result: rs) {
>
> }
>
> For table 1, rs.next() takes about 30ms.
> For table 2, rs.next() takes about 180ms.
>
> Both are returning the exact same result.  Why is it taking so much 
> longer on table 2 to get the same result?  The scan depth is the same.  
> The only difference is the column width.  But I'm filtering on a 
> single column and returning only that column.
>
> Am I missing something?  As I increase the number of columns, the 
> response time gets worse.  I do expect the response time to get worse 
> when increasing the number of rows, but not by increasing the number 
> of columns since I'm returning only 1 column in both cases.
>
> I appreciate any comments that you have.
>
> -Tony
>
>
>
> Tony Dean
> SAS Institute Inc.
> Principal Software Developer
> 919-531-6704          ...
>
> Confidentiality Notice:  The information contained in this message, 
> including any attachments hereto, may be confidential and is intended 
> to be read only by the individual or entity to whom this message is 
> addressed. If the reader of this message is not the intended recipient 
> or an agent or designee of the intended recipient, please note that 
> any review, use, disclosure or distribution of this message or its 
> attachments, in any form, is strictly prohibited.  If you have 
> received this message in error, please immediately notify the sender 
> and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.
>


RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
I was able to test scan performance with 0.94.9 with around 6000 rows X 40 columns and FuzzyRowFilter gave us 2-4 times better performance.  I was able to test this offline without any problems.  However, once I turned it on in our development cluster, we noticed that with some row keys that should have matched were not matching.  After reverting back to SingleColumnValueFilter the cases that were failing, began to work again.  We thought that the anomaly was due to certain data in row key, but we managed to create identical row keys in a different table and see the scan work.  So, bottom line I can't explain this behavior.  Has anyone seen this behavior and does anyone have debugging tips?

Thanks.

-----Original Message-----
From: Ted Yu [mailto:yuzhihong@gmail.com] 
Sent: Tuesday, July 02, 2013 6:11 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

Tony:
Take a look at
http://blog.sematext.com/2012/08/09/consider-using-fuzzyrowfilter-when-in-need-for-secondary-indexes-in-hbase/

Cheers

On Tue, Jul 2, 2013 at 2:31 PM, Tony Dean <To...@sas.com> wrote:

> The following information is what I discovered from Scan performance 
> testing.
>
> Setup
> -------
> row key format:
> positiion1,position2,position3
> where position1 is a fixed literal, and position2 and position3 are 
> variable data.
>
> I have created data with 6000 rows with ~40 columns in each row.  The 
> table contains only 1 column family.
>
> The row that I want to query is:
> vid,sid-0,Logon    event:customer value=?
>
> -------
>
> Case 1:
> use fully qualified row specification in start/stop row key (e.g.,
> vid,sid-0,Logon) with a SingleColumnValueFilter in the Scan.
>
> avg response time to get Scan iterator and iterate the single result 
> is ~5ms.  This is expected.
>
>
> Case 2:
> This is the normal case where position2 in the row key is unknown at 
> the time of the query: vid,?,Logon.
> Using a SingleColumnValueFilter in the Scan, the avg response time to 
> get Scan iterator and iterate the single result is ~100ms.
> This is the use case that I'm trying to improve upon.
>
> Case 3:
> After upgrading to 0.94.8 I was able to change Case2 by using 
> FuzzyRowFilter instead of SingleColumnValueFilter.  It's a good 
> candidate since I know position1 and position3.
> The avg response time to get Scan iterator and iterate the single 
> result was ~5ms (pretty much the same response time as case 1 where I 
> knew the complete row key).
>
> I didn't expect such an improvement.  Can you explain how 
> FuzzyRowFilter optimizes scanning rows from disk?  In my case it needs 
> to scan rows
> (vid,?,xxxx) until xxxx is greater than "Logon".  Then it can just 
> stop after that; thereby optimizing the scan, correct?  So, 
> optimization using FuzzyRowFilter is very dependent upon the data that you are scanning.
>
> Thanks for any insight.
>
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Monday, June 24, 2013 5:05 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> RowFilter can help. It depends on the setup.
> RowFilter skip all column of the row when the row key does not match.
> That will help with IO *if* your rows are larger than the HFile block 
> size (64k by default). Otherwise it still needs to touch each block.
>
> An HTable does some priming when it is created. The region information 
> for all tables could be substantial, so it does not make much sense to 
> prime the cache for all tables.
> How are you using the client. If you pre-create a reuse HTable and/or 
> HConnection you should be OK.
>
>
> -- Lars
>
>
>
> ________________________________
>  From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl < 
> larsh@apache.org>
> Sent: Monday, June 24, 2013 1:48 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I'm waiting for some time to exchange out hbase jars in cluster (that 
> support FuzzyRow filter) in order to try out.  In the meantime, I'm 
> wondering why RowFilter regex is not more helpful.  I'm guessing that 
> FuzzyRow filter helps in disk io while Row filter just filters after 
> the disk io has completed.  Also, I turned on row level bloom filter 
> which does not seem to help either.
>
> On a different performance note, I'm wondering if there is a way to 
> prime client connection information and such so that the first client 
> query isn't miserably slow.  After the first query, response times do 
> get considerably better due to caching necessary information.  Is 
> there a way to get around this first initial hit?  I assume any such 
> priming would have to be application specific.
>
> Thanks.
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Saturday, June 22, 2013 9:24 AM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> "essential column families" help when you filter on one column but 
> want to return *other* columns for the rows that matched the column.
>
> Check out HBASE-5416.
>
> -- Lars
>
>
>
> ________________________________
> From: Vladimir Rodionov <vr...@carrieriq.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl < 
> larsh@apache.org>
> Sent: Friday, June 21, 2013 5:00 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I thought that column family is the locality group and placement 
> columns which are frequently accessed together into the same column 
> family (locality group) is the obvious performance improvement tip. 
> What are the "essential column families" for in this context?
>
> As for original question..  Unless you place your column into a 
> separate column family in Table 2, you will need to scan (load from 
> disk if not cached) ~ 40x more data for the second table (because you 
> have 40 columns). This may explain why do  see such a difference in 
> execution time if all data needs to be loaded first from HDFS.
>
> Best regards,
> Vladimir Rodionov
> Principal Platform Engineer
> Carrier IQ, www.carrieriq.com
> e-mail: vrodionov@carrieriq.com
>
> ________________________________________
> From: lars hofhansl [larsh@apache.org]
> Sent: Friday, June 21, 2013 3:37 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> HBase is a key value (KV) store. Each column is stored in its own KV, 
> a row is just a set of KVs that happen to have the row key (which is 
> the first part of the key).
> I tried to summarize this here:
> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>
> In the StoreFiles all KVs are sorted in row/column order, but HBase 
> still needs to skip over many KVs in order to "reach" the next row. So 
> more disk and memory IO is needed.
>
> If you using 0.94 there is a new feature "essential column families". 
> If you always search by the same column you can place that one in its 
> own column family and all other column in another column family. In 
> that case your scan performance should be close identical.
>
>
> -- Lars
> ________________________________
>
> From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>
> Sent: Friday, June 21, 2013 2:08 PM
> Subject: Scan performance
>
>
>
>
> Hi,
>
> I hope that you can shed some light on these 2 scenarios below.
>
> I have 2 small tables of 6000 rows.
> Table 1 has only 1 column in each of its rows.
> Table 2 has 40 columns in each of its rows.
> Other than that the two tables are identical.
>
> In both tables there is only 1 row that contains a matching column that I
> am filtering on.   And the Scan performs correctly in both cases by
> returning only the single result.
>
> The code looks something like the following:
>
> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
> include all 6000 rows
> scan.addColumn(cf, qualifier); // only return the column that I am 
> interested in (should only be in 1 row and only 1 version)
>
> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new 
> SingleColumnValueFilter(cf, qualifier, CompareFilter.CompareOp.EQUALS, 
> value); scan.setFilter(new FilterList(f1, f2));
>
> scan .setTimeRange(0, MAX_LONG);
> scan.setMaxVersions(1);
>
> ResultScanner rs = t.getScanner(scan); for (Result result: rs) {
>
> }
>
> For table 1, rs.next() takes about 30ms.
> For table 2, rs.next() takes about 180ms.
>
> Both are returning the exact same result.  Why is it taking so much longer
> on table 2 to get the same result?  The scan depth is the same.  The only
> difference is the column width.  But I'm filtering on a single column and
> returning only that column.
>
> Am I missing something?  As I increase the number of columns, the response
> time gets worse.  I do expect the response time to get worse when
> increasing the number of rows, but not by increasing the number of columns
> since I'm returning only 1 column in
> both cases.
>
> I appreciate any comments that you have.
>
> -Tony
>
>
>
> Tony Dean
> SAS Institute Inc.
> Principal Software Developer
> 919-531-6704          ...
>
> Confidentiality Notice:  The information contained in this message,
> including any attachments hereto, may be confidential and is intended to be
> read only by the individual or entity to whom this message is addressed. If
> the reader of this message is not the intended recipient or an agent or
> designee of the intended recipient, please note that any review, use,
> disclosure or distribution of this message or its attachments, in any form,
> is strictly prohibited.  If you have received this message in error, please
> immediately notify the sender and/or Notifications@carrieriq.com and
> delete or destroy any copy of this message and its attachments.
>


RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
Thanks Ted.

-----Original Message-----
From: Ted Yu [mailto:yuzhihong@gmail.com] 
Sent: Tuesday, July 02, 2013 6:11 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

Tony:
Take a look at
http://blog.sematext.com/2012/08/09/consider-using-fuzzyrowfilter-when-in-need-for-secondary-indexes-in-hbase/

Cheers

On Tue, Jul 2, 2013 at 2:31 PM, Tony Dean <To...@sas.com> wrote:

> The following information is what I discovered from Scan performance 
> testing.
>
> Setup
> -------
> row key format:
> positiion1,position2,position3
> where position1 is a fixed literal, and position2 and position3 are 
> variable data.
>
> I have created data with 6000 rows with ~40 columns in each row.  The 
> table contains only 1 column family.
>
> The row that I want to query is:
> vid,sid-0,Logon    event:customer value=?
>
> -------
>
> Case 1:
> use fully qualified row specification in start/stop row key (e.g.,
> vid,sid-0,Logon) with a SingleColumnValueFilter in the Scan.
>
> avg response time to get Scan iterator and iterate the single result 
> is ~5ms.  This is expected.
>
>
> Case 2:
> This is the normal case where position2 in the row key is unknown at 
> the time of the query: vid,?,Logon.
> Using a SingleColumnValueFilter in the Scan, the avg response time to 
> get Scan iterator and iterate the single result is ~100ms.
> This is the use case that I'm trying to improve upon.
>
> Case 3:
> After upgrading to 0.94.8 I was able to change Case2 by using 
> FuzzyRowFilter instead of SingleColumnValueFilter.  It's a good 
> candidate since I know position1 and position3.
> The avg response time to get Scan iterator and iterate the single 
> result was ~5ms (pretty much the same response time as case 1 where I 
> knew the complete row key).
>
> I didn't expect such an improvement.  Can you explain how 
> FuzzyRowFilter optimizes scanning rows from disk?  In my case it needs 
> to scan rows
> (vid,?,xxxx) until xxxx is greater than "Logon".  Then it can just 
> stop after that; thereby optimizing the scan, correct?  So, 
> optimization using FuzzyRowFilter is very dependent upon the data that you are scanning.
>
> Thanks for any insight.
>
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Monday, June 24, 2013 5:05 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> RowFilter can help. It depends on the setup.
> RowFilter skip all column of the row when the row key does not match.
> That will help with IO *if* your rows are larger than the HFile block 
> size (64k by default). Otherwise it still needs to touch each block.
>
> An HTable does some priming when it is created. The region information 
> for all tables could be substantial, so it does not make much sense to 
> prime the cache for all tables.
> How are you using the client. If you pre-create a reuse HTable and/or 
> HConnection you should be OK.
>
>
> -- Lars
>
>
>
> ________________________________
>  From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl < 
> larsh@apache.org>
> Sent: Monday, June 24, 2013 1:48 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I'm waiting for some time to exchange out hbase jars in cluster (that 
> support FuzzyRow filter) in order to try out.  In the meantime, I'm 
> wondering why RowFilter regex is not more helpful.  I'm guessing that 
> FuzzyRow filter helps in disk io while Row filter just filters after 
> the disk io has completed.  Also, I turned on row level bloom filter 
> which does not seem to help either.
>
> On a different performance note, I'm wondering if there is a way to 
> prime client connection information and such so that the first client 
> query isn't miserably slow.  After the first query, response times do 
> get considerably better due to caching necessary information.  Is 
> there a way to get around this first initial hit?  I assume any such 
> priming would have to be application specific.
>
> Thanks.
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Saturday, June 22, 2013 9:24 AM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> "essential column families" help when you filter on one column but 
> want to return *other* columns for the rows that matched the column.
>
> Check out HBASE-5416.
>
> -- Lars
>
>
>
> ________________________________
> From: Vladimir Rodionov <vr...@carrieriq.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl < 
> larsh@apache.org>
> Sent: Friday, June 21, 2013 5:00 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I thought that column family is the locality group and placement 
> columns which are frequently accessed together into the same column 
> family (locality group) is the obvious performance improvement tip. 
> What are the "essential column families" for in this context?
>
> As for original question..  Unless you place your column into a 
> separate column family in Table 2, you will need to scan (load from 
> disk if not cached) ~ 40x more data for the second table (because you 
> have 40 columns). This may explain why do  see such a difference in 
> execution time if all data needs to be loaded first from HDFS.
>
> Best regards,
> Vladimir Rodionov
> Principal Platform Engineer
> Carrier IQ, www.carrieriq.com
> e-mail: vrodionov@carrieriq.com
>
> ________________________________________
> From: lars hofhansl [larsh@apache.org]
> Sent: Friday, June 21, 2013 3:37 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> HBase is a key value (KV) store. Each column is stored in its own KV, 
> a row is just a set of KVs that happen to have the row key (which is 
> the first part of the key).
> I tried to summarize this here:
> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>
> In the StoreFiles all KVs are sorted in row/column order, but HBase 
> still needs to skip over many KVs in order to "reach" the next row. So 
> more disk and memory IO is needed.
>
> If you using 0.94 there is a new feature "essential column families". 
> If you always search by the same column you can place that one in its 
> own column family and all other column in another column family. In 
> that case your scan performance should be close identical.
>
>
> -- Lars
> ________________________________
>
> From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>
> Sent: Friday, June 21, 2013 2:08 PM
> Subject: Scan performance
>
>
>
>
> Hi,
>
> I hope that you can shed some light on these 2 scenarios below.
>
> I have 2 small tables of 6000 rows.
> Table 1 has only 1 column in each of its rows.
> Table 2 has 40 columns in each of its rows.
> Other than that the two tables are identical.
>
> In both tables there is only 1 row that contains a matching column that I
> am filtering on.   And the Scan performs correctly in both cases by
> returning only the single result.
>
> The code looks something like the following:
>
> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
> include all 6000 rows
> scan.addColumn(cf, qualifier); // only return the column that I am 
> interested in (should only be in 1 row and only 1 version)
>
> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new 
> SingleColumnValueFilter(cf, qualifier, CompareFilter.CompareOp.EQUALS, 
> value); scan.setFilter(new FilterList(f1, f2));
>
> scan .setTimeRange(0, MAX_LONG);
> scan.setMaxVersions(1);
>
> ResultScanner rs = t.getScanner(scan); for (Result result: rs) {
>
> }
>
> For table 1, rs.next() takes about 30ms.
> For table 2, rs.next() takes about 180ms.
>
> Both are returning the exact same result.  Why is it taking so much longer
> on table 2 to get the same result?  The scan depth is the same.  The only
> difference is the column width.  But I'm filtering on a single column and
> returning only that column.
>
> Am I missing something?  As I increase the number of columns, the response
> time gets worse.  I do expect the response time to get worse when
> increasing the number of rows, but not by increasing the number of columns
> since I'm returning only 1 column in
> both cases.
>
> I appreciate any comments that you have.
>
> -Tony
>
>
>
> Tony Dean
> SAS Institute Inc.
> Principal Software Developer
> 919-531-6704          ...
>
> Confidentiality Notice:  The information contained in this message,
> including any attachments hereto, may be confidential and is intended to be
> read only by the individual or entity to whom this message is addressed. If
> the reader of this message is not the intended recipient or an agent or
> designee of the intended recipient, please note that any review, use,
> disclosure or distribution of this message or its attachments, in any form,
> is strictly prohibited.  If you have received this message in error, please
> immediately notify the sender and/or Notifications@carrieriq.com and
> delete or destroy any copy of this message and its attachments.
>


Re: Scan performance

Posted by Ted Yu <yu...@gmail.com>.
Tony:
Take a look at
http://blog.sematext.com/2012/08/09/consider-using-fuzzyrowfilter-when-in-need-for-secondary-indexes-in-hbase/

Cheers

On Tue, Jul 2, 2013 at 2:31 PM, Tony Dean <To...@sas.com> wrote:

> The following information is what I discovered from Scan performance
> testing.
>
> Setup
> -------
> row key format:
> positiion1,position2,position3
> where position1 is a fixed literal, and position2 and position3 are
> variable data.
>
> I have created data with 6000 rows with ~40 columns in each row.  The
> table contains only 1 column family.
>
> The row that I want to query is:
> vid,sid-0,Logon    event:customer value=?
>
> -------
>
> Case 1:
> use fully qualified row specification in start/stop row key (e.g.,
> vid,sid-0,Logon) with a SingleColumnValueFilter in the Scan.
>
> avg response time to get Scan iterator and iterate the single result is
> ~5ms.  This is expected.
>
>
> Case 2:
> This is the normal case where position2 in the row key is unknown at the
> time of the query: vid,?,Logon.
> Using a SingleColumnValueFilter in the Scan, the avg response time to get
> Scan iterator and iterate the single result is ~100ms.
> This is the use case that I'm trying to improve upon.
>
> Case 3:
> After upgrading to 0.94.8 I was able to change Case2 by using
> FuzzyRowFilter instead of SingleColumnValueFilter.  It's a good candidate
> since I know position1 and position3.
> The avg response time to get Scan iterator and iterate the single result
> was ~5ms (pretty much the same response time as case 1 where I knew the
> complete row key).
>
> I didn't expect such an improvement.  Can you explain how FuzzyRowFilter
> optimizes scanning rows from disk?  In my case it needs to scan rows
> (vid,?,xxxx) until xxxx is greater than "Logon".  Then it can just stop
> after that; thereby optimizing the scan, correct?  So, optimization using
> FuzzyRowFilter is very dependent upon the data that you are scanning.
>
> Thanks for any insight.
>
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Monday, June 24, 2013 5:05 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> RowFilter can help. It depends on the setup.
> RowFilter skip all column of the row when the row key does not match.
> That will help with IO *if* your rows are larger than the HFile block size
> (64k by default). Otherwise it still needs to touch each block.
>
> An HTable does some priming when it is created. The region information for
> all tables could be substantial, so it does not make much sense to prime
> the cache for all tables.
> How are you using the client. If you pre-create a reuse HTable and/or
> HConnection you should be OK.
>
>
> -- Lars
>
>
>
> ________________________________
>  From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <
> larsh@apache.org>
> Sent: Monday, June 24, 2013 1:48 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I'm waiting for some time to exchange out hbase jars in cluster (that
> support FuzzyRow filter) in order to try out.  In the meantime, I'm
> wondering why RowFilter regex is not more helpful.  I'm guessing that
> FuzzyRow filter helps in disk io while Row filter just filters after the
> disk io has completed.  Also, I turned on row level bloom filter which does
> not seem to help either.
>
> On a different performance note, I'm wondering if there is a way to prime
> client connection information and such so that the first client query isn't
> miserably slow.  After the first query, response times do get considerably
> better due to caching necessary information.  Is there a way to get around
> this first initial hit?  I assume any such priming would have to be
> application specific.
>
> Thanks.
>
> -----Original Message-----
> From: lars hofhansl [mailto:larsh@apache.org]
> Sent: Saturday, June 22, 2013 9:24 AM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> "essential column families" help when you filter on one column but want to
> return *other* columns for the rows that matched the column.
>
> Check out HBASE-5416.
>
> -- Lars
>
>
>
> ________________________________
> From: Vladimir Rodionov <vr...@carrieriq.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <
> larsh@apache.org>
> Sent: Friday, June 21, 2013 5:00 PM
> Subject: RE: Scan performance
>
>
> Lars,
> I thought that column family is the locality group and placement columns
> which are frequently accessed together into
> the same column family (locality group) is the obvious performance
> improvement tip. What are the "essential column families" for in this
> context?
>
> As for original question..  Unless you place your column into a separate
> column family in Table 2, you will
> need to scan (load from disk if not cached) ~ 40x more data for the second
> table (because you have 40 columns). This may explain why do  see such a
> difference in
> execution time if all data needs to be loaded first from HDFS.
>
> Best regards,
> Vladimir Rodionov
> Principal Platform Engineer
> Carrier IQ, www.carrieriq.com
> e-mail: vrodionov@carrieriq.com
>
> ________________________________________
> From: lars hofhansl [larsh@apache.org]
> Sent: Friday, June 21, 2013 3:37 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> HBase is a key value (KV) store. Each column is stored in its own KV, a
> row is just a set of KVs that happen to have the row key (which is the
> first part of the key).
> I tried to summarize this here:
> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>
> In the StoreFiles all KVs are sorted in row/column order, but HBase still
> needs to skip over many KVs in order to "reach" the next row. So more disk
> and memory IO is needed.
>
> If you using 0.94 there is a new feature "essential column families". If
> you always search by the same column you can place that one in its own
> column family and all other column in another column family. In that case
> your scan performance should be close identical.
>
>
> -- Lars
> ________________________________
>
> From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>
> Sent: Friday, June 21, 2013 2:08 PM
> Subject: Scan performance
>
>
>
>
> Hi,
>
> I hope that you can shed some light on these 2 scenarios below.
>
> I have 2 small tables of 6000 rows.
> Table 1 has only 1 column in each of its rows.
> Table 2 has 40 columns in each of its rows.
> Other than that the two tables are identical.
>
> In both tables there is only 1 row that contains a matching column that I
> am filtering on.   And the Scan performs correctly in both cases by
> returning only the single result.
>
> The code looks something like the following:
>
> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
> include all 6000 rows
> scan.addColumn(cf, qualifier); // only return the column that I am
> interested in (should only be in 1 row and only 1 version)
>
> Filter f1 = new InclusiveStopFilter(stopRow);
> Filter f2 = new SingleColumnValueFilter(cf, qualifier,
> CompareFilter.CompareOp.EQUALS, value);
> scan.setFilter(new FilterList(f1, f2));
>
> scan .setTimeRange(0, MAX_LONG);
> scan.setMaxVersions(1);
>
> ResultScanner rs = t.getScanner(scan);
> for (Result result: rs)
> {
>
> }
>
> For table 1, rs.next() takes about 30ms.
> For table 2, rs.next() takes about 180ms.
>
> Both are returning the exact same result.  Why is it taking so much longer
> on table 2 to get the same result?  The scan depth is the same.  The only
> difference is the column width.  But I’m filtering on a single column and
> returning only that column.
>
> Am I missing something?  As I increase the number of columns, the response
> time gets worse.  I do expect the response time to get worse when
> increasing the number of rows, but not by increasing the number of columns
> since I’m returning only 1 column in
> both cases.
>
> I appreciate any comments that you have.
>
> -Tony
>
>
>
> Tony Dean
> SAS Institute Inc.
> Principal Software Developer
> 919-531-6704          …
>
> Confidentiality Notice:  The information contained in this message,
> including any attachments hereto, may be confidential and is intended to be
> read only by the individual or entity to whom this message is addressed. If
> the reader of this message is not the intended recipient or an agent or
> designee of the intended recipient, please note that any review, use,
> disclosure or distribution of this message or its attachments, in any form,
> is strictly prohibited.  If you have received this message in error, please
> immediately notify the sender and/or Notifications@carrieriq.com and
> delete or destroy any copy of this message and its attachments.
>

RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
The following information is what I discovered from Scan performance testing.

Setup
-------
row key format:
positiion1,position2,position3
where position1 is a fixed literal, and position2 and position3 are variable data.

I have created data with 6000 rows with ~40 columns in each row.  The table contains only 1 column family.

The row that I want to query is:
vid,sid-0,Logon    event:customer value=?

-------

Case 1:
use fully qualified row specification in start/stop row key (e.g., vid,sid-0,Logon) with a SingleColumnValueFilter in the Scan.

avg response time to get Scan iterator and iterate the single result is ~5ms.  This is expected.


Case 2:
This is the normal case where position2 in the row key is unknown at the time of the query: vid,?,Logon.
Using a SingleColumnValueFilter in the Scan, the avg response time to get Scan iterator and iterate the single result is ~100ms.
This is the use case that I'm trying to improve upon.

Case 3:
After upgrading to 0.94.8 I was able to change Case2 by using FuzzyRowFilter instead of SingleColumnValueFilter.  It's a good candidate since I know position1 and position3.
The avg response time to get Scan iterator and iterate the single result was ~5ms (pretty much the same response time as case 1 where I knew the complete row key).

I didn't expect such an improvement.  Can you explain how FuzzyRowFilter optimizes scanning rows from disk?  In my case it needs to scan rows (vid,?,xxxx) until xxxx is greater than "Logon".  Then it can just stop after that; thereby optimizing the scan, correct?  So, optimization using FuzzyRowFilter is very dependent upon the data that you are scanning.

Thanks for any insight.


-----Original Message-----
From: lars hofhansl [mailto:larsh@apache.org] 
Sent: Monday, June 24, 2013 5:05 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

RowFilter can help. It depends on the setup.
RowFilter skip all column of the row when the row key does not match.
That will help with IO *if* your rows are larger than the HFile block size (64k by default). Otherwise it still needs to touch each block.

An HTable does some priming when it is created. The region information for all tables could be substantial, so it does not make much sense to prime the cache for all tables.
How are you using the client. If you pre-create a reuse HTable and/or HConnection you should be OK.


-- Lars



________________________________
 From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Monday, June 24, 2013 1:48 PM
Subject: RE: Scan performance
 

Lars,
I'm waiting for some time to exchange out hbase jars in cluster (that support FuzzyRow filter) in order to try out.  In the meantime, I'm wondering why RowFilter regex is not more helpful.  I'm guessing that FuzzyRow filter helps in disk io while Row filter just filters after the disk io has completed.  Also, I turned on row level bloom filter which does not seem to help either.

On a different performance note, I'm wondering if there is a way to prime client connection information and such so that the first client query isn't miserably slow.  After the first query, response times do get considerably better due to caching necessary information.  Is there a way to get around this first initial hit?  I assume any such priming would have to be application specific.

Thanks.

-----Original Message-----
From: lars hofhansl [mailto:larsh@apache.org] 
Sent: Saturday, June 22, 2013 9:24 AM
To: user@hbase.apache.org
Subject: Re: Scan performance

"essential column families" help when you filter on one column but want to return *other* columns for the rows that matched the column.

Check out HBASE-5416.

-- Lars



________________________________
From: Vladimir Rodionov <vr...@carrieriq.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Friday, June 21, 2013 5:00 PM
Subject: RE: Scan performance


Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into
the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will
need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in
execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.

Re: Scan performance

Posted by lars hofhansl <la...@apache.org>.
RowFilter can help. It depends on the setup.
RowFilter skip all column of the row when the row key does not match.
That will help with IO *if* your rows are larger than the HFile block size (64k by default). Otherwise it still needs to touch each block.

An HTable does some priming when it is created. The region information for all tables could be substantial, so it does not make much sense to prime the cache for all tables.
How are you using the client. If you pre-create a reuse HTable and/or HConnection you should be OK.


-- Lars



________________________________
 From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Monday, June 24, 2013 1:48 PM
Subject: RE: Scan performance
 

Lars,
I'm waiting for some time to exchange out hbase jars in cluster (that support FuzzyRow filter) in order to try out.  In the meantime, I'm wondering why RowFilter regex is not more helpful.  I'm guessing that FuzzyRow filter helps in disk io while Row filter just filters after the disk io has completed.  Also, I turned on row level bloom filter which does not seem to help either.

On a different performance note, I'm wondering if there is a way to prime client connection information and such so that the first client query isn't miserably slow.  After the first query, response times do get considerably better due to caching necessary information.  Is there a way to get around this first initial hit?  I assume any such priming would have to be application specific.

Thanks.

-----Original Message-----
From: lars hofhansl [mailto:larsh@apache.org] 
Sent: Saturday, June 22, 2013 9:24 AM
To: user@hbase.apache.org
Subject: Re: Scan performance

"essential column families" help when you filter on one column but want to return *other* columns for the rows that matched the column.

Check out HBASE-5416.

-- Lars



________________________________
From: Vladimir Rodionov <vr...@carrieriq.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Friday, June 21, 2013 5:00 PM
Subject: RE: Scan performance


Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into
the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will
need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in
execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.

RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
Lars,
I'm waiting for some time to exchange out hbase jars in cluster (that support FuzzyRow filter) in order to try out.  In the meantime, I'm wondering why RowFilter regex is not more helpful.  I'm guessing that FuzzyRow filter helps in disk io while Row filter just filters after the disk io has completed.  Also, I turned on row level bloom filter which does not seem to help either.

On a different performance note, I'm wondering if there is a way to prime client connection information and such so that the first client query isn't miserably slow.  After the first query, response times do get considerably better due to caching necessary information.  Is there a way to get around this first initial hit?  I assume any such priming would have to be application specific.

Thanks.

-----Original Message-----
From: lars hofhansl [mailto:larsh@apache.org] 
Sent: Saturday, June 22, 2013 9:24 AM
To: user@hbase.apache.org
Subject: Re: Scan performance

"essential column families" help when you filter on one column but want to return *other* columns for the rows that matched the column.

Check out HBASE-5416.

-- Lars



________________________________
 From: Vladimir Rodionov <vr...@carrieriq.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Friday, June 21, 2013 5:00 PM
Subject: RE: Scan performance
 

Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into
the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will
need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in
execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.

Re: Scan performance

Posted by lars hofhansl <la...@apache.org>.
"essential column families" help when you filter on one column but want to return *other* columns for the rows that matched the column.

Check out HBASE-5416.

-- Lars



________________________________
 From: Vladimir Rodionov <vr...@carrieriq.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>; lars hofhansl <la...@apache.org> 
Sent: Friday, June 21, 2013 5:00 PM
Subject: RE: Scan performance
 

Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into
the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will
need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in
execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.

RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
Hi James,

I do plan on looking more closely at Phoenix for SQL access to HBase.  Thanks.

-----Original Message-----
From: James Taylor [mailto:jtaylor@salesforce.com] 
Sent: Saturday, June 22, 2013 1:18 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

Hi Tony,
Have you had a look at Phoenix(https://github.com/forcedotcom/phoenix), a SQL skin over HBase? It has a skip scan that will let you model a multi part row key and skip through it efficiently as you've described. Take a look at this blog for more info: http://phoenix-hbase.blogspot.com/2013/05/demystifying-skip-scan-in-phoenix.html?m=1

Regards,
James

On Jun 22, 2013, at 6:29 AM, "lars hofhansl" <la...@apache.org> wrote:

> Yep generally you should design your keys such that start/stopKey can efficiently narrow the scope.
> 
> If that really cannot be done (and you should try hard), the 2nd  best option are "skip scans".
> 
> Filters in HBase allow for providing the scanner framework with hints where to go next.
> They can skip to the next column (to avoid looking at many versions), to the next row (to avoid looking at many columns), or they can provide a custom seek hint to a specific key value. The latter is what FuzzyRowFilter does.
> 
> 
> -- Lars
> 
> 
> 
> ________________________________
> From: Anoop John <an...@gmail.com>
> To: user@hbase.apache.org
> Sent: Friday, June 21, 2013 11:58 PM
> Subject: Re: Scan performance
> 
> 
> Have a look at FuzzyRowFilter
> 
> -Anoop-
> 
> On Sat, Jun 22, 2013 at 9:20 AM, Tony Dean <To...@sas.com> wrote:
> 
>> I understand more, but have additional questions about the internals...
>> 
>> So, in this example I have 6000 rows X 40 columns in this table.  In 
>> this test my startRow and stopRow do not narrow the scan criterior 
>> therefore all
>> 6000x40 KVs must be included in the search and thus read from disk 
>> and into memory.
>> 
>> The first filter that I used was:
>> Filter f2 = new SingleColumnValueFilter(cf, qualifier, 
>> CompareFilter.CompareOp.EQUALS, value);
>> 
>> This means that HBase must look for the qualifier column on all 6000 rows.
>> As you mention I could add certain columns to a different cf; but 
>> unfortunately, in my case there is no such small set of columns that 
>> will need to be compared (filtered on).  I could try to use indexes 
>> so that a complete row key can be calculated from a secondary index 
>> in order to perform a faster search against data in a primary table.  
>> This requires additional tables and maintenance that I would like to avoid.
>> 
>> I did try a row key filter with regex hoping that it would limit the 
>> number of rows that were read from disk.
>> Filter f2 = new RowFilter(CompareFilter.CompareOp.EQUAL, new 
>> RegexStringComparator(row_regexpr));
>> 
>> My row keys are something like: vid,sid,event.  sid is not known at 
>> query time so I can use a regex similar to: vid,.*,Logon where Logon 
>> is the event that I am looking for in a particular visit.  In my test 
>> data this should have narrowed the scan to 1 row X 40 columns.  The 
>> best I could do for start/stop row is: vid,0 and vid,~ respectively.  
>> I guess that is still going to cause all 6000 rows to be scanned, but 
>> the filtering should be more specific with the rowKey filter.  
>> However, I did not see any performance improvement.  Anything obvious?
>> 
>> Do you have any other ideas to help out with performance when row key is:
>> vid,sid,event and sid is not known at query time which leaves a gap 
>> in the start/stop row?  Too bad regex can't be used in start/stop row 
>> specification.  That's really what I need.
>> 
>> Thanks again.
>> -Tony
>> 
>> -----Original Message-----
>> From: Vladimir Rodionov [mailto:vrodionov@carrieriq.com]
>> Sent: Friday, June 21, 2013 8:00 PM
>> To: user@hbase.apache.org; lars hofhansl
>> Subject: RE: Scan performance
>> 
>> Lars,
>> I thought that column family is the locality group and placement 
>> columns which are frequently accessed together into the same column 
>> family (locality group) is the obvious performance improvement tip. 
>> What are the "essential column families" for in this context?
>> 
>> As for original question..  Unless you place your column into a 
>> separate column family in Table 2, you will need to scan (load from 
>> disk if not
>> cached) ~ 40x more data for the second table (because you have 40 columns).
>> This may explain why do  see such a difference in execution time if 
>> all data needs to be loaded first from HDFS.
>> 
>> Best regards,
>> Vladimir Rodionov
>> Principal Platform Engineer
>> Carrier IQ, www.carrieriq.com
>> e-mail: vrodionov@carrieriq.com
>> 
>> ________________________________________
>> From: lars hofhansl [larsh@apache.org]
>> Sent: Friday, June 21, 2013 3:37 PM
>> To: user@hbase.apache.org
>> Subject: Re: Scan performance
>> 
>> HBase is a key value (KV) store. Each column is stored in its own KV, 
>> a row is just a set of KVs that happen to have the row key (which is 
>> the first part of the key).
>> I tried to summarize this here:
>> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>> 
>> In the StoreFiles all KVs are sorted in row/column order, but HBase 
>> still needs to skip over many KVs in order to "reach" the next row. 
>> So more disk and memory IO is needed.
>> 
>> If you using 0.94 there is a new feature "essential column families". 
>> If you always search by the same column you can place that one in its 
>> own column family and all other column in another column family. In 
>> that case your scan performance should be close identical.
>> 
>> 
>> -- Lars
>> ________________________________
>> 
>> From: Tony Dean <To...@sas.com>
>> To: "user@hbase.apache.org" <us...@hbase.apache.org>
>> Sent: Friday, June 21, 2013 2:08 PM
>> Subject: Scan performance
>> 
>> 
>> 
>> 
>> Hi,
>> 
>> I hope that you can shed some light on these 2 scenarios below.
>> 
>> I have 2 small tables of 6000 rows.
>> Table 1 has only 1 column in each of its rows.
>> Table 2 has 40 columns in each of its rows.
>> Other than that the two tables are identical.
>> 
>> In both tables there is only 1 row that contains a matching column that I
>> am filtering on.   And the Scan performs correctly in both cases by
>> returning only the single result.
>> 
>> The code looks something like the following:
>> 
>> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
>> include all 6000 rows
>> scan.addColumn(cf, qualifier); // only return the column that I am 
>> interested in (should only be in 1 row and only 1 version)
>> 
>> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new 
>> SingleColumnValueFilter(cf, qualifier,  
>> CompareFilter.CompareOp.EQUALS, value); scan.setFilter(new 
>> FilterList(f1, f2));
>> 
>> scan .setTimeRange(0, MAX_LONG);
>> scan.setMaxVersions(1);
>> 
>> ResultScanner rs = t.getScanner(scan); for (Result result: rs) {
>> 
>> }
>> 
>> For table 1, rs.next() takes about 30ms.
>> For table 2, rs.next() takes about 180ms.
>> 
>> Both are returning the exact same result.  Why is it taking so much 
>> longer on table 2 to get the same result?  The scan depth is the 
>> same.  The only difference is the column width.  But I'm filtering on 
>> a single column and returning only that column.
>> 
>> Am I missing something?  As I increase the number of columns, the 
>> response time gets worse.  I do expect the response time to get worse 
>> when increasing the number of rows, but not by increasing the number 
>> of columns since I'm returning only 1 column in both cases.
>> 
>> I appreciate any comments that you have.
>> 
>> -Tony
>> 
>> 
>> 
>> Tony Dean
>> SAS Institute Inc.
>> Principal Software Developer
>> 919-531-6704          ...
>> 
>> Confidentiality Notice:  The information contained in this message, 
>> including any attachments hereto, may be confidential and is intended 
>> to be read only by the individual or entity to whom this message is 
>> addressed. If the reader of this message is not the intended 
>> recipient or an agent or designee of the intended recipient, please 
>> note that any review, use, disclosure or distribution of this message 
>> or its attachments, in any form, is strictly prohibited.  If you have 
>> received this message in error, please immediately notify the sender 
>> and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.
>> 
>> 
>> 



Re: Scan performance

Posted by James Taylor <jt...@salesforce.com>.
Hi Tony,
Have you had a look at Phoenix(https://github.com/forcedotcom/phoenix), a SQL skin over HBase? It has a skip scan that will let you model a multi part row key and skip through it efficiently as you've described. Take a look at this blog for more info: http://phoenix-hbase.blogspot.com/2013/05/demystifying-skip-scan-in-phoenix.html?m=1

Regards,
James

On Jun 22, 2013, at 6:29 AM, "lars hofhansl" <la...@apache.org> wrote:

> Yep generally you should design your keys such that start/stopKey can efficiently narrow the scope.
> 
> If that really cannot be done (and you should try hard), the 2nd  best option are "skip scans".
> 
> Filters in HBase allow for providing the scanner framework with hints where to go next.
> They can skip to the next column (to avoid looking at many versions), to the next row (to avoid looking at many columns), or they can provide a custom seek hint to a specific key value. The latter is what FuzzyRowFilter does.
> 
> 
> -- Lars
> 
> 
> 
> ________________________________
> From: Anoop John <an...@gmail.com>
> To: user@hbase.apache.org
> Sent: Friday, June 21, 2013 11:58 PM
> Subject: Re: Scan performance
> 
> 
> Have a look at FuzzyRowFilter
> 
> -Anoop-
> 
> On Sat, Jun 22, 2013 at 9:20 AM, Tony Dean <To...@sas.com> wrote:
> 
>> I understand more, but have additional questions about the internals...
>> 
>> So, in this example I have 6000 rows X 40 columns in this table.  In this
>> test my startRow and stopRow do not narrow the scan criterior therefore all
>> 6000x40 KVs must be included in the search and thus read from disk and into
>> memory.
>> 
>> The first filter that I used was:
>> Filter f2 = new SingleColumnValueFilter(cf, qualifier,
>> CompareFilter.CompareOp.EQUALS, value);
>> 
>> This means that HBase must look for the qualifier column on all 6000 rows.
>> As you mention I could add certain columns to a different cf; but
>> unfortunately, in my case there is no such small set of columns that will
>> need to be compared (filtered on).  I could try to use indexes so that a
>> complete row key can be calculated from a secondary index in order to
>> perform a faster search against data in a primary table.  This requires
>> additional tables and maintenance that I would like to avoid.
>> 
>> I did try a row key filter with regex hoping that it would limit the
>> number of rows that were read from disk.
>> Filter f2 = new RowFilter(CompareFilter.CompareOp.EQUAL, new
>> RegexStringComparator(row_regexpr));
>> 
>> My row keys are something like: vid,sid,event.  sid is not known at query
>> time so I can use a regex similar to: vid,.*,Logon where Logon is the event
>> that I am looking for in a particular visit.  In my test data this should
>> have narrowed the scan to 1 row X 40 columns.  The best I could do for
>> start/stop row is: vid,0 and vid,~ respectively.  I guess that is still
>> going to cause all 6000 rows to be scanned, but the filtering should be
>> more specific with the rowKey filter.  However, I did not see any
>> performance improvement.  Anything obvious?
>> 
>> Do you have any other ideas to help out with performance when row key is:
>> vid,sid,event and sid is not known at query time which leaves a gap in the
>> start/stop row?  Too bad regex can't be used in start/stop row
>> specification.  That's really what I need.
>> 
>> Thanks again.
>> -Tony
>> 
>> -----Original Message-----
>> From: Vladimir Rodionov [mailto:vrodionov@carrieriq.com]
>> Sent: Friday, June 21, 2013 8:00 PM
>> To: user@hbase.apache.org; lars hofhansl
>> Subject: RE: Scan performance
>> 
>> Lars,
>> I thought that column family is the locality group and placement columns
>> which are frequently accessed together into the same column family
>> (locality group) is the obvious performance improvement tip. What are the
>> "essential column families" for in this context?
>> 
>> As for original question..  Unless you place your column into a separate
>> column family in Table 2, you will need to scan (load from disk if not
>> cached) ~ 40x more data for the second table (because you have 40 columns).
>> This may explain why do  see such a difference in execution time if all
>> data needs to be loaded first from HDFS.
>> 
>> Best regards,
>> Vladimir Rodionov
>> Principal Platform Engineer
>> Carrier IQ, www.carrieriq.com
>> e-mail: vrodionov@carrieriq.com
>> 
>> ________________________________________
>> From: lars hofhansl [larsh@apache.org]
>> Sent: Friday, June 21, 2013 3:37 PM
>> To: user@hbase.apache.org
>> Subject: Re: Scan performance
>> 
>> HBase is a key value (KV) store. Each column is stored in its own KV, a
>> row is just a set of KVs that happen to have the row key (which is the
>> first part of the key).
>> I tried to summarize this here:
>> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>> 
>> In the StoreFiles all KVs are sorted in row/column order, but HBase still
>> needs to skip over many KVs in order to "reach" the next row. So more disk
>> and memory IO is needed.
>> 
>> If you using 0.94 there is a new feature "essential column families". If
>> you always search by the same column you can place that one in its own
>> column family and all other column in another column family. In that case
>> your scan performance should be close identical.
>> 
>> 
>> -- Lars
>> ________________________________
>> 
>> From: Tony Dean <To...@sas.com>
>> To: "user@hbase.apache.org" <us...@hbase.apache.org>
>> Sent: Friday, June 21, 2013 2:08 PM
>> Subject: Scan performance
>> 
>> 
>> 
>> 
>> Hi,
>> 
>> I hope that you can shed some light on these 2 scenarios below.
>> 
>> I have 2 small tables of 6000 rows.
>> Table 1 has only 1 column in each of its rows.
>> Table 2 has 40 columns in each of its rows.
>> Other than that the two tables are identical.
>> 
>> In both tables there is only 1 row that contains a matching column that I
>> am filtering on.   And the Scan performs correctly in both cases by
>> returning only the single result.
>> 
>> The code looks something like the following:
>> 
>> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
>> include all 6000 rows
>> scan.addColumn(cf, qualifier); // only return the column that I am
>> interested in (should only be in 1 row and only 1 version)
>> 
>> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new
>> SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS,
>> value); scan.setFilter(new FilterList(f1, f2));
>> 
>> scan .setTimeRange(0, MAX_LONG);
>> scan.setMaxVersions(1);
>> 
>> ResultScanner rs = t.getScanner(scan);
>> for (Result result: rs)
>> {
>> 
>> }
>> 
>> For table 1, rs.next() takes about 30ms.
>> For table 2, rs.next() takes about 180ms.
>> 
>> Both are returning the exact same result.  Why is it taking so much longer
>> on table 2 to get the same result?  The scan depth is the same.  The only
>> difference is the column width.  But I'm filtering on a single column and
>> returning only that column.
>> 
>> Am I missing something?  As I increase the number of columns, the response
>> time gets worse.  I do expect the response time to get worse when
>> increasing the number of rows, but not by increasing the number of columns
>> since I'm returning only 1 column in both cases.
>> 
>> I appreciate any comments that you have.
>> 
>> -Tony
>> 
>> 
>> 
>> Tony Dean
>> SAS Institute Inc.
>> Principal Software Developer
>> 919-531-6704          ...
>> 
>> Confidentiality Notice:  The information contained in this message,
>> including any attachments hereto, may be confidential and is intended to be
>> read only by the individual or entity to whom this message is addressed. If
>> the reader of this message is not the intended recipient or an agent or
>> designee of the intended recipient, please note that any review, use,
>> disclosure or distribution of this message or its attachments, in any form,
>> is strictly prohibited.  If you have received this message in error, please
>> immediately notify the sender and/or Notifications@carrieriq.com and
>> delete or destroy any copy of this message and its attachments.
>> 
>> 
>> 

Re: Scan performance

Posted by lars hofhansl <la...@apache.org>.
Yep generally you should design your keys such that start/stopKey can efficiently narrow the scope.

If that really cannot be done (and you should try hard), the 2nd  best option are "skip scans".

Filters in HBase allow for providing the scanner framework with hints where to go next.
They can skip to the next column (to avoid looking at many versions), to the next row (to avoid looking at many columns), or they can provide a custom seek hint to a specific key value. The latter is what FuzzyRowFilter does.


-- Lars



________________________________
 From: Anoop John <an...@gmail.com>
To: user@hbase.apache.org 
Sent: Friday, June 21, 2013 11:58 PM
Subject: Re: Scan performance
 

Have a look at FuzzyRowFilter

-Anoop-

On Sat, Jun 22, 2013 at 9:20 AM, Tony Dean <To...@sas.com> wrote:

> I understand more, but have additional questions about the internals...
>
> So, in this example I have 6000 rows X 40 columns in this table.  In this
> test my startRow and stopRow do not narrow the scan criterior therefore all
> 6000x40 KVs must be included in the search and thus read from disk and into
> memory.
>
> The first filter that I used was:
> Filter f2 = new SingleColumnValueFilter(cf, qualifier,
>  CompareFilter.CompareOp.EQUALS, value);
>
> This means that HBase must look for the qualifier column on all 6000 rows.
>  As you mention I could add certain columns to a different cf; but
> unfortunately, in my case there is no such small set of columns that will
> need to be compared (filtered on).  I could try to use indexes so that a
> complete row key can be calculated from a secondary index in order to
> perform a faster search against data in a primary table.  This requires
> additional tables and maintenance that I would like to avoid.
>
> I did try a row key filter with regex hoping that it would limit the
> number of rows that were read from disk.
> Filter f2 = new RowFilter(CompareFilter.CompareOp.EQUAL, new
> RegexStringComparator(row_regexpr));
>
> My row keys are something like: vid,sid,event.  sid is not known at query
> time so I can use a regex similar to: vid,.*,Logon where Logon is the event
> that I am looking for in a particular visit.  In my test data this should
> have narrowed the scan to 1 row X 40 columns.  The best I could do for
> start/stop row is: vid,0 and vid,~ respectively.  I guess that is still
> going to cause all 6000 rows to be scanned, but the filtering should be
> more specific with the rowKey filter.  However, I did not see any
> performance improvement.  Anything obvious?
>
> Do you have any other ideas to help out with performance when row key is:
> vid,sid,event and sid is not known at query time which leaves a gap in the
> start/stop row?  Too bad regex can't be used in start/stop row
> specification.  That's really what I need.
>
> Thanks again.
> -Tony
>
> -----Original Message-----
> From: Vladimir Rodionov [mailto:vrodionov@carrieriq.com]
> Sent: Friday, June 21, 2013 8:00 PM
> To: user@hbase.apache.org; lars hofhansl
> Subject: RE: Scan performance
>
> Lars,
> I thought that column family is the locality group and placement columns
> which are frequently accessed together into the same column family
> (locality group) is the obvious performance improvement tip. What are the
> "essential column families" for in this context?
>
> As for original question..  Unless you place your column into a separate
> column family in Table 2, you will need to scan (load from disk if not
> cached) ~ 40x more data for the second table (because you have 40 columns).
> This may explain why do  see such a difference in execution time if all
> data needs to be loaded first from HDFS.
>
> Best regards,
> Vladimir Rodionov
> Principal Platform Engineer
> Carrier IQ, www.carrieriq.com
> e-mail: vrodionov@carrieriq.com
>
> ________________________________________
> From: lars hofhansl [larsh@apache.org]
> Sent: Friday, June 21, 2013 3:37 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> HBase is a key value (KV) store. Each column is stored in its own KV, a
> row is just a set of KVs that happen to have the row key (which is the
> first part of the key).
> I tried to summarize this here:
> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>
> In the StoreFiles all KVs are sorted in row/column order, but HBase still
> needs to skip over many KVs in order to "reach" the next row. So more disk
> and memory IO is needed.
>
> If you using 0.94 there is a new feature "essential column families". If
> you always search by the same column you can place that one in its own
> column family and all other column in another column family. In that case
> your scan performance should be close identical.
>
>
> -- Lars
> ________________________________
>
> From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>
> Sent: Friday, June 21, 2013 2:08 PM
> Subject: Scan performance
>
>
>
>
> Hi,
>
> I hope that you can shed some light on these 2 scenarios below.
>
> I have 2 small tables of 6000 rows.
> Table 1 has only 1 column in each of its rows.
> Table 2 has 40 columns in each of its rows.
> Other than that the two tables are identical.
>
> In both tables there is only 1 row that contains a matching column that I
> am filtering on.   And the Scan performs correctly in both cases by
> returning only the single result.
>
> The code looks something like the following:
>
> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
> include all 6000 rows
> scan.addColumn(cf, qualifier); // only return the column that I am
> interested in (should only be in 1 row and only 1 version)
>
> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new
> SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS,
> value); scan.setFilter(new FilterList(f1, f2));
>
> scan .setTimeRange(0, MAX_LONG);
> scan.setMaxVersions(1);
>
> ResultScanner rs = t.getScanner(scan);
> for (Result result: rs)
> {
>
> }
>
> For table 1, rs.next() takes about 30ms.
> For table 2, rs.next() takes about 180ms.
>
> Both are returning the exact same result.  Why is it taking so much longer
> on table 2 to get the same result?  The scan depth is the same.  The only
> difference is the column width.  But I'm filtering on a single column and
> returning only that column.
>
> Am I missing something?  As I increase the number of columns, the response
> time gets worse.  I do expect the response time to get worse when
> increasing the number of rows, but not by increasing the number of columns
> since I'm returning only 1 column in both cases.
>
> I appreciate any comments that you have.
>
> -Tony
>
>
>
> Tony Dean
> SAS Institute Inc.
> Principal Software Developer
> 919-531-6704          ...
>
> Confidentiality Notice:  The information contained in this message,
> including any attachments hereto, may be confidential and is intended to be
> read only by the individual or entity to whom this message is addressed. If
> the reader of this message is not the intended recipient or an agent or
> designee of the intended recipient, please note that any review, use,
> disclosure or distribution of this message or its attachments, in any form,
> is strictly prohibited.  If you have received this message in error, please
> immediately notify the sender and/or Notifications@carrieriq.com and
> delete or destroy any copy of this message and its attachments.
>
>
>

Re: Scan performance

Posted by Anoop John <an...@gmail.com>.
Have a look at FuzzyRowFilter

-Anoop-

On Sat, Jun 22, 2013 at 9:20 AM, Tony Dean <To...@sas.com> wrote:

> I understand more, but have additional questions about the internals...
>
> So, in this example I have 6000 rows X 40 columns in this table.  In this
> test my startRow and stopRow do not narrow the scan criterior therefore all
> 6000x40 KVs must be included in the search and thus read from disk and into
> memory.
>
> The first filter that I used was:
> Filter f2 = new SingleColumnValueFilter(cf, qualifier,
>  CompareFilter.CompareOp.EQUALS, value);
>
> This means that HBase must look for the qualifier column on all 6000 rows.
>  As you mention I could add certain columns to a different cf; but
> unfortunately, in my case there is no such small set of columns that will
> need to be compared (filtered on).  I could try to use indexes so that a
> complete row key can be calculated from a secondary index in order to
> perform a faster search against data in a primary table.  This requires
> additional tables and maintenance that I would like to avoid.
>
> I did try a row key filter with regex hoping that it would limit the
> number of rows that were read from disk.
> Filter f2 = new RowFilter(CompareFilter.CompareOp.EQUAL, new
> RegexStringComparator(row_regexpr));
>
> My row keys are something like: vid,sid,event.  sid is not known at query
> time so I can use a regex similar to: vid,.*,Logon where Logon is the event
> that I am looking for in a particular visit.  In my test data this should
> have narrowed the scan to 1 row X 40 columns.  The best I could do for
> start/stop row is: vid,0 and vid,~ respectively.  I guess that is still
> going to cause all 6000 rows to be scanned, but the filtering should be
> more specific with the rowKey filter.  However, I did not see any
> performance improvement.  Anything obvious?
>
> Do you have any other ideas to help out with performance when row key is:
> vid,sid,event and sid is not known at query time which leaves a gap in the
> start/stop row?  Too bad regex can't be used in start/stop row
> specification.  That's really what I need.
>
> Thanks again.
> -Tony
>
> -----Original Message-----
> From: Vladimir Rodionov [mailto:vrodionov@carrieriq.com]
> Sent: Friday, June 21, 2013 8:00 PM
> To: user@hbase.apache.org; lars hofhansl
> Subject: RE: Scan performance
>
> Lars,
> I thought that column family is the locality group and placement columns
> which are frequently accessed together into the same column family
> (locality group) is the obvious performance improvement tip. What are the
> "essential column families" for in this context?
>
> As for original question..  Unless you place your column into a separate
> column family in Table 2, you will need to scan (load from disk if not
> cached) ~ 40x more data for the second table (because you have 40 columns).
> This may explain why do  see such a difference in execution time if all
> data needs to be loaded first from HDFS.
>
> Best regards,
> Vladimir Rodionov
> Principal Platform Engineer
> Carrier IQ, www.carrieriq.com
> e-mail: vrodionov@carrieriq.com
>
> ________________________________________
> From: lars hofhansl [larsh@apache.org]
> Sent: Friday, June 21, 2013 3:37 PM
> To: user@hbase.apache.org
> Subject: Re: Scan performance
>
> HBase is a key value (KV) store. Each column is stored in its own KV, a
> row is just a set of KVs that happen to have the row key (which is the
> first part of the key).
> I tried to summarize this here:
> http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)
>
> In the StoreFiles all KVs are sorted in row/column order, but HBase still
> needs to skip over many KVs in order to "reach" the next row. So more disk
> and memory IO is needed.
>
> If you using 0.94 there is a new feature "essential column families". If
> you always search by the same column you can place that one in its own
> column family and all other column in another column family. In that case
> your scan performance should be close identical.
>
>
> -- Lars
> ________________________________
>
> From: Tony Dean <To...@sas.com>
> To: "user@hbase.apache.org" <us...@hbase.apache.org>
> Sent: Friday, June 21, 2013 2:08 PM
> Subject: Scan performance
>
>
>
>
> Hi,
>
> I hope that you can shed some light on these 2 scenarios below.
>
> I have 2 small tables of 6000 rows.
> Table 1 has only 1 column in each of its rows.
> Table 2 has 40 columns in each of its rows.
> Other than that the two tables are identical.
>
> In both tables there is only 1 row that contains a matching column that I
> am filtering on.   And the Scan performs correctly in both cases by
> returning only the single result.
>
> The code looks something like the following:
>
> Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should
> include all 6000 rows
> scan.addColumn(cf, qualifier); // only return the column that I am
> interested in (should only be in 1 row and only 1 version)
>
> Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new
> SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS,
> value); scan.setFilter(new FilterList(f1, f2));
>
> scan .setTimeRange(0, MAX_LONG);
> scan.setMaxVersions(1);
>
> ResultScanner rs = t.getScanner(scan);
> for (Result result: rs)
> {
>
> }
>
> For table 1, rs.next() takes about 30ms.
> For table 2, rs.next() takes about 180ms.
>
> Both are returning the exact same result.  Why is it taking so much longer
> on table 2 to get the same result?  The scan depth is the same.  The only
> difference is the column width.  But I'm filtering on a single column and
> returning only that column.
>
> Am I missing something?  As I increase the number of columns, the response
> time gets worse.  I do expect the response time to get worse when
> increasing the number of rows, but not by increasing the number of columns
> since I'm returning only 1 column in both cases.
>
> I appreciate any comments that you have.
>
> -Tony
>
>
>
> Tony Dean
> SAS Institute Inc.
> Principal Software Developer
> 919-531-6704          ...
>
> Confidentiality Notice:  The information contained in this message,
> including any attachments hereto, may be confidential and is intended to be
> read only by the individual or entity to whom this message is addressed. If
> the reader of this message is not the intended recipient or an agent or
> designee of the intended recipient, please note that any review, use,
> disclosure or distribution of this message or its attachments, in any form,
> is strictly prohibited.  If you have received this message in error, please
> immediately notify the sender and/or Notifications@carrieriq.com and
> delete or destroy any copy of this message and its attachments.
>
>
>

RE: Scan performance

Posted by Tony Dean <To...@sas.com>.
I understand more, but have additional questions about the internals...

So, in this example I have 6000 rows X 40 columns in this table.  In this test my startRow and stopRow do not narrow the scan criterior therefore all 6000x40 KVs must be included in the search and thus read from disk and into memory.

The first filter that I used was:
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);

This means that HBase must look for the qualifier column on all 6000 rows.  As you mention I could add certain columns to a different cf; but unfortunately, in my case there is no such small set of columns that will need to be compared (filtered on).  I could try to use indexes so that a complete row key can be calculated from a secondary index in order to perform a faster search against data in a primary table.  This requires additional tables and maintenance that I would like to avoid.

I did try a row key filter with regex hoping that it would limit the number of rows that were read from disk.
Filter f2 = new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(row_regexpr));

My row keys are something like: vid,sid,event.  sid is not known at query time so I can use a regex similar to: vid,.*,Logon where Logon is the event that I am looking for in a particular visit.  In my test data this should have narrowed the scan to 1 row X 40 columns.  The best I could do for start/stop row is: vid,0 and vid,~ respectively.  I guess that is still going to cause all 6000 rows to be scanned, but the filtering should be more specific with the rowKey filter.  However, I did not see any performance improvement.  Anything obvious?

Do you have any other ideas to help out with performance when row key is: vid,sid,event and sid is not known at query time which leaves a gap in the start/stop row?  Too bad regex can't be used in start/stop row specification.  That's really what I need.

Thanks again.
-Tony

-----Original Message-----
From: Vladimir Rodionov [mailto:vrodionov@carrieriq.com] 
Sent: Friday, June 21, 2013 8:00 PM
To: user@hbase.apache.org; lars hofhansl
Subject: RE: Scan performance

Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow); Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value); scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I'm filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I'm returning only 1 column in both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          ...

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.



RE: Scan performance

Posted by Vladimir Rodionov <vr...@carrieriq.com>.
Lars,
I thought that column family is the locality group and placement columns which are frequently accessed together into
the same column family (locality group) is the obvious performance improvement tip. What are the "essential column families" for in this context?

As for original question..  Unless you place your column into a separate column family in Table 2, you will
need to scan (load from disk if not cached) ~ 40x more data for the second table (because you have 40 columns). This may explain why do  see such a difference in
execution time if all data needs to be loaded first from HDFS.

Best regards,
Vladimir Rodionov
Principal Platform Engineer
Carrier IQ, www.carrieriq.com
e-mail: vrodionov@carrieriq.com

________________________________________
From: lars hofhansl [larsh@apache.org]
Sent: Friday, June 21, 2013 3:37 PM
To: user@hbase.apache.org
Subject: Re: Scan performance

HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org>
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …

Confidentiality Notice:  The information contained in this message, including any attachments hereto, may be confidential and is intended to be read only by the individual or entity to whom this message is addressed. If the reader of this message is not the intended recipient or an agent or designee of the intended recipient, please note that any review, use, disclosure or distribution of this message or its attachments, in any form, is strictly prohibited.  If you have received this message in error, please immediately notify the sender and/or Notifications@carrieriq.com and delete or destroy any copy of this message and its attachments.

Re: Scan performance

Posted by lars hofhansl <la...@apache.org>.
HBase is a key value (KV) store. Each column is stored in its own KV, a row is just a set of KVs that happen to have the row key (which is the first part of the key).
I tried to summarize this here: http://hadoop-hbase.blogspot.de/2011/12/introduction-to-hbase.html)

In the StoreFiles all KVs are sorted in row/column order, but HBase still needs to skip over many KVs in order to "reach" the next row. So more disk and memory IO is needed.

If you using 0.94 there is a new feature "essential column families". If you always search by the same column you can place that one in its own column family and all other column in another column family. In that case your scan performance should be close identical.


-- Lars
________________________________

From: Tony Dean <To...@sas.com>
To: "user@hbase.apache.org" <us...@hbase.apache.org> 
Sent: Friday, June 21, 2013 2:08 PM
Subject: Scan performance




Hi,

I hope that you can shed some light on these 2 scenarios below.

I have 2 small tables of 6000 rows.
Table 1 has only 1 column in each of its rows.
Table 2 has 40 columns in each of its rows.
Other than that the two tables are identical.

In both tables there is only 1 row that contains a matching column that I am filtering on.   And the Scan performs correctly in both cases by returning only the single result.

The code looks something like the following:

Scan scan = new Scan(startRow, stopRow);   // the start/stop rows should include all 6000 rows
scan.addColumn(cf, qualifier); // only return the column that I am interested in (should only be in 1 row and only 1 version)

Filter f1 = new InclusiveStopFilter(stopRow);
Filter f2 = new SingleColumnValueFilter(cf, qualifier,  CompareFilter.CompareOp.EQUALS, value);
scan.setFilter(new FilterList(f1, f2));

scan .setTimeRange(0, MAX_LONG);
scan.setMaxVersions(1);

ResultScanner rs = t.getScanner(scan);
for (Result result: rs)
{

}

For table 1, rs.next() takes about 30ms.
For table 2, rs.next() takes about 180ms.

Both are returning the exact same result.  Why is it taking so much longer on table 2 to get the same result?  The scan depth is the same.  The only difference is the column width.  But I’m filtering on a single column and returning only that column.

Am I missing something?  As I increase the number of columns, the response time gets worse.  I do expect the response time to get worse when increasing the number of rows, but not by increasing the number of columns since I’m returning only 1 column in
both cases.

I appreciate any comments that you have.

-Tony



Tony Dean
SAS Institute Inc.
Principal Software Developer
919-531-6704          …