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 java.net.URI; 022import java.net.URISyntaxException; 023import org.apache.commons.lang3.StringUtils; 024import org.apache.hadoop.conf.Configurable; 025import org.apache.hadoop.conf.Configuration; 026import org.apache.hadoop.hbase.HBaseConfiguration; 027import org.apache.hadoop.hbase.HConstants; 028import org.apache.hadoop.hbase.TableName; 029import org.apache.hadoop.hbase.TableNotEnabledException; 030import org.apache.hadoop.hbase.TableNotFoundException; 031import org.apache.hadoop.hbase.client.Admin; 032import org.apache.hadoop.hbase.client.BufferedMutator; 033import org.apache.hadoop.hbase.client.Connection; 034import org.apache.hadoop.hbase.client.ConnectionFactory; 035import org.apache.hadoop.hbase.client.Delete; 036import org.apache.hadoop.hbase.client.Durability; 037import org.apache.hadoop.hbase.client.Mutation; 038import org.apache.hadoop.hbase.client.Put; 039import org.apache.hadoop.mapreduce.JobContext; 040import org.apache.hadoop.mapreduce.OutputCommitter; 041import org.apache.hadoop.mapreduce.OutputFormat; 042import org.apache.hadoop.mapreduce.RecordWriter; 043import org.apache.hadoop.mapreduce.TaskAttemptContext; 044import org.apache.yetus.audience.InterfaceAudience; 045import org.slf4j.Logger; 046import org.slf4j.LoggerFactory; 047 048/** 049 * Convert Map/Reduce output and write it to an HBase table. The KEY is ignored while the output 050 * value <u>must</u> be either a {@link Put} or a {@link Delete} instance. 051 */ 052@InterfaceAudience.Public 053public class TableOutputFormat<KEY> extends OutputFormat<KEY, Mutation> implements Configurable { 054 055 private static final Logger LOG = LoggerFactory.getLogger(TableOutputFormat.class); 056 057 /** Job parameter that specifies the output table. */ 058 public static final String OUTPUT_TABLE = "hbase.mapred.outputtable"; 059 060 /** Property value to use write-ahead logging */ 061 public static final boolean WAL_ON = true; 062 063 /** Property value to disable write-ahead logging */ 064 public static final boolean WAL_OFF = false; 065 066 /** Set this to {@link #WAL_OFF} to turn off write-ahead logging (WAL) */ 067 public static final String WAL_PROPERTY = "hbase.mapreduce.tableoutputformat.write.wal"; 068 069 /** 070 * Optional job parameter to specify a peer cluster. Used specifying remote cluster when copying 071 * between hbase clusters (the source is picked up from <code>hbase-site.xml</code>). 072 * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, 073 * Class, java.net.URI) 074 */ 075 public static final String OUTPUT_CLUSTER = "hbase.mapred.outputcluster"; 076 077 /** 078 * Prefix for configuration property overrides to apply in {@link #setConf(Configuration)}. For 079 * keys matching this prefix, the prefix is stripped, and the value is set in the configuration 080 * with the resulting key, ie. the entry "hbase.mapred.output.key1 = value1" would be set in the 081 * configuration as "key1 = value1". Use this to set properties which should only be applied to 082 * the {@code TableOutputFormat} configuration and not the input configuration. 083 * @deprecated Since 3.0.0, will be removed in 4.0.0. You do not need to use this way for 084 * specifying configurations any more, you can specify any configuration with the 085 * connection uri's queries specified by the {@link #OUTPUT_CLUSTER} parameter. 086 */ 087 @Deprecated 088 public static final String OUTPUT_CONF_PREFIX = "hbase.mapred.output."; 089 090 /** 091 * Optional job parameter to specify a peer cluster. Used specifying remote cluster when copying 092 * between hbase clusters (the source is picked up from <code>hbase-site.xml</code>). 093 * @see TableMapReduceUtil#initTableReducerJob(String, Class, org.apache.hadoop.mapreduce.Job, 094 * Class, String) 095 * @deprecated Since 3.0.0, will be removed in 4.0.0. Use {@link #OUTPUT_CLUSTER} to specify the 096 * peer cluster instead. 097 */ 098 @Deprecated 099 public static final String QUORUM_ADDRESS = OUTPUT_CONF_PREFIX + "quorum"; 100 101 /** 102 * Optional job parameter to specify peer cluster's ZK client port. 103 * @deprecated Since 3.0.0, will be removed in 4.0.0. You do not need to use this way for 104 * specifying configurations any more, you can specify any configuration with the 105 * connection uri's queries specified by the {@link #OUTPUT_CLUSTER} parameter. 106 */ 107 @Deprecated 108 public static final String QUORUM_PORT = OUTPUT_CONF_PREFIX + "quorum.port"; 109 110 /** 111 * Optional specification of the rs class name of the peer cluster. 112 * @deprecated Since 2.5.9, 2.6.1 and 2.7.0, will be removed in 4.0.0. Does not take effect from 113 * long ago, see HBASE-6044. 114 */ 115 @Deprecated 116 public static final String REGION_SERVER_CLASS = OUTPUT_CONF_PREFIX + "rs.class"; 117 /** 118 * Optional specification of the rs impl name of the peer cluster 119 * @deprecated Since 2.5.9, 2.6.1 and 2.7.0, will be removed in 4.0.0. Does not take effect from 120 * long ago, see HBASE-6044. 121 */ 122 @Deprecated 123 public static final String REGION_SERVER_IMPL = OUTPUT_CONF_PREFIX + "rs.impl"; 124 125 /** The configuration. */ 126 private Configuration conf = null; 127 128 private static Connection createConnection(Configuration conf) throws IOException { 129 String outputCluster = conf.get(OUTPUT_CLUSTER); 130 if (!StringUtils.isBlank(outputCluster)) { 131 URI uri; 132 try { 133 uri = new URI(outputCluster); 134 } catch (URISyntaxException e) { 135 throw new IOException( 136 "malformed connection uri: " + outputCluster + ", please check config " + OUTPUT_CLUSTER, 137 e); 138 } 139 return ConnectionFactory.createConnection(uri, conf); 140 } else { 141 return ConnectionFactory.createConnection(conf); 142 } 143 } 144 145 /** 146 * Writes the reducer output to an HBase table. 147 */ 148 protected class TableRecordWriter extends RecordWriter<KEY, Mutation> { 149 150 private Connection connection; 151 private BufferedMutator mutator; 152 boolean useWriteAheadLogging; 153 154 public TableRecordWriter() throws IOException { 155 this.connection = createConnection(conf); 156 String tableName = conf.get(OUTPUT_TABLE); 157 this.mutator = connection.getBufferedMutator(TableName.valueOf(tableName)); 158 LOG.info("Created table instance for " + tableName); 159 this.useWriteAheadLogging = conf.getBoolean(WAL_PROPERTY, WAL_ON); 160 } 161 162 /** 163 * Closes the writer, in this case flush table commits. 164 * @param context The context. 165 * @throws IOException When closing the writer fails. 166 * @see RecordWriter#close(TaskAttemptContext) 167 */ 168 @Override 169 public void close(TaskAttemptContext context) throws IOException { 170 try { 171 if (mutator != null) { 172 mutator.close(); 173 } 174 } finally { 175 if (connection != null) { 176 connection.close(); 177 } 178 } 179 } 180 181 /** 182 * Writes a key/value pair into the table. 183 * @param key The key. 184 * @param value The value. 185 * @throws IOException When writing fails. 186 * @see RecordWriter#write(Object, Object) 187 */ 188 @Override 189 public void write(KEY key, Mutation value) throws IOException { 190 if (!(value instanceof Put) && !(value instanceof Delete)) { 191 throw new IOException("Pass a Delete or a Put"); 192 } 193 if (!useWriteAheadLogging) { 194 value.setDurability(Durability.SKIP_WAL); 195 } 196 mutator.mutate(value); 197 } 198 } 199 200 /** 201 * Creates a new record writer. Be aware that the baseline javadoc gives the impression that there 202 * is a single {@link RecordWriter} per job but in HBase, it is more natural if we give you a new 203 * RecordWriter per call of this method. You must close the returned RecordWriter when done. 204 * Failure to do so will drop writes. 205 * @param context The current task context. 206 * @return The newly created writer instance. 207 * @throws IOException When creating the writer fails. 208 * @throws InterruptedException When the job is cancelled. 209 */ 210 @Override 211 public RecordWriter<KEY, Mutation> getRecordWriter(TaskAttemptContext context) 212 throws IOException, InterruptedException { 213 return new TableRecordWriter(); 214 } 215 216 /** 217 * Checks if the output table exists and is enabled. 218 * @param context The current context. 219 * @throws IOException When the check fails. 220 * @throws InterruptedException When the job is aborted. 221 * @see OutputFormat#checkOutputSpecs(JobContext) 222 */ 223 @Override 224 public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { 225 Configuration hConf = getConf(); 226 if (hConf == null) { 227 hConf = context.getConfiguration(); 228 } 229 230 try (Connection connection = createConnection(hConf); Admin admin = connection.getAdmin()) { 231 TableName tableName = TableName.valueOf(hConf.get(OUTPUT_TABLE)); 232 if (!admin.tableExists(tableName)) { 233 throw new TableNotFoundException( 234 "Can't write, table does not exist:" + tableName.getNameAsString()); 235 } 236 237 if (!admin.isTableEnabled(tableName)) { 238 throw new TableNotEnabledException( 239 "Can't write, table is not enabled: " + tableName.getNameAsString()); 240 } 241 } 242 } 243 244 /** 245 * Returns the output committer. 246 * @param context The current context. 247 * @return The committer. 248 * @throws IOException When creating the committer fails. 249 * @throws InterruptedException When the job is aborted. 250 * @see OutputFormat#getOutputCommitter(TaskAttemptContext) 251 */ 252 @Override 253 public OutputCommitter getOutputCommitter(TaskAttemptContext context) 254 throws IOException, InterruptedException { 255 return new TableOutputCommitter(); 256 } 257 258 @Override 259 public Configuration getConf() { 260 return conf; 261 } 262 263 @Override 264 public void setConf(Configuration otherConf) { 265 String tableName = otherConf.get(OUTPUT_TABLE); 266 if (tableName == null || tableName.length() <= 0) { 267 throw new IllegalArgumentException("Must specify table name"); 268 } 269 270 String address = otherConf.get(QUORUM_ADDRESS); 271 int zkClientPort = otherConf.getInt(QUORUM_PORT, 0); 272 273 try { 274 this.conf = HBaseConfiguration.createClusterConf(otherConf, address, OUTPUT_CONF_PREFIX); 275 if (zkClientPort != 0) { 276 this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort); 277 } 278 } catch (IOException e) { 279 LOG.error(e.toString(), e); 280 throw new RuntimeException(e); 281 } 282 } 283}