Hi Jianshi, Is there any reason why you need to split dynamically the table? Users usually pre-split their tables with a specific number of splits or they pick a region split policy that fits their needs:
https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/DelimitedKeyPrefixRegionSplitPolicy.html https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/ConstantSizeRegionSplitPolicy.html https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/IncreasingToUpperBoundRegionSplitPolicy.html https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/KeyPrefixRegionSplitPolicy.html https://hbase.apache.org/apidocs/org/apache/hadoop/hbase/regionserver/DisabledRegionSplitPolicy.html or they have the options to implement their own. See for some details http://hbase.apache.org/book/regions.arch.html#arch.region.split cheers, esteban. -- Cloudera, Inc. On Wed, Sep 17, 2014 at 5:06 AM, Shahab Yunus <[email protected]> wrote: > Split is an async operation. When you call it, and the call returns, it > does not mean that the region has been created yet. > > So either you wait for a while (using Thread.sleep) or check for the number > of regions in a loop and until they have increased to the value you want > and then access the region. The former is not a good idea, though you can > try it out just to make sure that this is indeed the issue. > > What am I suggesting is something like (pseudo code): > > while(new#regions > old#regions) > { > new#regions = admin.getLatest#regions > } > > Regards, > Shahab > > On Wed, Sep 17, 2014 at 5:39 AM, Jianshi Huang <[email protected]> > wrote: > > > I constantly get the following errors when I tried to add splits to a > > table. > > > > > > > org.apache.hadoop.hbase.ipc.RemoteWithExtrasException(org.apache.hadoop.hbase.NotServingRegionException): > > org.apache.hadoop.hbase.NotServingRegionException: Region > > > grapple_vertices,cust|rval#7ffffeb7cffca280|1636500018299676757,1410945568 > > 484.e7743495366df3c82a8571b36c2bdac3. is not online on > > lvshdc5dn0193.lvs.paypal.com,60020,1405014719359 > > at > > > > > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegionByEncodedName(HRegionServer.java:2676) > > at > > > > > org.apache.hadoop.hbase.regionserver.HRegionServer.getRegion(HRegionServer.java:4095) > > at > > > > > org.apache.hadoop.hbase.regionserver.HRegionServer.splitRegion(HRegionServer.java:3818) > > at > > > > > > But when I checked the region server (from hbase' webUI), the region is > > actually listed there. > > > > What does the error mean actually? How can I solve it? > > > > Currently I'm adding splits single-threaded, and I want to make it > > parallel, is there anything I need to be careful about? > > > > Here's the code for adding splits: > > > > def addSplits(tableName: String, splitKeys: Seq[Array[Byte]]): Unit = { > > val admin = new HBaseAdmin(conn) > > > > try { > > val regions = admin.getTableRegions(tableName.getBytes("UTF8")) > > val regionStartKeys = regions.map(_.getStartKey) > > val splits = splitKeys.diff(regionStartKeys) > > > > splits.foreach { splitPoint => > > admin.split(tableName.getBytes("UTF8"), splitPoint) > > } > > // NOTE: important! > > admin.balancer() > > } > > finally { > > admin.close() > > } > > } > > > > > > Any help is appreciated. > > > > -- > > Jianshi Huang > > > > LinkedIn: jianshi > > Twitter: @jshuang > > Github & Blog: http://huangjs.github.com/ > > >
