You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2013/03/26 16:04:07 UTC

svn commit: r1461164 - in /hbase/branches/0.95/bin: get-active-master.rb region_mover.rb region_status.rb

Author: stack
Date: Tue Mar 26 15:04:07 2013
New Revision: 1461164

URL: http://svn.apache.org/r1461164
Log:
HBASE-8172 Fix ./bin/*.rb scripts or remove them

Modified:
    hbase/branches/0.95/bin/get-active-master.rb
    hbase/branches/0.95/bin/region_mover.rb
    hbase/branches/0.95/bin/region_status.rb

Modified: hbase/branches/0.95/bin/get-active-master.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/bin/get-active-master.rb?rev=1461164&r1=1461163&r2=1461164&view=diff
==============================================================================
--- hbase/branches/0.95/bin/get-active-master.rb (original)
+++ hbase/branches/0.95/bin/get-active-master.rb Tue Mar 26 15:04:07 2013
@@ -34,7 +34,7 @@ zk = ZooKeeperWatcher.new(config, 'get-a
 begin
   master_address = ZKUtil.getData(zk, zk.masterAddressZNode)
   if master_address
-    puts ServerName.parseVersionedServerName(master_address).getHostname()
+    puts ServerName.parseFrom(master_address).getHostname()
   else
     puts 'Master not running'
   end

Modified: hbase/branches/0.95/bin/region_mover.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/bin/region_mover.rb?rev=1461164&r1=1461163&r2=1461164&view=diff
==============================================================================
--- hbase/branches/0.95/bin/region_mover.rb (original)
+++ hbase/branches/0.95/bin/region_mover.rb Tue Mar 26 15:04:07 2013
@@ -32,25 +32,18 @@ import org.apache.hadoop.hbase.client.Sc
 import org.apache.hadoop.hbase.client.HTable
 import org.apache.hadoop.hbase.client.HConnectionManager
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
-import org.apache.hadoop.hbase.HServerAddress
 import org.apache.hadoop.hbase.util.Bytes
 import org.apache.hadoop.hbase.util.Writables
 import org.apache.hadoop.conf.Configuration
 import org.apache.commons.logging.Log
 import org.apache.commons.logging.LogFactory
+import org.apache.hadoop.hbase.protobuf.ProtobufUtil
+import org.apache.hadoop.hbase.ServerName
+import org.apache.hadoop.hbase.HRegionInfo
 
 # Name of this script
 NAME = "region_mover"
 
-# Get root table reference
-def getRootTable(config)
-  # Keep meta reference in ruby global
-  if not $ROOT
-    $ROOT = HTable.new(config, HConstants::ROOT_TABLE_NAME)
-  end
-  return $ROOT
-end
-
 # Get meta table reference
 def getMetaTable(config)
   # Keep meta reference in ruby global
@@ -93,24 +86,20 @@ end
 # Get servername that is up in .META.; this is hostname + port + startcode comma-delimited.
 # Can return nil
 def getServerNameForRegion(admin, r)
-  if r.isRootRegion()
+  if r.isMetaRegion()
     # Hack
-    tracker = org.apache.hadoop.hbase.zookeeper.RootRegionTracker.new(admin.getConnection().getZooKeeperWatcher(), RubyAbortable.new())
+    tracker = org.apache.hadoop.hbase.zookeeper.MetaRegionTracker.new(admin.getConnection().getZooKeeperWatcher(), RubyAbortable.new())
     tracker.start()
     while not tracker.isLocationAvailable()
       sleep 0.1
     end
     # Make a fake servername by appending ','
-    rootServer = tracker.getRootRegionLocation().toString() + ","
+    metaServer = tracker.getMetaRegionLocation().toString() + ","
     tracker.stop()
-    return rootServer
+    return metaServer
   end
   table = nil
-  if r.isMetaRegion()
-    table = getRootTable(admin.getConfiguration()) 
-  else
-    table = getMetaTable(admin.getConfiguration())
-  end
+  table = getMetaTable(admin.getConfiguration())
   g = Get.new(r.getRegionName())
   g.addColumn(HConstants::CATALOG_FAMILY, HConstants::SERVER_QUALIFIER)
   g.addColumn(HConstants::CATALOG_FAMILY, HConstants::STARTCODE_QUALIFIER)
@@ -259,10 +248,8 @@ end
 
 # Now get list of regions on targetServer
 def getRegions(config, servername)
-  connection = HConnectionManager::getConnection(config)
-  parts = servername.split(',')
-  rs = connection.getHRegionConnection(parts[0], parts[1].to_i)
-  return rs.getOnlineRegions()
+  connection = HConnectionManager::getConnection(config);
+  return ProtobufUtil::getOnlineRegions(connection.getAdmin(ServerName.new(servername)));
 end
 
 def deleteFile(filename)
@@ -280,8 +267,7 @@ def writeFile(filename, regions)
   dos.writeInt(regions.size())
   # Write actual region names.
   for r in regions
-    bytes = Writables.getBytes(r)
-    Bytes.writeByteArray(dos, bytes)
+    Bytes.writeByteArray(dos, r.toByteArray())
   end
   dos.close()
 end
@@ -298,7 +284,7 @@ def readFile(filename)
   regions = java.util.ArrayList.new(count)
   index = 0
   while index < count
-    regions.add(Writables.getHRegionInfo(Bytes.readByteArray(dis)))
+    regions.add(HRegionInfo.parseFromOrNull(Bytes.readByteArray(dis)))
     index = index + 1
   end
   dis.close()

Modified: hbase/branches/0.95/bin/region_status.rb
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/bin/region_status.rb?rev=1461164&r1=1461163&r2=1461164&view=diff
==============================================================================
--- hbase/branches/0.95/bin/region_status.rb (original)
+++ hbase/branches/0.95/bin/region_status.rb Tue Mar 26 15:04:07 2013
@@ -53,7 +53,8 @@ import org.apache.hadoop.hbase.client.HT
 import org.apache.hadoop.hbase.client.Scan
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter
 import org.apache.hadoop.hbase.util.Bytes
-import org.apache.hadoop.hbase.util.Writables
+import org.apache.hadoop.hbase.HRegionInfo
+import org.apache.hadoop.hbase.client.MetaScanner
 
 # disable debug logging on this script for clarity
 log_level = org.apache.log4j.Level::ERROR
@@ -114,7 +115,7 @@ while iter.hasNext
     # Gone too far, break
     break
   end
-  region = Writables.getHRegionInfo result.getValue(INFO, REGION_INFO)
+  region = HRegionInfo.getHRegionInfo result
   if not region.isOffline
     # only include regions that should be online
     meta_count += 1
@@ -122,9 +123,9 @@ while iter.hasNext
 end
 scanner.close
 # If we're trying to see the status of all HBase tables, we need to include the
-# -ROOT- & .META. tables, that are not included in our scan
+# .META. table, that is not included in our scan
 if $tablename.nil?
-  meta_count += 2
+  meta_count += 1
 end
 
 # query the master to see how many regions are on region servers
@@ -135,7 +136,7 @@ while true
   if $tablename.nil?
     server_count = admin.getClusterStatus().getRegionsCount()
   else
-    server_count = $tableq.getRegionsInfo().size()
+    server_count = MetaScanner::allTableRegions(config,$tablename.to_java_bytes,false).size()
   end
   print "Region Status: #{server_count} / #{meta_count}\n"
   if SHOULD_WAIT and server_count < meta_count