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 */
018
019package org.apache.hadoop.hbase.util.compaction;
020
021import java.io.IOException;
022import java.util.Arrays;
023import java.util.Optional;
024import java.util.Set;
025import java.util.concurrent.Executors;
026import org.apache.hadoop.conf.Configuration;
027import org.apache.hadoop.hbase.HBaseConfiguration;
028import org.apache.hadoop.hbase.HBaseInterfaceAudience;
029import org.apache.hadoop.hbase.HConstants;
030import org.apache.hadoop.hbase.TableName;
031import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
032import org.apache.hadoop.hbase.client.Connection;
033import org.apache.hadoop.hbase.client.ConnectionFactory;
034import org.apache.hadoop.hbase.client.RegionInfo;
035import org.apache.hadoop.hbase.client.TableDescriptor;
036import org.apache.hadoop.util.ToolRunner;
037import org.apache.yetus.audience.InterfaceAudience;
038import org.slf4j.Logger;
039import org.slf4j.LoggerFactory;
040
041import org.apache.hbase.thirdparty.com.google.common.collect.Sets;
042import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLine;
043import org.apache.hbase.thirdparty.org.apache.commons.cli.CommandLineParser;
044import org.apache.hbase.thirdparty.org.apache.commons.cli.DefaultParser;
045import org.apache.hbase.thirdparty.org.apache.commons.cli.Option;
046import org.apache.hbase.thirdparty.org.apache.commons.cli.Options;
047import org.apache.hbase.thirdparty.org.apache.commons.cli.ParseException;
048
049/**
050 * This tool compacts a table's regions that are beyond it's TTL. It helps to save disk space and
051 * regions become empty as a result of compaction.
052 */
053@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
054public class MajorCompactorTTL extends MajorCompactor {
055
056  private static final Logger LOG = LoggerFactory.getLogger(MajorCompactorTTL .class);
057
058  private TableDescriptor htd;
059
060  @InterfaceAudience.Private
061  public MajorCompactorTTL(Configuration conf, TableDescriptor htd, int concurrency,
062      long sleepForMs) throws IOException {
063    this.connection = ConnectionFactory.createConnection(conf);
064    this.htd = htd;
065    this.tableName = htd.getTableName();
066    this.storesToCompact = Sets.newHashSet(); // Empty set so all stores will be compacted
067    this.sleepForMs = sleepForMs;
068    this.executor = Executors.newFixedThreadPool(concurrency);
069    this.clusterCompactionQueues = new ClusterCompactionQueues(concurrency);
070  }
071
072  protected MajorCompactorTTL() {
073    super();
074  }
075
076  @Override
077  protected Optional<MajorCompactionRequest> getMajorCompactionRequest(RegionInfo hri)
078      throws IOException {
079    return MajorCompactionTTLRequest.newRequest(connection.getConfiguration(), hri, htd);
080  }
081
082  @Override
083  protected Set<String> getStoresRequiringCompaction(MajorCompactionRequest request)
084      throws IOException {
085    return ((MajorCompactionTTLRequest)request).getStoresRequiringCompaction(htd).keySet();
086  }
087
088  public int compactRegionsTTLOnTable(Configuration conf, String table, int concurrency,
089      long sleep, int numServers, int numRegions, boolean dryRun, boolean skipWait)
090      throws Exception {
091
092    Connection conn = ConnectionFactory.createConnection(conf);
093    TableName tableName = TableName.valueOf(table);
094
095    TableDescriptor htd = conn.getAdmin().getDescriptor(tableName);
096    if (!doesAnyColFamilyHaveTTL(htd)) {
097      LOG.info("No TTL present for CF of table: " + tableName + ", skipping compaction");
098      return 0;
099    }
100
101    LOG.info("Major compacting table " + tableName + " based on TTL");
102    MajorCompactor compactor = new MajorCompactorTTL(conf, htd, concurrency, sleep);
103    compactor.setNumServers(numServers);
104    compactor.setNumRegions(numRegions);
105    compactor.setSkipWait(skipWait);
106
107    compactor.initializeWorkQueues();
108    if (!dryRun) {
109      compactor.compactAllRegions();
110    }
111    compactor.shutdown();
112    return ERRORS.size();
113  }
114
115  private boolean doesAnyColFamilyHaveTTL(TableDescriptor htd) {
116    for (ColumnFamilyDescriptor descriptor : htd.getColumnFamilies()) {
117      if (descriptor.getTimeToLive() != HConstants.FOREVER) {
118        return true;
119      }
120    }
121    return false;
122  }
123
124  private Options getOptions() {
125    Options options = getCommonOptions();
126
127    options.addOption(
128        Option.builder("table")
129            .required()
130            .desc("table name")
131            .hasArg()
132            .build()
133    );
134
135    return options;
136  }
137
138  @Override
139  public int run(String[] args) throws Exception {
140    Options options = getOptions();
141
142    final CommandLineParser cmdLineParser = new DefaultParser();
143    CommandLine commandLine;
144    try {
145      commandLine = cmdLineParser.parse(options, args);
146    } catch (ParseException parseException) {
147      System.out.println(
148          "ERROR: Unable to parse command-line arguments " + Arrays.toString(args) + " due to: "
149              + parseException);
150      printUsage(options);
151      return -1;
152    }
153    if (commandLine == null) {
154      System.out.println("ERROR: Failed parse, empty commandLine; " + Arrays.toString(args));
155      printUsage(options);
156      return -1;
157    }
158
159    String table = commandLine.getOptionValue("table");
160    int numServers = Integer.parseInt(commandLine.getOptionValue("numservers", "-1"));
161    int numRegions = Integer.parseInt(commandLine.getOptionValue("numregions", "-1"));
162    int concurrency = Integer.parseInt(commandLine.getOptionValue("servers", "1"));
163    long sleep = Long.parseLong(commandLine.getOptionValue("sleep", Long.toString(30000)));
164    boolean dryRun = commandLine.hasOption("dryRun");
165    boolean skipWait = commandLine.hasOption("skipWait");
166
167    return compactRegionsTTLOnTable(HBaseConfiguration.create(), table, concurrency, sleep,
168        numServers, numRegions, dryRun, skipWait);
169  }
170
171  public static void main(String[] args) throws Exception {
172    ToolRunner.run(HBaseConfiguration.create(), new MajorCompactorTTL(), args);
173  }
174}