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