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