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