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