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