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.regionserver; 019 020import static org.apache.hadoop.hbase.regionserver.Store.PRIORITY_USER; 021 022import java.io.IOException; 023import java.util.ArrayList; 024import java.util.HashSet; 025import java.util.LinkedList; 026import java.util.List; 027import java.util.Optional; 028import java.util.Set; 029 030import org.apache.hadoop.conf.Configuration; 031import org.apache.hadoop.conf.Configured; 032import org.apache.hadoop.fs.FSDataOutputStream; 033import org.apache.hadoop.fs.FileStatus; 034import org.apache.hadoop.fs.FileSystem; 035import org.apache.hadoop.fs.Path; 036import org.apache.hadoop.hbase.HBaseConfiguration; 037import org.apache.hadoop.hbase.HBaseInterfaceAudience; 038import org.apache.hadoop.hbase.HDFSBlocksDistribution; 039import org.apache.hadoop.hbase.client.RegionInfo; 040import org.apache.hadoop.hbase.client.TableDescriptor; 041import org.apache.hadoop.hbase.mapreduce.JobUtil; 042import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil; 043import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; 044import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; 045import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; 046import org.apache.hadoop.hbase.util.Bytes; 047import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 048import org.apache.hadoop.hbase.util.FSTableDescriptors; 049import org.apache.hadoop.hbase.util.FSUtils; 050import org.apache.hadoop.io.LongWritable; 051import org.apache.hadoop.io.NullWritable; 052import org.apache.hadoop.io.Text; 053import org.apache.hadoop.mapreduce.InputSplit; 054import org.apache.hadoop.mapreduce.Job; 055import org.apache.hadoop.mapreduce.JobContext; 056import org.apache.hadoop.mapreduce.Mapper; 057import org.apache.hadoop.mapreduce.lib.input.FileSplit; 058import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; 059import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; 060import org.apache.hadoop.util.LineReader; 061import org.apache.hadoop.util.Tool; 062import org.apache.hadoop.util.ToolRunner; 063import org.apache.yetus.audience.InterfaceAudience; 064import org.slf4j.Logger; 065import org.slf4j.LoggerFactory; 066 067/* 068 * The CompactionTool allows to execute a compaction specifying a: 069 * <ul> 070 * <li>table folder (all regions and families will be compacted) 071 * <li>region folder (all families in the region will be compacted) 072 * <li>family folder (the store files will be compacted) 073 * </ul> 074 */ 075@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS) 076public class CompactionTool extends Configured implements Tool { 077 private static final Logger LOG = LoggerFactory.getLogger(CompactionTool.class); 078 079 private final static String CONF_TMP_DIR = "hbase.tmp.dir"; 080 private final static String CONF_COMPACT_ONCE = "hbase.compactiontool.compact.once"; 081 private final static String CONF_COMPACT_MAJOR = "hbase.compactiontool.compact.major"; 082 private final static String CONF_DELETE_COMPACTED = "hbase.compactiontool.delete"; 083 084 /** 085 * Class responsible to execute the Compaction on the specified path. 086 * The path can be a table, region or family directory. 087 */ 088 private static class CompactionWorker { 089 private final boolean deleteCompacted; 090 private final Configuration conf; 091 private final FileSystem fs; 092 private final Path tmpDir; 093 094 public CompactionWorker(final FileSystem fs, final Configuration conf) { 095 this.conf = conf; 096 this.deleteCompacted = conf.getBoolean(CONF_DELETE_COMPACTED, false); 097 this.tmpDir = new Path(conf.get(CONF_TMP_DIR)); 098 this.fs = fs; 099 } 100 101 /** 102 * Execute the compaction on the specified path. 103 * 104 * @param path Directory path on which to run compaction. 105 * @param compactOnce Execute just a single step of compaction. 106 * @param major Request major compaction. 107 */ 108 public void compact(final Path path, final boolean compactOnce, final boolean major) throws IOException { 109 if (isFamilyDir(fs, path)) { 110 Path regionDir = path.getParent(); 111 Path tableDir = regionDir.getParent(); 112 TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); 113 RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); 114 compactStoreFiles(tableDir, htd, hri, 115 path.getName(), compactOnce, major); 116 } else if (isRegionDir(fs, path)) { 117 Path tableDir = path.getParent(); 118 TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); 119 compactRegion(tableDir, htd, path, compactOnce, major); 120 } else if (isTableDir(fs, path)) { 121 compactTable(path, compactOnce, major); 122 } else { 123 throw new IOException( 124 "Specified path is not a table, region or family directory. path=" + path); 125 } 126 } 127 128 private void compactTable(final Path tableDir, final boolean compactOnce, final boolean major) 129 throws IOException { 130 TableDescriptor htd = FSTableDescriptors.getTableDescriptorFromFs(fs, tableDir); 131 for (Path regionDir: FSUtils.getRegionDirs(fs, tableDir)) { 132 compactRegion(tableDir, htd, regionDir, compactOnce, major); 133 } 134 } 135 136 private void compactRegion(final Path tableDir, final TableDescriptor htd, 137 final Path regionDir, final boolean compactOnce, final boolean major) 138 throws IOException { 139 RegionInfo hri = HRegionFileSystem.loadRegionInfoFileContent(fs, regionDir); 140 for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) { 141 compactStoreFiles(tableDir, htd, hri, familyDir.getName(), compactOnce, major); 142 } 143 } 144 145 /** 146 * Execute the actual compaction job. 147 * If the compact once flag is not specified, execute the compaction until 148 * no more compactions are needed. Uses the Configuration settings provided. 149 */ 150 private void compactStoreFiles(final Path tableDir, final TableDescriptor htd, 151 final RegionInfo hri, final String familyName, final boolean compactOnce, 152 final boolean major) throws IOException { 153 HStore store = getStore(conf, fs, tableDir, htd, hri, familyName, tmpDir); 154 LOG.info("Compact table=" + htd.getTableName() + 155 " region=" + hri.getRegionNameAsString() + 156 " family=" + familyName); 157 if (major) { 158 store.triggerMajorCompaction(); 159 } 160 do { 161 Optional<CompactionContext> compaction = 162 store.requestCompaction(PRIORITY_USER, CompactionLifeCycleTracker.DUMMY, null); 163 if (!compaction.isPresent()) { 164 break; 165 } 166 List<HStoreFile> storeFiles = 167 store.compact(compaction.get(), NoLimitThroughputController.INSTANCE, null); 168 if (storeFiles != null && !storeFiles.isEmpty()) { 169 if (deleteCompacted) { 170 for (HStoreFile storeFile: storeFiles) { 171 fs.delete(storeFile.getPath(), false); 172 } 173 } 174 } 175 } while (store.needsCompaction() && !compactOnce); 176 } 177 178 /** 179 * Create a "mock" HStore that uses the tmpDir specified by the user and 180 * the store dir to compact as source. 181 */ 182 private static HStore getStore(final Configuration conf, final FileSystem fs, 183 final Path tableDir, final TableDescriptor htd, final RegionInfo hri, 184 final String familyName, final Path tempDir) throws IOException { 185 HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, hri) { 186 @Override 187 public Path getTempDir() { 188 return tempDir; 189 } 190 }; 191 HRegion region = new HRegion(regionFs, null, conf, htd, null); 192 return new HStore(region, htd.getColumnFamily(Bytes.toBytes(familyName)), conf); 193 } 194 } 195 196 private static boolean isRegionDir(final FileSystem fs, final Path path) throws IOException { 197 Path regionInfo = new Path(path, HRegionFileSystem.REGION_INFO_FILE); 198 return fs.exists(regionInfo); 199 } 200 201 private static boolean isTableDir(final FileSystem fs, final Path path) throws IOException { 202 return FSTableDescriptors.getTableInfoPath(fs, path) != null; 203 } 204 205 private static boolean isFamilyDir(final FileSystem fs, final Path path) throws IOException { 206 return isRegionDir(fs, path.getParent()); 207 } 208 209 private static class CompactionMapper 210 extends Mapper<LongWritable, Text, NullWritable, NullWritable> { 211 private CompactionWorker compactor = null; 212 private boolean compactOnce = false; 213 private boolean major = false; 214 215 @Override 216 public void setup(Context context) { 217 Configuration conf = context.getConfiguration(); 218 compactOnce = conf.getBoolean(CONF_COMPACT_ONCE, false); 219 major = conf.getBoolean(CONF_COMPACT_MAJOR, false); 220 221 try { 222 FileSystem fs = FileSystem.get(conf); 223 this.compactor = new CompactionWorker(fs, conf); 224 } catch (IOException e) { 225 throw new RuntimeException("Could not get the input FileSystem", e); 226 } 227 } 228 229 @Override 230 public void map(LongWritable key, Text value, Context context) 231 throws InterruptedException, IOException { 232 Path path = new Path(value.toString()); 233 this.compactor.compact(path, compactOnce, major); 234 } 235 } 236 237 /** 238 * Input format that uses store files block location as input split locality. 239 */ 240 private static class CompactionInputFormat extends TextInputFormat { 241 @Override 242 protected boolean isSplitable(JobContext context, Path file) { 243 return true; 244 } 245 246 /** 247 * Returns a split for each store files directory using the block location 248 * of each file as locality reference. 249 */ 250 @Override 251 public List<InputSplit> getSplits(JobContext job) throws IOException { 252 List<InputSplit> splits = new ArrayList<>(); 253 List<FileStatus> files = listStatus(job); 254 255 Text key = new Text(); 256 for (FileStatus file: files) { 257 Path path = file.getPath(); 258 FileSystem fs = path.getFileSystem(job.getConfiguration()); 259 LineReader reader = new LineReader(fs.open(path)); 260 long pos = 0; 261 int n; 262 try { 263 while ((n = reader.readLine(key)) > 0) { 264 String[] hosts = getStoreDirHosts(fs, path); 265 splits.add(new FileSplit(path, pos, n, hosts)); 266 pos += n; 267 } 268 } finally { 269 reader.close(); 270 } 271 } 272 273 return splits; 274 } 275 276 /** 277 * return the top hosts of the store files, used by the Split 278 */ 279 private static String[] getStoreDirHosts(final FileSystem fs, final Path path) 280 throws IOException { 281 FileStatus[] files = FSUtils.listStatus(fs, path); 282 if (files == null) { 283 return new String[] {}; 284 } 285 286 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); 287 for (FileStatus hfileStatus: files) { 288 HDFSBlocksDistribution storeFileBlocksDistribution = 289 FSUtils.computeHDFSBlocksDistribution(fs, hfileStatus, 0, hfileStatus.getLen()); 290 hdfsBlocksDistribution.add(storeFileBlocksDistribution); 291 } 292 293 List<String> hosts = hdfsBlocksDistribution.getTopHosts(); 294 return hosts.toArray(new String[hosts.size()]); 295 } 296 297 /** 298 * Create the input file for the given directories to compact. 299 * The file is a TextFile with each line corrisponding to a 300 * store files directory to compact. 301 */ 302 public static void createInputFile(final FileSystem fs, final Path path, 303 final Set<Path> toCompactDirs) throws IOException { 304 // Extract the list of store dirs 305 List<Path> storeDirs = new LinkedList<>(); 306 for (Path compactDir: toCompactDirs) { 307 if (isFamilyDir(fs, compactDir)) { 308 storeDirs.add(compactDir); 309 } else if (isRegionDir(fs, compactDir)) { 310 for (Path familyDir: FSUtils.getFamilyDirs(fs, compactDir)) { 311 storeDirs.add(familyDir); 312 } 313 } else if (isTableDir(fs, compactDir)) { 314 // Lookup regions 315 for (Path regionDir: FSUtils.getRegionDirs(fs, compactDir)) { 316 for (Path familyDir: FSUtils.getFamilyDirs(fs, regionDir)) { 317 storeDirs.add(familyDir); 318 } 319 } 320 } else { 321 throw new IOException( 322 "Specified path is not a table, region or family directory. path=" + compactDir); 323 } 324 } 325 326 // Write Input File 327 FSDataOutputStream stream = fs.create(path); 328 LOG.info("Create input file=" + path + " with " + storeDirs.size() + " dirs to compact."); 329 try { 330 final byte[] newLine = Bytes.toBytes("\n"); 331 for (Path storeDir: storeDirs) { 332 stream.write(Bytes.toBytes(storeDir.toString())); 333 stream.write(newLine); 334 } 335 } finally { 336 stream.close(); 337 } 338 } 339 } 340 341 /** 342 * Execute compaction, using a Map-Reduce job. 343 */ 344 private int doMapReduce(final FileSystem fs, final Set<Path> toCompactDirs, 345 final boolean compactOnce, final boolean major) throws Exception { 346 Configuration conf = getConf(); 347 conf.setBoolean(CONF_COMPACT_ONCE, compactOnce); 348 conf.setBoolean(CONF_COMPACT_MAJOR, major); 349 350 Job job = new Job(conf); 351 job.setJobName("CompactionTool"); 352 job.setJarByClass(CompactionTool.class); 353 job.setMapperClass(CompactionMapper.class); 354 job.setInputFormatClass(CompactionInputFormat.class); 355 job.setOutputFormatClass(NullOutputFormat.class); 356 job.setMapSpeculativeExecution(false); 357 job.setNumReduceTasks(0); 358 359 // add dependencies (including HBase ones) 360 TableMapReduceUtil.addDependencyJars(job); 361 362 Path stagingDir = JobUtil.getStagingDir(conf); 363 try { 364 // Create input file with the store dirs 365 Path inputPath = new Path(stagingDir, "compact-"+ EnvironmentEdgeManager.currentTime()); 366 CompactionInputFormat.createInputFile(fs, inputPath, toCompactDirs); 367 CompactionInputFormat.addInputPath(job, inputPath); 368 369 // Initialize credential for secure cluster 370 TableMapReduceUtil.initCredentials(job); 371 372 // Start the MR Job and wait 373 return job.waitForCompletion(true) ? 0 : 1; 374 } finally { 375 fs.delete(stagingDir, true); 376 } 377 } 378 379 /** 380 * Execute compaction, from this client, one path at the time. 381 */ 382 private int doClient(final FileSystem fs, final Set<Path> toCompactDirs, 383 final boolean compactOnce, final boolean major) throws IOException { 384 CompactionWorker worker = new CompactionWorker(fs, getConf()); 385 for (Path path: toCompactDirs) { 386 worker.compact(path, compactOnce, major); 387 } 388 return 0; 389 } 390 391 @Override 392 public int run(String[] args) throws Exception { 393 Set<Path> toCompactDirs = new HashSet<>(); 394 boolean compactOnce = false; 395 boolean major = false; 396 boolean mapred = false; 397 398 Configuration conf = getConf(); 399 FileSystem fs = FileSystem.get(conf); 400 401 try { 402 for (int i = 0; i < args.length; ++i) { 403 String opt = args[i]; 404 if (opt.equals("-compactOnce")) { 405 compactOnce = true; 406 } else if (opt.equals("-major")) { 407 major = true; 408 } else if (opt.equals("-mapred")) { 409 mapred = true; 410 } else if (!opt.startsWith("-")) { 411 Path path = new Path(opt); 412 FileStatus status = fs.getFileStatus(path); 413 if (!status.isDirectory()) { 414 printUsage("Specified path is not a directory. path=" + path); 415 return 1; 416 } 417 toCompactDirs.add(path); 418 } else { 419 printUsage(); 420 } 421 } 422 } catch (Exception e) { 423 printUsage(e.getMessage()); 424 return 1; 425 } 426 427 if (toCompactDirs.isEmpty()) { 428 printUsage("No directories to compact specified."); 429 return 1; 430 } 431 432 // Execute compaction! 433 if (mapred) { 434 return doMapReduce(fs, toCompactDirs, compactOnce, major); 435 } else { 436 return doClient(fs, toCompactDirs, compactOnce, major); 437 } 438 } 439 440 private void printUsage() { 441 printUsage(null); 442 } 443 444 private void printUsage(final String message) { 445 if (message != null && message.length() > 0) { 446 System.err.println(message); 447 } 448 System.err.println("Usage: java " + this.getClass().getName() + " \\"); 449 System.err.println(" [-compactOnce] [-major] [-mapred] [-D<property=value>]* files..."); 450 System.err.println(); 451 System.err.println("Options:"); 452 System.err.println(" mapred Use MapReduce to run compaction."); 453 System.err.println(" compactOnce Execute just one compaction step. (default: while needed)"); 454 System.err.println(" major Trigger major compaction."); 455 System.err.println(); 456 System.err.println("Note: -D properties will be applied to the conf used. "); 457 System.err.println("For example: "); 458 System.err.println(" To stop delete of compacted file, pass -D"+CONF_DELETE_COMPACTED+"=false"); 459 System.err.println(" To set tmp dir, pass -D"+CONF_TMP_DIR+"=ALTERNATE_DIR"); 460 System.err.println(); 461 System.err.println("Examples:"); 462 System.err.println(" To compact the full 'TestTable' using MapReduce:"); 463 System.err.println(" $ hbase " + this.getClass().getName() + " -mapred hdfs://hbase/data/default/TestTable"); 464 System.err.println(); 465 System.err.println(" To compact column family 'x' of the table 'TestTable' region 'abc':"); 466 System.err.println(" $ hbase " + this.getClass().getName() + " hdfs://hbase/data/default/TestTable/abc/x"); 467 } 468 469 public static void main(String[] args) throws Exception { 470 System.exit(ToolRunner.run(HBaseConfiguration.create(), new CompactionTool(), args)); 471 } 472}