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