View Javadoc

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