View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.mapreduce;
19  
20  import java.io.IOException;
21  import java.text.ParseException;
22  import java.text.SimpleDateFormat;
23  import java.util.Map;
24  import java.util.TreeMap;
25  
26  import org.apache.commons.logging.Log;
27  import org.apache.commons.logging.LogFactory;
28  import org.apache.hadoop.conf.Configuration;
29  import org.apache.hadoop.conf.Configured;
30  import org.apache.hadoop.fs.Path;
31  import org.apache.hadoop.hbase.Cell;
32  import org.apache.hadoop.hbase.CellUtil;
33  import org.apache.hadoop.hbase.HBaseConfiguration;
34  import org.apache.hadoop.hbase.KeyValue;
35  import org.apache.hadoop.hbase.KeyValueUtil;
36  import org.apache.hadoop.hbase.TableName;
37  import org.apache.hadoop.hbase.classification.InterfaceAudience;
38  import org.apache.hadoop.hbase.classification.InterfaceStability;
39  import org.apache.hadoop.hbase.client.Connection;
40  import org.apache.hadoop.hbase.client.ConnectionFactory;
41  import org.apache.hadoop.hbase.client.Delete;
42  import org.apache.hadoop.hbase.client.Mutation;
43  import org.apache.hadoop.hbase.client.Put;
44  import org.apache.hadoop.hbase.client.RegionLocator;
45  import org.apache.hadoop.hbase.client.Table;
46  import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
47  import org.apache.hadoop.hbase.regionserver.wal.WALCellCodec;
48  import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
49  import org.apache.hadoop.hbase.util.Bytes;
50  import org.apache.hadoop.hbase.wal.WALKey;
51  import org.apache.hadoop.mapreduce.Job;
52  import org.apache.hadoop.mapreduce.Mapper;
53  import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
54  import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
55  import org.apache.hadoop.util.Tool;
56  import org.apache.hadoop.util.ToolRunner;
57
58  /**
59   * A tool to replay WAL files as a M/R job.
60   * The WAL can be replayed for a set of tables or all tables,
61   * and a timerange can be provided (in milliseconds).
62   * The WAL is filtered to the passed set of tables and  the output
63   * can optionally be mapped to another set of tables.
64   *
65   * WAL replay can also generate HFiles for later bulk importing,
66   * in that case the WAL is replayed for a single table only.
67   */
68  @InterfaceAudience.Public
69  @InterfaceStability.Stable
70  public class WALPlayer extends Configured implements Tool {
71    private static final Log LOG = LogFactory.getLog(WALPlayer.class);
72    final static String NAME = "WALPlayer";
73    final static String BULK_OUTPUT_CONF_KEY = "wal.bulk.output";
74    final static String TABLES_KEY = "wal.input.tables";
75    final static String TABLE_MAP_KEY = "wal.input.tablesmap";
76
77    // This relies on Hadoop Configuration to handle warning about deprecated configs and
78    // to set the correct non-deprecated configs when an old one shows up.
79    static {
80      Configuration.addDeprecation("hlog.bulk.output", BULK_OUTPUT_CONF_KEY);
81      Configuration.addDeprecation("hlog.input.tables", TABLES_KEY);
82      Configuration.addDeprecation("hlog.input.tablesmap", TABLE_MAP_KEY);
83      Configuration.addDeprecation(HLogInputFormat.START_TIME_KEY, WALInputFormat.START_TIME_KEY);
84      Configuration.addDeprecation(HLogInputFormat.END_TIME_KEY, WALInputFormat.END_TIME_KEY);
85    }
86
87    private final static String JOB_NAME_CONF_KEY = "mapreduce.job.name";
88
89    protected WALPlayer(final Configuration c) {
90      super(c);
91    }
92
93    /**
94     * A mapper that just writes out KeyValues.
95     * This one can be used together with {@link KeyValueSortReducer}
96     */
97    static class WALKeyValueMapper
98    extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, KeyValue> {
99      private byte[] table;
100
101     @Override
102     public void map(WALKey key, WALEdit value,
103       Context context)
104     throws IOException {
105       try {
106         // skip all other tables
107         if (Bytes.equals(table, key.getTablename().getName())) {
108           for (Cell cell : value.getCells()) {
109             KeyValue kv = KeyValueUtil.ensureKeyValue(cell);
110             if (WALEdit.isMetaEditFamily(kv)) continue;
111             context.write(new ImmutableBytesWritable(CellUtil.cloneRow(kv)), kv);
112           }
113         }
114       } catch (InterruptedException e) {
115         e.printStackTrace();
116       }
117     }
118
119     @Override
120     public void setup(Context context) throws IOException {
121       // only a single table is supported when HFiles are generated with HFileOutputFormat
122       String[] tables = context.getConfiguration().getStrings(TABLES_KEY);
123       if (tables == null || tables.length != 1) {
124         // this can only happen when WALMapper is used directly by a class other than WALPlayer
125         throw new IOException("Exactly one table must be specified for bulk HFile case.");
126       }
127       table = Bytes.toBytes(tables[0]);
128     }
129   }
130
131   /**
132    * A mapper that writes out {@link Mutation} to be directly applied to
133    * a running HBase instance.
134    */
135   protected static class WALMapper
136   extends Mapper<WALKey, WALEdit, ImmutableBytesWritable, Mutation> {
137     private Map<TableName, TableName> tables = new TreeMap<TableName, TableName>();
138
139     @Override
140     public void map(WALKey key, WALEdit value, Context context)
141     throws IOException {
142       try {
143         if (tables.isEmpty() || tables.containsKey(key.getTablename())) {
144           TableName targetTable = tables.isEmpty() ?
145                 key.getTablename() :
146                 tables.get(key.getTablename());
147           ImmutableBytesWritable tableOut = new ImmutableBytesWritable(targetTable.getName());
148           Put put = null;
149           Delete del = null;
150           Cell lastCell = null;
151           for (Cell cell : value.getCells()) {
152             // filtering WAL meta entries
153             if (WALEdit.isMetaEditFamily(cell)) continue;
154
155             // Allow a subclass filter out this cell.
156             if (filter(context, cell)) {
157               // A WALEdit may contain multiple operations (HBASE-3584) and/or
158               // multiple rows (HBASE-5229).
159               // Aggregate as much as possible into a single Put/Delete
160               // operation before writing to the context.
161               if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
162                   || !CellUtil.matchingRow(lastCell, cell)) {
163                 // row or type changed, write out aggregate KVs.
164                 if (put != null) context.write(tableOut, put);
165                 if (del != null) context.write(tableOut, del);
166                 if (CellUtil.isDelete(cell)) {
167                   del = new Delete(CellUtil.cloneRow(cell));
168                 } else {
169                   put = new Put(CellUtil.cloneRow(cell));
170                 }
171               }
172               if (CellUtil.isDelete(cell)) {
173                 del.addDeleteMarker(cell);
174               } else {
175                 put.add(cell);
176               }
177             }
178             lastCell = cell;
179           }
180           // write residual KVs
181           if (put != null) context.write(tableOut, put);
182           if (del != null) context.write(tableOut, del);
183         }
184       } catch (InterruptedException e) {
185         e.printStackTrace();
186       }
187     }
188
189     /**
190      * @param cell
191      * @return Return true if we are to emit this cell.
192      */
193     protected boolean filter(Context context, final Cell cell) {
194       return true;
195     }
196
197     @Override
198     public void setup(Context context) throws IOException {
199       String[] tableMap = context.getConfiguration().getStrings(TABLE_MAP_KEY);
200       String[] tablesToUse = context.getConfiguration().getStrings(TABLES_KEY);
201       if (tablesToUse == null && tableMap == null) {
202         // Then user wants all tables.
203       } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
204         // this can only happen when WALMapper is used directly by a class other than WALPlayer
205         throw new IOException("No tables or incorrect table mapping specified.");
206       }
207       int i = 0;
208       if (tablesToUse != null) {
209         for (String table : tablesToUse) {
210           tables.put(TableName.valueOf(table),
211             TableName.valueOf(tableMap[i++]));
212         }
213       }
214     }
215   }
216
217   void setupTime(Configuration conf, String option) throws IOException {
218     String val = conf.get(option);
219     if (null == val) return;
220     long ms;
221     try {
222       // first try to parse in user friendly form
223       ms = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SS").parse(val).getTime();
224     } catch (ParseException pe) {
225       try {
226         // then see if just a number of ms's was specified
227         ms = Long.parseLong(val);
228       } catch (NumberFormatException nfe) {
229         throw new IOException(option
230             + " must be specified either in the form 2001-02-20T16:35:06.99 "
231             + "or as number of milliseconds");
232       }
233     }
234     conf.setLong(option, ms);
235   }
236
237   /**
238    * Sets up the actual job.
239    *
240    * @param args  The command line parameters.
241    * @return The newly created job.
242    * @throws IOException When setting up the job fails.
243    */
244   public Job createSubmittableJob(String[] args)
245   throws IOException {
246     Configuration conf = getConf();
247     setupTime(conf, HLogInputFormat.START_TIME_KEY);
248     setupTime(conf, HLogInputFormat.END_TIME_KEY);
249     Path inputDir = new Path(args[0]);
250     String[] tables = args[1].split(",");
251     String[] tableMap;
252     if (args.length > 2) {
253       tableMap = args[2].split(",");
254       if (tableMap.length != tables.length) {
255         throw new IOException("The same number of tables and mapping must be provided.");
256       }
257     } else {
258       // if not mapping is specified map each table to itself
259       tableMap = tables;
260     }
261     conf.setStrings(TABLES_KEY, tables);
262     conf.setStrings(TABLE_MAP_KEY, tableMap);
263     Job job = Job.getInstance(conf, conf.get(JOB_NAME_CONF_KEY, NAME + "_" + inputDir));
264     job.setJarByClass(WALPlayer.class);
265     FileInputFormat.setInputPaths(job, inputDir);
266     job.setInputFormatClass(WALInputFormat.class);
267     job.setMapOutputKeyClass(ImmutableBytesWritable.class);
268     String hfileOutPath = conf.get(BULK_OUTPUT_CONF_KEY);
269     if (hfileOutPath != null) {
270       // the bulk HFile case
271       if (tables.length != 1) {
272         throw new IOException("Exactly one table must be specified for the bulk export option");
273       }
274       TableName tableName = TableName.valueOf(tables[0]);
275       job.setMapperClass(WALKeyValueMapper.class);
276       job.setReducerClass(KeyValueSortReducer.class);
277       Path outputDir = new Path(hfileOutPath);
278       FileOutputFormat.setOutputPath(job, outputDir);
279       job.setMapOutputValueClass(KeyValue.class);
280       try (Connection conn = ConnectionFactory.createConnection(conf);
281           Table table = conn.getTable(tableName);
282           RegionLocator regionLocator = conn.getRegionLocator(tableName)) {
283         HFileOutputFormat2.configureIncrementalLoad(job, table.getTableDescriptor(), regionLocator);
284       }
285       TableMapReduceUtil.addDependencyJars(job.getConfiguration(),
286           com.google.common.base.Preconditions.class);
287     } else {
288       // output to live cluster
289       job.setMapperClass(WALMapper.class);
290       job.setOutputFormatClass(MultiTableOutputFormat.class);
291       TableMapReduceUtil.addDependencyJars(job);
292       TableMapReduceUtil.initCredentials(job);
293       // No reducers.
294       job.setNumReduceTasks(0);
295     }
296     String codecCls = WALCellCodec.getWALCellCodecClass(conf);
297     try {
298       TableMapReduceUtil.addDependencyJars(job.getConfiguration(), Class.forName(codecCls));
299     } catch (Exception e) {
300       throw new IOException("Cannot determine wal codec class " + codecCls, e);
301     }
302     return job;
303   }
304
305   /*
306    * @param errorMsg Error message.  Can be null.
307    */
308   private void usage(final String errorMsg) {
309     if (errorMsg != null && errorMsg.length() > 0) {
310       System.err.println("ERROR: " + errorMsg);
311     }
312     System.err.println("Usage: " + NAME + " [options] <wal inputdir> <tables> [<tableMappings>]");
313     System.err.println("Read all WAL entries for <tables>.");
314     System.err.println("If no tables (\"\") are specific, all tables are imported.");
315     System.err.println("(Careful, even -ROOT- and hbase:meta entries will be imported in that case.)");
316     System.err.println("Otherwise <tables> is a comma separated list of tables.\n");
317     System.err.println("The WAL entries can be mapped to new set of tables via <tableMapping>.");
318     System.err.println("<tableMapping> is a command separated list of targettables.");
319     System.err.println("If specified, each table in <tables> must have a mapping.\n");
320     System.err.println("By default " + NAME + " will load data directly into HBase.");
321     System.err.println("To generate HFiles for a bulk data load instead, pass the option:");
322     System.err.println("  -D" + BULK_OUTPUT_CONF_KEY + "=/path/for/output");
323     System.err.println("  (Only one table can be specified, and no mapping is allowed!)");
324     System.err.println("Other options: (specify time range to WAL edit to consider)");
325     System.err.println("  -D" + WALInputFormat.START_TIME_KEY + "=[date|ms]");
326     System.err.println("  -D" + WALInputFormat.END_TIME_KEY + "=[date|ms]");
327     System.err.println("   -D " + JOB_NAME_CONF_KEY
328         + "=jobName - use the specified mapreduce job name for the wal player");
329     System.err.println("For performance also consider the following options:\n"
330         + "  -Dmapreduce.map.speculative=false\n"
331         + "  -Dmapreduce.reduce.speculative=false");
332   }
333
334   /**
335    * Main entry point.
336    *
337    * @param args  The command line parameters.
338    * @throws Exception When running the job fails.
339    */
340   public static void main(String[] args) throws Exception {
341     int ret = ToolRunner.run(new WALPlayer(HBaseConfiguration.create()), args);
342     System.exit(ret);
343   }
344
345   @Override
346   public int run(String[] args) throws Exception {
347     if (args.length < 2) {
348       usage("Wrong number of arguments: " + args.length);
349       System.exit(-1);
350     }
351     Job job = createSubmittableJob(args);
352     return job.waitForCompletion(true) ? 0 : 1;
353   }
354 }