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}