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.master.cleaner; 019 020import static org.apache.hadoop.hbase.HConstants.HBASE_MASTER_LOGCLEANER_PLUGINS; 021 022import java.io.IOException; 023import java.util.ArrayList; 024import java.util.List; 025import java.util.Map; 026import java.util.concurrent.CountDownLatch; 027import java.util.concurrent.LinkedBlockingQueue; 028import java.util.concurrent.TimeUnit; 029import java.util.concurrent.atomic.AtomicBoolean; 030import java.util.stream.Collectors; 031import org.apache.hadoop.conf.Configuration; 032import org.apache.hadoop.fs.FileStatus; 033import org.apache.hadoop.fs.FileSystem; 034import org.apache.hadoop.fs.Path; 035import org.apache.hadoop.hbase.Stoppable; 036import org.apache.hadoop.hbase.conf.ConfigurationObserver; 037import org.apache.hadoop.hbase.master.procedure.MasterProcedureUtil; 038import org.apache.hadoop.hbase.master.region.MasterRegionFactory; 039import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; 040import org.apache.yetus.audience.InterfaceAudience; 041import org.slf4j.Logger; 042import org.slf4j.LoggerFactory; 043 044import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 045 046/** 047 * This Chore, every time it runs, will attempt to delete the WALs and Procedure WALs in the old 048 * logs folder. The WAL is only deleted if none of the cleaner delegates says otherwise. 049 * @see BaseLogCleanerDelegate 050 */ 051@InterfaceAudience.Private 052public class LogCleaner extends CleanerChore<BaseLogCleanerDelegate> 053 implements ConfigurationObserver { 054 private static final Logger LOG = LoggerFactory.getLogger(LogCleaner.class); 055 056 public static final String OLD_WALS_CLEANER_THREAD_SIZE = "hbase.oldwals.cleaner.thread.size"; 057 public static final int DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE = 2; 058 059 public static final String OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 060 "hbase.oldwals.cleaner.thread.timeout.msec"; 061 static final long DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC = 60 * 1000L; 062 063 private final LinkedBlockingQueue<CleanerContext> pendingDelete; 064 private List<Thread> oldWALsCleaner; 065 private long cleanerThreadTimeoutMsec; 066 067 /** 068 * @param period the period of time to sleep between each run 069 * @param stopper the stopper 070 * @param conf configuration to use 071 * @param fs handle to the FS 072 * @param oldLogDir the path to the archived logs 073 * @param pool the thread pool used to scan directories 074 */ 075 public LogCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs, 076 Path oldLogDir, DirScanPool pool, Map<String, Object> params) { 077 super("LogsCleaner", period, stopper, conf, fs, oldLogDir, HBASE_MASTER_LOGCLEANER_PLUGINS, 078 pool, params, null); 079 this.pendingDelete = new LinkedBlockingQueue<>(); 080 int size = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE); 081 if (size <= 0) { 082 size = DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE; 083 LOG.warn( 084 "The configuration {} has been set to an invalid value {}, " 085 + "the default value {} will be used.", 086 OLD_WALS_CLEANER_THREAD_SIZE, size, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE); 087 } 088 this.oldWALsCleaner = createOldWalsCleaner(size); 089 this.cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC, 090 DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC); 091 } 092 093 @Override 094 protected boolean validate(Path file) { 095 return AbstractFSWALProvider.validateWALFilename(file.getName()) 096 || MasterProcedureUtil.validateProcedureWALFilename(file.getName()) 097 || file.getName().endsWith(MasterRegionFactory.ARCHIVED_WAL_SUFFIX); 098 } 099 100 @Override 101 public void onConfigurationChange(Configuration conf) { 102 int newSize = conf.getInt(OLD_WALS_CLEANER_THREAD_SIZE, DEFAULT_OLD_WALS_CLEANER_THREAD_SIZE); 103 if (newSize == oldWALsCleaner.size()) { 104 LOG.debug( 105 "Size from configuration is the same as previous which " + "is {}, no need to update.", 106 newSize); 107 return; 108 } 109 interruptOldWALsCleaner(); 110 oldWALsCleaner = createOldWalsCleaner(newSize); 111 cleanerThreadTimeoutMsec = conf.getLong(OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC, 112 DEFAULT_OLD_WALS_CLEANER_THREAD_TIMEOUT_MSEC); 113 } 114 115 @Override 116 protected int deleteFiles(Iterable<FileStatus> filesToDelete) { 117 List<CleanerContext> results = new ArrayList<>(); 118 for (FileStatus file : filesToDelete) { 119 LOG.trace("Scheduling file {} for deletion", file); 120 if (file != null) { 121 results.add(new CleanerContext(file)); 122 } 123 } 124 if (results.isEmpty()) { 125 return 0; 126 } 127 128 LOG.debug("Old WALs for delete: {}", 129 results.stream().map(cc -> cc.target.getPath().getName()).collect(Collectors.joining(", "))); 130 pendingDelete.addAll(results); 131 132 int deletedFiles = 0; 133 for (CleanerContext res : results) { 134 LOG.trace("Awaiting the results for deletion of old WAL file: {}", res); 135 deletedFiles += res.getResult(this.cleanerThreadTimeoutMsec) ? 1 : 0; 136 } 137 return deletedFiles; 138 } 139 140 @Override 141 public synchronized void cleanup() { 142 super.cleanup(); 143 interruptOldWALsCleaner(); 144 } 145 146 int getSizeOfCleaners() { 147 return oldWALsCleaner.size(); 148 } 149 150 long getCleanerThreadTimeoutMsec() { 151 return cleanerThreadTimeoutMsec; 152 } 153 154 private List<Thread> createOldWalsCleaner(int size) { 155 LOG.info("Creating {} old WALs cleaner threads", size); 156 157 List<Thread> oldWALsCleaner = new ArrayList<>(size); 158 for (int i = 0; i < size; i++) { 159 Thread cleaner = new Thread(() -> deleteFile()); 160 cleaner.setName("OldWALsCleaner-" + i); 161 cleaner.setDaemon(true); 162 cleaner.start(); 163 oldWALsCleaner.add(cleaner); 164 } 165 return oldWALsCleaner; 166 } 167 168 private void interruptOldWALsCleaner() { 169 for (Thread cleaner : oldWALsCleaner) { 170 LOG.trace("Interrupting thread: {}", cleaner); 171 cleaner.interrupt(); 172 } 173 oldWALsCleaner.clear(); 174 } 175 176 private void deleteFile() { 177 while (true) { 178 try { 179 final CleanerContext context = pendingDelete.take(); 180 Preconditions.checkNotNull(context); 181 FileStatus oldWalFile = context.getTargetToClean(); 182 try { 183 LOG.debug("Deleting {}", oldWalFile); 184 boolean succeed = this.fs.delete(oldWalFile.getPath(), false); 185 context.setResult(succeed); 186 } catch (IOException e) { 187 // fs.delete() fails. 188 LOG.warn("Failed to delete old WAL file", e); 189 context.setResult(false); 190 } 191 } catch (InterruptedException ite) { 192 // It is most likely from configuration changing request 193 LOG.warn( 194 "Interrupted while cleaning old WALs, will " + "try to clean it next round. Exiting."); 195 // Restore interrupt status 196 Thread.currentThread().interrupt(); 197 return; 198 } 199 LOG.trace("Exiting"); 200 } 201 } 202 203 @Override 204 public synchronized void cancel(boolean mayInterruptIfRunning) { 205 LOG.debug("Cancelling LogCleaner"); 206 super.cancel(mayInterruptIfRunning); 207 interruptOldWALsCleaner(); 208 } 209 210 private static final class CleanerContext { 211 212 final FileStatus target; 213 final AtomicBoolean result; 214 final CountDownLatch remainingResults; 215 216 private CleanerContext(FileStatus status) { 217 this.target = status; 218 this.result = new AtomicBoolean(false); 219 this.remainingResults = new CountDownLatch(1); 220 } 221 222 void setResult(boolean res) { 223 this.result.set(res); 224 this.remainingResults.countDown(); 225 } 226 227 boolean getResult(long waitIfNotFinished) { 228 try { 229 boolean completed = this.remainingResults.await(waitIfNotFinished, TimeUnit.MILLISECONDS); 230 if (!completed) { 231 LOG.warn("Spent too much time [{}ms] deleting old WAL file: {}", waitIfNotFinished, 232 target); 233 return false; 234 } 235 } catch (InterruptedException e) { 236 LOG.warn("Interrupted while awaiting deletion of WAL file: {}", target); 237 return false; 238 } 239 return result.get(); 240 } 241 242 FileStatus getTargetToClean() { 243 return target; 244 } 245 246 @Override 247 public String toString() { 248 return "CleanerContext [target=" + target + ", result=" + result + "]"; 249 } 250 } 251}