From 7eaba369e745f8095204a39a037c1d96ae7deac6 Mon Sep 17 00:00:00 2001 From: chenheng Date: Sat, 27 Aug 2016 10:36:21 +0800 Subject: [PATCH] HBASE016490 Fix race condition between SnapshotManager and SnapshotCleaner --- .../apache/hadoop/hbase/master/HMaster.java | 5 +++- .../cleaner/BaseFileCleanerDelegate.java | 7 +++++ .../hbase/master/cleaner/CleanerChore.java | 29 ++++++++++++------- .../master/cleaner/FileCleanerDelegate.java | 8 +++++ .../hbase/master/cleaner/HFileCleaner.java | 12 ++++++-- .../DisabledTableSnapshotHandler.java | 4 +-- .../snapshot/EnabledTableSnapshotHandler.java | 2 +- .../master/snapshot/SnapshotFileCache.java | 28 +++++++++++------- .../master/snapshot/SnapshotHFileCleaner.java | 15 +++++++++- .../master/snapshot/SnapshotManager.java | 22 +++++++++++++- .../master/snapshot/TakeSnapshotHandler.java | 10 +++++-- .../snapshot/TestSnapshotFileCache.java | 11 +++---- .../snapshot/TestSnapshotHFileCleaner.java | 19 ++++++------ 13 files changed, 127 insertions(+), 45 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index 5b3984b9bb2..f4c2c1c0672 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -973,8 +974,10 @@ public class HMaster extends HRegionServer implements MasterServices { //start the hfile archive cleaner thread Path archiveDir = HFileArchiveUtil.getArchivePath(conf); + Map params = new HashMap(); + params.put(MASTER, this); this.hfileCleaner = new HFileCleaner(cleanerInterval, this, conf, getMasterFileSystem() - .getFileSystem(), archiveDir); + .getFileSystem(), archiveDir, params); getChoreService().scheduleChore(hfileCleaner); serviceStarted = true; if (LOG.isTraceEnabled()) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseFileCleanerDelegate.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseFileCleanerDelegate.java index c6955d0d501..891db22e98c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseFileCleanerDelegate.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/BaseFileCleanerDelegate.java @@ -23,6 +23,8 @@ import org.apache.hadoop.hbase.BaseConfigurable; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; +import java.util.Map; + /** * Base class for file cleaners which allows subclasses to implement a simple * isFileDeletable method (which used to be the FileCleanerDelegate contract). @@ -39,6 +41,11 @@ implements FileCleanerDelegate { }}); } + @Override + public void init(Map params) { + // subclass could override it if needed. + } + /** * Should the master delete the file or keep it? * @param fStat file status of the file to check diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java index 5a93a6d0c4d..b094507c9aa 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java @@ -17,10 +17,10 @@ */ package org.apache.hadoop.hbase.master.cleaner; -import java.io.IOException; -import java.util.LinkedList; -import java.util.List; - +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; @@ -32,10 +32,10 @@ import org.apache.hadoop.hbase.Stoppable; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.ipc.RemoteException; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; +import java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; /** * Abstract Cleaner that uses a chain of delegates to clean a directory of files @@ -49,6 +49,12 @@ public abstract class CleanerChore extends Schedu private final Path oldFileDir; private final Configuration conf; protected List cleanersChain; + protected Map params; + + public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf, + FileSystem fs, Path oldFileDir, String confKey) { + this(name, sleepPeriod, s, conf, fs, oldFileDir, confKey, null); + } /** * @param name name of the chore being run @@ -58,17 +64,19 @@ public abstract class CleanerChore extends Schedu * @param fs handle to the FS * @param oldFileDir the path to the archived files * @param confKey configuration key for the classes to instantiate + * @param params members could be used in cleaner */ public CleanerChore(String name, final int sleepPeriod, final Stoppable s, Configuration conf, - FileSystem fs, Path oldFileDir, String confKey) { + FileSystem fs, Path oldFileDir, String confKey, Map params) { super(name, s, sleepPeriod); this.fs = fs; this.oldFileDir = oldFileDir; this.conf = conf; - + this.params = params; initCleanerChain(confKey); } + /** * Validate the file to see if it even belongs in the directory. If it is valid, then the file * will go through the cleaner delegates, but otherwise the file is just deleted. @@ -109,6 +117,7 @@ public abstract class CleanerChore extends Schedu @SuppressWarnings("unchecked") T cleaner = (T) c.newInstance(); cleaner.setConf(conf); + cleaner.init(this.params); return cleaner; } catch (Exception e) { LOG.warn("Can NOT create CleanerDelegate: " + className, e); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java index b11fd80e0bf..7a15b9671d4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/FileCleanerDelegate.java @@ -22,6 +22,8 @@ import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.hbase.Stoppable; +import java.util.Map; + /** * General interface for cleaning files from a folder (generally an archive or * backup folder). These are chained via the {@link CleanerChore} to determine @@ -36,4 +38,10 @@ public interface FileCleanerDelegate extends Configurable, Stoppable { * @return files that are ok to delete according to this cleaner */ Iterable getDeletableFiles(Iterable files); + + + /** + * this method is used to pass some instance into subclass + * */ + void init(Map params); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java index 27851550dbc..89c316b2e45 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/HFileCleaner.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hbase.master.cleaner; import java.util.List; +import java.util.Map; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -35,16 +36,23 @@ public class HFileCleaner extends CleanerChore { public static final String MASTER_HFILE_CLEANER_PLUGINS = "hbase.master.hfilecleaner.plugins"; + public HFileCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs, + Path directory) { + this(period, stopper, conf, fs, directory, null); + } + /** * @param period the period of time to sleep between each run * @param stopper the stopper * @param conf configuration to use * @param fs handle to the FS * @param directory directory to be cleaned + * @param params params could be used in subclass of BaseHFileCleanerDelegate */ public HFileCleaner(final int period, final Stoppable stopper, Configuration conf, FileSystem fs, - Path directory) { - super("HFileCleaner", period, stopper, conf, fs, directory, MASTER_HFILE_CLEANER_PLUGINS); + Path directory, Map params) { + super("HFileCleaner", period, stopper, conf, fs, + directory, MASTER_HFILE_CLEANER_PLUGINS, params); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java index 5d5922997cb..a7c26520074 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/DisabledTableSnapshotHandler.java @@ -58,8 +58,8 @@ public class DisabledTableSnapshotHandler extends TakeSnapshotHandler { * @param masterServices master services provider */ public DisabledTableSnapshotHandler(SnapshotDescription snapshot, - final MasterServices masterServices) { - super(snapshot, masterServices); + final MasterServices masterServices, final SnapshotManager snapshotManager) { + super(snapshot, masterServices, snapshotManager); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java index 7e047acdc67..f545a825954 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/EnabledTableSnapshotHandler.java @@ -50,7 +50,7 @@ public class EnabledTableSnapshotHandler extends TakeSnapshotHandler { public EnabledTableSnapshotHandler(SnapshotDescription snapshot, MasterServices master, final SnapshotManager manager) { - super(snapshot, master); + super(snapshot, master, manager); this.coordinator = manager.getCoordinator(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java index 5b367c501ee..f80d962cc3f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotFileCache.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import java.util.Timer; import java.util.TimerTask; +import java.util.concurrent.locks.ReentrantLock; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -177,7 +178,8 @@ public class SnapshotFileCache implements Stoppable { // XXX this is inefficient to synchronize on the method, when what we really need to guard against // is an illegal access to the cache. Really we could do a mutex-guarded pointer swap on the // cache, but that seems overkill at the moment and isn't necessarily a bottleneck. - public synchronized Iterable getUnreferencedFiles(Iterable files) + public synchronized Iterable getUnreferencedFiles(Iterable files, + final SnapshotManager snapshotManager) throws IOException { List unReferencedFiles = Lists.newArrayList(); List snapshotsInProgress = null; @@ -192,7 +194,7 @@ public class SnapshotFileCache implements Stoppable { continue; } if (snapshotsInProgress == null) { - snapshotsInProgress = getSnapshotsInProgress(); + snapshotsInProgress = getSnapshotsInProgress(snapshotManager); } if (snapshotsInProgress.contains(fileName)) { continue; @@ -292,8 +294,9 @@ public class SnapshotFileCache implements Stoppable { this.snapshots.clear(); this.snapshots.putAll(known); } - - @VisibleForTesting List getSnapshotsInProgress() throws IOException { + + @VisibleForTesting List getSnapshotsInProgress( + final SnapshotManager snapshotManager) throws IOException { List snapshotInProgress = Lists.newArrayList(); // only add those files to the cache, but not to the known snapshots Path snapshotTmpDir = new Path(snapshotDir, SnapshotDescriptionUtils.SNAPSHOT_TMP_DIR_NAME); @@ -301,20 +304,25 @@ public class SnapshotFileCache implements Stoppable { FileStatus[] running = FSUtils.listStatus(fs, snapshotTmpDir); if (running != null) { for (FileStatus run : running) { + ReentrantLock lock = null; + if (snapshotManager != null) { + lock = snapshotManager.getLocks().acquireLock(run.getPath().getName()); + } try { snapshotInProgress.addAll(fileInspector.filesUnderSnapshot(run.getPath())); } catch (CorruptedSnapshotException e) { // See HBASE-16464 if (e.getCause() instanceof FileNotFoundException) { - // If the snapshot is not in progress, we will delete it - if (!fs.exists(new Path(run.getPath(), - SnapshotDescriptionUtils.SNAPSHOT_IN_PROGRESS))) { - fs.delete(run.getPath(), true); - LOG.warn("delete the " + run.getPath() + " due to exception:", e.getCause()); - } + // If the snapshot is corrupt, we will delete it + fs.delete(run.getPath(), true); + LOG.warn("delete the " + run.getPath() + " due to exception:", e.getCause()); } else { throw e; } + } finally { + if (lock != null) { + lock.unlock(); + } } } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java index df03d63db2c..2fdbd55761f 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotHFileCleaner.java @@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.master.snapshot; import java.io.IOException; import java.util.Collection; import java.util.Collections; +import java.util.Map; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -30,6 +31,8 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hbase.HBaseInterfaceAudience; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.master.MasterServices; import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate; import org.apache.hadoop.hbase.snapshot.CorruptedSnapshotException; import org.apache.hadoop.hbase.snapshot.SnapshotReferenceUtil; @@ -57,10 +60,12 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate { /** File cache for HFiles in the completed and currently running snapshots */ private SnapshotFileCache cache; + private MasterServices master; + @Override public synchronized Iterable getDeletableFiles(Iterable files) { try { - return cache.getUnreferencedFiles(files); + return cache.getUnreferencedFiles(files, master.getSnapshotManager()); } catch (CorruptedSnapshotException cse) { LOG.debug("Corrupted in-progress snapshot file exception, ignored ", cse); } catch (IOException e) { @@ -69,6 +74,13 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate { return Collections.emptyList(); } + @Override + public void init(Map params) { + if (params.containsKey(HMaster.MASTER)) { + this.master = (MasterServices) params.get(HMaster.MASTER); + } + } + @Override protected boolean isFileDeletable(FileStatus fStat) { return false; @@ -93,6 +105,7 @@ public class SnapshotHFileCleaner extends BaseHFileCleanerDelegate { } } + @Override public void stop(String why) { this.cache.stop(why); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java index 0304e381ab8..abd69ad18d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/SnapshotManager.java @@ -27,7 +27,10 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -84,6 +87,8 @@ import org.apache.hadoop.hbase.snapshot.TablePartiallyOpenException; import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; import org.apache.hadoop.hbase.util.FSUtils; +import org.apache.hadoop.hbase.util.KeyLocker; +import org.apache.hadoop.hbase.wal.WAL; import org.apache.zookeeper.KeeperException; /** @@ -160,6 +165,16 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable private Path rootDir; private ExecutorService executorService; + /** + * Locks for snapshot operations + * key is snapshot's filename in progress, value is the related lock + * - create snapshot + * - SnapshotCleaner + * */ + private KeyLocker locks = new KeyLocker(); + + + public SnapshotManager() {} /** @@ -471,7 +486,7 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable // Take the snapshot of the disabled table DisabledTableSnapshotHandler handler = - new DisabledTableSnapshotHandler(snapshot, master); + new DisabledTableSnapshotHandler(snapshot, master, this); snapshotTable(snapshot, handler); } @@ -1172,4 +1187,9 @@ public class SnapshotManager extends MasterProcedureManager implements Stoppable builder.setType(SnapshotDescription.Type.FLUSH); return builder.build(); } + + public KeyLocker getLocks() { + return locks; + } + } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java index 8967a7093ed..a0e5b936f88 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/snapshot/TakeSnapshotHandler.java @@ -23,6 +23,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CancellationException; +import java.util.concurrent.locks.ReentrantLock; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -87,6 +88,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh protected final MonitoredTask status; protected final TableName snapshotTable; protected final SnapshotManifest snapshotManifest; + protected final SnapshotManager snapshotManager; protected HTableDescriptor htd; @@ -94,13 +96,15 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh * @param snapshot descriptor of the snapshot to take * @param masterServices master services provider */ - public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices) { + public TakeSnapshotHandler(SnapshotDescription snapshot, final MasterServices masterServices, + final SnapshotManager snapshotManager) { super(masterServices, EventType.C_M_SNAPSHOT_TABLE); assert snapshot != null : "SnapshotDescription must not be nul1"; assert masterServices != null : "MasterServices must not be nul1"; this.master = masterServices; this.snapshot = snapshot; + this.snapshotManager = snapshotManager; this.snapshotTable = TableName.valueOf(snapshot.getTable()); this.conf = this.master.getConfiguration(); this.fs = this.master.getMasterFileSystem().getFileSystem(); @@ -160,11 +164,12 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh String msg = "Running " + snapshot.getType() + " table snapshot " + snapshot.getName() + " " + eventType + " on table " + snapshotTable; LOG.info(msg); + ReentrantLock lock = snapshotManager.getLocks().acquireLock(snapshot.getName()); status.setStatus(msg); try { // If regions move after this meta scan, the region specific snapshot should fail, triggering // an external exception that gets captured here. - SnapshotDescriptionUtils.createInProgressTag(workingDir, fs); + // write down the snapshot info in the working directory SnapshotDescriptionUtils.writeSnapshotInfo(snapshot, workingDir, fs); snapshotManifest.addTableDescriptor(this.htd); @@ -228,6 +233,7 @@ public abstract class TakeSnapshotHandler extends EventHandler implements Snapsh } catch (IOException e) { LOG.error("Couldn't delete snapshot working directory:" + workingDir); } + lock.unlock(); releaseTableLock(); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java index 2e0c14c1032..eb2c84c1581 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotFileCache.java @@ -137,8 +137,9 @@ public class TestSnapshotFileCache { SnapshotFileCache cache = new SnapshotFileCache(fs, rootDir, period, 10000000, "test-snapshot-file-cache-refresh", new SnapshotFiles()) { @Override - List getSnapshotsInProgress() throws IOException { - List result = super.getSnapshotsInProgress(); + List getSnapshotsInProgress(final SnapshotManager snapshotManager) + throws IOException { + List result = super.getSnapshotsInProgress(snapshotManager); count.incrementAndGet(); return result; } @@ -159,7 +160,7 @@ public class TestSnapshotFileCache { FSUtils.logFileSystemState(fs, rootDir, LOG); List allStoreFiles = getStoreFilesForSnapshot(complete); - Iterable deletableFiles = cache.getUnreferencedFiles(allStoreFiles); + Iterable deletableFiles = cache.getUnreferencedFiles(allStoreFiles, null); assertTrue(Iterables.isEmpty(deletableFiles)); // no need for tmp dir check as all files are accounted for. assertEquals(0, count.get() - countBeforeCheck); @@ -168,7 +169,7 @@ public class TestSnapshotFileCache { // add a random file to make sure we refresh FileStatus randomFile = mockStoreFile(UUID.randomUUID().toString()); allStoreFiles.add(randomFile); - deletableFiles = cache.getUnreferencedFiles(allStoreFiles); + deletableFiles = cache.getUnreferencedFiles(allStoreFiles, null); assertEquals(randomFile, Iterables.getOnlyElement(deletableFiles)); assertEquals(1, count.get() - countBeforeCheck); // we check the tmp directory } @@ -275,7 +276,7 @@ public class TestSnapshotFileCache { private Iterable getNonSnapshotFiles(SnapshotFileCache cache, Path storeFile) throws IOException { return cache.getUnreferencedFiles( - Arrays.asList(FSUtils.listStatus(fs, storeFile.getParent())) + Arrays.asList(FSUtils.listStatus(fs, storeFile.getParent())), null ); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java index 88ca5a08fb8..1a27433eb43 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/snapshot/TestSnapshotHFileCleaner.java @@ -134,7 +134,7 @@ public class TestSnapshotHFileCleaner { SnapshotFileCache cache = new SnapshotFileCache(fs, rootDir, period, 10000000, "test-snapshot-file-cache-refresh", new SnapshotFiles()); try { - cache.getSnapshotsInProgress(); + cache.getSnapshotsInProgress(null); } catch (CorruptedSnapshotException cse) { LOG.info("Expected exception " + cse); } finally { @@ -161,7 +161,7 @@ public class TestSnapshotHFileCleaner { SnapshotFileCache cache = new SnapshotFileCache(fs, rootDir, period, 10000000, "test-snapshot-file-cache-refresh", new SnapshotFiles()); try { - cache.getSnapshotsInProgress(); + cache.getSnapshotsInProgress(null); } catch (CorruptedSnapshotException cse) { LOG.info("Expected exception " + cse); } finally { @@ -169,23 +169,22 @@ public class TestSnapshotHFileCleaner { } } - /** - * HBASE-16464 - */ + * HBASE-16464 + */ @Test public void testMissedTmpSnapshot() throws IOException { SnapshotTestingUtils.SnapshotMock - snapshotMock = new SnapshotTestingUtils.SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir); + snapshotMock = new SnapshotTestingUtils.SnapshotMock(TEST_UTIL.getConfiguration(), fs, rootDir); SnapshotTestingUtils.SnapshotMock.SnapshotBuilder builder = snapshotMock.createSnapshotV2( - SNAPSHOT_NAME_STR, TABLE_NAME_STR); + SNAPSHOT_NAME_STR, TABLE_NAME_STR); builder.addRegionV2(); builder.missOneRegionSnapshotFile(); - long period = Long.MAX_VALUE; + long period = Long.MAX_VALUE; SnapshotFileCache cache = new SnapshotFileCache(fs, rootDir, period, 10000000, - "test-snapshot-file-cache-refresh", new SnapshotFiles()); - cache.getSnapshotsInProgress(); + "test-snapshot-file-cache-refresh", new SnapshotFiles()); + cache.getSnapshotsInProgress(null); assertFalse(fs.exists(builder.getSnapshotsDir())); } }