Merging Regions in HBase

So, like with any job using physical hardware, getting new machines for your growing Hadoop cluster takes time. And sometimes, the amount of ingest starts putting pressure on components before new hosts can be added. Today, our HBase hosts are edging dangerously close to >=200 regions per host. The HBase Book suggests 200+ regions per host is bad.

Okay, so we’re edging closer and closer to worse and worse performance as we add data (and regions naturally split) without adding hosts. While there isn’t much we can do to reduce our ingest rate (in fact, we only want to increase it), we can try to reduce the current region count. HBase can set TTLs for tables easily, and for our primary OpenTSDB table, we have. In fact, data is already aging out of our primary table! So we have some empty regions. Prime candidates for clean up, which should take the pressure down.

Digging around the internet for other people who have worked on similar problems with HBase, I found the following: http://appsintheopen.com/posts/51-merge-empty-hbase-regions. Great! We can merge regions instead of closing them! We’ll skip the compaction queue that way and reduce our size faster! The problems:

  1. The script was written before HBase 2.0 came out, so we have some cleanup to do around deprecated/removed commands.
  2. It only merges empty regions together, which doesn’t actually solve the problem of removing all empty regions.

So okay, let’s write some JRuby and fix these problems. The following tool cleaned up ~25 empty regions for me in a minute or so:

# Test Mode:
#
# hbase org.jruby.Main merge_empty_regions.rb namespace.tablename
#
# Non Test - ie actually do the merge:
#
# hbase org.jruby.Main merge_empty_regions.rb namespace.tablename merge
#
# Note: Please replace namespace.tablename with your namespace and table, eg NS1.MyTable. This value is case sensitive.

require 'digest'
require 'java'
java_import org.apache.hadoop.hbase.HBaseConfiguration
java_import org.apache.hadoop.hbase.client.HBaseAdmin
java_import org.apache.hadoop.hbase.TableName
java_import org.apache.hadoop.hbase.client.RegionInfo
java_import org.apache.hadoop.hbase.client.Connection
java_import org.apache.hadoop.hbase.client.ConnectionFactory
java_import org.apache.hadoop.hbase.client.Table
java_import org.apache.hadoop.hbase.util.Bytes

def list_non_empty_regions(admin, table)
  #cluster_status = admin.getClusterMetrics()
  non_empty = []
  admin.getRegionServers.each do |s|
    admin.getRegionMetrics(s, table).each do |r|
      #next unless r.get_name_as_string =~ /#{table}\,/
      if r.getStoreFileSize().get() > 0
        if r.get_name_as_string =~ /\.([^\.]+)\.$/
          non_empty.push $1
        else
          raise "Failed to get the encoded name for #{r.get_name_as_string}"
        end
      end
    end
  end
  non_empty
end

# Handle command line parameters
table_name = ARGV[0]
real_table_name = TableName.valueOf(table_name.split(".")[1..-1].join("."))
do_merge_single = false
do_merge = false
if ARGV[1] == 'merge'
  do_merge = true
end

config = HBaseConfiguration.create()
connection = ConnectionFactory.createConnection(config)
admin = connection.getAdmin()

non_empty_regions = list_non_empty_regions(admin, real_table_name)
regions = []
admin.getRegionServers.each do |s|
  admin.getRegions(s).each do |r|
    next unless r.get_region_name_as_string =~ /#{real_table_name}\,/
    regions.push r
  end
end

puts "Total Table Regions: #{regions.length}"
puts "Total non empty regions: #{non_empty_regions.length}"

filtered_regions = regions.reject do |r|
  non_empty_regions.include?(r.get_encoded_name)
end

puts "Total regions to consider for Merge: #{filtered_regions.length}"

if do_merge
  if filtered_regions.length < 1
    puts "There are not enough regions to merge"
  end
  filtered_regions.each do |r|
    if r.nil? || r.is_split()
      next
    end
    regions.each do |nearby|
      if nearby.nil? || nearby.is_split()
        next
      end
      if RegionInfo.are_adjacent(r, nearby)
        puts "#{r.get_encoded_name} is adjacent to #{nearby.get_encoded_name}, merging"
        admin.mergeRegions(r.getEncodedNameAsBytes, nearby.getEncodedNameAsBytes, false)
        puts "Successfully Merged #{r.get_encoded_name} with #{nearby.get_encoded_name}"
        sleep 2
        break
      end
    end
  end
end
admin.close