001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, software 013 * distributed under the License is distributed on an "AS IS" BASIS, 014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 015 * See the License for the specific language governing permissions and 016 * limitations under the License. 017 */ 018package org.apache.hadoop.hbase.mapreduce; 019 020import java.io.IOException; 021import org.apache.hadoop.conf.Configurable; 022import org.apache.hadoop.conf.Configuration; 023import org.apache.hadoop.hbase.HBaseConfiguration; 024import org.apache.hadoop.hbase.HConstants; 025import org.apache.hadoop.hbase.TableName; 026import org.apache.hadoop.hbase.TableNotEnabledException; 027import org.apache.hadoop.hbase.TableNotFoundException; 028import org.apache.hadoop.hbase.client.Admin; 029import org.apache.hadoop.hbase.client.BufferedMutator; 030import org.apache.hadoop.hbase.client.Connection; 031import org.apache.hadoop.hbase.client.ConnectionFactory; 032import org.apache.hadoop.hbase.client.Delete; 033import org.apache.hadoop.hbase.client.Mutation; 034import org.apache.hadoop.hbase.client.Put; 035import org.apache.hadoop.mapreduce.JobContext; 036import org.apache.hadoop.mapreduce.OutputCommitter; 037import org.apache.hadoop.mapreduce.OutputFormat; 038import org.apache.hadoop.mapreduce.RecordWriter; 039import org.apache.hadoop.mapreduce.TaskAttemptContext; 040import org.apache.yetus.audience.InterfaceAudience; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044/** 045 * Convert Map/Reduce output and write it to an HBase table. The KEY is ignored while the output 046 * value <u>must</u> be either a {@link Put} or a {@link Delete} instance. 047 */ 048@InterfaceAudience.Public 049public class TableOutputFormat<KEY> extends OutputFormat<KEY, Mutation> implements Configurable { 050 051 private static final Logger LOG = LoggerFactory.getLogger(TableOutputFormat.class); 052 053 /** Job parameter that specifies the output table. */ 054 public static final String OUTPUT_TABLE = "hbase.mapred.outputtable"; 055 056 /** 057 * Prefix for configuration property overrides to apply in {@link #setConf(Configuration)}. For 058 * keys matching this prefix, the prefix is stripped, and the value is set in the configuration 059 * with the resulting key, ie. the entry "hbase.mapred.output.key1 = value1" would be set in the 060 * configuration as "key1 = value1". Use this to set properties which should only be applied to 061 * the {@code TableOutputFormat} configuration and not the input configuration. 062 */ 063 public static final String OUTPUT_CONF_PREFIX = "hbase.mapred.output."; 064 065 /** 066 * Optional job parameter to specify a peer cluster. Used specifying remote cluster when copying 067 * between hbase clusters (the source is picked up from <code>hbase-site.xml</code>). 068 * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, 069 * Class, String, String, String) 070 */ 071 public static final String QUORUM_ADDRESS = OUTPUT_CONF_PREFIX + "quorum"; 072 073 /** Optional job parameter to specify peer cluster's ZK client port */ 074 public static final String QUORUM_PORT = OUTPUT_CONF_PREFIX + "quorum.port"; 075 076 /** Optional specification of the rs class name of the peer cluster */ 077 public static final String REGION_SERVER_CLASS = OUTPUT_CONF_PREFIX + "rs.class"; 078 /** Optional specification of the rs impl name of the peer cluster */ 079 public static final String REGION_SERVER_IMPL = OUTPUT_CONF_PREFIX + "rs.impl"; 080 081 /** The configuration. */ 082 private Configuration conf = null; 083 084 /** 085 * Writes the reducer output to an HBase table. 086 */ 087 protected class TableRecordWriter extends RecordWriter<KEY, Mutation> { 088 089 private Connection connection; 090 private BufferedMutator mutator; 091 092 /** 093 * 094 * 095 */ 096 public TableRecordWriter() throws IOException { 097 String tableName = conf.get(OUTPUT_TABLE); 098 this.connection = ConnectionFactory.createConnection(conf); 099 this.mutator = connection.getBufferedMutator(TableName.valueOf(tableName)); 100 LOG.info("Created table instance for " + tableName); 101 } 102 103 /** 104 * Closes the writer, in this case flush table commits. 105 * @param context The context. 106 * @throws IOException When closing the writer fails. 107 * @see RecordWriter#close(TaskAttemptContext) 108 */ 109 @Override 110 public void close(TaskAttemptContext context) throws IOException { 111 try { 112 if (mutator != null) { 113 mutator.close(); 114 } 115 } finally { 116 if (connection != null) { 117 connection.close(); 118 } 119 } 120 } 121 122 /** 123 * Writes a key/value pair into the table. 124 * @param key The key. 125 * @param value The value. 126 * @throws IOException When writing fails. 127 * @see RecordWriter#write(Object, Object) 128 */ 129 @Override 130 public void write(KEY key, Mutation value) throws IOException { 131 if (!(value instanceof Put) && !(value instanceof Delete)) { 132 throw new IOException("Pass a Delete or a Put"); 133 } 134 mutator.mutate(value); 135 } 136 } 137 138 /** 139 * Creates a new record writer. Be aware that the baseline javadoc gives the impression that there 140 * is a single {@link RecordWriter} per job but in HBase, it is more natural if we give you a new 141 * RecordWriter per call of this method. You must close the returned RecordWriter when done. 142 * Failure to do so will drop writes. 143 * @param context The current task context. 144 * @return The newly created writer instance. 145 * @throws IOException When creating the writer fails. 146 * @throws InterruptedException When the job is cancelled. 147 */ 148 @Override 149 public RecordWriter<KEY, Mutation> getRecordWriter(TaskAttemptContext context) 150 throws IOException, InterruptedException { 151 return new TableRecordWriter(); 152 } 153 154 /** 155 * Checks if the output table exists and is enabled. 156 * @param context The current context. 157 * @throws IOException When the check fails. 158 * @throws InterruptedException When the job is aborted. 159 * @see OutputFormat#checkOutputSpecs(JobContext) 160 */ 161 @Override 162 public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { 163 Configuration hConf = getConf(); 164 if (hConf == null) { 165 hConf = context.getConfiguration(); 166 } 167 168 try (Connection connection = ConnectionFactory.createConnection(hConf); 169 Admin admin = connection.getAdmin()) { 170 TableName tableName = TableName.valueOf(hConf.get(OUTPUT_TABLE)); 171 if (!admin.tableExists(tableName)) { 172 throw new TableNotFoundException( 173 "Can't write, table does not exist:" + tableName.getNameAsString()); 174 } 175 176 if (!admin.isTableEnabled(tableName)) { 177 throw new TableNotEnabledException( 178 "Can't write, table is not enabled: " + tableName.getNameAsString()); 179 } 180 } 181 } 182 183 /** 184 * Returns the output committer. 185 * @param context The current context. 186 * @return The committer. 187 * @throws IOException When creating the committer fails. 188 * @throws InterruptedException When the job is aborted. 189 * @see OutputFormat#getOutputCommitter(TaskAttemptContext) 190 */ 191 @Override 192 public OutputCommitter getOutputCommitter(TaskAttemptContext context) 193 throws IOException, InterruptedException { 194 return new TableOutputCommitter(); 195 } 196 197 @Override 198 public Configuration getConf() { 199 return conf; 200 } 201 202 @Override 203 public void setConf(Configuration otherConf) { 204 String tableName = otherConf.get(OUTPUT_TABLE); 205 if (tableName == null || tableName.length() <= 0) { 206 throw new IllegalArgumentException("Must specify table name"); 207 } 208 209 String address = otherConf.get(QUORUM_ADDRESS); 210 int zkClientPort = otherConf.getInt(QUORUM_PORT, 0); 211 String serverClass = otherConf.get(REGION_SERVER_CLASS); 212 String serverImpl = otherConf.get(REGION_SERVER_IMPL); 213 214 try { 215 this.conf = HBaseConfiguration.createClusterConf(otherConf, address, OUTPUT_CONF_PREFIX); 216 217 if (serverClass != null) { 218 this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl); 219 } 220 if (zkClientPort != 0) { 221 this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort); 222 } 223 } catch (IOException e) { 224 LOG.error(e.toString(), e); 225 throw new RuntimeException(e); 226 } 227 } 228}