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