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