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