View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    * http://www.apache.org/licenses/LICENSE-2.0
10   * Unless required by applicable law or agreed to in writing, software
11   * distributed under the License is distributed on an "AS IS" BASIS,
12   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13   * See the License for the specific language governing permissions and
14   * limitations under the License.
15   */
16  
17  package org.apache.hadoop.hbase.util.compaction;
18  
19  import com.google.common.base.Joiner;
20  import com.google.common.base.Optional;
21  import com.google.common.base.Predicate;
22  import com.google.common.base.Splitter;
23  import com.google.common.collect.Iterables;
24  import com.google.common.collect.Lists;
25  import com.google.common.collect.Maps;
26  import com.google.common.collect.Sets;
27  import java.io.IOException;
28  import java.util.Arrays;
29  import java.util.Collection;
30  import java.util.Collections;
31  import java.util.List;
32  import java.util.Map;
33  import java.util.Set;
34  import java.util.concurrent.ExecutorService;
35  import java.util.concurrent.Executors;
36  import java.util.concurrent.Future;
37  import java.util.concurrent.TimeUnit;
38  import org.apache.commons.cli.BasicParser;
39  import org.apache.commons.cli.CommandLine;
40  import org.apache.commons.cli.CommandLineParser;
41  import org.apache.commons.cli.HelpFormatter;
42  import org.apache.commons.cli.Option;
43  import org.apache.commons.cli.Options;
44  import org.apache.commons.cli.ParseException;
45  import org.apache.hadoop.conf.Configuration;
46  import org.apache.hadoop.conf.Configured;
47  import org.apache.hadoop.hbase.HBaseConfiguration;
48  import org.apache.hadoop.hbase.HBaseInterfaceAudience;
49  import org.apache.hadoop.hbase.HColumnDescriptor;
50  import org.apache.hadoop.hbase.HConstants;
51  import org.apache.hadoop.hbase.HRegionInfo;
52  import org.apache.hadoop.hbase.HRegionLocation;
53  import org.apache.hadoop.hbase.NotServingRegionException;
54  import org.apache.hadoop.hbase.ServerName;
55  import org.apache.hadoop.hbase.TableName;
56  import org.apache.hadoop.hbase.classification.InterfaceAudience;
57  import org.apache.hadoop.hbase.client.Admin;
58  import org.apache.hadoop.hbase.client.Connection;
59  import org.apache.hadoop.hbase.client.ConnectionFactory;
60  import org.apache.hadoop.hbase.protobuf.generated.AdminProtos;
61  import org.apache.hadoop.hbase.util.Bytes;
62  import org.apache.hadoop.util.Tool;
63  import org.apache.hadoop.util.ToolRunner;
64  import org.slf4j.Logger;
65  import org.slf4j.LoggerFactory;
66  
67  @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.TOOLS)
68  public class MajorCompactor extends Configured implements Tool {
69  
70    private static final Logger LOG = LoggerFactory.getLogger(MajorCompactor.class);
71    static final Set<MajorCompactionRequest> ERRORS = Sets.newHashSet();
72  
73    protected ClusterCompactionQueues clusterCompactionQueues;
74    private long timestamp;
75    protected Set<String> storesToCompact;
76    protected ExecutorService executor;
77    protected long sleepForMs;
78    protected Connection connection;
79    protected TableName tableName;
80    private int numServers = -1;
81    private int numRegions = -1;
82    private boolean skipWait = false;
83  
84    MajorCompactor() {
85    }
86  
87    public MajorCompactor(Configuration conf, TableName tableName, Set<String> storesToCompact,
88        int concurrency, long timestamp, long sleepForMs) throws IOException {
89      this.connection = ConnectionFactory.createConnection(conf);
90      this.tableName = tableName;
91      this.timestamp = timestamp;
92      this.storesToCompact = storesToCompact;
93      this.executor = Executors.newFixedThreadPool(concurrency);
94      this.clusterCompactionQueues = new ClusterCompactionQueues(concurrency);
95      this.sleepForMs = sleepForMs;
96    }
97  
98    public void compactAllRegions() throws Exception {
99      List<Future<?>> futures = Lists.newArrayList();
100     while (clusterCompactionQueues.hasWorkItems() || !futuresComplete(futures)) {
101       while (clusterCompactionQueues.atCapacity()) {
102         LOG.debug("Waiting for servers to complete Compactions");
103         Thread.sleep(sleepForMs);
104       }
105       Optional<ServerName> serverToProcess =
106           clusterCompactionQueues.getLargestQueueFromServersNotCompacting();
107       if (serverToProcess.isPresent() && clusterCompactionQueues.hasWorkItems()) {
108         ServerName serverName = serverToProcess.get();
109         // check to see if the region has moved... if so we have to enqueue it again with
110         // the proper serverName
111         MajorCompactionRequest request = clusterCompactionQueues.reserveForCompaction(serverName);
112 
113         ServerName currentServer = connection.getRegionLocator(tableName)
114             .getRegionLocation(request.getRegion().getStartKey()).getServerName();
115 
116         if (!currentServer.equals(serverName)) {
117           // add it back to the queue with the correct server it should be picked up in the future.
118           LOG.info("Server changed for region: " + request.getRegion().getEncodedName() + " from: "
119               + serverName + " to: " + currentServer + " re-queuing request");
120           clusterCompactionQueues.addToCompactionQueue(currentServer, request);
121           clusterCompactionQueues.releaseCompaction(serverName);
122         } else {
123           LOG.info("Firing off compaction request for server: " + serverName + ", " + request
124               + " total queue size left: " + clusterCompactionQueues
125               .getCompactionRequestsLeftToFinish());
126           futures.add(executor.submit(new Compact(serverName, request)));
127         }
128       } else {
129         // haven't assigned anything so we sleep.
130         Thread.sleep(sleepForMs);
131       }
132     }
133     LOG.info("All compactions have completed");
134   }
135 
136   private boolean futuresComplete(List<Future<?>> futures) {
137     Iterables.removeIf(futures, new Predicate<Future<?>>() {
138       @Override public boolean apply(Future<?> input) {
139         return input.isDone();
140       }
141     });
142     return futures.isEmpty();
143   }
144 
145   public void shutdown() throws Exception {
146     executor.shutdown();
147     executor.awaitTermination(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
148     if (!ERRORS.isEmpty()) {
149       StringBuilder builder =
150           new StringBuilder().append("Major compaction failed, there were: ").append(ERRORS.size())
151               .append(" regions / stores that failed compacting\n")
152               .append("Failed compaction requests\n").append("--------------------------\n")
153               .append(Joiner.on("\n").join(ERRORS));
154       LOG.error(builder.toString());
155     }
156     if (connection != null) {
157       connection.close();
158     }
159     LOG.info("All regions major compacted successfully");
160   }
161 
162   @InterfaceAudience.Private
163   void initializeWorkQueues() throws IOException {
164     if (storesToCompact.isEmpty()) {
165       for (HColumnDescriptor a : connection.getTable(tableName).getTableDescriptor()
166           .getFamilies()) {
167         storesToCompact.add(Bytes.toString(a.getName()));
168       }
169       LOG.info("No family specified, will execute for all families");
170     }
171     LOG.info(
172         "Initializing compaction queues for table:  " + tableName + " with cf: " + storesToCompact);
173 
174     Map<ServerName, List<HRegionInfo>> snRegionMap = getServerRegionsMap();
175     /*
176      * If numservers is specified, stop inspecting regions beyond the numservers, it will serve
177      * to throttle and won't end up scanning all the regions in the event there are not many
178      * regions to compact based on the criteria.
179      */
180     for (ServerName sn : getServersToCompact(snRegionMap.keySet())) {
181       List<HRegionInfo> regions = snRegionMap.get(sn);
182       LOG.debug("Table: " + tableName + " Server: " + sn + " No of regions: " + regions.size());
183 
184       /*
185        * If the tool is run periodically, then we could shuffle the regions and provide
186        * some random order to select regions. Helps if numregions is specified.
187        */
188       Collections.shuffle(regions);
189       int regionsToCompact = numRegions;
190       for (HRegionInfo hri : regions) {
191         if (numRegions > 0 && regionsToCompact <= 0) {
192           LOG.debug("Reached region limit for server: " + sn);
193           break;
194         }
195 
196         Optional<MajorCompactionRequest> request = getMajorCompactionRequest(hri);
197         if (request.isPresent()) {
198           LOG.debug("Adding region " + hri + " to queue " + sn + " for compaction");
199           clusterCompactionQueues.addToCompactionQueue(sn, request.get());
200           if (numRegions > 0) {
201             regionsToCompact--;
202           }
203         }
204       }
205     }
206   }
207 
208   protected Optional<MajorCompactionRequest> getMajorCompactionRequest(HRegionInfo hri)
209       throws IOException {
210     return MajorCompactionRequest.newRequest(connection.getConfiguration(), hri, storesToCompact,
211             timestamp);
212   }
213 
214   private Collection<ServerName> getServersToCompact(Set<ServerName> snSet) {
215     if(numServers < 0 || snSet.size() <= numServers) {
216       return snSet;
217 
218     } else {
219       List<ServerName> snList = Lists.newArrayList(snSet);
220       Collections.shuffle(snList);
221       return snList.subList(0, numServers);
222     }
223   }
224 
225   private Map<ServerName, List<HRegionInfo>> getServerRegionsMap() throws IOException {
226     Map<ServerName, List<HRegionInfo>> snRegionMap = Maps.newHashMap();
227     List<HRegionLocation> regionLocations =
228         connection.getRegionLocator(tableName).getAllRegionLocations();
229     for (HRegionLocation regionLocation : regionLocations) {
230       ServerName sn = regionLocation.getServerName();
231       HRegionInfo hri = regionLocation.getRegionInfo();
232       if (!snRegionMap.containsKey(sn)) {
233         snRegionMap.put(sn, Lists.<HRegionInfo>newArrayList());
234       }
235       snRegionMap.get(sn).add(hri);
236     }
237     return snRegionMap;
238   }
239 
240   public void setNumServers(int numServers) {
241     this.numServers = numServers;
242   }
243 
244   public void setNumRegions(int numRegions) {
245     this.numRegions = numRegions;
246   }
247 
248   public void setSkipWait(boolean skipWait) {
249     this.skipWait = skipWait;
250   }
251 
252   class Compact implements Runnable {
253 
254     private final ServerName serverName;
255     private final MajorCompactionRequest request;
256 
257     Compact(ServerName serverName, MajorCompactionRequest request) {
258       this.serverName = serverName;
259       this.request = request;
260     }
261 
262     @Override public void run() {
263       try {
264         compactAndWait(request);
265       } catch (NotServingRegionException e) {
266         // this region has split or merged
267         LOG.warn("Region is invalid, requesting updated regions", e);
268         // lets updated the cluster compaction queues with these newly created regions.
269         addNewRegions();
270       } catch (Exception e) {
271         LOG.warn("Error compacting:", e);
272       } finally {
273         clusterCompactionQueues.releaseCompaction(serverName);
274       }
275     }
276 
277     void compactAndWait(MajorCompactionRequest request) throws Exception {
278       Admin admin = connection.getAdmin();
279       try {
280         // only make the request if the region is not already major compacting
281         if (!isCompacting(request)) {
282           Set<String> stores = getStoresRequiringCompaction(request);
283           if (!stores.isEmpty()) {
284             request.setStores(stores);
285             for (String store : request.getStores()) {
286               compactRegionOnServer(request, admin, store);
287             }
288           }
289         }
290 
291         /*
292          * In some scenarios like compacting TTLed regions, the compaction itself won't take time
293          * and hence we can skip the wait. An external tool will also be triggered frequently and
294          * the next run can identify region movements and compact them.
295          */
296         if (!skipWait) {
297           while (isCompacting(request)) {
298             Thread.sleep(sleepForMs);
299             LOG.debug("Waiting for compaction to complete for region: " + request.getRegion()
300                 .getEncodedName());
301           }
302         }
303       } finally {
304         if (!skipWait) {
305           // Make sure to wait for the CompactedFileDischarger chore to do its work
306           int waitForArchive = connection.getConfiguration()
307               .getInt("hbase.hfile.compaction.discharger.interval", 2 * 60 * 1000);
308           Thread.sleep(waitForArchive);
309           // check if compaction completed successfully, otherwise put that request back in the
310           // proper queue
311           Set<String> storesRequiringCompaction = getStoresRequiringCompaction(request);
312           if (!storesRequiringCompaction.isEmpty()) {
313             // this happens, when a region server is marked as dead, flushes a store file and
314             // the new regionserver doesn't pick it up because its accounted for in the WAL replay,
315             // thus you have more store files on the filesystem than the regionserver knows about.
316             boolean regionHasNotMoved = connection.getRegionLocator(tableName)
317                 .getRegionLocation(request.getRegion().getStartKey()).getServerName()
318                 .equals(serverName);
319             if (regionHasNotMoved) {
320               LOG.error(
321                   "Not all store files were compacted, this may be due to the regionserver not "
322                       + "being aware of all store files.  Will not reattempt compacting, "
323                       + request);
324               ERRORS.add(request);
325             } else {
326               request.setStores(storesRequiringCompaction);
327               clusterCompactionQueues.addToCompactionQueue(serverName, request);
328               LOG.info("Compaction failed for the following stores: " + storesRequiringCompaction
329                   + " region: " + request.getRegion().getEncodedName());
330             }
331           } else {
332             LOG.info("Compaction complete for region: " + request.getRegion().getEncodedName()
333                 + " -> cf(s): " + request.getStores());
334           }
335         }
336       }
337     }
338 
339     private void compactRegionOnServer(MajorCompactionRequest request, Admin admin, String store)
340         throws IOException {
341       admin.majorCompactRegion(request.getRegion().getEncodedNameAsBytes(),
342           Bytes.toBytes(store));
343     }
344   }
345 
346   private boolean isCompacting(MajorCompactionRequest request) throws Exception {
347     AdminProtos.GetRegionInfoResponse.CompactionState compactionState = connection.getAdmin()
348         .getCompactionStateForRegion(request.getRegion().getEncodedNameAsBytes());
349     return compactionState.equals(AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR)
350         || compactionState
351         .equals(AdminProtos.GetRegionInfoResponse.CompactionState.MAJOR_AND_MINOR);
352   }
353 
354   private void addNewRegions() {
355     try {
356       List<HRegionLocation> locations =
357           connection.getRegionLocator(tableName).getAllRegionLocations();
358       for (HRegionLocation location : locations) {
359         if (location.getRegionInfo().getRegionId() > timestamp) {
360           Optional<MajorCompactionRequest> compactionRequest =
361               getMajorCompactionRequest(location.getRegionInfo());
362           if (compactionRequest.isPresent()) {
363             clusterCompactionQueues
364                 .addToCompactionQueue(location.getServerName(), compactionRequest.get());
365           }
366         }
367       }
368     } catch (IOException e) {
369       throw new RuntimeException(e);
370     }
371   }
372 
373   protected Set<String> getStoresRequiringCompaction(MajorCompactionRequest request)
374       throws IOException {
375     return request.getStoresRequiringCompaction(storesToCompact, timestamp);
376   }
377 
378   protected Options getCommonOptions() {
379     Options options = new Options();
380     Option serverOption = new Option("servers", true, "Concurrent servers compacting");
381     serverOption.setRequired(true);
382     options.addOption(serverOption);
383 
384     Option sleepOption = new Option("sleep", true, "Time to sleepForMs (ms) for checking "
385         + "compaction status per region and available "
386         + "work queues: default 30s");
387     options.addOption(sleepOption);
388 
389     Option retryOption = new Option("retries", true, "Max # of retries for a compaction request,"
390         + " defaults to 3");
391     options.addOption(retryOption);
392 
393     options.addOption(new Option("dryRun", false, "Dry run, will just output a list of regions"
394         + " that require compaction based on parameters passed"));
395 
396     options.addOption(new Option("skipWait", false, "Skip waiting after triggering compaction."));
397 
398     Option numServersOption = new Option("numservers", true, "Number of servers to compact in "
399         + "this run, defaults to all");
400     options.addOption(numServersOption);
401 
402     Option numRegionsOption = new Option("numregions", true, "Number of regions to compact per"
403         + "server, defaults to all");
404     options.addOption(numRegionsOption);
405     return options;
406   }
407 
408   @Override
409   public int run(String[] args) throws Exception {
410     Options options = getCommonOptions();
411     Option tableOption = new Option("table", true, "table name");
412     tableOption.setRequired(true);
413     options.addOption(tableOption);
414 
415     Option cfOption = new Option("cf", true, "column families: comma separated eg: a,b,c");
416     cfOption.setOptionalArg(true);
417     options.addOption(cfOption);
418 
419     options.addOption(new Option("minModTime", true,
420         "Compact if store files have modification time < minModTime"));
421 
422     Option zkOption = new Option("zk", true, "zk quorum");
423     zkOption.setOptionalArg(true);
424     options.addOption(zkOption);
425 
426     Option rootDirOption = new Option("rootDir", true, "hbase.rootDir");
427     rootDirOption.setOptionalArg(true);
428     options.addOption(rootDirOption);
429 
430     final CommandLineParser cmdLineParser =  new BasicParser();
431     CommandLine commandLine = null;
432     try {
433       commandLine = cmdLineParser.parse(options, args);
434     } catch (ParseException parseException) {
435       System.out.println(
436           "ERROR: Unable to parse command-line arguments " + Arrays.toString(args) + " due to: "
437               + parseException);
438       printUsage(options);
439       throw parseException;
440     }
441 
442     String tableName = commandLine.getOptionValue("table");
443     String cf = commandLine.getOptionValue("cf", null);
444     Set<String> families = Sets.newHashSet();
445     if (cf != null) {
446       Iterables.addAll(families, Splitter.on(",").split(cf));
447     }
448 
449     Configuration configuration = HBaseConfiguration.create();
450     int concurrency = Integer.parseInt(commandLine.getOptionValue("servers"));
451     long minModTime = Long.parseLong(
452         commandLine.getOptionValue("minModTime", String.valueOf(System.currentTimeMillis())));
453     String quorum =
454         commandLine.getOptionValue("zk", configuration.get(HConstants.ZOOKEEPER_QUORUM));
455     String rootDir = commandLine.getOptionValue("rootDir", configuration.get(HConstants.HBASE_DIR));
456     long sleep = Long.parseLong(commandLine.getOptionValue("sleep", "30000"));
457 
458     int numServers = Integer.parseInt(commandLine.getOptionValue("numservers", "-1"));
459     int numRegions = Integer.parseInt(commandLine.getOptionValue("numregions", "-1"));
460 
461     configuration.set(HConstants.HBASE_DIR, rootDir);
462     configuration.set(HConstants.ZOOKEEPER_QUORUM, quorum);
463 
464     MajorCompactor compactor =
465         new MajorCompactor(configuration, TableName.valueOf(tableName), families, concurrency,
466             minModTime, sleep);
467     compactor.setNumServers(numServers);
468     compactor.setNumRegions(numRegions);
469     compactor.setSkipWait(commandLine.hasOption("skipWait"));
470 
471     compactor.initializeWorkQueues();
472     if (!commandLine.hasOption("dryRun")) {
473       compactor.compactAllRegions();
474     }
475     compactor.shutdown();
476     return ERRORS.size();
477   }
478 
479   protected static void printUsage(final Options options) {
480     String header = "\nUsage instructions\n\n";
481     String footer = "\n";
482     HelpFormatter formatter = new HelpFormatter();
483     formatter.printHelp(MajorCompactor.class.getSimpleName(), header, options, footer, true);
484   }
485 
486   public static void main(String[] args) throws Exception {
487     ToolRunner.run(HBaseConfiguration.create(), new MajorCompactor(), args);
488   }
489 }