hbase-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Sean Busbey <bus...@cloudera.com>
Subject Re: Splitting up an HBase Table into partitions
Date Tue, 17 Mar 2015 19:27:03 GMT
You should ask for a RegionLocator if you want to know the boundaries of
all the regions in a table


final Connection connection = ConnectionFactory.createConnection(config);

try {

  final RegionLocator locator =
connection.getRegionLocator(TableName.valueOf("myTable"));

  final Pair<byte[][], byte[][]> startEndKeys = locator.getStartEndKeys();

  final byte[][] startKeys = startEndKeys.getFirst();

  final byte[][] endKeys = startEndKeys.getSecond();

  for (int i=0; i < startKeys.length && i < endKeys.length; i++) {

     System.out.println("Region " + i + " starts at '" +
Bytes.toStringBinary(startKeys[i]) +

         "' and ends at '" + Bytes.toStringBinary(endKeys[i]));

  }

} finally {

  connection.close();

}


There are other methods in RegionLocator if you need other details.

On Tue, Mar 17, 2015 at 2:09 PM, Gokul Balakrishnan <royalgok@gmail.com>
wrote:

> Hi Michael,
>
> Thanks for the reply. Yes, I do realise that HBase has regions, perhaps my
> usage of the term partitions was misleading. What I'm looking for is
> exactly what you've mentioned - a means of creating splits based on
> regions, without having to iterate over all rows in the table through the
> client API. Do you have any idea how I might achieve this?
>
> Thanks,
>
> On Tuesday, March 17, 2015, Michael Segel <michael_segel@hotmail.com>
> wrote:
>
> > Hbase doesn't have partitions.  It has regions.
> >
> > The split occurs against the regions so that if you have n regions, you
> > have n splits.
> >
> > Please don't confuse partitions and regions because they are not the same
> > or synonymous.
> >
> > > On Mar 17, 2015, at 7:30 AM, Gokul Balakrishnan <royalgok@gmail.com
> > <javascript:;>> wrote:
> > >
> > > Hi,
> > >
> > > My requirement is to partition an HBase Table and return a group of
> > records
> > > (i.e. rows having a specific format) without having to iterate over all
> > of
> > > its rows. These partitions (which should ideally be along regions) will
> > > eventually be sent to Spark but rather than use the HBase or Hadoop
> RDDs
> > > directly, I'll be using a custom RDD which recognizes partitions as the
> > > aforementioned group of records.
> > >
> > > I was looking at achieving this through creating InputSplits through
> > > TableInputFormat.getSplits(), as being done in the HBase RDD [1] but I
> > > can't figure out a way to do this without having access to the mapred
> > > context etc.
> > >
> > > Would greatly appreciate if someone could point me in the right
> > direction.
> > >
> > > [1]
> > >
> >
> https://github.com/tmalaska/SparkOnHBase/blob/master/src/main/scala/com/cloudera/spark/hbase/HBaseScanRDD.scala
> > >
> > > Thanks,
> > > Gokul
> >
> > 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
> >
> >
> >
> >
> >
> >
>



-- 
Sean

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