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.hbase.HBaseConfiguration;
27 import org.apache.hadoop.hbase.HConstants;
28 import org.apache.hadoop.hbase.client.Delete;
29 import org.apache.hadoop.hbase.client.HTable;
30 import org.apache.hadoop.hbase.client.Put;
31 import org.apache.hadoop.io.Writable;
32 import org.apache.hadoop.mapreduce.JobContext;
33 import org.apache.hadoop.mapreduce.OutputCommitter;
34 import org.apache.hadoop.mapreduce.OutputFormat;
35 import org.apache.hadoop.mapreduce.RecordWriter;
36 import org.apache.hadoop.mapreduce.TaskAttemptContext;
37 import org.apache.hadoop.conf.Configuration;
38
39 /**
40 * Convert Map/Reduce output and write it to an HBase table. The KEY is ignored
41 * while the output value <u>must</u> be either a {@link Put} or a
42 * {@link Delete} instance.
43 *
44 * @param <KEY> The type of the key. Ignored in this class.
45 */
46 public class TableOutputFormat<KEY> extends OutputFormat<KEY, Writable> {
47
48 private final Log LOG = LogFactory.getLog(TableOutputFormat.class);
49 /** Job parameter that specifies the output table. */
50 public static final String OUTPUT_TABLE = "hbase.mapred.outputtable";
51 /** Optional job parameter to specify a peer cluster */
52 public static final String QUORUM_ADDRESS = "hbase.mapred.output.quorum";
53 /** Optional specification of the rs class name of the peer cluster */
54 public static final String
55 REGION_SERVER_CLASS = "hbase.mapred.output.rs.class";
56 /** Optional specification of the rs impl name of the peer cluster */
57 public static final String
58 REGION_SERVER_IMPL = "hbase.mapred.output.rs.impl";
59
60 /**
61 * Writes the reducer output to an HBase table.
62 *
63 * @param <KEY> The type of the key.
64 */
65 protected static class TableRecordWriter<KEY>
66 extends RecordWriter<KEY, Writable> {
67
68 /** The table to write to. */
69 private HTable table;
70
71 /**
72 * Instantiate a TableRecordWriter with the HBase HClient for writing.
73 *
74 * @param table The table to write to.
75 */
76 public TableRecordWriter(HTable table) {
77 this.table = table;
78 }
79
80 /**
81 * Closes the writer, in this case flush table commits.
82 *
83 * @param context The context.
84 * @throws IOException When closing the writer fails.
85 * @see org.apache.hadoop.mapreduce.RecordWriter#close(org.apache.hadoop.mapreduce.TaskAttemptContext)
86 */
87 @Override
88 public void close(TaskAttemptContext context)
89 throws IOException {
90 table.flushCommits();
91 }
92
93 /**
94 * Writes a key/value pair into the table.
95 *
96 * @param key The key.
97 * @param value The value.
98 * @throws IOException When writing fails.
99 * @see org.apache.hadoop.mapreduce.RecordWriter#write(java.lang.Object, java.lang.Object)
100 */
101 @Override
102 public void write(KEY key, Writable value)
103 throws IOException {
104 if (value instanceof Put) this.table.put(new Put((Put)value));
105 else if (value instanceof Delete) this.table.delete(new Delete((Delete)value));
106 else throw new IOException("Pass a Delete or a Put");
107 }
108 }
109
110 /**
111 * Creates a new record writer.
112 *
113 * @param context The current task context.
114 * @return The newly created writer instance.
115 * @throws IOException When creating the writer fails.
116 * @throws InterruptedException When the jobs is cancelled.
117 * @see org.apache.hadoop.mapreduce.lib.output.FileOutputFormat#getRecordWriter(org.apache.hadoop.mapreduce.TaskAttemptContext)
118 */
119 @Override
120 public RecordWriter<KEY, Writable> getRecordWriter(
121 TaskAttemptContext context)
122 throws IOException, InterruptedException {
123 // expecting exactly one path
124 Configuration conf = new Configuration(context.getConfiguration());
125 String tableName = conf.get(OUTPUT_TABLE);
126 String address = conf.get(QUORUM_ADDRESS);
127 String serverClass = conf.get(REGION_SERVER_CLASS);
128 String serverImpl = conf.get(REGION_SERVER_IMPL);
129 HTable table = null;
130 try {
131 HBaseConfiguration.addHbaseResources(conf);
132 if (address != null) {
133 // Check is done in TMRU
134 String[] parts = address.split(":");
135 conf.set(HConstants.ZOOKEEPER_QUORUM, parts[0]);
136 conf.set(HConstants.ZOOKEEPER_ZNODE_PARENT, parts[1]);
137 }
138 if (serverClass != null) {
139 conf.set(HConstants.REGION_SERVER_CLASS, serverClass);
140 conf.set(HConstants.REGION_SERVER_IMPL, serverImpl);
141 }
142 table = new HTable(conf, tableName);
143 } catch(IOException e) {
144 LOG.error(e);
145 throw e;
146 }
147 table.setAutoFlush(false);
148 return new TableRecordWriter<KEY>(table);
149 }
150
151 /**
152 * Checks if the output target exists.
153 *
154 * @param context The current context.
155 * @throws IOException When the check fails.
156 * @throws InterruptedException When the job is aborted.
157 * @see org.apache.hadoop.mapreduce.OutputFormat#checkOutputSpecs(org.apache.hadoop.mapreduce.JobContext)
158 */
159 @Override
160 public void checkOutputSpecs(JobContext context) throws IOException,
161 InterruptedException {
162 // TODO Check if the table exists?
163
164 }
165
166 /**
167 * Returns the output committer.
168 *
169 * @param context The current context.
170 * @return The committer.
171 * @throws IOException When creating the committer fails.
172 * @throws InterruptedException When the job is aborted.
173 * @see org.apache.hadoop.mapreduce.OutputFormat#getOutputCommitter(org.apache.hadoop.mapreduce.TaskAttemptContext)
174 */
175 @Override
176 public OutputCommitter getOutputCommitter(TaskAttemptContext context)
177 throws IOException, InterruptedException {
178 return new TableOutputCommitter();
179 }
180
181 }