1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
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
59
60
61
62
63
64
65
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
77
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
94
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
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
121 String[] tables = context.getConfiguration().getStrings(TABLES_KEY);
122 if (tables == null || tables.length != 1) {
123
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
132
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
152 if (WALEdit.isMetaEditFamily(cell)) continue;
153
154
155 if (filter(context, cell)) {
156
157
158
159
160 if (lastCell == null || lastCell.getTypeByte() != cell.getTypeByte()
161 || !CellUtil.matchingRow(lastCell, cell)) {
162
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
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
190
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
202 } else if (tablesToUse == null || tableMap == null || tablesToUse.length != tableMap.length) {
203
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
222 ms = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SS").parse(val).getTime();
223 } catch (ParseException pe) {
224 try {
225
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
238
239
240
241
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
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
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
288 job.setMapperClass(WALMapper.class);
289 job.setOutputFormatClass(MultiTableOutputFormat.class);
290 TableMapReduceUtil.addDependencyJars(job);
291 TableMapReduceUtil.initCredentials(job);
292
293 job.setNumReduceTasks(0);
294 }
295 return job;
296 }
297
298
299
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
329
330
331
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 }