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