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