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}