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    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.util.compaction;
20  
21  import com.google.common.base.Optional;
22  import com.google.common.collect.Maps;
23  import java.io.IOException;
24  import java.util.Collection;
25  import java.util.Map;
26  import org.apache.hadoop.conf.Configuration;
27  import org.apache.hadoop.hbase.HColumnDescriptor;
28  import org.apache.hadoop.hbase.HConstants;
29  import org.apache.hadoop.hbase.HRegionInfo;
30  import org.apache.hadoop.hbase.HTableDescriptor;
31  import org.apache.hadoop.hbase.classification.InterfaceAudience;
32  import org.apache.hadoop.hbase.client.Connection;
33  import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
34  import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
35  import org.slf4j.Logger;
36  import org.slf4j.LoggerFactory;
37  
38  /**
39   * This request helps determine if a region has to be compacted based on table's TTL.
40   */
41  @InterfaceAudience.Private
42  public class MajorCompactionTTLRequest extends MajorCompactionRequest {
43  
44    private static final Logger LOG = LoggerFactory.getLogger(MajorCompactionTTLRequest.class);
45  
46    MajorCompactionTTLRequest(Configuration conf, HRegionInfo region) {
47      super(conf, region);
48    }
49  
50    static Optional<MajorCompactionRequest> newRequest(Configuration conf, HRegionInfo info,
51        HTableDescriptor htd) throws IOException {
52      MajorCompactionTTLRequest request = new MajorCompactionTTLRequest(conf, info);
53      return request.createRequest(conf, htd);
54    }
55  
56    private Optional<MajorCompactionRequest> createRequest(Configuration conf, HTableDescriptor htd)
57        throws IOException {
58      Map<String, Long> familiesToCompact = getStoresRequiringCompaction(htd);
59      MajorCompactionRequest request = null;
60      if (!familiesToCompact.isEmpty()) {
61        LOG.debug("Compaction families for region: " + region + " CF: " + familiesToCompact.keySet());
62        request = new MajorCompactionTTLRequest(conf, region);
63      }
64      return Optional.fromNullable(request);
65    }
66  
67    Map<String, Long> getStoresRequiringCompaction(HTableDescriptor htd) throws IOException {
68      try(Connection connection = getConnection(configuration)) {
69        HRegionFileSystem fileSystem = getFileSystem(connection);
70        Map<String, Long> familyTTLMap = Maps.newHashMap();
71        for (HColumnDescriptor descriptor : htd.getColumnFamilies()) {
72          long ts = getColFamilyCutoffTime(descriptor);
73          // If the table's TTL is forever, lets not compact any of the regions.
74          if (ts > 0 && shouldCFBeCompacted(fileSystem, descriptor.getNameAsString(), ts)) {
75            familyTTLMap.put(descriptor.getNameAsString(), ts);
76          }
77        }
78        return familyTTLMap;
79      }
80    }
81  
82    // If the CF has no TTL, return -1, else return the current time - TTL.
83    private long getColFamilyCutoffTime(HColumnDescriptor colDesc) {
84      if (colDesc.getTimeToLive() == HConstants.FOREVER) {
85        return -1;
86      }
87      return System.currentTimeMillis() - (colDesc.getTimeToLive() * 1000L);
88    }
89  
90    @Override
91    protected boolean shouldIncludeStore(HRegionFileSystem fileSystem, String family,
92        Collection<StoreFileInfo> storeFiles, long ts) throws IOException {
93  
94      for (StoreFileInfo storeFile : storeFiles) {
95        // Lets only compact when all files are older than TTL
96        if (storeFile.getModificationTime() >= ts) {
97          LOG.info("There is atleast one file in store: " + family + " file: " + storeFile.getPath()
98              + " with timestamp " + storeFile.getModificationTime()
99              + " for region: " + fileSystem.getRegionInfo().getEncodedName()
100             + " older than TTL: " + ts);
101         return false;
102       }
103     }
104     return true;
105   }
106 }