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 * n * 094 */ 095 public TableRecordWriter() throws IOException { 096 String tableName = conf.get(OUTPUT_TABLE); 097 this.connection = ConnectionFactory.createConnection(conf); 098 this.mutator = connection.getBufferedMutator(TableName.valueOf(tableName)); 099 LOG.info("Created table instance for " + tableName); 100 } 101 102 /** 103 * Closes the writer, in this case flush table commits. 104 * @param context The context. 105 * @throws IOException When closing the writer fails. 106 * @see RecordWriter#close(TaskAttemptContext) 107 */ 108 @Override 109 public void close(TaskAttemptContext context) throws IOException { 110 try { 111 if (mutator != null) { 112 mutator.close(); 113 } 114 } finally { 115 if (connection != null) { 116 connection.close(); 117 } 118 } 119 } 120 121 /** 122 * Writes a key/value pair into the table. 123 * @param key The key. 124 * @param value The value. 125 * @throws IOException When writing fails. 126 * @see RecordWriter#write(Object, Object) 127 */ 128 @Override 129 public void write(KEY key, Mutation value) throws IOException { 130 if (!(value instanceof Put) && !(value instanceof Delete)) { 131 throw new IOException("Pass a Delete or a Put"); 132 } 133 mutator.mutate(value); 134 } 135 } 136 137 /** 138 * Creates a new record writer. Be aware that the baseline javadoc gives the impression that there 139 * is a single {@link RecordWriter} per job but in HBase, it is more natural if we give you a new 140 * RecordWriter per call of this method. You must close the returned RecordWriter when done. 141 * Failure to do so will drop writes. 142 * @param context The current task context. 143 * @return The newly created writer instance. 144 * @throws IOException When creating the writer fails. 145 * @throws InterruptedException When the job is cancelled. 146 */ 147 @Override 148 public RecordWriter<KEY, Mutation> getRecordWriter(TaskAttemptContext context) 149 throws IOException, InterruptedException { 150 return new TableRecordWriter(); 151 } 152 153 /** 154 * Checks if the output table exists and is enabled. 155 * @param context The current context. 156 * @throws IOException When the check fails. 157 * @throws InterruptedException When the job is aborted. 158 * @see OutputFormat#checkOutputSpecs(JobContext) 159 */ 160 @Override 161 public void checkOutputSpecs(JobContext context) throws IOException, InterruptedException { 162 Configuration hConf = getConf(); 163 if (hConf == null) { 164 hConf = context.getConfiguration(); 165 } 166 167 try (Connection connection = ConnectionFactory.createConnection(hConf); 168 Admin admin = connection.getAdmin()) { 169 TableName tableName = TableName.valueOf(hConf.get(OUTPUT_TABLE)); 170 if (!admin.tableExists(tableName)) { 171 throw new TableNotFoundException( 172 "Can't write, table does not exist:" + tableName.getNameAsString()); 173 } 174 175 if (!admin.isTableEnabled(tableName)) { 176 throw new TableNotEnabledException( 177 "Can't write, table is not enabled: " + tableName.getNameAsString()); 178 } 179 } 180 } 181 182 /** 183 * Returns the output committer. 184 * @param context The current context. 185 * @return The committer. 186 * @throws IOException When creating the committer fails. 187 * @throws InterruptedException When the job is aborted. 188 * @see OutputFormat#getOutputCommitter(TaskAttemptContext) 189 */ 190 @Override 191 public OutputCommitter getOutputCommitter(TaskAttemptContext context) 192 throws IOException, InterruptedException { 193 return new TableOutputCommitter(); 194 } 195 196 @Override 197 public Configuration getConf() { 198 return conf; 199 } 200 201 @Override 202 public void setConf(Configuration otherConf) { 203 String tableName = otherConf.get(OUTPUT_TABLE); 204 if (tableName == null || tableName.length() <= 0) { 205 throw new IllegalArgumentException("Must specify table name"); 206 } 207 208 String address = otherConf.get(QUORUM_ADDRESS); 209 int zkClientPort = otherConf.getInt(QUORUM_PORT, 0); 210 String serverClass = otherConf.get(REGION_SERVER_CLASS); 211 String serverImpl = otherConf.get(REGION_SERVER_IMPL); 212 213 try { 214 this.conf = HBaseConfiguration.createClusterConf(otherConf, address, OUTPUT_CONF_PREFIX); 215 216 if (serverClass != null) { 217 this.conf.set(HConstants.REGION_SERVER_IMPL, serverImpl); 218 } 219 if (zkClientPort != 0) { 220 this.conf.setInt(HConstants.ZOOKEEPER_CLIENT_PORT, zkClientPort); 221 } 222 } catch (IOException e) { 223 LOG.error(e.toString(), e); 224 throw new RuntimeException(e); 225 } 226 } 227}