hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Shahab Yunus <shahab.yu...@gmail.com>
Subject Re: Splitting an existing table with new keys.
Date Wed, 20 Aug 2014 20:45:38 GMT
Thanks Ted.

I also wanted know that from recommendation perspective is this approach
even safe or desirable or not. Or if this is some kind of HBase
anti-pattern (splitting a same table before each bulk import.)

So I did try this and it works with and without existing data.

Now one follow-up question. As we know the split method is asynchronous. So
how do we know that the split is happened as I don't see any callback
mechanism.

In the POC I used 2 appraoches (just to see if it works or not)

1- sleeping for a while which is of course not workable.
2- using the table.getRegionLocations method and checking the size of the
returned map, and if it gets greater from the value captured before the
split, I assume we are all set.

#1 is of course not something that should be done. Is #2 makes sense? Or is
there some other better way to do this?

Thanks again.

Regards,
Shahab


On Tue, Aug 19, 2014 at 7:00 PM, Ted Yu <yuzhihong@gmail.com> wrote:

> My suggestion wasn't about pre-splitting.
>
> You can insert dummy values as part of your proof-of-concept code -
> before admin.split()
> is called.
>
>
> On Tue, Aug 19, 2014 at 3:50 PM, Shahab Yunus <shahab.yunus@gmail.com>
> wrote:
>
> > Ted,
> >
> > Hmmm. So basically, if I understand you correctly, what you are proposing
> > is to insert dummy values corresponding to the new region boundaries that
> > we want as a pre-processing step instead of presplitting?
> >
> > But inserting* only few rows* per-desired-region-range won't guarantee a
> > new region, right? We have to then insert *enough *dummy rows to force a
> > region split? Then there is the question of performing this and how long
> > one should wait to have the region split actually occur?
> >
> > Or am I totally misunderstanding your question/suggestion?
> >
> > Right now the application does not expect empty values but I find the
> idea
> > interesting enough to add logic for it as long as it is workable.
> >
> > Regards,
> > Shahab
> >
> >
> > On Tue, Aug 19, 2014 at 4:49 PM, Ted Yu <yuzhihong@gmail.com> wrote:
> >
> > > Shahab:
> > > How does your application deal with KeyValue whose value is empty ?
> > >
> > > Can you insert rows with empty value whose keys correspond to the
> splits
> > ?
> > >
> > > Cheers
> > >
> > >
> > > On Tue, Aug 19, 2014 at 1:29 PM, Shahab Yunus <shahab.yunus@gmail.com>
> > > wrote:
> > >
> > > > So the situation here is that we are trying to bulk load data in to a
> > > > table. But each load of data has such range of keys that it will go
> to
> > a
> > > > specific continuous chunk of the region servers.
> > > >
> > > > In other other words, at each bulk load, we face hot-spotting but not
> > at
> > > > the end like the conventional case but it can be any where in between
> > the
> > > > row-key range of our table.
> > > >
> > > > Please note that the split point that I am trying to split on does
> not
> > > > exist in the table yet. I am trying to prepare the existing table
> with
> > > > data, by splitting into regions into which I will then bulk import my
> > new
> > > > data, to avoid hotspotting on one region server.
> > > >
> > > > The proof-of-concept code is below. Trying to split data into 16
> > regions
> > > > ('0' to 'f' of the guid since each row in this current load shares
> the
> > > same
> > > > value for the first 2 fields of the row key).
> > > >
> > > > Key is:
> > > > data_source + time-in-long + 32-bytes-random-guid
> > > >
> > > > /*****/
> > > >
> > > > byte[][] splits = new byte[16][];
> > > > byte[] dataSourceId = Bytes.toBytes(dataSource.getDataSourceID());
> > > > byte[] loadTime = Bytes.toBytes(batchLoadTime);
> > > > byte[] guidPrefix = null;
> > > >
> > > >   for(int i=0; i<splitPointsPrefixes.length; i++)  {
> > > >
> > > >    guidPrefix = Bytes.toBytes(splitPointsPrefixes[i]);
> > > >    splits[i] = new byte[dataSourceId.length + loadTime.length +
> > > guidPrefix.
> > > > length];
> > > >    ByteBuffer splitBuffer = ByteBuffer.wrap(splits[i]);
> > > >    splitBuffer.put(dataSourceId);
> > > >    splitBuffer.put(loadTime);
> > > >    splitBuffer.put(guidPrefix);
> > > > }
> > > >
> > > > byte[] tableNameInBytes = Bytes.toBytes(tableName);
> > > > HBaseAdmin admin = new
> > HBaseAdmin(HBaseConfiguration.create(getConf()));
> > > >
> > > > for(byte[] split : splits)  {
> > > >    //This is asynchronous. Should I wait here after each split to
> move
> > > onto
> > > > next one?
> > > >    admin.split(tableNameInBytes, split);
> > > > }
> > > > /*****/
> > > >
> > > > Regards,
> > > > Shahab
> > > >
> > > >
> > > > On Tue, Aug 19, 2014 at 4:13 PM, Jean-Marc Spaggiari <
> > > > jean-marc@spaggiari.org> wrote:
> > > >
> > > > > Hi Shahab,
> > > > >
> > > > > can you sahre your code? Seems that the RS you reached did not have
> > the
> > > > > expected region. How is your table status in the web interface?
> > > > >
> > > > > JM
> > > > >
> > > > >
> > > > > 2014-08-19 16:11 GMT-04:00 Shahab Yunus <shahab.yunus@gmail.com>:
> > > > >
> > > > > > I have a table already created and with some data. I want to
> split
> > it
> > > > > > trough code using HBaseAdmin api into multiple regions, while
> > > > specifying
> > > > > > keys that do not exist in the table.
> > > > > >
> > > > > > I am getting the exception below which makes sense because the
> key
> > > > > doesn't
> > > > > > exist yet. But at the time of creation of the table we can indeed
> > > > > pre-split
> > > > > > it using keys that don't exist.
> > > > > >
> > > > > > Is it possible to do it for table that already exists and has
> data?
> > > > > >
> > > > > > *Caused by:
> > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
> org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException):
> > > > > > org.apache.hadoop.hbase.NotServingRegionException: *
> > > > > >
> > > > > >
> > > > > > Using Hbase: 0.98.1-cdh5.1.0
> > > > > >
> > > > > > Thanks a lot.
> > > > > >
> > > > > > Regards,
> > > > > > Shahab
> > > > > >
> > > > >
> > > >
> > >
> >
>

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