hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Michael Segel <michael_se...@hotmail.com>
Subject Re: Hbase row ingestion ..
Date Thu, 30 Apr 2015 19:36:54 GMT
Heh.. I just did a talk at BDTC in Boston… of course at the end of the last day… small

Bucketing is a bit different from just hashing the rowkey.  If you are doing get(), then having
480 buckets isn’t a problem. 
Doing a range scan over the 480 buckets makes getting your sort ordered result set a bit more
interesting. (a merge sort of n ordered lists). 

Your use case is why I started HBASE-12853.  The idea is that you can specify the number of
buckets at table creation if you wanted a bucketed table, and then not worry about it. No
special interface just the standard HTable interface and you’re good to go for either bucketed
or not bucketed tables.
Truthfully it should be a straight forward design and a quick piece of code…. but I digress.

Yes, having a longer rowkey may be problematic, but if you need it to make each row unique,
you have no choice. 
Writing the data as an Avro (JSON) record will help quite a bit in that respect. 
Then if you need secondary indexes, you can manually create inverted tables and you’re able
to now find your RS quickly. 

This approach is really independent of the specific version. 

> On Apr 30, 2015, at 11:27 AM, Gautam <gautamkowshik@gmail.com> wrote:
> Thanks Guys for responding!
> Michael,
>   I indeed should have elaborated on our current rowkey design. Re:
> hotspotting, We'r doing exactly what you'r suggesting, i.e. fanning out
> into buckets where the bucket location is a hash(message_unique_fields)
> (we use murmur3). So our write pattern is extremely even on the regions
> and region-servers. We also pre-split our table into 480 buckets (that
> number is based on our experience with the rate of change of cluster size).
> So no complaints on the relative load on regions. We'v designed the rowkey
> as per our usecase and are pretty happy with it. I'm happy to keep the
> rowkey size the way it is but was concerned that we redundantly write that
> very rowkey for each column (which isn't really needed). This column
> qualifier optimization is over and above what we'r already doing to scale
> on writes.  And was wondering if that could get use improvements on write
> times. But I could be wrong if that cost, of repeating rowkey for each
> cell, is purely incurred on the RS side and doesn't affect the write call
> directly.
> Lemme also point out we'r on Hbase 0.98.6 currently.
> James,
>            That talk is awesome sauce! Especially the way you guys
> analyzed your design with that lovely visualization. Any chance that's on a
> github repo :-) ? Would be extremely useful for folks like us. Rowkey
> design has been the center of our attention for weeks/months on end and a
> quicker feedback loop like this viz would really speed up that process.
> Thanks again guys. All of this helps.
> -Gautam.
> On Thu, Apr 30, 2015 at 7:35 AM, James Estes <james.estes@gmail.com> wrote:
>> Guatam,
>> Michael makes a lot of good points. Especially the importance of analyzing
>> your use case for determining the row key design. We (Jive) did a talk at
>> HBasecon a couple years back talking about our row key redesign to vastly
>> improve performance. It also talks a little about the write path and has a
>> (crude) visualization of the impact of the old and new row key designs.
>> Your use case is likely different than ours was, but it may be helpful to
>> hear our experience with row key design
>> http://www.cloudera.com/content/cloudera/en/resources/library/hbasecon/video-hbasecon-2012-real-performance-gains-with-real-time-data.html
>> James
>> On Apr 30, 2015, at 7:51 AM, Michael Segel <michael_segel@hotmail.com>
>> wrote:
>>> I wouldn’t call storing attributes in separate columns a ‘rigid schema’.
>>> You are correct that you could write your data as a CLOB/BLOB and store
>> it in a single cell.
>>> The upside is that its more efficient.
>>> The downside is that its really an all or nothing fetch and then you
>> need to write the extra code to pull data from the Avro CLOB.  (Which does
>> fit your use case.)
>>> This is a normal pattern and gives HBase an extra dimension of storage.
>>> With respect to the row key… look at your main use case.
>>> The size of the row key may be a necessary evil in terms of getting the
>> unique document. (clob/blob).
>>> In terms of performance gains… you need to look at it this way… the cost
>> of inserting a row is what it is.
>>> There will always be a cost for insertion.
>>> There will always be a minimum rowkey size required by your use case.
>>> The next issue is if you are ‘hot spotting’.  Note that I’m not talking
>> about the initial start of loading in to a table, but if all of your data
>> is going to the last region written because the rowkey is sequential.
>>> Here, you may look at hashing the rowkey (SHA-1 or SHA-2) which may
>> shrink your row key (depending on your current rowkey length). The downside
>> here is that you will lose your ability to perform range scans. So if your
>> access pattern is get() rather than scan(), this will work.  Note too that
>> I recommended SHA-1 or SHA-2 for the hash. MD5 works, and is faster, but
>> there’s a greater chance of a hash collision. SHA-1 has a mathematical
>> chance of a collision depending on data set, but I’ve never heard of anyone
>> finding a collision. SHA-2 doesn’t have that problem, but I don’t know if
>> its part of the core java packages.
>>> Again here, the upside is that you’re going to get a fairly even
>> distribution across your cluster. (Which you didn’t describe. That too
>> could be a factor in performance.)
>>> HTH
>>>> On Apr 29, 2015, at 8:03 PM, Gautam <gautamkowshik@gmail.com> wrote:
>>>> Thanks for the quick response!
>>>> Our read path is fairly straightforward and very deterministic. We
>> always
>>>> push down predicates at the rowkey level and read the row's full
>> payload (
>>>> never do projection/filtering over CQs ).  So.. I could, in theory,
>> expect
>>>> a gain as much as the current overhead of  [ 40 * sizeof(rowkey) ] ?
>>>> Curious to understand more about how much of that overhead is actually
>>>> incurred over the network and how much on the RS side. At least to the
>>>> extent it affects the put() / flush()  calls. Lemme know if there are
>>>> particular parts of the code or documentation I should be looking at for
>>>> this. Would like to learn about the memory/netwokr footprint of write
>> calls.
>>>> thank you,
>>>> -Gautam.
>>>> On Wed, Apr 29, 2015 at 5:48 PM, Esteban Gutierrez <
>> esteban@cloudera.com>
>>>> wrote:
>>>>> Hi Gautam,
>>>>> Your reasoning is correct and that will improve the write performance,
>>>>> specially if you always need to write all the qualifiers in a row
>> (sort of
>>>>> a rigid schema). However you should consider to use qualifiers at some
>>>>> extent if the read pattern might include some conditional search, e.g.
>> if
>>>>> you are interested to filter rows that have a qualifier on it.
>>>>> cheers,
>>>>> esteban.
>>>>> --
>>>>> Cloudera, Inc.
>>>>> On Wed, Apr 29, 2015 at 5:31 PM, Gautam <gautamkowshik@gmail.com>
>> wrote:
>>>>>> .. I'd like to add that we have a very fat rowkey.
>>>>>> - Thanks.
>>>>>> On Wed, Apr 29, 2015 at 5:30 PM, Gautam <gautamkowshik@gmail.com>
>> wrote:
>>>>>>> Hello,
>>>>>>>     We'v been fighting some ingestion perf issues on hbase and
>>>>> have
>>>>>>> been looking at the write path in particular. Trying to optimize
>>>>> write
>>>>>>> path currently.
>>>>>>> We have around 40 column qualifiers (under single CF) for each
>> So
>>>>> I
>>>>>>> understand that each put(row) written into hbase would translate
>>>>> 40
>>>>>>> (rowkey, cq, ts)  cells in Hbase.  If I switched to an Avro object
>>>>> based
>>>>>>> schema instead there would be a single (rowkey, avro_cq, ts)
cell per
>>>>>> row (
>>>>>>> all fields shoved into a single Avro blob).  Question is, would
>>>>>>> approach really translate into any write-path perf benefits?
>>>>>>> Cheers,
>>>>>>> -Gautam.
>>>>>> --
>>>>>> "If you really want something in this life, you have to work for
>> Now,
>>>>>> quiet! They're about to announce the lottery numbers..."
>>>> --
>>>> "If you really want something in this life, you have to work for it.
>> Now,
>>>> quiet! They're about to announce the lottery numbers..."
> -- 
> "If you really want something in this life, you have to work for it. Now,
> quiet! They're about to announce the lottery numbers..."

The opinions expressed here are mine, while they may reflect a cognitive thought, that is
purely accidental. 
Use at your own risk. 
Michael Segel
michael_segel (AT) hotmail.com

View raw message