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.HConstants;
31  import org.apache.hadoop.hbase.client.Delete;
32  import org.apache.hadoop.hbase.client.HTable;
33  import org.apache.hadoop.hbase.client.Mutation;
34  import org.apache.hadoop.hbase.client.Put;
35  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
36  import org.apache.hadoop.mapreduce.JobContext;
37  import org.apache.hadoop.mapreduce.OutputCommitter;
38  import org.apache.hadoop.mapreduce.OutputFormat;
39  import org.apache.hadoop.mapreduce.RecordWriter;
40  import org.apache.hadoop.mapreduce.TaskAttemptContext;
41  
42  /**
43   * Convert Map/Reduce output and write it to an HBase table. The KEY is ignored
44   * while the output value <u>must</u> be either a {@link Put} or a
45   * {@link Delete} instance.
46   *
47   * @param <KEY>  The type of the key. Ignored in this class.
48   */
49  @InterfaceAudience.Public
50  @InterfaceStability.Stable
51  public class TableOutputFormat<KEY> extends OutputFormat<KEY, Mutation>
52  implements Configurable {
53  
54    private final Log LOG = LogFactory.getLog(TableOutputFormat.class);
55  
56    /** Job parameter that specifies the output table. */
57    public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
58  
59    /**
60     * Optional job parameter to specify a peer cluster.
61     * Used specifying remote cluster when copying between hbase clusters (the
62     * source is picked up from <code>hbase-site.xml</code>).
63     * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, Class, String, String, String)
64     */
65    public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum";
66  
67    /** Optional job parameter to specify peer cluster's ZK client port */
68    public static final String QUORUM_PORT = "hbase.mapred.output.quorum.port";
69  
70    /** Optional specification of the rs class name of the peer cluster */
71    public static final String
72        REGION_SERVER_CLASS = "hbase.mapred.output.rs.class";
73    /** Optional specification of the rs impl name of the peer cluster */
74    public static final String
75        REGION_SERVER_IMPL = "hbase.mapred.output.rs.impl";
76  
77    /** The configuration. */
78    private Configuration conf = null;
79  
80    private HTable table;
81  
82    /**
83     * Writes the reducer output to an HBase table.
84     *
85     * @param <KEY>  The type of the key.
86     */
87    protected static class TableRecordWriter<KEY>
88    extends RecordWriter<KEY, Mutation> {
89  
90      /** The table to write to. */
91      private HTable table;
92  
93      /**
94       * Instantiate a TableRecordWriter with the HBase HClient for writing.
95       *
96       * @param table  The table to write to.
97       */
98      public TableRecordWriter(HTable table) {
99        this.table = table;
100     }
101 
102     /**
103      * Closes the writer, in this case flush table commits.
104      *
105      * @param context  The context.
106      * @throws IOException When closing the writer fails.
107      * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext)
108      */
109     @Override
110     public void close(TaskAttemptContext context)
111     throws IOException {
112       table.close();
113     }
114 
115     /**
116      * Writes a key/value pair into the table.
117      *
118      * @param key  The key.
119      * @param value  The value.
120      * @throws IOException When writing fails.
121      * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object)
122      */
123     @Override
124     public void write(KEY key, Mutation value)
125     throws IOException {
126       if (value instanceof Put) this.table.put(new Put((Put)value));
127       else if (value instanceof Delete) this.table.delete(new Delete((Delete)value));
128       else throw new IOException("Pass a Delete or a Put");
129     }
130   }
131 
132   /**
133    * Creates a new record writer.
134    *
135    * @param context  The current task context.
136    * @return The newly created writer instance.
137    * @throws IOException When creating the writer fails.
138    * @throws InterruptedException When the jobs is cancelled.
139    * @see org.apache.hadoop.mapreduce.lib.output.FileOutputFormat#getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext)
140    */
141   @Override
142   public RecordWriter<KEY, Mutation> getRecordWriter(
143     TaskAttemptContext context)
144   throws IOException, InterruptedException {
145     return new TableRecordWriter<KEY>(this.table);
146   }
147 
148   /**
149    * Checks if the output target exists.
150    *
151    * @param context  The current context.
152    * @throws IOException When the check fails.
153    * @throws InterruptedException When the job is aborted.
154    * @see org.apache.hadoop.mapreduce.OutputFormat#checkOutputSpecs(org.apache.hadoop.mapreduce.JobContext)
155    */
156   @Override
157   public void checkOutputSpecs(JobContext context) throws IOException,
158       InterruptedException {
159     // TODO Check if the table exists?
160 
161   }
162 
163   /**
164    * Returns the output committer.
165    *
166    * @param context  The current context.
167    * @return The committer.
168    * @throws IOException When creating the committer fails.
169    * @throws InterruptedException When the job is aborted.
170    * @see org.apache.hadoop.mapreduce.OutputFormat#getOutputCommitter(org.apache.hadoop.mapreduce.TaskAttemptContext)
171    */
172   @Override
173   public OutputCommitter getOutputCommitter(TaskAttemptContext context)
174   throws IOException, InterruptedException {
175     return new TableOutputCommitter();
176   }
177 
178   public Configuration getConf() {
179     return conf;
180   }
181 
182   @Override
183   public void setConf(Configuration otherConf) {
184     this.conf = HBaseConfiguration.create(otherConf);
185 
186     String tableName = this.conf.get(OUTPUT_TABLE);
187     if(tableName == null || tableName.length() <= 0) {
188       throw new IllegalArgumentException("Must specify table name");
189     }
190 
191     String address = this.conf.get(QUORUM_ADDRESS);
192     int zkClientPort = this.conf.getInt(QUORUM_PORT, 0);
193     String serverClass = this.conf.get(REGION_SERVER_CLASS);
194     String serverImpl = this.conf.get(REGION_SERVER_IMPL);
195 
196     try {
197       if (address != null) {
198         ZKUtil.applyClusterKeyToConf(this.conf, address);
199       }
200       if (serverClass != null) {
201         this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl);
202       }
203       if (zkClientPort != 0) {
204         this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort);
205       }
206       this.table = new HTable(this.conf, tableName);
207       this.table.setAutoFlush(false, true);
208       LOG.info("Created table instance for "  + tableName);
209     } catch(IOException e) {
210       LOG.error(e);
211       throw new RuntimeException(e);
212     }
213   }
214 }