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.classification.InterfaceAudience;
26  import org.apache.hadoop.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.client.HTable;
31  import org.apache.hadoop.hbase.client.RegionLocator;
32  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33  import org.apache.hadoop.hbase.util.Bytes;
34  import org.apache.hadoop.mapreduce.Partitioner;
35  
36  /**
37   * This is used to partition the output keys into groups of keys.
38   * Keys are grouped according to the regions that currently exist
39   * so that each reducer fills a single region so load is distributed.
40   *
41   * <p>This class is not suitable as partitioner creating hfiles
42   * for incremental bulk loads as region spread will likely change between time of
43   * hfile creation and load time. See {@link LoadIncrementalHFiles}
44   * and <a href="http://hbase.apache.org/docs/current/bulk-loads.html">Bulk Load</a>.
45   *
46   * @param <KEY>  The type of the key.
47   * @param <VALUE>  The type of the value.
48   */
49  @InterfaceAudience.Public
50  @InterfaceStability.Stable
51  public class HRegionPartitioner<KEY, VALUE>
52  extends Partitioner<ImmutableBytesWritable, VALUE>
53  implements Configurable {
54  
55    private static final Log LOG = LogFactory.getLog(HRegionPartitioner.class);
56    private Configuration conf = null;
57    private RegionLocator table;
58    private byte[][] startKeys;
59  
60    /**
61     * Gets the partition number for a given key (hence record) given the total
62     * number of partitions i.e. number of reduce-tasks for the job.
63     *
64     * <p>Typically a hash function on a all or a subset of the key.</p>
65     *
66     * @param key  The key to be partitioned.
67     * @param value  The entry value.
68     * @param numPartitions  The total number of partitions.
69     * @return The partition number for the <code>key</code>.
70     * @see org.apache.hadoop.mapreduce.Partitioner#getPartition(
71     *   java.lang.Object, java.lang.Object, int)
72     */
73    @Override
74    public int getPartition(ImmutableBytesWritable key,
75        VALUE value, int numPartitions) {
76      byte[] region = null;
77      // Only one region return 0
78      if (this.startKeys.length == 1){
79        return 0;
80      }
81      try {
82        // Not sure if this is cached after a split so we could have problems
83        // here if a region splits while mapping
84        region = table.getRegionLocation(key.get()).getRegionInfo().getStartKey();
85      } catch (IOException e) {
86        LOG.error(e);
87      }
88      for (int i = 0; i < this.startKeys.length; i++){
89        if (Bytes.compareTo(region, this.startKeys[i]) == 0 ){
90          if (i >= numPartitions-1){
91            // cover if we have less reduces then regions.
92            return (Integer.toString(i).hashCode()
93                & Integer.MAX_VALUE) % numPartitions;
94          }
95          return i;
96        }
97      }
98      // if above fails to find start key that match we need to return something
99      return 0;
100   }
101 
102   /**
103    * Returns the current configuration.
104    *
105    * @return The current configuration.
106    * @see org.apache.hadoop.conf.Configurable#getConf()
107    */
108   @Override
109   public Configuration getConf() {
110     return conf;
111   }
112 
113   /**
114    * Sets the configuration. This is used to determine the start keys for the
115    * given table.
116    *
117    * @param configuration  The configuration to set.
118    * @see org.apache.hadoop.conf.Configurable#setConf(
119    *   org.apache.hadoop.conf.Configuration)
120    */
121   @Override
122   public void setConf(Configuration configuration) {
123     this.conf = HBaseConfiguration.create(configuration);
124     try {
125       this.table = new HTable(this.conf,
126         configuration.get(TableOutputFormat.OUTPUT_TABLE));
127     } catch (IOException e) {
128       LOG.error(e);
129     }
130     try {
131       this.startKeys = this.table.getStartKeys();
132     } catch (IOException e) {
133       LOG.error(e);
134     }
135   }
136 }