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