Re: Streaming data to htable

2015-02-16 Thread hongbin ma
Hi Andrey, thanks for your reply, I found them very inspiring. I'm curious about the scheme you described: It looks to me you've sharded your data into separate regions, are you fixing the number of these regions? If yes, each of the growing region will spawn lots of compactions while writing da

Re: Streaming data to htable

2015-02-14 Thread Abraham Tom
useful for other folks to know. > From: Geovanie Marquez > To: "user@hbase.apache.org" > Sent: Friday, February 13, 2015 12:14 PM > Subject: Re: Streaming data to htable > > We use Spark to convert large batches of data directly into HFiles. We've > foun

Re: Streaming data to htable

2015-02-14 Thread lars hofhansl
That's pretty cool. Have you documented somewhere how exactly you do that (a blog post or something)? That'd be useful for other folks to know. From: Geovanie Marquez To: "user@hbase.apache.org" Sent: Friday, February 13, 2015 12:14 PM Subject: Re: Streaming data

Re: Streaming data to htable

2015-02-13 Thread Nicolas Liochon
You should first try with the 'autoflush' boolean on the htable: set it to false. it buffers the writes for you and does the writes asynchronously. So all the multithreading / buffering work is done for you. If you need a synchronisation point (to free the resources on the sending side), you can ca

Re: Streaming data to htable

2015-02-13 Thread Geovanie Marquez
We use Spark to convert large batches of data directly into HFiles. We've found it to be extremely performant, but we do not batch since our use case is not streaming. We bring it in about 50GB at a time so we would not suffer from the small files issue mentioned, but we do manually manage our regi

Re: Streaming data to htable

2015-02-13 Thread Alok Singh
Have you considered placing something like Kafka queue in between the data stream and hbase consumer/writer? I have used Kafka in the past to consume very high volume of event data and write it to hbase. Problems we ran into when writing large amounts of data continuously to hbase are stalls/timeo

Re: Streaming data to htable

2015-02-13 Thread Andrey Stepachev
Hi Jaime. That a bit of magic to use HFiles directly without considering keys and data layout (as mentioned by Nick you will face with a task of manually splitting keys, so effectively you will do what hbase already does effectively). Original answer was for concrete usecase: it is known where ke

Re: Streaming data to htable

2015-02-13 Thread Nick Dimiduk
Writing HFiles can become cumbersome if the data is spread evenly across regions -- you'll end up with lots of small files rather than a few big ones. You can batch writes through the client API. I would recommend you start with HTableInterface$put(List). You can tune the client-side buffer (#setW

Re: Streaming data to htable

2015-02-13 Thread Jaime Solano
Hi Andrey, We're facing a similar situation, where we plan to load a lot of data into HBase direclty. We considered writing the Hfiles without MapReduce. Is this something you've done in the past? Are there any sample codes we could use as guide? On another side, what would you consider "big enoug

Re: Streaming data to htable

2015-02-13 Thread Andrey Stepachev
Hi hongbin, It seems that depend on how many data you ingest. In case of big enough I'd look at creating HFiles directly without mapreduce (for example using HFileOutputFormat without mapreduce or using HFileWriter directly). Created files can be imported by LoadIncrementalHFiles#doBulkLoad direct

Re: Streaming data to htable

2015-02-13 Thread Wilm Schumacher
Am 13.02.2015 um 10:39 schrieb Sleiman Jneidi: > I would go with second option, HtableInterface.put(List). The first > option sounds dodgy, where 5 minutes is a good time for things to go wrong > and you lose your data I agree with Sleiman. In my opinion the "multi put" option is the best plan. T

Re: Streaming data to htable

2015-02-13 Thread Sleiman Jneidi
I would go with second option, HtableInterface.put(List). The first option sounds dodgy, where 5 minutes is a good time for things to go wrong and you lose your data On Fri, Feb 13, 2015 at 6:20 AM, hongbin ma wrote: > hi, > > I'm trying to use a htable to store data that comes in a streaming fa

Streaming data to htable

2015-02-12 Thread hongbin ma
hi, I'm trying to use a htable to store data that comes in a streaming fashion. The streaming in data is guaranteed to have a larger KEY than ANY existing keys in the table. And the data will be READONLY. The data is streaming in at a very high rate, I don't want to issue a PUT operation for each