001/** 002 * 003 * Licensed to the Apache Software Foundation (ASF) under one 004 * or more contributor license agreements. See the NOTICE file 005 * distributed with this work for additional information 006 * regarding copyright ownership. The ASF licenses this file 007 * to you under the Apache License, Version 2.0 (the 008 * "License"); you may not use this file except in compliance 009 * with the License. You may obtain a copy of the License at 010 * 011 * http://www.apache.org/licenses/LICENSE-2.0 012 * 013 * Unless required by applicable law or agreed to in writing, software 014 * distributed under the License is distributed on an "AS IS" BASIS, 015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 016 * See the License for the specific language governing permissions and 017 * limitations under the License. 018 */ 019package org.apache.hadoop.hbase.mob.mapreduce; 020 021import java.io.IOException; 022import java.util.Arrays; 023import java.util.Base64; 024import java.util.HashSet; 025import java.util.Set; 026import java.util.UUID; 027 028import org.apache.hadoop.conf.Configuration; 029import org.apache.hadoop.conf.Configured; 030import org.apache.hadoop.fs.FileStatus; 031import org.apache.hadoop.fs.FileSystem; 032import org.apache.hadoop.fs.Path; 033import org.apache.hadoop.hbase.Cell; 034import org.apache.hadoop.hbase.HBaseConfiguration; 035import org.apache.hadoop.hbase.HConstants; 036import org.apache.hadoop.hbase.TableName; 037import org.apache.hadoop.hbase.client.Admin; 038import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 039import org.apache.hadoop.hbase.client.Connection; 040import org.apache.hadoop.hbase.client.ConnectionFactory; 041import org.apache.hadoop.hbase.client.Result; 042import org.apache.hadoop.hbase.client.Scan; 043import org.apache.hadoop.hbase.client.TableDescriptor; 044import org.apache.hadoop.hbase.io.HFileLink; 045import org.apache.hadoop.hbase.io.ImmutableBytesWritable; 046import org.apache.hadoop.hbase.mapreduce.TableInputFormat; 047import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; 048import org.apache.hadoop.hbase.mapreduce.TableMapper; 049import org.apache.hadoop.hbase.mob.MobConstants; 050import org.apache.hadoop.hbase.mob.MobUtils; 051import org.apache.hadoop.hbase.util.Bytes; 052import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 053import org.apache.hadoop.hbase.util.FSUtils; 054import org.apache.hadoop.hbase.util.HFileArchiveUtil; 055import org.apache.hadoop.hbase.util.Pair; 056import org.apache.hadoop.io.Text; 057import org.apache.hadoop.mapreduce.Job; 058import org.apache.hadoop.mapreduce.Reducer; 059import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; 060import org.apache.hadoop.security.UserGroupInformation; 061import org.apache.hadoop.util.Tool; 062import org.apache.hadoop.util.ToolRunner; 063import org.apache.yetus.audience.InterfaceAudience; 064 065import org.slf4j.Logger; 066import org.slf4j.LoggerFactory; 067 068 069/** 070 * Scans a given table + CF for all mob reference cells to get the list of backing mob files. 071 * For each referenced file we attempt to verify that said file is on the FileSystem in a place 072 * that the MOB system will look when attempting to resolve the actual value. 073 * 074 * The job includes counters that can help provide a rough sketch of the mob data. 075 * 076 * <pre> 077 * Map-Reduce Framework 078 * Map input records=10000 079 * ... 080 * Reduce output records=99 081 * ... 082 * CELLS PER ROW 083 * Number of rows with 1s of cells per row=10000 084 * MOB 085 * NUM_CELLS=52364 086 * PROBLEM 087 * Affected rows=338 088 * Problem MOB files=2 089 * ROWS WITH PROBLEMS PER FILE 090 * Number of HFiles with 100s of affected rows=2 091 * SIZES OF CELLS 092 * Number of cells with size in the 10,000s of bytes=627 093 * Number of cells with size in the 100,000s of bytes=51392 094 * Number of cells with size in the 1,000,000s of bytes=345 095 * SIZES OF ROWS 096 * Number of rows with total size in the 100,000s of bytes=6838 097 * Number of rows with total size in the 1,000,000s of bytes=3162 098 * </pre> 099 * 100 * * Map-Reduce Framework:Map input records - the number of rows with mob references 101 * * Map-Reduce Framework:Reduce output records - the number of unique hfiles referenced 102 * * MOB:NUM_CELLS - the total number of mob reference cells 103 * * PROBLEM:Affected rows - the number of rows that reference hfiles with an issue 104 * * PROBLEM:Problem MOB files - the number of unique hfiles that have an issue 105 * * CELLS PER ROW: - this counter group gives a histogram of the order of magnitude of the 106 * number of cells in a given row by grouping by the number of digits used in each count. 107 * This allows us to see more about the distribution of cells than what we can determine 108 * with just the cell count and the row count. In this particular example we can see that 109 * all of our rows have somewhere between 1 - 9 cells. 110 * * ROWS WITH PROBLEMS PER FILE: - this counter group gives a histogram of the order of 111 * magnitude of the number of rows in each of the hfiles with a problem. e.g. in the 112 * example there are 2 hfiles and they each have the same order of magnitude number of rows, 113 * specifically between 100 and 999. 114 * * SIZES OF CELLS: - this counter group gives a histogram of the order of magnitude of 115 * the size of mob values according to our reference cells. e.g. in the example above we 116 * have cell sizes that are all between 10,000 bytes and 9,999,999 bytes. From this 117 * histogram we can also see that _most_ cells are 100,000 - 999,000 bytes and the smaller 118 * and bigger ones are outliers making up less than 2% of mob cells. 119 * * SIZES OF ROWS: - this counter group gives a histogram of the order of magnitude of the 120 * size of mob values across each row according to our reference cells. In the example above 121 * we have rows that are are between 100,000 bytes and 9,999,999 bytes. We can also see that 122 * about 2/3rd of our rows are 100,000 - 999,999 bytes. 123 * 124 * Generates a report that gives one file status per line, with tabs dividing fields. 125 * 126 * <pre> 127 * RESULT OF LOOKUP FILE REF comma seperated, base64 encoded rows when there's a problem 128 * </pre> 129 * 130 * e.g. 131 * 132 * <pre> 133 * MOB DIR 09c576e28a65ed2ead0004d192ffaa382019110184b30a1c7e034573bf8580aef8393402 134 * MISSING FILE 28e252d7f013973174750d483d358fa020191101f73536e7133f4cd3ab1065edf588d509 MmJiMjMyYzBiMTNjNzc0OTY1ZWY4NTU4ZjBmYmQ2MTUtNTIz,MmEzOGE0YTkzMTZjNDllNWE4MzM1MTdjNDVkMzEwNzAtODg= 135 * </pre> 136 * 137 * Possible results are listed; the first three indicate things are working properly. 138 * * MOB DIR - the reference is in the normal MOB area for the given table and CF 139 * * HLINK TO ARCHIVE FOR SAME TABLE - the reference is present in the archive area for this 140 * table and CF 141 * * HLINK TO ARCHIVE FOR OTHER TABLE - the reference is present in a different table and CF, 142 * either in the MOB or archive areas (e.g. from a snapshot restore or clone) 143 * * ARCHIVE WITH HLINK BUT NOT FROM OUR TABLE - the reference is currently present in the archive 144 * area for this table and CF, but it is kept there because a _different_ table has a 145 * reference to it (e.g. from a snapshot clone). If these other tables are removed then 146 * the file will likely be deleted unless there is a snapshot also referencing it. 147 * * ARCHIVE BUT NO HLINKS - the reference is currently present in the archive for this table and 148 * CF, but there are no references present to prevent its removal. Unless it is newer than 149 * the general TTL (default 5 minutes) or referenced in a snapshot it will be subject to 150 * cleaning. 151 * * ARCHIVE BUT FAILURE WHILE CHECKING HLINKS - Check the job logs to see why things failed while 152 * looking for why this file is being kept around. 153 * * MISSING FILE - We couldn't find the reference on the FileSystem. Either there is dataloss due 154 * to a bug in the MOB storage system or the MOB storage is damaged but in an edge case that 155 * allows it to work for now. You can verify which by doing a raw reference scan to get the 156 * referenced hfile and check the underlying filesystem. See the ref guide section on mob 157 * for details. 158 * * HLINK BUT POINT TO MISSING FILE - There is a pointer in our mob area for this table and CF 159 * to a file elsewhere on the FileSystem, however the file it points to no longer exists. 160 * * MISSING FILE BUT FAILURE WHILE CHECKING HLINKS - We could not find the referenced file, 161 * however you should check the job logs to see why we couldn't check to see if there is a 162 * pointer to the referenced file in our archive or another table's archive or mob area. 163 * 164 */ 165@InterfaceAudience.Private 166public class MobRefReporter extends Configured implements Tool { 167 private static Logger LOG = LoggerFactory.getLogger(MobRefReporter.class); 168 public static final String NAME = "mobrefs"; 169 static final String REPORT_JOB_ID = "mob.report.job.id"; 170 static final String REPORT_START_DATETIME = "mob.report.job.start"; 171 172 public static class MobRefMapper extends TableMapper<Text, ImmutableBytesWritable> { 173 @Override 174 public void map(ImmutableBytesWritable r, Result columns, Context context) throws IOException, 175 InterruptedException { 176 if (columns == null) { 177 return; 178 } 179 Cell[] cells = columns.rawCells(); 180 if (cells == null || cells.length == 0) { 181 return; 182 } 183 Set<String> files = new HashSet<>(); 184 long count = 0; 185 long size = 0; 186 for (Cell c : cells) { 187 if (MobUtils.hasValidMobRefCellValue(c)) { 188 // TODO confirm there aren't tags 189 String fileName = MobUtils.getMobFileName(c); 190 if (!files.contains(fileName)) { 191 context.write(new Text(fileName), r); 192 files.add(fileName); 193 } 194 final int cellsize = MobUtils.getMobValueLength(c); 195 context.getCounter("SIZES OF CELLS", "Number of cells with size in the " + 196 log10GroupedString(cellsize) + "s of bytes").increment(1L); 197 size += cellsize; 198 count++; 199 } else { 200 LOG.debug("cell is not a mob ref, even though we asked for only refs. cell={}", c); 201 } 202 } 203 context.getCounter("CELLS PER ROW", "Number of rows with " + log10GroupedString(count) + 204 "s of cells per row").increment(1L); 205 context.getCounter("SIZES OF ROWS", "Number of rows with total size in the " + 206 log10GroupedString(size) + "s of bytes").increment(1L); 207 context.getCounter("MOB","NUM_CELLS").increment(count); 208 } 209 } 210 211 public static class MobRefReducer extends 212 Reducer<Text, ImmutableBytesWritable, Text, Text> { 213 214 TableName table; 215 String mobRegion; 216 Path mob; 217 Path archive; 218 String seperator; 219 220 /* Results that mean things are fine */ 221 final Text OK_MOB_DIR = new Text("MOB DIR"); 222 final Text OK_HLINK_RESTORE = new Text("HLINK TO ARCHIVE FOR SAME TABLE"); 223 final Text OK_HLINK_CLONE = new Text("HLINK TO ARCHIVE FOR OTHER TABLE"); 224 /* Results that mean something is incorrect */ 225 final Text INCONSISTENT_ARCHIVE_BAD_LINK = 226 new Text("ARCHIVE WITH HLINK BUT NOT FROM OUR TABLE"); 227 final Text INCONSISTENT_ARCHIVE_STALE = new Text("ARCHIVE BUT NO HLINKS"); 228 final Text INCONSISTENT_ARCHIVE_IOE = new Text("ARCHIVE BUT FAILURE WHILE CHECKING HLINKS"); 229 /* Results that mean data is probably already gone */ 230 final Text DATALOSS_MISSING = new Text("MISSING FILE"); 231 final Text DATALOSS_HLINK_DANGLING = new Text("HLINK BUT POINTS TO MISSING FILE"); 232 final Text DATALOSS_MISSING_IOE = new Text("MISSING FILE BUT FAILURE WHILE CHECKING HLINKS"); 233 final Base64.Encoder base64 = Base64.getEncoder(); 234 235 @Override 236 public void setup(Context context) throws IOException, InterruptedException { 237 final Configuration conf = context.getConfiguration(); 238 final String tableName = conf.get(TableInputFormat.INPUT_TABLE); 239 if (null == tableName) { 240 throw new IOException("Job configuration did not include table."); 241 } 242 table = TableName.valueOf(tableName); 243 mobRegion = MobUtils.getMobRegionInfo(table).getEncodedName(); 244 final String family = conf.get(TableInputFormat.SCAN_COLUMN_FAMILY); 245 if (null == family) { 246 throw new IOException("Job configuration did not include column family"); 247 } 248 mob = MobUtils.getMobFamilyPath(conf, table, family); 249 LOG.info("Using active mob area '{}'", mob); 250 archive = HFileArchiveUtil.getStoreArchivePath(conf, table, 251 MobUtils.getMobRegionInfo(table).getEncodedName(), family); 252 LOG.info("Using archive mob area '{}'", archive); 253 seperator = conf.get(TextOutputFormat.SEPERATOR, "\t"); 254 } 255 256 @Override 257 public void reduce(Text key, Iterable<ImmutableBytesWritable> rows, Context context) 258 throws IOException, InterruptedException { 259 final Configuration conf = context.getConfiguration(); 260 final String file = key.toString(); 261 // active mob area 262 if (mob.getFileSystem(conf).exists(new Path(mob, file))) { 263 LOG.debug("Found file '{}' in mob area", file); 264 context.write(OK_MOB_DIR, key); 265 // archive area - is there an hlink back reference (from a snapshot from same table) 266 } else if (archive.getFileSystem(conf).exists(new Path(archive, file))) { 267 268 Path backRefDir = HFileLink.getBackReferencesDir(archive, file); 269 try { 270 FileStatus[] backRefs = FSUtils.listStatus(archive.getFileSystem(conf), backRefDir); 271 if (backRefs != null) { 272 boolean found = false; 273 for (FileStatus backRef : backRefs) { 274 Pair<TableName, String> refParts = HFileLink.parseBackReferenceName( 275 backRef.getPath().getName()); 276 if (table.equals(refParts.getFirst()) && mobRegion.equals(refParts.getSecond())) { 277 Path hlinkPath = HFileLink.getHFileFromBackReference(MobUtils.getMobHome(conf), 278 backRef.getPath()); 279 if (hlinkPath.getFileSystem(conf).exists(hlinkPath)) { 280 found = true; 281 } else { 282 LOG.warn("Found file '{}' in archive area with a back reference to the mob area " 283 + "for our table, but the mob area does not have a corresponding hfilelink.", 284 file); 285 } 286 } 287 } 288 if (found) { 289 LOG.debug("Found file '{}' in archive area. has proper hlink back references to " 290 + "suggest it is from a restored snapshot for this table.", file); 291 context.write(OK_HLINK_RESTORE, key); 292 } else { 293 LOG.warn("Found file '{}' in archive area, but the hlink back references do not " 294 + "properly point to the mob area for our table.", file); 295 context.write(INCONSISTENT_ARCHIVE_BAD_LINK, encodeRows(context, key, rows)); 296 } 297 } else { 298 LOG.warn("Found file '{}' in archive area, but there are no hlinks pointing to it. Not " 299 + "yet used snapshot or an error.", file); 300 context.write(INCONSISTENT_ARCHIVE_STALE, encodeRows(context, key, rows)); 301 } 302 } catch (IOException e) { 303 LOG.warn("Found file '{}' in archive area, but got an error while checking " 304 + "on back references.", file, e); 305 context.write(INCONSISTENT_ARCHIVE_IOE, encodeRows(context, key, rows)); 306 } 307 308 } else { 309 // check for an hlink in the active mob area (from a snapshot of a different table) 310 try { 311 /** 312 * we are doing this ourselves instead of using FSUtils.getReferenceFilePaths because 313 * we know the mob region never splits, so we can only have HFileLink references 314 * and looking for just them is cheaper then listing everything. 315 * 316 * This glob should match the naming convention for HFileLinks to our referenced hfile. 317 * As simplified explanation those file names look like "table=region-hfile". For details 318 * see the {@link HFileLink#createHFileLinkName HFileLink implementation}. 319 */ 320 FileStatus[] hlinks = mob.getFileSystem(conf).globStatus(new Path(mob + "/*=*-" + file)); 321 if (hlinks != null && hlinks.length != 0) { 322 if (hlinks.length != 1) { 323 LOG.warn("Found file '{}' as hfilelinks in the mob area, but there are more than " + 324 "one: {}", file, Arrays.deepToString(hlinks)); 325 } 326 HFileLink found = null; 327 for (FileStatus hlink : hlinks) { 328 HFileLink tmp = HFileLink.buildFromHFileLinkPattern(conf, hlink.getPath()); 329 if (tmp.exists(archive.getFileSystem(conf))) { 330 found = tmp; 331 break; 332 } else { 333 LOG.debug("Target file does not exist for ref {}", tmp); 334 } 335 } 336 if (found != null) { 337 LOG.debug("Found file '{}' as a ref in the mob area: {}", file, found); 338 context.write(OK_HLINK_CLONE, key); 339 } else { 340 LOG.warn("Found file '{}' as ref(s) in the mob area but they do not point to an hfile" 341 + " that exists.", file); 342 context.write(DATALOSS_HLINK_DANGLING, encodeRows(context, key, rows)); 343 } 344 } else { 345 LOG.error("Could not find referenced file '{}'. See the docs on this tool.", file); 346 LOG.debug("Note that we don't have the server-side tag from the mob cells that says " 347 + "what table the reference is originally from. So if the HFileLink in this table " 348 + "is missing but the referenced file is still in the table from that tag, then " 349 + "lookups of these impacted rows will work. Do a scan of the reference details " 350 + "of the cell for the hfile name and then check the entire hbase install if this " 351 + "table was made from a snapshot of another table. see the ref guide section on " 352 + "mob for details."); 353 context.write(DATALOSS_MISSING, encodeRows(context, key, rows)); 354 } 355 } catch (IOException e) { 356 LOG.error( 357 "Exception while checking mob area of our table for HFileLinks that point to {}", 358 file, e); 359 context.write(DATALOSS_MISSING_IOE, encodeRows(context, key, rows)); 360 } 361 } 362 } 363 364 /** 365 * reuses the passed Text key. appends the configured seperator and then a comma seperated list 366 * of base64 encoded row keys 367 */ 368 private Text encodeRows(Context context, Text key, Iterable<ImmutableBytesWritable> rows) 369 throws IOException { 370 StringBuilder sb = new StringBuilder(key.toString()); 371 sb.append(seperator); 372 boolean moreThanOne = false; 373 long count = 0; 374 for (ImmutableBytesWritable row : rows) { 375 if (moreThanOne) { 376 sb.append(","); 377 } 378 sb.append(base64.encodeToString(row.copyBytes())); 379 moreThanOne = true; 380 count++; 381 } 382 context.getCounter("PROBLEM", "Problem MOB files").increment(1L); 383 context.getCounter("PROBLEM", "Affected rows").increment(count); 384 context.getCounter("ROWS WITH PROBLEMS PER FILE", "Number of HFiles with " + 385 log10GroupedString(count) + "s of affected rows").increment(1L); 386 key.set(sb.toString()); 387 return key; 388 } 389 } 390 391 /** 392 * Returns the string representation of the given number after grouping it 393 * into log10 buckets. e.g. 0-9 -> 1, 10-99 -> 10, ..., 100,000-999,999 -> 100,000, etc. 394 */ 395 static String log10GroupedString(long number) { 396 return String.format("%,d", (long)(Math.pow(10d, Math.floor(Math.log10(number))))); 397 } 398 399 /** 400 * Main method for the tool. 401 * @return 0 if success, 1 for bad args. 2 if job aborted with an exception, 402 * 3 if mr job was unsuccessful 403 */ 404 public int run(String[] args) throws IOException, InterruptedException { 405 // TODO make family and table optional 406 if (args.length != 3) { 407 printUsage(); 408 return 1; 409 } 410 final String output = args[0]; 411 final String tableName = args[1]; 412 final String familyName = args[2]; 413 final long reportStartTime = EnvironmentEdgeManager.currentTime(); 414 Configuration conf = getConf(); 415 try { 416 FileSystem fs = FileSystem.get(conf); 417 // check whether the current user is the same one with the owner of hbase root 418 String currentUserName = UserGroupInformation.getCurrentUser().getShortUserName(); 419 FileStatus[] hbaseRootFileStat = fs.listStatus(new Path(conf.get(HConstants.HBASE_DIR))); 420 if (hbaseRootFileStat.length > 0) { 421 String owner = hbaseRootFileStat[0].getOwner(); 422 if (!owner.equals(currentUserName)) { 423 String errorMsg = "The current user[" + currentUserName 424 + "] does not have hbase root credentials." 425 + " If this job fails due to an inability to read HBase's internal directories, " 426 + "you will need to rerun as a user with sufficient permissions. The HBase superuser " 427 + "is a safe choice."; 428 LOG.warn(errorMsg); 429 } 430 } else { 431 LOG.error("The passed configs point to an HBase dir does not exist: {}", 432 conf.get(HConstants.HBASE_DIR)); 433 throw new IOException("The target HBase does not exist"); 434 } 435 436 byte[] family; 437 int maxVersions; 438 TableName tn = TableName.valueOf(tableName); 439 try (Connection connection = ConnectionFactory.createConnection(conf); 440 Admin admin = connection.getAdmin()) { 441 TableDescriptor htd = admin.getDescriptor(tn); 442 ColumnFamilyDescriptor hcd = htd.getColumnFamily(Bytes.toBytes(familyName)); 443 if (hcd == null || !hcd.isMobEnabled()) { 444 throw new IOException("Column family " + familyName + " is not a MOB column family"); 445 } 446 family = hcd.getName(); 447 maxVersions = hcd.getMaxVersions(); 448 } 449 450 451 String id = getClass().getSimpleName() + UUID.randomUUID().toString().replace("-", ""); 452 Job job = null; 453 Scan scan = new Scan(); 454 scan.addFamily(family); 455 // Do not retrieve the mob data when scanning 456 scan.setAttribute(MobConstants.MOB_SCAN_RAW, Bytes.toBytes(Boolean.TRUE)); 457 scan.setAttribute(MobConstants.MOB_SCAN_REF_ONLY, Bytes.toBytes(Boolean.TRUE)); 458 // If a scanner caching value isn't set, pick a smaller default since we know we're doing 459 // a full table scan and don't want to impact other clients badly. 460 scan.setCaching(conf.getInt(HConstants.HBASE_CLIENT_SCANNER_CACHING, 10000)); 461 scan.setCacheBlocks(false); 462 scan.setMaxVersions(maxVersions); 463 conf.set(REPORT_JOB_ID, id); 464 465 job = Job.getInstance(conf); 466 job.setJarByClass(getClass()); 467 TableMapReduceUtil.initTableMapperJob(tn, scan, 468 MobRefMapper.class, Text.class, ImmutableBytesWritable.class, job); 469 470 job.setReducerClass(MobRefReducer.class); 471 job.setOutputFormatClass(TextOutputFormat.class); 472 TextOutputFormat.setOutputPath(job, new Path(output)); 473 474 job.setJobName(getClass().getSimpleName() + "-" + tn + "-" + familyName); 475 // for use in the reducer. easier than re-parsing it out of the scan string. 476 job.getConfiguration().set(TableInputFormat.SCAN_COLUMN_FAMILY, familyName); 477 478 // Use when we start this job as the base point for file "recency". 479 job.getConfiguration().setLong(REPORT_START_DATETIME, reportStartTime); 480 481 if (job.waitForCompletion(true)) { 482 LOG.info("Finished creating report for '{}', family='{}'", tn, familyName); 483 } else { 484 System.err.println("Job was not successful"); 485 return 3; 486 } 487 return 0; 488 489 } catch (ClassNotFoundException | RuntimeException | IOException | InterruptedException e) { 490 System.err.println("Job aborted due to exception " + e); 491 return 2; // job failed 492 } 493 } 494 495 public static void main(String[] args) throws Exception { 496 Configuration conf = HBaseConfiguration.create(); 497 int ret = ToolRunner.run(conf, new MobRefReporter(), args); 498 System.exit(ret); 499 } 500 501 private void printUsage() { 502 System.err.println("Usage:\n" + "--------------------------\n" + MobRefReporter.class.getName() 503 + " output-dir tableName familyName"); 504 System.err.println(" output-dir Where to write output report."); 505 System.err.println(" tableName The table name"); 506 System.err.println(" familyName The column family name"); 507 } 508 509}