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