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