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