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