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