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.snapshot;
019
020import java.io.IOException;
021import java.util.Collection;
022import java.util.Collections;
023import java.util.List;
024import java.util.Map;
025import java.util.stream.Collectors;
026import java.util.stream.StreamSupport;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.fs.FileStatus;
029import org.apache.hadoop.fs.FileSystem;
030import org.apache.hadoop.fs.Path;
031import org.apache.hadoop.hbase.HBaseInterfaceAudience;
032import org.apache.hadoop.hbase.master.HMaster;
033import org.apache.hadoop.hbase.master.MasterServices;
034import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
035import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
036import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
037import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil;
038import org.apache.hadoop.hbase.util.CommonFSUtils;
039import org.apache.yetus.audience.InterfaceAudience;
040import org.apache.yetus.audience.InterfaceStability;
041import org.slf4j.Logger;
042import org.slf4j.LoggerFactory;
043
044/**
045 * Implementation of a file cleaner that checks if a hfile is still used by snapshots of HBase
046 * tables.
047 */
048@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.CONFIG)
049@InterfaceStability.Evolving
050public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate {
051  private static final Logger LOG = LoggerFactory.getLogger(SnapshotHFileCleaner.class);
052
053  /**
054   * Conf key for the frequency to attempt to refresh the cache of hfiles currently used in
055   * snapshots (ms)
056   */
057  public static final String HFILE_CACHE_REFRESH_PERIOD_CONF_KEY =
058    "hbase.master.hfilecleaner.plugins.snapshot.period";
059
060  /** Refresh cache, by default, every 5 minutes */
061  private static final long DEFAULT_HFILE_CACHE_REFRESH_PERIOD = 300000;
062
063  /** File cache for HFiles in the completed and currently running snapshots */
064  private SnapshotFileCache cache;
065
066  private MasterServices master;
067
068  @Override
069  public Iterable<FileStatus> getDeletableFiles(Iterable<FileStatus> files) {
070    // The Iterable is lazy evaluated, so if we just pass this Iterable in, we will access the HFile
071    // storage inside the snapshot lock, which could take a lot of time (for example, several
072    // seconds), and block all other operations, especially other cleaners.
073    // So here we convert it to List first, to force it evaluated before calling
074    // getUnreferencedFiles, so we will not hold snapshot lock for a long time.
075    List<FileStatus> filesList =
076      StreamSupport.stream(files.spliterator(), false).collect(Collectors.toList());
077    try {
078      return cache.getUnreferencedFiles(filesList, master.getSnapshotManager());
079    } catch (CorruptedSnapshotException cse) {
080      LOG.debug("Corrupted in-progress snapshot file exception, ignored ", cse);
081    } catch (IOException e) {
082      LOG.error("Exception while checking if files were valid, keeping them just in case.", e);
083    }
084    return Collections.emptyList();
085  }
086
087  @Override
088  public void init(Map<String, Object> params) {
089    if (params != null && params.containsKey(HMaster.MASTER)) {
090      this.master = (MasterServices) params.get(HMaster.MASTER);
091    }
092  }
093
094  @Override
095  protected boolean isFileDeletable(FileStatus fStat) {
096    return false;
097  }
098
099  @Override
100  public void setConf(final Configuration conf) {
101    super.setConf(conf);
102    try {
103      long cacheRefreshPeriod =
104        conf.getLong(HFILE_CACHE_REFRESH_PERIOD_CONF_KEY, DEFAULT_HFILE_CACHE_REFRESH_PERIOD);
105      final FileSystem fs = CommonFSUtils.getCurrentFileSystem(conf);
106      Path rootDir = CommonFSUtils.getRootDir(conf);
107      Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(rootDir, conf);
108      FileSystem workingFs = workingDir.getFileSystem(conf);
109
110      cache = new SnapshotFileCache(fs, rootDir, workingFs, workingDir, cacheRefreshPeriod,
111        cacheRefreshPeriod, "snapshot-hfile-cleaner-cache-refresher",
112        new SnapshotFileCache.SnapshotFileInspector() {
113          @Override
114          public Collection<String> filesUnderSnapshot(final FileSystem fs, final Path snapshotDir)
115            throws IOException {
116            return SnapshotReferenceUtil.getHFileNames(conf, fs, snapshotDir);
117          }
118        });
119    } catch (IOException e) {
120      LOG.error("Failed to create cleaner util", e);
121    }
122  }
123
124  @Override
125  public void stop(String why) {
126    this.cache.stop(why);
127  }
128
129  @Override
130  public boolean isStopped() {
131    return this.cache.isStopped();
132  }
133
134  /**
135   * Exposed for Testing!
136   * @return the cache of all hfiles
137   */
138  public SnapshotFileCache getFileCacheForTesting() {
139    return this.cache;
140  }
141}