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 import java.util.ArrayList;
23
24 import org.apache.hadoop.classification.InterfaceAudience;
25 import org.apache.hadoop.classification.InterfaceStability;
26 import org.apache.hadoop.conf.Configurable;
27 import org.apache.hadoop.conf.Configuration;
28 import org.apache.hadoop.hbase.KeyValue;
29 import org.apache.hadoop.hbase.client.Result;
30 import org.apache.hadoop.hbase.client.Scan;
31 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
32 import org.apache.hadoop.hbase.util.Bytes;
33 import org.apache.hadoop.mapreduce.Job;
34
35 /**
36 * Extract grouping columns from input record.
37 */
38 @InterfaceAudience.Public
39 @InterfaceStability.Stable
40 public class GroupingTableMapper
41 extends TableMapper<ImmutableBytesWritable,Result> implements Configurable {
42
43 /**
44 * JobConf parameter to specify the columns used to produce the key passed to
45 * collect from the map phase.
46 */
47 public static final String GROUP_COLUMNS =
48 "hbase.mapred.groupingtablemap.columns";
49
50 /** The grouping columns. */
51 protected byte [][] columns;
52 /** The current configuration. */
53 private Configuration conf = null;
54
55 /**
56 * Use this before submitting a TableMap job. It will appropriately set up
57 * the job.
58 *
59 * @param table The table to be processed.
60 * @param scan The scan with the columns etc.
61 * @param groupColumns A space separated list of columns used to form the
62 * key used in collect.
63 * @param mapper The mapper class.
64 * @param job The current job.
65 * @throws IOException When setting up the job fails.
66 */
67 @SuppressWarnings("unchecked")
68 public static void initJob(String table, Scan scan, String groupColumns,
69 Class<? extends TableMapper> mapper, Job job) throws IOException {
70 TableMapReduceUtil.initTableMapperJob(table, scan, mapper,
71 ImmutableBytesWritable.class, Result.class, job);
72 job.getConfiguration().set(GROUP_COLUMNS, groupColumns);
73 }
74
75 /**
76 * Extract the grouping columns from value to construct a new key. Pass the
77 * new key and value to reduce. If any of the grouping columns are not found
78 * in the value, the record is skipped.
79 *
80 * @param key The current key.
81 * @param value The current value.
82 * @param context The current context.
83 * @throws IOException When writing the record fails.
84 * @throws InterruptedException When the job is aborted.
85 */
86 @Override
87 public void map(ImmutableBytesWritable key, Result value, Context context)
88 throws IOException, InterruptedException {
89 byte[][] keyVals = extractKeyValues(value);
90 if(keyVals != null) {
91 ImmutableBytesWritable tKey = createGroupKey(keyVals);
92 context.write(tKey, value);
93 }
94 }
95
96 /**
97 * Extract columns values from the current record. This method returns
98 * null if any of the columns are not found.
99 * <p>
100 * Override this method if you want to deal with nulls differently.
101 *
102 * @param r The current values.
103 * @return Array of byte values.
104 */
105 protected byte[][] extractKeyValues(Result r) {
106 byte[][] keyVals = null;
107 ArrayList<byte[]> foundList = new ArrayList<byte[]>();
108 int numCols = columns.length;
109 if (numCols > 0) {
110 for (KeyValue value: r.list()) {
111 byte [] column = KeyValue.makeColumn(value.getFamily(),
112 value.getQualifier());
113 for (int i = 0; i < numCols; i++) {
114 if (Bytes.equals(column, columns[i])) {
115 foundList.add(value.getValue());
116 break;
117 }
118 }
119 }
120 if(foundList.size() == numCols) {
121 keyVals = foundList.toArray(new byte[numCols][]);
122 }
123 }
124 return keyVals;
125 }
126
127 /**
128 * Create a key by concatenating multiple column values.
129 * <p>
130 * Override this function in order to produce different types of keys.
131 *
132 * @param vals The current key/values.
133 * @return A key generated by concatenating multiple column values.
134 */
135 protected ImmutableBytesWritable createGroupKey(byte[][] vals) {
136 if(vals == null) {
137 return null;
138 }
139 StringBuilder sb = new StringBuilder();
140 for(int i = 0; i < vals.length; i++) {
141 if(i > 0) {
142 sb.append(" ");
143 }
144 sb.append(Bytes.toString(vals[i]));
145 }
146 return new ImmutableBytesWritable(Bytes.toBytes(sb.toString()));
147 }
148
149 /**
150 * Returns the current configuration.
151 *
152 * @return The current configuration.
153 * @see org.apache.hadoop.conf.Configurable#getConf()
154 */
155 @Override
156 public Configuration getConf() {
157 return conf;
158 }
159
160 /**
161 * Sets the configuration. This is used to set up the grouping details.
162 *
163 * @param configuration The configuration to set.
164 * @see org.apache.hadoop.conf.Configurable#setConf(
165 * org.apache.hadoop.conf.Configuration)
166 */
167 @Override
168 public void setConf(Configuration configuration) {
169 this.conf = configuration;
170 String[] cols = conf.get(GROUP_COLUMNS, "").split(" ");
171 columns = new byte[cols.length][];
172 for(int i = 0; i < cols.length; i++) {
173 columns[i] = Bytes.toBytes(cols[i]);
174 }
175 }
176
177 }