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