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