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.text.ParseException;
022import java.text.SimpleDateFormat;
023import java.util.ArrayList;
024import java.util.Collections;
025import java.util.HashSet;
026import java.util.List;
027import java.util.Map;
028import java.util.Set;
029import java.util.TreeMap;
030import org.apache.hadoop.conf.Configuration;
031import org.apache.hadoop.conf.Configured;
032import org.apache.hadoop.fs.Path;
033import org.apache.hadoop.hbase.Cell;
034import org.apache.hadoop.hbase.CellUtil;
035import org.apache.hadoop.hbase.ExtendedCell;
036import org.apache.hadoop.hbase.HBaseConfiguration;
037import org.apache.hadoop.hbase.KeyValue;
038import org.apache.hadoop.hbase.PrivateCellUtil;
039import org.apache.hadoop.hbase.TableName;
040import org.apache.hadoop.hbase.client.Connection;
041import org.apache.hadoop.hbase.client.ConnectionFactory;
042import org.apache.hadoop.hbase.client.Delete;
043import org.apache.hadoop.hbase.client.Mutation;
044import org.apache.hadoop.hbase.client.Put;
045import org.apache.hadoop.hbase.client.RegionLocator;
046import org.apache.hadoop.hbase.client.Table;
047import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
048import org.apache.hadoop.hbase.mapreduce.HFileOutputFormat2.TableInfo;
049import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
050import org.apache.hadoop.hbase.snapshot.SnapshotRegionLocator;
051import org.apache.hadoop.hbase.util.Bytes;
052import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
053import org.apache.hadoop.hbase.util.MapReduceExtendedCell;
054import org.apache.hadoop.hbase.wal.WALEdit;
055import org.apache.hadoop.hbase.wal.WALEditInternalHelper;
056import org.apache.hadoop.hbase.wal.WALKey;
057import org.apache.hadoop.io.WritableComparable;
058import org.apache.hadoop.mapreduce.Job;
059import org.apache.hadoop.mapreduce.Mapper;
060import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
061import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
062import org.apache.hadoop.util.Tool;
063import org.apache.hadoop.util.ToolRunner;
064import org.apache.yetus.audience.InterfaceAudience;
065import org.slf4j.Logger;
066import org.slf4j.LoggerFactory;
067
068/**
069 * A tool to replay WAL files as a M/R job. The WAL can be replayed for a set of tables or all
070 * tables, and a time range can be provided (in milliseconds). The WAL is filtered to the passed set
071 * of tables and the output can optionally be mapped to another set of tables. WAL replay can also
072 * generate HFiles for later bulk importing, in that case the WAL is replayed for a single table
073 * only.
074 */
075@InterfaceAudience.Public
076public class WALPlayer extends Configured implements Tool {
077  private static final Logger LOG = LoggerFactory.getLogger(WALPlayer.class);
078  final static String NAME = "WALPlayer";
079  public final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
080  public final static String TABLES_KEY = "wal.input.tables";
081  public final static String TABLE_MAP_KEY = "wal.input.tablesmap";
082  public final static String INPUT_FILES_SEPARATOR_KEY = "wal.input.separator";
083  public final static String IGNORE_MISSING_FILES = "wal.input.ignore.missing.files";
084  public final static String MULTI_TABLES_SUPPORT = "wal.multi.tables.support";
085
086  protected static final String tableSeparator = ";";
087
088  private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
089
090  public WALPlayer() {
091  }
092
093  protected WALPlayer(final Configuration c) {
094    super(c);
095  }
096
097  /**
098   * A mapper that just writes out KeyValues. This one can be used together with
099   * {@link CellSortReducer}
100   */
101  static class WALKeyValueMapper extends Mapper<WALKey, WALEdit, WritableComparable<?>, Cell> {
102    private Set<String> tableSet = new HashSet<String>();
103    private boolean multiTableSupport = false;
104    private boolean diskBasedSortingEnabled = false;
105
106    @Override
107    public void map(WALKey key, WALEdit value, Context context) throws IOException {
108      try {
109        // skip all other tables
110        TableName table = key.getTableName();
111        if (tableSet.contains(table.getNameAsString())) {
112          for (Cell cell : value.getCells()) {
113            if (WALEdit.isMetaEditFamily(cell)) {
114              continue;
115            }
116
117            // Set sequenceId from WALKey, since it is not included by WALCellCodec. The sequenceId
118            // on WALKey is the same value that was on the cells in the WALEdit. This enables
119            // CellSortReducer to use sequenceId to disambiguate duplicate cell timestamps.
120            // See HBASE-27649
121            PrivateCellUtil.setSequenceId(cell, key.getSequenceId());
122
123            byte[] outKey = multiTableSupport
124              ? Bytes.add(table.getName(), Bytes.toBytes(tableSeparator), CellUtil.cloneRow(cell))
125              : CellUtil.cloneRow(cell);
126            ExtendedCell extendedCell = PrivateCellUtil.ensureExtendedCell(cell);
127            context.write(wrapKey(outKey, extendedCell), new MapReduceExtendedCell(extendedCell));
128          }
129        }
130      } catch (InterruptedException e) {
131        LOG.error("Interrupted while emitting Cell", e);
132        Thread.currentThread().interrupt();
133      }
134    }
135
136    @Override
137    public void setup(Context context) throws IOException {
138      Configuration conf = context.getConfiguration();
139      String[] tables = conf.getStrings(TABLES_KEY);
140      this.multiTableSupport = conf.getBoolean(MULTI_TABLES_SUPPORT, false);
141      this.diskBasedSortingEnabled = HFileOutputFormat2.diskBasedSortingEnabled(conf);
142      Collections.addAll(tableSet, tables);
143    }
144
145    private WritableComparable<?> wrapKey(byte[] key, ExtendedCell cell) {
146      if (this.diskBasedSortingEnabled) {
147        // Important to build a new cell with the updated key to maintain multi-table support
148        KeyValue kv = new KeyValue(key, 0, key.length, cell.getFamilyArray(),
149          cell.getFamilyOffset(), cell.getFamilyLength(), cell.getQualifierArray(),
150          cell.getQualifierOffset(), cell.getQualifierLength(), cell.getTimestamp(),
151          KeyValue.Type.codeToType(PrivateCellUtil.getTypeByte(cell)), null, 0, 0);
152        kv.setSequenceId(cell.getSequenceId());
153        return new KeyOnlyCellComparable(kv);
154      } else {
155        return new ImmutableBytesWritable(key);
156      }
157    }
158  }
159
160  /**
161   * Enum for map metrics. Keep it out here rather than inside in the Map inner-class so we can find
162   * associated properties.
163   */
164  protected static enum Counter {
165    /** Number of aggregated writes */
166    PUTS,
167    /** Number of aggregated deletes */
168    DELETES,
169    CELLS_READ,
170    CELLS_WRITTEN,
171    WALEDITS
172  }
173
174  /**
175   * A mapper that writes out {@link Mutation} to be directly applied to a running HBase instance.
176   */
177  protected static class WALMapper
178    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
179    private Map<TableName, TableName> tables = new TreeMap<>();
180
181    @Override
182    public void map(WALKey key, WALEdit value, Context context) throws IOException {
183      context.getCounter(Counter.WALEDITS).increment(1);
184      try {
185        if (tables.isEmpty() || tables.containsKey(key.getTableName())) {
186          TableName targetTable =
187            tables.isEmpty() ? key.getTableName() : tables.get(key.getTableName());
188          ImmutableBytesWritable tableOut = new ImmutableBytesWritable(targetTable.getName());
189          Put put = null;
190          Delete del = null;
191          ExtendedCell lastCell = null;
192          for (ExtendedCell cell : WALEditInternalHelper.getExtendedCells(value)) {
193            context.getCounter(Counter.CELLS_READ).increment(1);
194            // Filtering WAL meta marker entries.
195            if (WALEdit.isMetaEditFamily(cell)) {
196              continue;
197            }
198            // Allow a subclass filter out this cell.
199            if (filter(context, cell)) {
200              // A WALEdit may contain multiple operations (HBASE-3584) and/or
201              // multiple rows (HBASE-5229).
202              // Aggregate as much as possible into a single Put/Delete
203              // operation before writing to the context.
204              if (
205                lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
206                  || !CellUtil.matchingRows(lastCell, cell)
207              ) {
208                // row or type changed, write out aggregate KVs.
209                if (put != null) {
210                  context.write(tableOut, put);
211                  context.getCounter(Counter.PUTS).increment(1);
212                }
213                if (del != null) {
214                  context.write(tableOut, del);
215                  context.getCounter(Counter.DELETES).increment(1);
216                }
217                if (CellUtil.isDelete(cell)) {
218                  del = new Delete(CellUtil.cloneRow(cell));
219                } else {
220                  put = new Put(CellUtil.cloneRow(cell));
221                }
222              }
223              if (CellUtil.isDelete(cell)) {
224                del.add(cell);
225              } else {
226                put.add(cell);
227              }
228              context.getCounter(Counter.CELLS_WRITTEN).increment(1);
229            }
230            lastCell = cell;
231          }
232          // write residual KVs
233          if (put != null) {
234            context.write(tableOut, put);
235            context.getCounter(Counter.PUTS).increment(1);
236          }
237          if (del != null) {
238            context.getCounter(Counter.DELETES).increment(1);
239            context.write(tableOut, del);
240          }
241        }
242      } catch (InterruptedException e) {
243        LOG.error("Interrupted while writing results", e);
244        Thread.currentThread().interrupt();
245      }
246    }
247
248    protected boolean filter(Context context, final Cell cell) {
249      return true;
250    }
251
252    @Override
253    protected void
254      cleanup(Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation>.Context context)
255        throws IOException, InterruptedException {
256      super.cleanup(context);
257    }
258
259    @SuppressWarnings("checkstyle:EmptyBlock")
260    @Override
261    public void setup(Context context) throws IOException {
262      String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
263      String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
264      if (tableMap == null) {
265        tableMap = tablesToUse;
266      }
267      if (tablesToUse == null) {
268        // Then user wants all tables.
269      } else if (tablesToUse.length != tableMap.length) {
270        // this can only happen when WALMapper is used directly by a class other than WALPlayer
271        throw new IOException("Incorrect table mapping specified .");
272      }
273      int i = 0;
274      if (tablesToUse != null) {
275        for (String table : tablesToUse) {
276          tables.put(TableName.valueOf(table), TableName.valueOf(tableMap[i++]));
277        }
278      }
279    }
280  }
281
282  void setupTime(Configuration conf, String option) throws IOException {
283    String val = conf.get(option);
284    if (null == val) {
285      return;
286    }
287    long ms;
288    try {
289      // first try to parse in user friendly form
290      ms = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SS").parse(val).getTime();
291    } catch (ParseException pe) {
292      try {
293        // then see if just a number of ms's was specified
294        ms = Long.parseLong(val);
295      } catch (NumberFormatException nfe) {
296        throw new IOException(
297          option + " must be specified either in the form 2001-02-20T16:35:06.99 "
298            + "or as number of milliseconds");
299      }
300    }
301    conf.setLong(option, ms);
302  }
303
304  /**
305   * Sets up the actual job.
306   * @param args The command line parameters.
307   * @return The newly created job.
308   * @throws IOException When setting up the job fails.
309   */
310  public Job createSubmittableJob(String[] args) throws IOException {
311    Configuration conf = getConf();
312    setupTime(conf, WALInputFormat.START_TIME_KEY);
313    setupTime(conf, WALInputFormat.END_TIME_KEY);
314    String inputDirs = args[0];
315    String[] tables = args.length == 1 ? new String[] {} : args[1].split(",");
316    String[] tableMap;
317    if (args.length > 2) {
318      tableMap = args[2].split(",");
319      if (tableMap.length != tables.length) {
320        throw new IOException("The same number of tables and mapping must be provided.");
321      }
322    } else {
323      // if no mapping is specified, map each table to itself
324      tableMap = tables;
325    }
326    conf.setStrings(TABLES_KEY, tables);
327    conf.setStrings(TABLE_MAP_KEY, tableMap);
328    conf.set(FileInputFormat.INPUT_DIR, inputDirs);
329    Job job = Job.getInstance(conf,
330      conf.get(JOB_NAME_CONF_KEY, NAME + "_" + EnvironmentEdgeManager.currentTime()));
331    job.setJarByClass(WALPlayer.class);
332
333    job.setInputFormatClass(WALInputFormat.class);
334    boolean diskBasedSortingEnabled = HFileOutputFormat2.diskBasedSortingEnabled(conf);
335    if (diskBasedSortingEnabled) {
336      job.setMapOutputKeyClass(KeyOnlyCellComparable.class);
337      job.setSortComparatorClass(KeyOnlyCellComparable.KeyOnlyCellComparator.class);
338    } else {
339      job.setMapOutputKeyClass(ImmutableBytesWritable.class);
340    }
341
342    String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
343    if (hfileOutPath != null) {
344      LOG.debug("add incremental job :" + hfileOutPath + " from " + inputDirs);
345
346      // WALPlayer needs ExtendedCellSerialization so that sequenceId can be propagated when
347      // sorting cells in CellSortReducer
348      job.getConfiguration().setBoolean(HFileOutputFormat2.EXTENDED_CELL_SERIALIZATION_ENABLED_KEY,
349        true);
350
351      // the bulk HFile case
352      List<TableName> tableNames = getTableNameList(tables);
353
354      job.setMapperClass(WALKeyValueMapper.class);
355      if (diskBasedSortingEnabled) {
356        job.setReducerClass(PreSortedCellsReducer.class);
357      } else {
358        job.setReducerClass(CellSortReducer.class);
359      }
360      Path outputDir = new Path(hfileOutPath);
361      FileOutputFormat.setOutputPath(job, outputDir);
362      job.setMapOutputValueClass(MapReduceExtendedCell.class);
363      try (Connection conn = ConnectionFactory.createConnection(conf);) {
364        List<TableInfo> tableInfoList = new ArrayList<TableInfo>();
365        for (TableName tableName : tableNames) {
366          Table table = conn.getTable(tableName);
367          RegionLocator regionLocator = getRegionLocator(tableName, conf, conn);
368          tableInfoList.add(new TableInfo(table.getDescriptor(), regionLocator));
369        }
370        MultiTableHFileOutputFormat.configureIncrementalLoad(job, tableInfoList);
371      }
372      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
373        org.apache.hbase.thirdparty.com.google.common.base.Preconditions.class);
374    } else {
375      // output to live cluster
376      job.setMapperClass(WALMapper.class);
377      job.setOutputFormatClass(MultiTableOutputFormat.class);
378      TableMapReduceUtil.addDependencyJars(job);
379      TableMapReduceUtil.initCredentials(job);
380      // No reducers.
381      job.setNumReduceTasks(0);
382    }
383    String codecCls = WALCellCodec.getWALCellCodecClass(conf).getName();
384    try {
385      TableMapReduceUtil.addDependencyJarsForClasses(job.getConfiguration(),
386        Class.forName(codecCls));
387    } catch (Exception e) {
388      throw new IOException("Cannot determine wal codec class " + codecCls, e);
389    }
390    return job;
391  }
392
393  private List<TableName> getTableNameList(String[] tables) {
394    List<TableName> list = new ArrayList<TableName>();
395    for (String name : tables) {
396      list.add(TableName.valueOf(name));
397    }
398    return list;
399  }
400
401  /**
402   * Print usage
403   * @param errorMsg Error message. Can be null.
404   */
405  private void usage(final String errorMsg) {
406    if (errorMsg != null && errorMsg.length() > 0) {
407      System.err.println("ERROR: " + errorMsg);
408    }
409    System.err.println("Usage: " + NAME + " [options] <WAL inputdir> [<tables> <tableMappings>]");
410    System.err.println(" <WAL inputdir>   directory of WALs to replay.");
411    System.err.println(" <tables>         comma separated list of tables. If no tables specified,");
412    System.err.println("                  all are imported (even hbase:meta if present).");
413    System.err.println(
414      " <tableMappings>  WAL entries can be mapped to a new set of tables by " + "passing");
415    System.err
416      .println("                  <tableMappings>, a comma separated list of target " + "tables.");
417    System.err
418      .println("                  If specified, each table in <tables> must have a " + "mapping.");
419    System.err.println("To generate HFiles to bulk load instead of loading HBase directly, pass:");
420    System.err.println(" -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
421    System.err.println(" Only one table can be specified, and no mapping allowed!");
422    System.err.println("To specify a time range, pass:");
423    System.err.println(" -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]");
424    System.err.println(" -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]");
425    System.err.println(" The start and the end date of timerange (inclusive). The dates can be");
426    System.err
427      .println(" expressed in milliseconds-since-epoch or yyyy-MM-dd'T'HH:mm:ss.SS " + "format.");
428    System.err.println(" E.g. 1234567890120 or 2009-02-13T23:32:30.12");
429    System.err.println("Other options:");
430    System.err.println(" -D" + JOB_NAME_CONF_KEY + "=jobName");
431    System.err.println(" Use the specified mapreduce job name for the wal player");
432    System.err.println(" -Dwal.input.separator=' '");
433    System.err.println(" Change WAL filename separator (WAL dir names use default ','.)");
434    System.err.println("For performance also consider the following options:\n"
435      + "  -Dmapreduce.map.speculative=false\n" + "  -Dmapreduce.reduce.speculative=false");
436  }
437
438  /**
439   * Main entry point.
440   * @param args The command line parameters.
441   * @throws Exception When running the job fails.
442   */
443  public static void main(String[] args) throws Exception {
444    int ret = ToolRunner.run(new WALPlayer(HBaseConfiguration.create()), args);
445    System.exit(ret);
446  }
447
448  @Override
449  public int run(String[] args) throws Exception {
450    if (args.length < 1) {
451      usage("Wrong number of arguments: " + args.length);
452      System.exit(-1);
453    }
454    Job job = createSubmittableJob(args);
455    return job.waitForCompletion(true) ? 0 : 1;
456  }
457
458  private static RegionLocator getRegionLocator(TableName tableName, Configuration conf,
459    Connection conn) throws IOException {
460    if (SnapshotRegionLocator.shouldUseSnapshotRegionLocator(conf, tableName)) {
461      return SnapshotRegionLocator.create(conf, tableName);
462    }
463
464    return conn.getRegionLocator(tableName);
465  }
466}