View Javadoc

1   /**
2    * Copyright 2009 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  
24  import org.apache.commons.logging.Log;
25  import org.apache.commons.logging.LogFactory;
26  import org.apache.hadoop.conf.Configurable;
27  import org.apache.hadoop.conf.Configuration;
28  import org.apache.hadoop.hbase.HBaseConfiguration;
29  import org.apache.hadoop.hbase.HConstants;
30  import org.apache.hadoop.hbase.client.Delete;
31  import org.apache.hadoop.hbase.client.HConnectionManager;
32  import org.apache.hadoop.hbase.client.HTable;
33  import org.apache.hadoop.hbase.client.Put;
34  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
35  import org.apache.hadoop.io.Writable;
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  public class TableOutputFormat<KEY> extends OutputFormat<KEY, Writable>
50  implements Configurable {
51  
52    private final Log LOG = LogFactory.getLog(TableOutputFormat.class);
53  
54    /** Job parameter that specifies the output table. */
55    public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
56  
57    /**
58     * Optional job parameter to specify a peer cluster.
59     * Used specifying remote cluster when copying between hbase clusters (the
60     * source is picked up from <code>hbase-site.xml</code>).
61     * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, Class, String, String, String)
62     */
63    public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum";
64  
65    /** Optional specification of the rs class name of the peer cluster */
66    public static final String
67        REGION_SERVER_CLASS = "hbase.mapred.output.rs.class";
68    /** Optional specification of the rs impl name of the peer cluster */
69    public static final String
70        REGION_SERVER_IMPL = "hbase.mapred.output.rs.impl";
71  
72    /** The configuration. */
73    private Configuration conf = null;
74  
75    private HTable table;
76  
77    /**
78     * Writes the reducer output to an HBase table.
79     *
80     * @param <KEY>  The type of the key.
81     */
82    protected static class TableRecordWriter<KEY>
83    extends RecordWriter<KEY, Writable> {
84  
85      /** The table to write to. */
86      private HTable table;
87  
88      /**
89       * Instantiate a TableRecordWriter with the HBase HClient for writing.
90       *
91       * @param table  The table to write to.
92       */
93      public TableRecordWriter(HTable table) {
94        this.table = table;
95      }
96  
97      /**
98       * Closes the writer, in this case flush table commits.
99       *
100      * @param context  The context.
101      * @throws IOException When closing the writer fails.
102      * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext)
103      */
104     @Override
105     public void close(TaskAttemptContext context)
106     throws IOException {
107       table.flushCommits();
108       // The following call will shutdown all connections to the cluster from
109       // this JVM.  It will close out our zk session otherwise zk wil log
110       // expired sessions rather than closed ones.  If any other HTable instance
111       // running in this JVM, this next call will cause it damage.  Presumption
112       // is that the above this.table is only instance.
113       HConnectionManager.deleteAllConnections(true);
114     }
115 
116     /**
117      * Writes a key/value pair into the table.
118      *
119      * @param key  The key.
120      * @param value  The value.
121      * @throws IOException When writing fails.
122      * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object)
123      */
124     @Override
125     public void write(KEY key, Writable value)
126     throws IOException {
127       if (value instanceof Put) this.table.put(new Put((Put)value));
128       else if (value instanceof Delete) this.table.delete(new Delete((Delete)value));
129       else throw new IOException("Pass a Delete or a Put");
130     }
131   }
132 
133   /**
134    * Creates a new record writer.
135    *
136    * @param context  The current task context.
137    * @return The newly created writer instance.
138    * @throws IOException When creating the writer fails.
139    * @throws InterruptedException When the jobs is cancelled.
140    * @see org.apache.hadoop.mapreduce.lib.output.FileOutputFormat#getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext)
141    */
142   @Override
143   public RecordWriter<KEY, Writable> getRecordWriter(
144     TaskAttemptContext context)
145   throws IOException, InterruptedException {
146     return new TableRecordWriter<KEY>(this.table);
147   }
148 
149   /**
150    * Checks if the output target exists.
151    *
152    * @param context  The current context.
153    * @throws IOException When the check fails.
154    * @throws InterruptedException When the job is aborted.
155    * @see org.apache.hadoop.mapreduce.OutputFormat#checkOutputSpecs(org.apache.hadoop.mapreduce.JobContext)
156    */
157   @Override
158   public void checkOutputSpecs(JobContext context) throws IOException,
159       InterruptedException {
160     // TODO Check if the table exists?
161 
162   }
163 
164   /**
165    * Returns the output committer.
166    *
167    * @param context  The current context.
168    * @return The committer.
169    * @throws IOException When creating the committer fails.
170    * @throws InterruptedException When the job is aborted.
171    * @see org.apache.hadoop.mapreduce.OutputFormat#getOutputCommitter(org.apache.hadoop.mapreduce.TaskAttemptContext)
172    */
173   @Override
174   public OutputCommitter getOutputCommitter(TaskAttemptContext context)
175   throws IOException, InterruptedException {
176     return new TableOutputCommitter();
177   }
178 
179   public Configuration getConf() {
180     return conf;
181   }
182 
183   @Override
184   public void setConf(Configuration otherConf) {
185     this.conf = HBaseConfiguration.create(otherConf);
186     String tableName = this.conf.get(OUTPUT_TABLE);
187     String address = this.conf.get(QUORUM_ADDRESS);
188     String serverClass = this.conf.get(REGION_SERVER_CLASS);
189     String serverImpl = this.conf.get(REGION_SERVER_IMPL);
190     try {
191       if (address != null) {
192         ZKUtil.applyClusterKeyToConf(this.conf, address);
193       }
194       if (serverClass != null) {
195         this.conf.set(HConstants.REGION_SERVER_CLASS, serverClass);
196         this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl);
197       }
198       this.table = new HTable(this.conf, tableName);
199       this.table.setAutoFlush(false);
200       LOG.info("Created table instance for "  + tableName);
201     } catch(IOException e) {
202       LOG.error(e);
203     }
204   }
205 }