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.util.compaction;
019
020import java.io.IOException;
021import java.util.Collection;
022import java.util.Map;
023import java.util.Optional;
024import org.apache.hadoop.hbase.HConstants;
025import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
026import org.apache.hadoop.hbase.client.Connection;
027import org.apache.hadoop.hbase.client.RegionInfo;
028import org.apache.hadoop.hbase.client.TableDescriptor;
029import org.apache.hadoop.hbase.regionserver.HRegionFileSystem;
030import org.apache.hadoop.hbase.regionserver.StoreFileInfo;
031import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
032import org.apache.yetus.audience.InterfaceAudience;
033import org.slf4j.Logger;
034import org.slf4j.LoggerFactory;
035
036import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
037
038/**
039 * This request helps determine if a region has to be compacted based on table's TTL.
040 */
041@InterfaceAudience.Private
042public class MajorCompactionTTLRequest extends MajorCompactionRequest {
043
044  private static final Logger LOG = LoggerFactory.getLogger(MajorCompactionTTLRequest.class);
045
046  MajorCompactionTTLRequest(Connection connection, RegionInfo region) {
047    super(connection, region);
048  }
049
050  static Optional<MajorCompactionRequest> newRequest(Connection connection, RegionInfo info,
051    TableDescriptor htd) throws IOException {
052    MajorCompactionTTLRequest request = new MajorCompactionTTLRequest(connection, info);
053    return request.createRequest(connection, htd);
054  }
055
056  private Optional<MajorCompactionRequest> createRequest(Connection connection, TableDescriptor htd)
057    throws IOException {
058    Map<String, Long> familiesToCompact = getStoresRequiringCompaction(htd);
059    MajorCompactionRequest request = null;
060    if (!familiesToCompact.isEmpty()) {
061      LOG.debug("Compaction families for region: " + region + " CF: " + familiesToCompact.keySet());
062      request = new MajorCompactionTTLRequest(connection, region);
063    }
064    return Optional.ofNullable(request);
065  }
066
067  Map<String, Long> getStoresRequiringCompaction(TableDescriptor htd) throws IOException {
068    HRegionFileSystem fileSystem = getFileSystem();
069    Map<String, Long> familyTTLMap = Maps.newHashMap();
070    for (ColumnFamilyDescriptor descriptor : htd.getColumnFamilies()) {
071      long ts = getColFamilyCutoffTime(descriptor);
072      // If the table's TTL is forever, lets not compact any of the regions.
073      if (ts > 0 && shouldCFBeCompacted(fileSystem, descriptor.getNameAsString(), ts)) {
074        familyTTLMap.put(descriptor.getNameAsString(), ts);
075      }
076    }
077    return familyTTLMap;
078  }
079
080  // If the CF has no TTL, return -1, else return the current time - TTL.
081  private long getColFamilyCutoffTime(ColumnFamilyDescriptor colDesc) {
082    if (colDesc.getTimeToLive() == HConstants.FOREVER) {
083      return -1;
084    }
085    return EnvironmentEdgeManager.currentTime() - (colDesc.getTimeToLive() * 1000L);
086  }
087
088  @Override
089  protected boolean shouldIncludeStore(HRegionFileSystem fileSystem, String family,
090    Collection<StoreFileInfo> storeFiles, long ts) throws IOException {
091
092    for (StoreFileInfo storeFile : storeFiles) {
093      // Lets only compact when all files are older than TTL
094      if (storeFile.getModificationTime() >= ts) {
095        LOG.info("There is atleast one file in store: " + family + " file: " + storeFile.getPath()
096          + " with timestamp " + storeFile.getModificationTime() + " for region: "
097          + fileSystem.getRegionInfo().getEncodedName() + " older than TTL: " + ts);
098        return false;
099      }
100    }
101    return true;
102  }
103}