You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2012/09/07 05:43:11 UTC

svn commit: r1381852 - in /hbase/trunk/hbase-server/src: main/java/org/apache/hadoop/hbase/backup/example/ main/java/org/apache/hadoop/hbase/master/cleaner/ test/java/org/apache/hadoop/hbase/backup/example/

Author: stack
Date: Fri Sep  7 03:43:11 2012
New Revision: 1381852

URL: http://svn.apache.org/viewvc?rev=1381852&view=rev
Log:
HBASE-6707 TEST org.apache.hadoop.hbase.backup.example.TestZooKeeperTableArchiveClient.testMultipleTables flaps

Modified:
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java
    hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
    hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java?rev=1381852&r1=1381851&r2=1381852&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/backup/example/LongTermArchivingHFileCleaner.java Fri Sep  7 03:43:11 2012
@@ -100,7 +100,10 @@ public class LongTermArchivingHFileClean
       LOG.info("Stopping " + this.archiveTracker);
       this.archiveTracker.stop();
     }
+  }
 
+  boolean isArchiving(String tableName) {
+    return this.archiveTracker.keepHFiles(tableName);
   }
 
 }

Modified: hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java?rev=1381852&r1=1381851&r2=1381852&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java (original)
+++ hbase/trunk/hbase-server/src/main/java/org/apache/hadoop/hbase/master/cleaner/CleanerChore.java Fri Sep  7 03:43:11 2012
@@ -235,4 +235,14 @@ public abstract class CleanerChore<T ext
       }
     }
   }
+
+  /**
+   * Get the cleaner chain.
+   * <p>
+   * Exposed for testing.
+   * @return the list of currently used cleaners
+   */
+  public List<T> getCleanerChain() {
+    return this.cleanersChain;
+  }
 }

Modified: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java?rev=1381852&r1=1381851&r2=1381852&view=diff
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java (original)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/backup/example/TestZooKeeperTableArchiveClient.java Fri Sep  7 03:43:11 2012
@@ -19,17 +19,15 @@ package org.apache.hadoop.hbase.backup.e
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.List;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
@@ -41,11 +39,12 @@ import org.apache.hadoop.hbase.regionser
 import org.apache.hadoop.hbase.regionserver.HRegion;
 import org.apache.hadoop.hbase.regionserver.HRegionServer;
 import org.apache.hadoop.hbase.regionserver.Store;
-import org.apache.hadoop.hbase.regionserver.HStore;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.util.HFileArchiveTestingUtil;
 import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -66,12 +65,15 @@ public class TestZooKeeperTableArchiveCl
   private static final byte[] TEST_FAM = Bytes.toBytes("fam");
   private static final byte[] TABLE_NAME = Bytes.toBytes(STRING_TABLE_NAME);
   private static final int numRS = 2;
-  private static final int maxTries = 5;
+  /** time in ms to let hfiles live in the archive */
   private static final long ttl = 1000;
+  /** ms to wait for the archiver to realize it needs to change archiving state */
+  private static final long WAIT_FOR_ZK_ARCHIVE_STATE_CHANGE = 100;
   private static ZKTableArchiveClient archivingClient;
 
   /**
    * Setup the config for the cluster
+   * @throws Exception on failure
    */
   @BeforeClass
   public static void setupCluster() throws Exception {
@@ -121,23 +123,19 @@ public class TestZooKeeperTableArchiveCl
 
   @AfterClass
   public static void cleanupTest() throws Exception {
-    try {
-      UTIL.shutdownMiniCluster();
-    } catch (Exception e) {
-      LOG.warn("problem shutting down cluster", e);
-    }
+    UTIL.shutdownMiniCluster();
   }
 
   /**
    * Test turning on/off archiving
+   * @throws Exception on failure
    */
   @Test
   public void testArchivingEnableDisable() throws Exception {
     // 1. turn on hfile backups
     LOG.debug("----Starting archiving");
     archivingClient.enableHFileBackupAsync(TABLE_NAME);
-    assertTrue("Archving didn't get turned on", archivingClient
-        .getArchivingEnabled(TABLE_NAME));
+    assertTrue("Archving didn't get turned on", archivingClient.getArchivingEnabled(TABLE_NAME));
 
     // 2. Turn off archiving and make sure its off
     archivingClient.disableHFileBackup();
@@ -145,8 +143,7 @@ public class TestZooKeeperTableArchiveCl
 
     // 3. Check enable/disable on a single table
     archivingClient.enableHFileBackupAsync(TABLE_NAME);
-    assertTrue("Archving didn't get turned on", archivingClient
-        .getArchivingEnabled(TABLE_NAME));
+    assertTrue("Archving didn't get turned on", archivingClient.getArchivingEnabled(TABLE_NAME));
 
     // 4. Turn off archiving and make sure its off
     archivingClient.disableHFileBackup(TABLE_NAME);
@@ -156,171 +153,127 @@ public class TestZooKeeperTableArchiveCl
 
   @Test
   public void testArchivingOnSingleTable() throws Exception {
-    // turn on hfile retention
-    LOG.debug("----Starting archiving");
-    archivingClient.enableHFileBackupAsync(TABLE_NAME);
-    assertTrue("Archving didn't get turned on", archivingClient
-        .getArchivingEnabled(TABLE_NAME));
-
-    // get the RS and region serving our table
-    List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(TABLE_NAME);
-    // make sure we only have 1 region serving this table
-    assertEquals(1, servingRegions.size());
-    HRegion region = servingRegions.get(0);
-
-    // get the parent RS and monitor
-    HRegionServer hrs = UTIL.getRSForFirstRegionInTable(TABLE_NAME);
-    FileSystem fs = hrs.getFileSystem();
-
-    // put some data on the region
-    LOG.debug("-------Loading table");
-    UTIL.loadRegion(region, TEST_FAM);
-    loadAndCompact(region);
+    // turn on archiving for our table
+    enableArchiving(STRING_TABLE_NAME);
 
-    // check that we actually have some store files that were archived
-    Store store = region.getStore(TEST_FAM);
-    Path storeArchiveDir = HFileArchiveTestingUtil.getStoreArchivePath(UTIL.getConfiguration(),
-      region, store);
+    FileSystem fs = FSUtils.getCurrentFileSystem(UTIL.getConfiguration());
 
-    // check to make sure we archived some files
-    assertTrue("Didn't create a store archive directory", fs.exists(storeArchiveDir));
-    assertTrue("No files in the store archive",
-      FSUtils.listStatus(fs, storeArchiveDir, null).length > 0);
+    // make sure there are files to archive
+    int archived = loadAndCompact(STRING_TABLE_NAME, TEST_FAM);
 
-    // and then put some non-tables files in the archive
-    Configuration conf = UTIL.getConfiguration();
-    Path archiveDir = HFileArchiveUtil.getArchivePath(conf);
-    // write a tmp file to the archive dir
-    Path tmpFile = new Path(archiveDir, "toDelete");
-    FSDataOutputStream out = fs.create(tmpFile);
-    out.write(1);
-    out.close();
-
-    assertTrue(fs.exists(tmpFile));
-    // make sure we wait long enough for the files to expire
-    Thread.sleep(ttl);
+    waitForFilesToExpire();
 
-    // print currrent state for comparison
-    FSUtils.logFileSystemState(fs, archiveDir, LOG);
+    ensureAllTableFilesinArchive(STRING_TABLE_NAME, TEST_FAM, fs, archived);
 
-    // ensure there are no archived files after waiting for a timeout
-    ensureHFileCleanersRun();
+    // turn off archiving
+    disableArchiving(STRING_TABLE_NAME);
 
-    // check to make sure the right things get deleted
-    assertTrue("Store archive got deleted", fs.exists(storeArchiveDir));
-    assertTrue("Archived HFiles got deleted",
-      FSUtils.listStatus(fs, storeArchiveDir, null).length > 0);
-
-    assertFalse(
-      "Tmp file (non-table archive file) didn't " + "get deleted, archive dir: "
-          + fs.listStatus(archiveDir), fs.exists(tmpFile));
-    LOG.debug("Turning off hfile backup.");
-    // stop archiving the table
-    archivingClient.disableHFileBackup();
-    LOG.debug("Deleting table from archive.");
-    // now remove the archived table
-    Path primaryTable = new Path(HFileArchiveUtil.getArchivePath(UTIL.getConfiguration()),
-        STRING_TABLE_NAME);
-    fs.delete(primaryTable, true);
-    LOG.debug("Deleted primary table, waiting for file cleaners to run");
-    // and make sure the archive directory is retained after a cleanup
-    // have to do this manually since delegates aren't run if there isn't any files in the archive
-    // dir to cleanup
-    Thread.sleep(ttl);
-    UTIL.getHBaseCluster().getMaster().getHFileCleaner().triggerNow();
-    Thread.sleep(ttl);
-    LOG.debug("File cleaners done, checking results.");
-    // but we still have the archive directory
+    // then ensure that those files are deleted after the timeout
+    waitForFilesToExpire();
+    ensureAllTableFilesinArchive(STRING_TABLE_NAME, TEST_FAM, fs, 0);
+    // but that we still have the archive directory around
     assertTrue(fs.exists(HFileArchiveUtil.getArchivePath(UTIL.getConfiguration())));
   }
 
   /**
-   * Make sure all the {@link HFileCleaner} run.
-   * <p>
-   * Blocking operation up to 3x ttl
-   * @throws InterruptedException
-   */
-  private void ensureHFileCleanersRun() throws InterruptedException {
-    LOG.debug("Waiting on archive cleaners to run...");
-    CheckedArchivingHFileCleaner.resetCheck();
-    do {
-      UTIL.getHBaseCluster().getMaster().getHFileCleaner().triggerNow();
-      LOG.debug("Triggered, sleeping an amount until we can pass the check.");
-      Thread.sleep(ttl);
-    } while (!CheckedArchivingHFileCleaner.getChecked());
-  }
-
-  /**
    * Test archiving/cleaning across multiple tables, where some are retained, and others aren't
    * @throws Exception
    */
   @Test
   public void testMultipleTables() throws Exception {
-    archivingClient.enableHFileBackupAsync(TABLE_NAME);
-    assertTrue("Archving didn't get turned on", archivingClient
-        .getArchivingEnabled(TABLE_NAME));
-
     // create the another table that we don't archive
     String otherTable = "otherTable";
     UTIL.createTable(Bytes.toBytes(otherTable), TEST_FAM);
 
-    // get the parent RS and monitor
-    FileSystem fs = FileSystem.get(UTIL.getConfiguration());
-
-    // put data in the filesystem of the first table
-    LOG.debug("Loading data into:" + STRING_TABLE_NAME);
-    loadAndCompact(STRING_TABLE_NAME);
+    // archive the primary table
+    enableArchiving(STRING_TABLE_NAME);
 
-    // and some data in the other table
-    LOG.debug("Loading data into:" + otherTable);
-    loadAndCompact(otherTable);
+    FileSystem fs = FSUtils.getCurrentFileSystem(UTIL.getConfiguration());
 
-    // make sure we wait long enough for the other table's files to expire
-    ensureHFileCleanersRun();
+    // make sure there are files to archive for both tables
+    int primaryArchived = loadAndCompact(STRING_TABLE_NAME, TEST_FAM);
+    loadAndCompact(otherTable, TEST_FAM);
 
-    // check to make sure the right things get deleted
-    Path primaryStoreArchive = HFileArchiveTestingUtil.getStoreArchivePath(UTIL, STRING_TABLE_NAME,
-      TEST_FAM);
     Path otherStoreArchive = HFileArchiveTestingUtil
         .getStoreArchivePath(UTIL, otherTable, TEST_FAM);
-    // make sure the primary store doesn't have any files
-    assertTrue("Store archive got deleted", fs.exists(primaryStoreArchive));
-    assertTrue("Archived HFiles got deleted",
-      FSUtils.listStatus(fs, primaryStoreArchive, null).length > 0);
-    FileStatus[] otherArchiveFiles = FSUtils.listStatus(fs, otherStoreArchive, null);
-    assertNull("Archived HFiles (" + otherStoreArchive
-        + ") should have gotten deleted, but didn't, remaining files:"
-        + getPaths(otherArchiveFiles), otherArchiveFiles);
-    // sleep again to make sure we the other table gets cleaned up
-    ensureHFileCleanersRun();
-    // first pass removes the store archive
-    assertFalse(fs.exists(otherStoreArchive));
-    // second pass removes the region
-    ensureHFileCleanersRun();
-    Path parent = otherStoreArchive.getParent();
-    assertFalse(fs.exists(parent));
-    // third pass remove the table
-    ensureHFileCleanersRun();
-    parent = otherStoreArchive.getParent();
-    assertFalse(fs.exists(parent));
-    // but we still have the archive directory
-    assertTrue(fs.exists(HFileArchiveUtil.getArchivePath(UTIL.getConfiguration())));
+    // make sure we archive the primary table
+    ensureAllTableFilesinArchive(STRING_TABLE_NAME, TEST_FAM, fs, primaryArchived);
+
+    waitForFilesToExpire();
 
-    FSUtils.logFileSystemState(fs, HFileArchiveUtil.getArchivePath(UTIL.getConfiguration()), LOG);
-    UTIL.deleteTable(Bytes.toBytes(otherTable));
+    // make sure that we didn't long-term archive the non-archive table
+    assertThat(0, new HasExactFiles(fs, otherStoreArchive));
+
+    // make sure we still archive the primary table
+    ensureAllTableFilesinArchive(STRING_TABLE_NAME, TEST_FAM, fs, primaryArchived);
   }
 
-  private List<Path> getPaths(FileStatus[] files) {
-    if (files == null || files.length == 0) return null;
+  /**
+   * Turn on hfile archiving and ensure its enabled
+   * @param tableName name of the table to enable
+   */
+  private void enableArchiving(String tableName) throws Exception {
+    LOG.debug("----Starting archiving on table:" + tableName);
+    archivingClient.enableHFileBackupAsync(Bytes.toBytes(tableName));
+    assertTrue("Archving didn't get turned on", archivingClient.getArchivingEnabled(tableName));
+    waitForCleanerToChangeState(false);
+  }
 
-    List<Path> paths = new ArrayList<Path>(files.length);
-    for (FileStatus file : files) {
-      paths.add(file.getPath());
+  /**
+   * Turn off hfile archiving and ensure its enabled
+   * @param tableName name of the table to enable
+   */
+  private void disableArchiving(String tableName) throws Exception {
+    LOG.debug("----Disable archiving on table:" + tableName);
+    archivingClient.disableHFileBackup(Bytes.toBytes(tableName));
+    assertFalse("Archving didn't get turned of", archivingClient.getArchivingEnabled(tableName));
+    waitForCleanerToChangeState(true);
+  }
+
+  private void waitForCleanerToChangeState(boolean wasArchiving) throws InterruptedException {
+    // get the cleaner from the master
+    LongTermArchivingHFileCleaner cleaner = (LongTermArchivingHFileCleaner) UTIL
+        .getMiniHBaseCluster().getMaster().getHFileCleaner().getCleanerChain().get(1);
+    // wait for it to switch state
+    while (cleaner.isArchiving(STRING_TABLE_NAME) == wasArchiving) {
+      Thread.sleep(WAIT_FOR_ZK_ARCHIVE_STATE_CHANGE);
     }
-    return paths;
   }
 
-  private void loadAndCompact(String tableName) throws Exception {
+  /**
+   * Wait for files in the archive to expire
+   */
+  private void waitForFilesToExpire() throws Exception {
+    // sleep long enough for archived files to expire
+    Thread.sleep(ttl + 10);
+
+    // make sure we clean the archive
+    ensureHFileCleanersRun();
+  }
+
+  /**
+   * Make sure all the {@link HFileCleaner} run.
+   * <p>
+   * Blocking operation up to 3x ttl
+   * @throws InterruptedException
+   */
+  private void ensureHFileCleanersRun() throws InterruptedException {
+    LOG.debug("Waiting on archive cleaners to run...");
+    CheckedArchivingHFileCleaner.resetCheck();
+    do {
+      UTIL.getHBaseCluster().getMaster().getHFileCleaner().triggerNow();
+      LOG.debug("Triggered, sleeping an amount until we can pass the check.");
+      Thread.sleep(ttl);
+    } while (!CheckedArchivingHFileCleaner.getChecked());
+  }
+
+  /**
+   * Load a table with a single region and compact the files.
+   * @param tableName name of the table to load
+   * @return the number of archived store files
+   * @throws Exception on failure
+   */
+  private int loadAndCompact(String tableName, byte[] family) throws Exception {
     byte[] table = Bytes.toBytes(tableName);
     // get the RS and region serving our table
     List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(table);
@@ -334,49 +287,114 @@ public class TestZooKeeperTableArchiveCl
 
     // put some data on the region
     LOG.debug("-------Loading table");
-    UTIL.loadRegion(region, TEST_FAM);
-    loadAndCompact(region);
+    // load the region with data
+    UTIL.loadRegion(region, family);
+    // wait for the flushes/compactions to finish
+    region.waitForFlushesAndCompactions();
+    // then trigger a compaction to be sure we have files in the archive
+    compactRegion(region, family);
 
     // check that we actually have some store files that were archived
-    Store store = region.getStore(TEST_FAM);
+    Store store = region.getStore(family);
     Path storeArchiveDir = HFileArchiveTestingUtil.getStoreArchivePath(UTIL.getConfiguration(),
       region, store);
 
     // check to make sure we archived some files
+    int storeFiles = FSUtils.listStatus(fs, storeArchiveDir, null).length;
     assertTrue("Didn't create a store archive directory", fs.exists(storeArchiveDir));
-    assertTrue("No files in the store archive",
-      FSUtils.listStatus(fs, storeArchiveDir, null).length > 0);
-
-    // wait for the compactions to finish
-    region.waitForFlushesAndCompactions();
+    assertThat(null, new HasAnyFiles(fs, storeArchiveDir));
+    return storeFiles;
   }
 
   /**
-   * Load the given region and then ensure that it compacts some files
+   * Compact all the store files in a given region.
    */
-  private void loadAndCompact(HRegion region) throws Exception {
-    int tries = 0;
-    Exception last = null;
-    while (tries++ <= maxTries) {
+  private void compactRegion(HRegion region, byte[] family) throws IOException {
+    Store store = region.getStores().get(family);
+    if (store.getStorefilesCount() <= 1) {
+      LOG.debug("Not compacting store with just one store file!");
+      return;
+    }
+    LOG.debug("Attemping to compact:" + store.getStorefilesCount());
+    assertTrue("Only one store file, need more for compaction!", store.getStorefilesCount() > 1);
+    store.compactRecentForTesting(store.getStorefilesCount());
+  }
+
+  private void ensureAllTableFilesinArchive(String tablename, byte[] family, FileSystem fs,
+      int expectedArchiveFiles) throws Exception {
+    List<HRegion> servingRegions = UTIL.getHBaseCluster().getRegions(Bytes.toBytes(tablename));
+    // make sure we only have 1 region serving this table
+    assertEquals("Test doesn't support more than 1 region serving table: " + tablename + "", 1,
+      servingRegions.size());
+    HRegion region = servingRegions.get(0);
+    Store store = region.getStore(TEST_FAM);
+    Path storeArchiveDir = HFileArchiveTestingUtil.getStoreArchivePath(UTIL.getConfiguration(),
+      region, store);
+
+    // log the current fs state for the archive dir
+    Path regionArchiveDir = HFileArchiveUtil.getArchivePath(UTIL.getConfiguration());
+    FSUtils.logFileSystemState(fs, regionArchiveDir, LOG);
+
+    // if we don't have store files, we shouldn't have the directory, but otherwise should have the
+    // store archive directory to store the archived files
+    assertEquals("Didn't create a store archive directory:" + storeArchiveDir,
+      expectedArchiveFiles != 0, fs.exists(storeArchiveDir));
+    assertThat(expectedArchiveFiles, new HasExactFiles(fs, storeArchiveDir));
+  }
+
+  private static class HasAnyFiles extends BaseMatcher<Object> {
+    protected FileSystem fs;
+    protected Path storeArchiveDir;
+
+    public HasAnyFiles(FileSystem fs, Path storeArchiveDir) {
+      this.fs = fs;
+      this.storeArchiveDir = storeArchiveDir;
+    }
+
+    @Override
+    public boolean matches(Object arg0) {
       try {
-        // load the region with data
-        UTIL.loadRegion(region, TEST_FAM);
-        // and then trigger a compaction to be sure we try to archive
-        compactRegion(region, TEST_FAM);
-        return;
-      } catch (Exception e) {
-        // keep this around for if we fail later
-        last = e;
+        return FSUtils.listStatus(fs, storeArchiveDir, null).length > 0;
+      } catch (IOException e) {
+        LOG.error("Failed to read the FS!", e);
+        return false;
       }
     }
-    throw last;
+
+    @Override
+    public void describeTo(Description desc) {
+      desc.appendText("No store files in archive");
+    }
   }
 
-  /**
-   * Compact all the store files in a given region.
-   */
-  private void compactRegion(HRegion region, byte[] family) throws IOException {
-    Store store = region.getStores().get(TEST_FAM);
-    store.compactRecentForTesting(store.getStorefiles().size());
+  private static class HasExactFiles extends BaseMatcher<Integer> {
+    protected FileSystem fs;
+    protected Path storeArchiveDir;
+
+    public HasExactFiles(FileSystem fs, Path storeArchiveDir) {
+      this.fs = fs;
+      this.storeArchiveDir = storeArchiveDir;
+    }
+
+    @Override
+    public boolean matches(Object arg0) {
+      try {
+        int expected = ((Integer) arg0).intValue();
+        FileStatus[] files = FSUtils.listStatus(fs, storeArchiveDir, null);
+        if (expected == 0 && files == null) {
+          LOG.debug("Directory '" + storeArchiveDir + "' doesn't exist, therefore 0 files!");
+          return true;
+        }
+        return expected == files.length;
+      } catch (IOException e) {
+        LOG.error("Failed to read the FS!", e);
+        return false;
+      }
+    }
+
+    @Override
+    public void describeTo(Description desc) {
+      desc.appendText("Store files in archive doesn't match expected");
+    }
   }
-}
+}
\ No newline at end of file