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