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.mapred;
20  
21  import java.io.IOException;
22  
23  import org.apache.hadoop.fs.FileAlreadyExistsException;
24  import org.apache.hadoop.fs.FileSystem;
25  import org.apache.hadoop.hbase.TableName;
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.classification.InterfaceStability;
28  import org.apache.hadoop.hbase.client.BufferedMutator;
29  import org.apache.hadoop.hbase.client.Connection;
30  import org.apache.hadoop.hbase.client.ConnectionFactory;
31  import org.apache.hadoop.hbase.client.Put;
32  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
33  import org.apache.hadoop.mapred.FileOutputFormat;
34  import org.apache.hadoop.mapred.InvalidJobConfException;
35  import org.apache.hadoop.mapred.JobConf;
36  import org.apache.hadoop.mapred.RecordWriter;
37  import org.apache.hadoop.mapred.Reporter;
38  import org.apache.hadoop.util.Progressable;
39  
40  /**
41   * Convert Map/Reduce output and write it to an HBase table
42   */
43  @InterfaceAudience.Public
44  @InterfaceStability.Stable
45  public class TableOutputFormat extends FileOutputFormat<ImmutableBytesWritable, Put> {
46  
47    /** JobConf parameter that specifies the output table */
48    public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
49  
50    /**
51     * Convert Reduce output (key, value) to (HStoreKey, KeyedDataArrayWritable)
52     * and write to an HBase table.
53     */
54    protected static class TableRecordWriter implements RecordWriter<ImmutableBytesWritable, Put> {
55      private BufferedMutator m_mutator;
56      private Connection connection;
57      /**
58       * Instantiate a TableRecordWriter with the HBase HClient for writing. Assumes control over the
59       * lifecycle of {@code conn}.
60       */
61      public TableRecordWriter(final BufferedMutator mutator) throws IOException {
62        this.m_mutator = mutator;
63      }
64  
65      public TableRecordWriter(JobConf job) throws IOException {
66        // expecting exactly one path
67        TableName tableName = TableName.valueOf(job.get(OUTPUT_TABLE));
68        connection = ConnectionFactory.createConnection(job);
69        m_mutator = connection.getBufferedMutator(tableName);
70      }
71  
72      public void close(Reporter reporter) throws IOException {
73        this.m_mutator.close();
74        if (connection != null) {
75          connection.close();
76          connection = null;
77        }
78      }
79  
80      public void write(ImmutableBytesWritable key, Put value) throws IOException {
81        m_mutator.mutate(new Put(value));
82      }
83    }
84  
85    @Override
86    public RecordWriter getRecordWriter(FileSystem ignored, JobConf job, String name,
87        Progressable progress)
88    throws IOException {
89      return new TableRecordWriter(job);
90    }
91  
92    @Override
93    public void checkOutputSpecs(FileSystem ignored, JobConf job)
94    throws FileAlreadyExistsException, InvalidJobConfException, IOException {
95      String tableName = job.get(OUTPUT_TABLE);
96      if (tableName == null) {
97        throw new IOException("Must specify table name");
98      }
99    }
100 }