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.FileNotFoundException;
021import java.io.IOException;
022import java.util.Collection;
023import java.util.HashMap;
024import java.util.HashSet;
025import java.util.List;
026import java.util.Map;
027import java.util.Set;
028import java.util.Timer;
029import java.util.TimerTask;
030import java.util.concurrent.locks.Lock;
031
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.fs.FileStatus;
034import org.apache.hadoop.fs.FileSystem;
035import org.apache.hadoop.fs.Path;
036import org.apache.hadoop.hbase.Stoppable;
037import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException;
038import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
039import org.apache.hadoop.hbase.util.FSUtils;
040import org.apache.yetus.audience.InterfaceAudience;
041import org.apache.yetus.audience.InterfaceStability;
042import org.slf4j.Logger;
043import org.slf4j.LoggerFactory;
044
045import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
046import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
047
048/**
049 * Intelligently keep track of all the files for all the snapshots.
050 * <p>
051 * A cache of files is kept to avoid querying the {@link FileSystem} frequently. If there is a cache
052 * miss the directory modification time is used to ensure that we don't rescan directories that we
053 * already have in cache. We only check the modification times of the snapshot directories
054 * (/hbase/.snapshot/[snapshot_name]) to determine if the files need to be loaded into the cache.
055 * <p>
056 * New snapshots will be added to the cache and deleted snapshots will be removed when we refresh
057 * the cache. If the files underneath a snapshot directory are changed, but not the snapshot itself,
058 * we will ignore updates to that snapshot's files.
059 * <p>
060 * This is sufficient because each snapshot has its own directory and is added via an atomic rename
061 * <i>once</i>, when the snapshot is created. We don't need to worry about the data in the snapshot
062 * being run.
063 * <p>
064 * Further, the cache is periodically refreshed ensure that files in snapshots that were deleted are
065 * also removed from the cache.
066 * <p>
067 * A {@link SnapshotFileCache.SnapshotFileInspector} must be passed when creating <tt>this</tt> to
068 * allow extraction of files under /hbase/.snapshot/[snapshot name] directory, for each snapshot.
069 * This allows you to only cache files under, for instance, all the logs in the .logs directory or
070 * all the files under all the regions.
071 * <p>
072 * <tt>this</tt> also considers all running snapshots (those under /hbase/.snapshot/.tmp) as valid
073 * snapshots and will attempt to cache files from those snapshots as well.
074 * <p>
075 * Queries about a given file are thread-safe with respect to multiple queries and cache refreshes.
076 */
077@InterfaceAudience.Private
078@InterfaceStability.Evolving
079public class SnapshotFileCache implements Stoppable {
080  interface SnapshotFileInspector {
081    /**
082     * Returns a collection of file names needed by the snapshot.
083     * @param snapshotDir {@link Path} to the snapshot directory to scan.
084     * @return the collection of file names needed by the snapshot.
085     */
086    Collection<String> filesUnderSnapshot(final Path snapshotDir) throws IOException;
087  }
088
089  private static final Logger LOG = LoggerFactory.getLogger(SnapshotFileCache.class);
090  private volatile boolean stop = false;
091  private final FileSystem fs;
092  private final SnapshotFileInspector fileInspector;
093  private final Path snapshotDir;
094  private final Set<String> cache = new HashSet<>();
095  /**
096   * This is a helper map of information about the snapshot directories so we don't need to rescan
097   * them if they haven't changed since the last time we looked.
098   */
099  private final Map<String, SnapshotDirectoryInfo> snapshots = new HashMap<>();
100  private final Timer refreshTimer;
101
102  private long lastModifiedTime = Long.MIN_VALUE;
103
104  /**
105   * Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
106   * filesystem.
107   * <p>
108   * Immediately loads the file cache.
109   * @param conf to extract the configured {@link FileSystem} where the snapshots are stored and
110   *          hbase root directory
111   * @param cacheRefreshPeriod frequency (ms) with which the cache should be refreshed
112   * @param refreshThreadName name of the cache refresh thread
113   * @param inspectSnapshotFiles Filter to apply to each snapshot to extract the files.
114   * @throws IOException if the {@link FileSystem} or root directory cannot be loaded
115   */
116  public SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName,
117      SnapshotFileInspector inspectSnapshotFiles) throws IOException {
118    this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf), 0, cacheRefreshPeriod,
119        refreshThreadName, inspectSnapshotFiles);
120  }
121
122  /**
123   * Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
124   * filesystem
125   * @param fs {@link FileSystem} where the snapshots are stored
126   * @param rootDir hbase root directory
127   * @param cacheRefreshPeriod period (ms) with which the cache should be refreshed
128   * @param cacheRefreshDelay amount of time to wait for the cache to be refreshed
129   * @param refreshThreadName name of the cache refresh thread
130   * @param inspectSnapshotFiles Filter to apply to each snapshot to extract the files.
131   */
132  public SnapshotFileCache(FileSystem fs, Path rootDir, long cacheRefreshPeriod,
133      long cacheRefreshDelay, String refreshThreadName, SnapshotFileInspector inspectSnapshotFiles) {
134    this.fs = fs;
135    this.fileInspector = inspectSnapshotFiles;
136    this.snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
137    // periodically refresh the file cache to make sure we aren't superfluously saving files.
138    this.refreshTimer = new Timer(refreshThreadName, true);
139    this.refreshTimer.scheduleAtFixedRate(new RefreshCacheTask(), cacheRefreshDelay,
140      cacheRefreshPeriod);
141  }
142
143  /**
144   * Trigger a cache refresh, even if its before the next cache refresh. Does not affect pending
145   * cache refreshes.
146   * <p>
147   * Blocks until the cache is refreshed.
148   * <p>
149   * Exposed for TESTING.
150   */
151  public void triggerCacheRefreshForTesting() {
152    try {
153      SnapshotFileCache.this.refreshCache();
154    } catch (IOException e) {
155      LOG.warn("Failed to refresh snapshot hfile cache!", e);
156    }
157    LOG.debug("Current cache:" + cache);
158  }
159
160  /**
161   * Check to see if any of the passed file names is contained in any of the snapshots.
162   * First checks an in-memory cache of the files to keep. If its not in the cache, then the cache
163   * is refreshed and the cache checked again for that file.
164   * This ensures that we never return files that exist.
165   * <p>
166   * Note this may lead to periodic false positives for the file being referenced. Periodically, the
167   * cache is refreshed even if there are no requests to ensure that the false negatives get removed
168   * eventually. For instance, suppose you have a file in the snapshot and it gets loaded into the
169   * cache. Then at some point later that snapshot is deleted. If the cache has not been refreshed
170   * at that point, cache will still think the file system contains that file and return
171   * <tt>true</tt>, even if it is no longer present (false positive). However, if the file never was
172   * on the filesystem, we will never find it and always return <tt>false</tt>.
173   * @param files file to check, NOTE: Relies that files are loaded from hdfs before method
174   *              is called (NOT LAZY)
175   * @return <tt>unReferencedFiles</tt> the collection of files that do not have snapshot references
176   * @throws IOException if there is an unexpected error reaching the filesystem.
177   */
178  // XXX this is inefficient to synchronize on the method, when what we really need to guard against
179  // is an illegal access to the cache. Really we could do a mutex-guarded pointer swap on the
180  // cache, but that seems overkill at the moment and isn't necessarily a bottleneck.
181  public synchronized Iterable<FileStatus> getUnreferencedFiles(Iterable<FileStatus> files,
182      final SnapshotManager snapshotManager)
183      throws IOException {
184    List<FileStatus> unReferencedFiles = Lists.newArrayList();
185    List<String> snapshotsInProgress = null;
186    boolean refreshed = false;
187    Lock lock = null;
188    if (snapshotManager != null) {
189      lock = snapshotManager.getTakingSnapshotLock().writeLock();
190    }
191    if (lock == null || lock.tryLock()) {
192      try {
193        if (snapshotManager != null && snapshotManager.isTakingAnySnapshot()) {
194          LOG.warn("Not checking unreferenced files since snapshot is running, it will "
195              + "skip to clean the HFiles this time");
196          return unReferencedFiles;
197        }
198        for (FileStatus file : files) {
199          String fileName = file.getPath().getName();
200          if (!refreshed && !cache.contains(fileName)) {
201            refreshCache();
202            refreshed = true;
203          }
204          if (cache.contains(fileName)) {
205            continue;
206          }
207          if (snapshotsInProgress == null) {
208            snapshotsInProgress = getSnapshotsInProgress();
209          }
210          if (snapshotsInProgress.contains(fileName)) {
211            continue;
212          }
213          unReferencedFiles.add(file);
214        }
215      } finally {
216        if (lock != null) {
217          lock.unlock();
218        }
219      }
220    }
221    return unReferencedFiles;
222  }
223
224  private synchronized void refreshCache() throws IOException {
225    // get the status of the snapshots directory and check if it is has changes
226    FileStatus dirStatus;
227    try {
228      dirStatus = fs.getFileStatus(snapshotDir);
229    } catch (FileNotFoundException e) {
230      if (this.cache.size() > 0) {
231        LOG.error("Snapshot directory: " + snapshotDir + " doesn't exist");
232      }
233      return;
234    }
235
236    // if the snapshot directory wasn't modified since we last check, we are done
237    if (dirStatus.getModificationTime() <= this.lastModifiedTime) return;
238
239    // directory was modified, so we need to reload our cache
240    // there could be a slight race here where we miss the cache, check the directory modification
241    // time, then someone updates the directory, causing us to not scan the directory again.
242    // However, snapshot directories are only created once, so this isn't an issue.
243
244    // 1. update the modified time
245    this.lastModifiedTime = dirStatus.getModificationTime();
246
247    // 2.clear the cache
248    this.cache.clear();
249    Map<String, SnapshotDirectoryInfo> known = new HashMap<>();
250
251    // 3. check each of the snapshot directories
252    FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir);
253    if (snapshots == null) {
254      // remove all the remembered snapshots because we don't have any left
255      if (LOG.isDebugEnabled() && this.snapshots.size() > 0) {
256        LOG.debug("No snapshots on-disk, cache empty");
257      }
258      this.snapshots.clear();
259      return;
260    }
261
262    // 3.1 iterate through the on-disk snapshots
263    for (FileStatus snapshot : snapshots) {
264      String name = snapshot.getPath().getName();
265      // its not the tmp dir,
266      if (!name.equals(SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME)) {
267        SnapshotDirectoryInfo files = this.snapshots.remove(name);
268        // 3.1.1 if we don't know about the snapshot or its been modified, we need to update the
269        // files the latter could occur where I create a snapshot, then delete it, and then make a
270        // new snapshot with the same name. We will need to update the cache the information from
271        // that new snapshot, even though it has the same name as the files referenced have
272        // probably changed.
273        if (files == null || files.hasBeenModified(snapshot.getModificationTime())) {
274          // get all files for the snapshot and create a new info
275          Collection<String> storedFiles = fileInspector.filesUnderSnapshot(snapshot.getPath());
276          files = new SnapshotDirectoryInfo(snapshot.getModificationTime(), storedFiles);
277        }
278        // 3.2 add all the files to cache
279        this.cache.addAll(files.getFiles());
280        known.put(name, files);
281      }
282    }
283
284    // 4. set the snapshots we are tracking
285    this.snapshots.clear();
286    this.snapshots.putAll(known);
287  }
288
289  @VisibleForTesting
290  List<String> getSnapshotsInProgress() throws IOException {
291    List<String> snapshotInProgress = Lists.newArrayList();
292    // only add those files to the cache, but not to the known snapshots
293    Path snapshotTmpDir = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME);
294    FileStatus[] running = FSUtils.listStatus(fs, snapshotTmpDir);
295    if (running != null) {
296      for (FileStatus run : running) {
297        try {
298          snapshotInProgress.addAll(fileInspector.filesUnderSnapshot(run.getPath()));
299        } catch (CorruptedSnapshotException e) {
300          // See HBASE-16464
301          if (e.getCause() instanceof FileNotFoundException) {
302            // If the snapshot is corrupt, we will delete it
303            fs.delete(run.getPath(), true);
304            LOG.warn("delete the " + run.getPath() + " due to exception:", e.getCause());
305          } else {
306            throw e;
307          }
308        }
309      }
310    }
311    return snapshotInProgress;
312  }
313
314  /**
315   * Simple helper task that just periodically attempts to refresh the cache
316   */
317  public class RefreshCacheTask extends TimerTask {
318    @Override
319    public void run() {
320      try {
321        SnapshotFileCache.this.refreshCache();
322      } catch (IOException e) {
323        LOG.warn("Failed to refresh snapshot hfile cache!", e);
324      }
325    }
326  }
327
328  @Override
329  public void stop(String why) {
330    if (!this.stop) {
331      this.stop = true;
332      this.refreshTimer.cancel();
333    }
334
335  }
336
337  @Override
338  public boolean isStopped() {
339    return this.stop;
340  }
341
342  /**
343   * Information about a snapshot directory
344   */
345  private static class SnapshotDirectoryInfo {
346    long lastModified;
347    Collection<String> files;
348
349    public SnapshotDirectoryInfo(long mtime, Collection<String> files) {
350      this.lastModified = mtime;
351      this.files = files;
352    }
353
354    /**
355     * @return the hfiles in the snapshot when <tt>this</tt> was made.
356     */
357    public Collection<String> getFiles() {
358      return this.files;
359    }
360
361    /**
362     * Check if the snapshot directory has been modified
363     * @param mtime current modification time of the directory
364     * @return <tt>true</tt> if it the modification time of the directory is newer time when we
365     *         created <tt>this</tt>
366     */
367    public boolean hasBeenModified(long mtime) {
368      return this.lastModified < mtime;
369    }
370  }
371}