1 /**
2  * Licensed to the Apache Software Foundation (ASF) under one
3  * or more contributor license agreements.  See the NOTICE file
4  * distributed with this work for additional information
5  * regarding copyright ownership.  The ASF licenses this file
6  * to you under the Apache License, Version 2.0 (the
7  * "License"); you may not use this file except in compliance
8  * with the License.  You may obtain a copy of the License at
9  *
10  *     http://www.apache.org/licenses/LICENSE-2.0
11  *
12  * Unless required by applicable law or agreed to in writing, software
13  * distributed under the License is distributed on an "AS IS" BASIS,
14  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15  * See the License for the specific language governing permissions and
16  * limitations under the License.
17  */
18 package org.apache.hadoop.hbase.master.snapshot;
19 
20 import java.io.FileNotFoundException;
21 import java.io.IOException;
22 import java.util.Collection;
23 import java.util.HashMap;
24 import java.util.HashSet;
25 import java.util.List;
26 import java.util.Map;
27 import java.util.Set;
28 import java.util.Timer;
29 import java.util.TimerTask;
30 
31 import com.google.common.annotations.VisibleForTesting;
32 import com.google.common.collect.Lists;
33 import org.apache.commons.logging.Log;
34 import org.apache.commons.logging.LogFactory;
35 import org.apache.hadoop.hbase.classification.InterfaceAudience;
36 import org.apache.hadoop.hbase.classification.InterfaceStability;
37 import org.apache.hadoop.conf.Configuration;
38 import org.apache.hadoop.fs.FileStatus;
39 import org.apache.hadoop.fs.FileSystem;
40 import org.apache.hadoop.fs.Path;
41 import org.apache.hadoop.hbase.Stoppable;
42 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
43 import org.apache.hadoop.hbase.util.FSUtils;
44 
45 /**
46  * Intelligently keep track of all the files for all the snapshots.
47  * <p>
48  * A cache of files is kept to avoid querying the {@link FileSystem} frequently. If there is a cache
49  * miss the directory modification time is used to ensure that we don't rescan directories that we
50  * already have in cache. We only check the modification times of the snapshot directories
51  * (/hbase/.snapshot/[snapshot_name]) to determine if the files need to be loaded into the cache.
52  * <p>
53  * New snapshots will be added to the cache and deleted snapshots will be removed when we refresh
54  * the cache. If the files underneath a snapshot directory are changed, but not the snapshot itself,
55  * we will ignore updates to that snapshot's files.
56  * <p>
57  * This is sufficient because each snapshot has its own directory and is added via an atomic rename
58  * <i>once</i>, when the snapshot is created. We don't need to worry about the data in the snapshot
59  * being run.
60  * <p>
61  * Further, the cache is periodically refreshed ensure that files in snapshots that were deleted are
62  * also removed from the cache.
63  * <p>
64  * A {@link SnapshotFileCache.SnapshotFileInspector} must be passed when creating <tt>this</tt> to
65  * allow extraction of files under /hbase/.snapshot/[snapshot name] directory, for each snapshot.
66  * This allows you to only cache files under, for instance, all the logs in the .logs directory or
67  * all the files under all the regions.
68  * <p>
69  * <tt>this</tt> also considers all running snapshots (those under /hbase/.snapshot/.tmp) as valid
70  * snapshots and will attempt to cache files from those snapshots as well.
71  * <p>
72  * Queries about a given file are thread-safe with respect to multiple queries and cache refreshes.
73  */
74 @InterfaceAudience.Private
75 @InterfaceStability.Evolving
76 public class SnapshotFileCache implements Stoppable {
77   interface SnapshotFileInspector {
78     /**
79      * Returns a collection of file names needed by the snapshot.
80      * @param snapshotDir {@link Path} to the snapshot directory to scan.
81      * @return the collection of file names needed by the snapshot.
82      */
filesUnderSnapshot(final Path snapshotDir)83     Collection<String> filesUnderSnapshot(final Path snapshotDir) throws IOException;
84   }
85 
86   private static final Log LOG = LogFactory.getLog(SnapshotFileCache.class);
87   private volatile boolean stop = false;
88   private final FileSystem fs;
89   private final SnapshotFileInspector fileInspector;
90   private final Path snapshotDir;
91   private final Set<String> cache = new HashSet<String>();
92   /**
93    * This is a helper map of information about the snapshot directories so we don't need to rescan
94    * them if they haven't changed since the last time we looked.
95    */
96   private final Map<String, SnapshotDirectoryInfo> snapshots =
97       new HashMap<String, SnapshotDirectoryInfo>();
98   private final Timer refreshTimer;
99 
100   private long lastModifiedTime = Long.MIN_VALUE;
101 
102   /**
103    * Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
104    * filesystem.
105    * <p>
106    * Immediately loads the file cache.
107    * @param conf to extract the configured {@link FileSystem} where the snapshots are stored and
108    *          hbase root directory
109    * @param cacheRefreshPeriod frequency (ms) with which the cache should be refreshed
110    * @param refreshThreadName name of the cache refresh thread
111    * @param inspectSnapshotFiles Filter to apply to each snapshot to extract the files.
112    * @throws IOException if the {@link FileSystem} or root directory cannot be loaded
113    */
SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName, SnapshotFileInspector inspectSnapshotFiles)114   public SnapshotFileCache(Configuration conf, long cacheRefreshPeriod, String refreshThreadName,
115       SnapshotFileInspector inspectSnapshotFiles) throws IOException {
116     this(FSUtils.getCurrentFileSystem(conf), FSUtils.getRootDir(conf), 0, cacheRefreshPeriod,
117         refreshThreadName, inspectSnapshotFiles);
118   }
119 
120   /**
121    * Create a snapshot file cache for all snapshots under the specified [root]/.snapshot on the
122    * filesystem
123    * @param fs {@link FileSystem} where the snapshots are stored
124    * @param rootDir hbase root directory
125    * @param cacheRefreshPeriod period (ms) with which the cache should be refreshed
126    * @param cacheRefreshDelay amount of time to wait for the cache to be refreshed
127    * @param refreshThreadName name of the cache refresh thread
128    * @param inspectSnapshotFiles Filter to apply to each snapshot to extract the files.
129    */
SnapshotFileCache(FileSystem fs, Path rootDir, long cacheRefreshPeriod, long cacheRefreshDelay, String refreshThreadName, SnapshotFileInspector inspectSnapshotFiles)130   public SnapshotFileCache(FileSystem fs, Path rootDir, long cacheRefreshPeriod,
131       long cacheRefreshDelay, String refreshThreadName, SnapshotFileInspector inspectSnapshotFiles) {
132     this.fs = fs;
133     this.fileInspector = inspectSnapshotFiles;
134     this.snapshotDir = SnapshotDescriptionUtils.getSnapshotsDir(rootDir);
135     // periodically refresh the file cache to make sure we aren't superfluously saving files.
136     this.refreshTimer = new Timer(refreshThreadName, true);
137     this.refreshTimer.scheduleAtFixedRate(new RefreshCacheTask(), cacheRefreshDelay,
138       cacheRefreshPeriod);
139   }
140 
141   /**
142    * Trigger a cache refresh, even if its before the next cache refresh. Does not affect pending
143    * cache refreshes.
144    * <p>
145    * Blocks until the cache is refreshed.
146    * <p>
147    * Exposed for TESTING.
148    */
triggerCacheRefreshForTesting()149   public void triggerCacheRefreshForTesting() {
150     try {
151       SnapshotFileCache.this.refreshCache();
152     } catch (IOException e) {
153       LOG.warn("Failed to refresh snapshot hfile cache!", e);
154     }
155     LOG.debug("Current cache:" + cache);
156   }
157 
158   /**
159    * Check to see if any of the passed file names is contained in any of the snapshots.
160    * First checks an in-memory cache of the files to keep. If its not in the cache, then the cache
161    * is refreshed and the cache checked again for that file.
162    * This ensures that we never return files that exist.
163    * <p>
164    * Note this may lead to periodic false positives for the file being referenced. Periodically, the
165    * cache is refreshed even if there are no requests to ensure that the false negatives get removed
166    * eventually. For instance, suppose you have a file in the snapshot and it gets loaded into the
167    * cache. Then at some point later that snapshot is deleted. If the cache has not been refreshed
168    * at that point, cache will still think the file system contains that file and return
169    * <tt>true</tt>, even if it is no longer present (false positive). However, if the file never was
170    * on the filesystem, we will never find it and always return <tt>false</tt>.
171    * @param files file to check, NOTE: Relies that files are loaded from hdfs before method
172    *              is called (NOT LAZY)
173    * @return <tt>unReferencedFiles</tt> the collection of files that do not have snapshot references
174    * @throws IOException if there is an unexpected error reaching the filesystem.
175    */
176   // XXX this is inefficient to synchronize on the method, when what we really need to guard against
177   // is an illegal access to the cache. Really we could do a mutex-guarded pointer swap on the
178   // cache, but that seems overkill at the moment and isn't necessarily a bottleneck.
getUnreferencedFiles(Iterable<FileStatus> files)179   public synchronized Iterable<FileStatus> getUnreferencedFiles(Iterable<FileStatus> files)
180       throws IOException {
181     List<FileStatus> unReferencedFiles = Lists.newArrayList();
182     List<String> snapshotsInProgress = null;
183     boolean refreshed = false;
184     for (FileStatus file : files) {
185       String fileName = file.getPath().getName();
186       if (!refreshed && !cache.contains(fileName)) {
187         refreshCache();
188         refreshed = true;
189       }
190       if (cache.contains(fileName)) {
191         continue;
192       }
193       if (snapshotsInProgress == null) {
194         snapshotsInProgress = getSnapshotsInProgress();
195       }
196       if (snapshotsInProgress.contains(fileName)) {
197         continue;
198       }
199       unReferencedFiles.add(file);
200     }
201     return unReferencedFiles;
202   }
203 
refreshCache()204   private synchronized void refreshCache() throws IOException {
205     long lastTimestamp = Long.MAX_VALUE;
206     boolean hasChanges = false;
207 
208     // get the status of the snapshots directory and check if it is has changes
209     try {
210       FileStatus dirStatus = fs.getFileStatus(snapshotDir);
211       lastTimestamp = dirStatus.getModificationTime();
212       hasChanges |= (lastTimestamp >= lastModifiedTime);
213     } catch (FileNotFoundException e) {
214       if (this.cache.size() > 0) {
215         LOG.error("Snapshot directory: " + snapshotDir + " doesn't exist");
216       }
217       return;
218     }
219 
220     // get the status of the snapshots temporary directory and check if it has changes
221     // The top-level directory timestamp is not updated, so we have to check the inner-level.
222     try {
223       Path snapshotTmpDir = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME);
224       FileStatus tempDirStatus = fs.getFileStatus(snapshotTmpDir);
225       lastTimestamp = Math.min(lastTimestamp, tempDirStatus.getModificationTime());
226       hasChanges |= (lastTimestamp >= lastModifiedTime);
227       if (!hasChanges) {
228         FileStatus[] tmpSnapshots = FSUtils.listStatus(fs, snapshotDir);
229         if (tmpSnapshots != null) {
230           for (FileStatus dirStatus: tmpSnapshots) {
231             lastTimestamp = Math.min(lastTimestamp, dirStatus.getModificationTime());
232           }
233           hasChanges |= (lastTimestamp >= lastModifiedTime);
234         }
235       }
236     } catch (FileNotFoundException e) {
237       // Nothing todo, if the tmp dir is empty
238     }
239 
240     // if the snapshot directory wasn't modified since we last check, we are done
241     if (!hasChanges) {
242       return;
243     }
244 
245     // directory was modified, so we need to reload our cache
246     // there could be a slight race here where we miss the cache, check the directory modification
247     // time, then someone updates the directory, causing us to not scan the directory again.
248     // However, snapshot directories are only created once, so this isn't an issue.
249 
250     // 1. update the modified time
251     this.lastModifiedTime = lastTimestamp;
252 
253     // 2.clear the cache
254     this.cache.clear();
255     Map<String, SnapshotDirectoryInfo> known = new HashMap<String, SnapshotDirectoryInfo>();
256 
257     // 3. check each of the snapshot directories
258     FileStatus[] snapshots = FSUtils.listStatus(fs, snapshotDir);
259     if (snapshots == null) {
260       // remove all the remembered snapshots because we don't have any left
261       if (LOG.isDebugEnabled() && this.snapshots.size() > 0) {
262         LOG.debug("No snapshots on-disk, cache empty");
263       }
264       this.snapshots.clear();
265       return;
266     }
267 
268     // 3.1 iterate through the on-disk snapshots
269     for (FileStatus snapshot : snapshots) {
270       String name = snapshot.getPath().getName();
271       // its not the tmp dir,
272       if (!name.equals(SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME)) {
273         SnapshotDirectoryInfo files = this.snapshots.remove(name);
274         // 3.1.1 if we don't know about the snapshot or its been modified, we need to update the
275         // files the latter could occur where I create a snapshot, then delete it, and then make a
276         // new snapshot with the same name. We will need to update the cache the information from
277         // that new snapshot, even though it has the same name as the files referenced have
278         // probably changed.
279         if (files == null || files.hasBeenModified(snapshot.getModificationTime())) {
280           // get all files for the snapshot and create a new info
281           Collection<String> storedFiles = fileInspector.filesUnderSnapshot(snapshot.getPath());
282           files = new SnapshotDirectoryInfo(snapshot.getModificationTime(), storedFiles);
283         }
284         // 3.2 add all the files to cache
285         this.cache.addAll(files.getFiles());
286         known.put(name, files);
287       }
288     }
289 
290     // 4. set the snapshots we are tracking
291     this.snapshots.clear();
292     this.snapshots.putAll(known);
293   }
294 
getSnapshotsInProgress()295   @VisibleForTesting List<String> getSnapshotsInProgress() throws IOException {
296     List<String> snapshotInProgress = Lists.newArrayList();
297     // only add those files to the cache, but not to the known snapshots
298     Path snapshotTmpDir = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME);
299     // only add those files to the cache, but not to the known snapshots
300     FileStatus[] running = FSUtils.listStatus(fs, snapshotTmpDir);
301     if (running != null) {
302       for (FileStatus run : running) {
303         snapshotInProgress.addAll(fileInspector.filesUnderSnapshot(run.getPath()));
304       }
305     }
306     return snapshotInProgress;
307   }
308 
309   /**
310    * Simple helper task that just periodically attempts to refresh the cache
311    */
312   public class RefreshCacheTask extends TimerTask {
313     @Override
run()314     public void run() {
315       try {
316         SnapshotFileCache.this.refreshCache();
317       } catch (IOException e) {
318         LOG.warn("Failed to refresh snapshot hfile cache!", e);
319       }
320     }
321   }
322 
323   @Override
stop(String why)324   public void stop(String why) {
325     if (!this.stop) {
326       this.stop = true;
327       this.refreshTimer.cancel();
328     }
329 
330   }
331 
332   @Override
isStopped()333   public boolean isStopped() {
334     return this.stop;
335   }
336 
337   /**
338    * Information about a snapshot directory
339    */
340   private static class SnapshotDirectoryInfo {
341     long lastModified;
342     Collection<String> files;
343 
SnapshotDirectoryInfo(long mtime, Collection<String> files)344     public SnapshotDirectoryInfo(long mtime, Collection<String> files) {
345       this.lastModified = mtime;
346       this.files = files;
347     }
348 
349     /**
350      * @return the hfiles in the snapshot when <tt>this</tt> was made.
351      */
getFiles()352     public Collection<String> getFiles() {
353       return this.files;
354     }
355 
356     /**
357      * Check if the snapshot directory has been modified
358      * @param mtime current modification time of the directory
359      * @return <tt>true</tt> if it the modification time of the directory is newer time when we
360      *         created <tt>this</tt>
361      */
hasBeenModified(long mtime)362     public boolean hasBeenModified(long mtime) {
363       return this.lastModified < mtime;
364     }
365   }
366 }
367