1 /** 2 * Copyright 2008 The Apache Software Foundation 3 * 4 * Licensed to the Apache Software Foundation (ASF) under one 5 * or more contributor license agreements. See the NOTICE file 6 * distributed with this work for additional information 7 * regarding copyright ownership. The ASF licenses this file 8 * to you under the Apache License, Version 2.0 (the 9 * "License"); you may not use this file except in compliance 10 * with the License. You may obtain a copy of the License at 11 * 12 * http://www.apache.org/licenses/LICENSE-2.0 13 * 14 * Unless required by applicable law or agreed to in writing, software 15 * distributed under the License is distributed on an "AS IS" BASIS, 16 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 * See the License for the specific language governing permissions and 18 * limitations under the License. 19 */ 20 package org.apache.hadoop.hbase.mapreduce; 21 22 import java.io.IOException; 23 24 import org.apache.commons.logging.Log; 25 import org.apache.commons.logging.LogFactory; 26 import org.apache.hadoop.conf.Configurable; 27 import org.apache.hadoop.conf.Configuration; 28 import org.apache.hadoop.hbase.HBaseConfiguration; 29 import org.apache.hadoop.hbase.client.HTable; 30 import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 31 import org.apache.hadoop.hbase.util.Bytes; 32 import org.apache.hadoop.mapreduce.Partitioner; 33 34 /** 35 * This is used to partition the output keys into groups of keys. 36 * Keys are grouped according to the regions that currently exist 37 * so that each reducer fills a single region so load is distributed. 38 * 39 * @param <KEY> The type of the key. 40 * @param <VALUE> The type of the value. 41 */ 42 public class HRegionPartitioner<KEY, VALUE> 43 extends Partitioner<ImmutableBytesWritable, VALUE> 44 implements Configurable { 45 46 private final Log LOG = LogFactory.getLog(TableInputFormat.class); 47 private Configuration conf = null; 48 private HTable table; 49 private byte[][] startKeys; 50 51 /** 52 * Gets the partition number for a given key (hence record) given the total 53 * number of partitions i.e. number of reduce-tasks for the job. 54 * 55 * <p>Typically a hash function on a all or a subset of the key.</p> 56 * 57 * @param key The key to be partitioned. 58 * @param value The entry value. 59 * @param numPartitions The total number of partitions. 60 * @return The partition number for the <code>key</code>. 61 * @see org.apache.hadoop.mapreduce.Partitioner#getPartition( 62 * java.lang.Object, java.lang.Object, int) 63 */ 64 @Override 65 public int getPartition(ImmutableBytesWritable key, 66 VALUE value, int numPartitions) { 67 byte[] region = null; 68 // Only one region return 0 69 if (this.startKeys.length == 1){ 70 return 0; 71 } 72 try { 73 // Not sure if this is cached after a split so we could have problems 74 // here if a region splits while mapping 75 region = table.getRegionLocation(key.get()).getRegionInfo().getStartKey(); 76 } catch (IOException e) { 77 LOG.error(e); 78 } 79 for (int i = 0; i < this.startKeys.length; i++){ 80 if (Bytes.compareTo(region, this.startKeys[i]) == 0 ){ 81 if (i >= numPartitions-1){ 82 // cover if we have less reduces then regions. 83 return (Integer.toString(i).hashCode() 84 & Integer.MAX_VALUE) % numPartitions; 85 } 86 return i; 87 } 88 } 89 // if above fails to find start key that match we need to return something 90 return 0; 91 } 92 93 /** 94 * Returns the current configuration. 95 * 96 * @return The current configuration. 97 * @see org.apache.hadoop.conf.Configurable#getConf() 98 */ 99 @Override 100 public Configuration getConf() { 101 return conf; 102 } 103 104 /** 105 * Sets the configuration. This is used to determine the start keys for the 106 * given table. 107 * 108 * @param configuration The configuration to set. 109 * @see org.apache.hadoop.conf.Configurable#setConf( 110 * org.apache.hadoop.conf.Configuration) 111 */ 112 @Override 113 public void setConf(Configuration configuration) { 114 this.conf = configuration; 115 try { 116 HBaseConfiguration.addHbaseResources(conf); 117 this.table = new HTable(this.conf, 118 configuration.get(TableOutputFormat.OUTPUT_TABLE)); 119 } catch (IOException e) { 120 LOG.error(e); 121 } 122 try { 123 this.startKeys = this.table.getStartKeys(); 124 } catch (IOException e) { 125 LOG.error(e); 126 } 127 } 128 }