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

svn commit: r1364203 [3/3] - in /hbase/trunk: hbase-common/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/ hbase-server/src/main/java/org/apache/hadoop/hbase/backup/ hbase-server/src/main/java/org/apache/hadoo...

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java?rev=1364203&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestHFileCleaner.java Sun Jul 22 01:11:36 2012
@@ -0,0 +1,145 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.cleaner;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.Server;
+import org.apache.hadoop.hbase.ServerName;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.util.HFileArchiveUtil;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(SmallTests.class)
+public class TestHFileCleaner {
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  @Test
+  public void testHFileCleaning() throws Exception{
+    String prefix = "someHFileThatWouldBeAUUID";
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // set TTL
+    long ttl = 2000;
+    conf.setLong(TimeToLiveHFileCleaner.TTL_CONF_KEY, ttl);
+    Server server = new DummyServer();
+    Path archivedHfileDir = new Path(TEST_UTIL.getDataTestDir(),
+        HFileArchiveUtil.getConfiguredArchiveDirName(conf));
+    FileSystem fs = FileSystem.get(conf);
+    HFileCleaner cleaner = new HFileCleaner(1000, server, conf, fs, archivedHfileDir);
+
+    // Create 2 invalid files, 1 "recent" file, 1 very new file and 30 old files
+    long now = System.currentTimeMillis();
+    fs.delete(archivedHfileDir, true);
+    fs.mkdirs(archivedHfileDir);
+    // Case 1: 1 invalid file, which would be deleted directly
+    fs.createNewFile(new Path(archivedHfileDir, "dfd-dfd"));
+    // Case 2: 1 "recent" file, not even deletable for the first log cleaner
+    // (TimeToLiveLogCleaner), so we are not going down the chain
+    System.out.println("Now is: " + now);
+    for (int i = 1; i < 32; i++) {
+      // Case 3: old files which would be deletable for the first log cleaner
+      // (TimeToLiveHFileCleaner),
+      Path fileName = new Path(archivedHfileDir, (prefix + "." + (now - i)));
+      fs.createNewFile(fileName);
+    }
+
+    // sleep for sometime to get newer modifcation time
+    Thread.sleep(ttl);
+
+    // Case 2: 1 newer file, not even deletable for the first log cleaner
+    // (TimeToLiveLogCleaner), so we are not going down the chain
+    fs.createNewFile(new Path(archivedHfileDir, prefix + "." + (now + 10000)));
+
+    for (FileStatus stat : fs.listStatus(archivedHfileDir)) {
+      System.out.println(stat.getPath().toString());
+    }
+
+    assertEquals(33, fs.listStatus(archivedHfileDir).length);
+
+    cleaner.chore();
+
+    // We end up a small number - just the one newer one
+    assertEquals(1, fs.listStatus(archivedHfileDir).length);
+
+    for (FileStatus file : fs.listStatus(archivedHfileDir)) {
+      System.out.println("Kept log files: " + file.getPath().getName());
+    }
+
+    cleaner.interrupt();
+  }
+
+  static class DummyServer implements Server {
+
+    @Override
+    public Configuration getConfiguration() {
+      return TEST_UTIL.getConfiguration();
+    }
+
+    @Override
+    public ZooKeeperWatcher getZooKeeper() {
+      try {
+        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      return null;
+    }
+
+    @Override
+    public CatalogTracker getCatalogTracker() {
+      return null;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return new ServerName("regionserver,60020,000000");
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {}
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public void stop(String why) {}
+
+    @Override
+    public boolean isStopped() {
+      return false;
+    }
+  }
+
+  @org.junit.Rule
+  public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
+    new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
+}
+

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java?rev=1364203&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/master/cleaner/TestLogsCleaner.java Sun Jul 22 01:11:36 2012
@@ -0,0 +1,180 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.master.cleaner;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import java.net.URLEncoder;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.*;
+import org.apache.hadoop.hbase.catalog.CatalogTracker;
+import org.apache.hadoop.hbase.master.cleaner.LogCleaner;
+import org.apache.hadoop.hbase.replication.ReplicationZookeeper;
+import org.apache.hadoop.hbase.replication.regionserver.Replication;
+import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category(MediumTests.class)
+public class TestLogsCleaner {
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    TEST_UTIL.startMiniZKCluster();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @AfterClass
+  public static void tearDownAfterClass() throws Exception {
+    TEST_UTIL.shutdownMiniZKCluster();
+  }
+
+  @Test
+  public void testLogCleaning() throws Exception{
+    Configuration conf = TEST_UTIL.getConfiguration();
+    // set TTL
+    long ttl = 2000;
+    conf.setLong("hbase.master.logcleaner.ttl", ttl);
+    conf.setBoolean(HConstants.REPLICATION_ENABLE_KEY, true);
+    Replication.decorateMasterConfiguration(conf);
+    Server server = new DummyServer();
+    ReplicationZookeeper zkHelper =
+        new ReplicationZookeeper(server, new AtomicBoolean(true));
+
+    Path oldLogDir = new Path(TEST_UTIL.getDataTestDir(),
+        HConstants.HREGION_OLDLOGDIR_NAME);
+    String fakeMachineName =
+      URLEncoder.encode(server.getServerName().toString(), "UTF8");
+
+    FileSystem fs = FileSystem.get(conf);
+    LogCleaner cleaner  = new LogCleaner(1000, server, conf, fs, oldLogDir);
+
+    // Create 2 invalid files, 1 "recent" file, 1 very new file and 30 old files
+    long now = System.currentTimeMillis();
+    fs.delete(oldLogDir, true);
+    fs.mkdirs(oldLogDir);
+    // Case 1: 2 invalid files, which would be deleted directly
+    fs.createNewFile(new Path(oldLogDir, "a"));
+    fs.createNewFile(new Path(oldLogDir, fakeMachineName + "." + "a"));
+    // Case 2: 1 "recent" file, not even deletable for the first log cleaner
+    // (TimeToLiveLogCleaner), so we are not going down the chain
+    System.out.println("Now is: " + now);
+    for (int i = 1; i < 31; i++) {
+      // Case 3: old files which would be deletable for the first log cleaner
+      // (TimeToLiveLogCleaner), and also for the second (ReplicationLogCleaner)
+      Path fileName = new Path(oldLogDir, fakeMachineName + "." + (now - i) );
+      fs.createNewFile(fileName);
+      // Case 4: put 3 old log files in ZK indicating that they are scheduled
+      // for replication so these files would pass the first log cleaner
+      // (TimeToLiveLogCleaner) but would be rejected by the second
+      // (ReplicationLogCleaner)
+      if (i % (30/3) == 1) {
+        zkHelper.addLogToList(fileName.getName(), fakeMachineName);
+        System.out.println("Replication log file: " + fileName);
+      }
+    }
+
+    // sleep for sometime to get newer modifcation time
+    Thread.sleep(ttl);
+    fs.createNewFile(new Path(oldLogDir, fakeMachineName + "." + now));
+
+    // Case 2: 1 newer file, not even deletable for the first log cleaner
+    // (TimeToLiveLogCleaner), so we are not going down the chain
+    fs.createNewFile(new Path(oldLogDir, fakeMachineName + "." + (now + 10000) ));
+
+    for (FileStatus stat : fs.listStatus(oldLogDir)) {
+      System.out.println(stat.getPath().toString());
+    }
+
+    assertEquals(34, fs.listStatus(oldLogDir).length);
+
+    cleaner.chore();
+
+    // We end up with the current log file, a newer one and the 3 old log
+    // files which are scheduled for replication
+    assertEquals(5, fs.listStatus(oldLogDir).length);
+
+    for (FileStatus file : fs.listStatus(oldLogDir)) {
+      System.out.println("Kept log files: " + file.getPath().getName());
+    }
+  }
+
+  static class DummyServer implements Server {
+
+    @Override
+    public Configuration getConfiguration() {
+      return TEST_UTIL.getConfiguration();
+    }
+
+    @Override
+    public ZooKeeperWatcher getZooKeeper() {
+      try {
+        return new ZooKeeperWatcher(getConfiguration(), "dummy server", this);
+      } catch (IOException e) {
+        e.printStackTrace();
+      }
+      return null;
+    }
+
+    @Override
+    public CatalogTracker getCatalogTracker() {
+      return null;
+    }
+
+    @Override
+    public ServerName getServerName() {
+      return new ServerName("regionserver,60020,000000");
+    }
+
+    @Override
+    public void abort(String why, Throwable e) {}
+
+    @Override
+    public boolean isAborted() {
+      return false;
+    }
+
+    @Override
+    public void stop(String why) {}
+
+    @Override
+    public boolean isStopped() {
+      return false;
+    }
+  }
+
+  @org.junit.Rule
+  public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
+    new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
+}
+

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CheckedArchivingHFileCleaner.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CheckedArchivingHFileCleaner.java?rev=1364203&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CheckedArchivingHFileCleaner.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/CheckedArchivingHFileCleaner.java Sun Jul 22 01:11:36 2012
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.regionserver;
+
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.master.cleaner.BaseHFileCleanerDelegate;
+
+/**
+ * HFile archive cleaner that just tells you if it has been run already or not (and allows resets) -
+ * always attempts to delete the passed file.
+ * <p>
+ * Just a helper class for testing to make sure the cleaner has been run.
+ */
+public class CheckedArchivingHFileCleaner extends BaseHFileCleanerDelegate {
+
+  private static boolean checked;
+
+  @Override
+  public boolean isFileDeleteable(Path file) {
+    checked = true;
+    return true;
+  }
+
+  public static boolean getChecked() {
+    return checked;
+  }
+
+  public static void resetCheck() {
+    checked = false;
+  }
+}

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java?rev=1364203&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/HFileArchiveTestingUtil.java Sun Jul 22 01:11:36 2012
@@ -0,0 +1,239 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+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.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.apache.hadoop.hbase.regionserver.Store;
+
+/**
+ * Test helper for testing archiving of HFiles
+ */
+public class HFileArchiveTestingUtil {
+
+  private static final Log LOG = LogFactory.getLog(HFileArchiveTestingUtil.class);
+
+  private HFileArchiveTestingUtil() {
+    // NOOP private ctor since this is just a utility class
+  }
+
+  public static boolean compareArchiveToOriginal(FileStatus[] previous, FileStatus[] archived,
+      FileSystem fs, boolean hasTimedBackup) {
+
+    List<List<String>> lists = getFileLists(previous, archived);
+    List<String> original = lists.get(0);
+    Collections.sort(original);
+
+    List<String> currentFiles = lists.get(1);
+    Collections.sort(currentFiles);
+
+    List<String> backedup = lists.get(2);
+    Collections.sort(backedup);
+
+    // check the backed up files versus the current (should match up, less the
+    // backup time in the name)
+    if (!hasTimedBackup == (backedup.size() > 0)) {
+      LOG.debug("backedup files doesn't match expected.");
+      return false;
+    }
+    String msg = null;
+    if (hasTimedBackup) {
+      msg = assertArchiveEquality(original, backedup);
+      if (msg != null) {
+        LOG.debug(msg);
+        return false;
+      }
+    }
+    msg = assertArchiveEquality(original, currentFiles);
+    if (msg != null) {
+      LOG.debug(msg);
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * Compare the archived files to the files in the original directory
+   * @param previous original files that should have been archived
+   * @param archived files that were archived
+   * @param fs filessystem on which the archiving took place
+   * @throws IOException
+   */
+  public static void assertArchiveEqualToOriginal(FileStatus[] previous, FileStatus[] archived,
+      FileSystem fs) throws IOException {
+    assertArchiveEqualToOriginal(previous, archived, fs, false);
+  }
+
+  /**
+   * Compare the archived files to the files in the original directory
+   * @param previous original files that should have been archived
+   * @param archived files that were archived
+   * @param fs {@link FileSystem} on which the archiving took place
+   * @param hasTimedBackup <tt>true</tt> if we expect to find an archive backup directory with a
+   *          copy of the files in the archive directory (and the original files).
+   * @throws IOException
+   */
+  public static void assertArchiveEqualToOriginal(FileStatus[] previous, FileStatus[] archived,
+      FileSystem fs, boolean hasTimedBackup) throws IOException {
+
+    List<List<String>> lists = getFileLists(previous, archived);
+    List<String> original = lists.get(0);
+    Collections.sort(original);
+
+    List<String> currentFiles = lists.get(1);
+    Collections.sort(currentFiles);
+
+    List<String> backedup = lists.get(2);
+    Collections.sort(backedup);
+
+    // check the backed up files versus the current (should match up, less the
+    // backup time in the name)
+    assertEquals("Didn't expect any backup files, but got: " + backedup, hasTimedBackup,
+      backedup.size() > 0);
+    String msg = null;
+    if (hasTimedBackup) {
+      assertArchiveEquality(original, backedup);
+      assertNull(msg, msg);
+    }
+
+    // do the rest of the comparison
+    msg = assertArchiveEquality(original, currentFiles);
+    assertNull(msg, msg);
+  }
+
+  private static String assertArchiveEquality(List<String> expected, List<String> archived) {
+    String compare = compareFileLists(expected, archived);
+    if (!(expected.size() == archived.size())) return "Not the same number of current files\n"
+        + compare;
+    if (!expected.equals(archived)) return "Different backup files, but same amount\n" + compare;
+    return null;
+  }
+
+  /**
+   * @return <expected, gotten, backup>, where each is sorted
+   */
+  private static List<List<String>> getFileLists(FileStatus[] previous, FileStatus[] archived) {
+    List<List<String>> files = new ArrayList<List<String>>();
+
+    // copy over the original files
+    List<String> originalFileNames = convertToString(previous);
+    files.add(originalFileNames);
+
+    List<String> currentFiles = new ArrayList<String>(previous.length);
+    List<FileStatus> backedupFiles = new ArrayList<FileStatus>(previous.length);
+    for (FileStatus f : archived) {
+      String name = f.getPath().getName();
+      // if the file has been backed up
+      if (name.contains(".")) {
+        Path parent = f.getPath().getParent();
+        String shortName = name.split("[.]")[0];
+        Path modPath = new Path(parent, shortName);
+        FileStatus file = new FileStatus(f.getLen(), f.isDir(), f.getReplication(),
+            f.getBlockSize(), f.getModificationTime(), modPath);
+        backedupFiles.add(file);
+      } else {
+        // otherwise, add it to the list to compare to the original store files
+        currentFiles.add(name);
+      }
+    }
+
+    files.add(currentFiles);
+    files.add(convertToString(backedupFiles));
+    return files;
+  }
+
+  private static List<String> convertToString(FileStatus[] files) {
+    return convertToString(Arrays.asList(files));
+  }
+
+  private static List<String> convertToString(List<FileStatus> files) {
+    List<String> originalFileNames = new ArrayList<String>(files.size());
+    for (FileStatus f : files) {
+      originalFileNames.add(f.getPath().getName());
+    }
+    return originalFileNames;
+  }
+
+  /* Get a pretty representation of the differences */
+  private static String compareFileLists(List<String> expected, List<String> gotten) {
+    StringBuilder sb = new StringBuilder("Expected (" + expected.size() + "): \t\t Gotten ("
+        + gotten.size() + "):\n");
+    List<String> notFound = new ArrayList<String>();
+    for (String s : expected) {
+      if (gotten.contains(s)) sb.append(s + "\t\t" + s + "\n");
+      else notFound.add(s);
+    }
+    sb.append("Not Found:\n");
+    for (String s : notFound) {
+      sb.append(s + "\n");
+    }
+    sb.append("\nExtra:\n");
+    for (String s : gotten) {
+      if (!expected.contains(s)) sb.append(s + "\n");
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Helper method to get the archive directory for the specified region
+   * @param conf {@link Configuration} to check for the name of the archive directory
+   * @param region region that is being archived
+   * @return {@link Path} to the archive directory for the given region
+   */
+  public static Path getRegionArchiveDir(Configuration conf, HRegion region) {
+    return HFileArchiveUtil.getRegionArchiveDir(conf, region.getTableDir(), region.getRegionDir());
+  }
+
+  /**
+   * Helper method to get the store archive directory for the specified region
+   * @param conf {@link Configuration} to check for the name of the archive directory
+   * @param region region that is being archived
+   * @param store store that is archiving files
+   * @return {@link Path} to the store archive directory for the given region
+   */
+  public static Path getStoreArchivePath(Configuration conf, HRegion region, Store store) {
+    return HFileArchiveUtil.getStoreArchivePath(conf, region, store.getFamily().getName());
+  }
+
+  public static Path getStoreArchivePath(HBaseTestingUtility util, String tableName,
+      byte[] storeName) throws IOException {
+    byte[] table = Bytes.toBytes(tableName);
+    // get the RS and region serving our table
+    List<HRegion> servingRegions = util.getHBaseCluster().getRegions(table);
+    HRegion region = servingRegions.get(0);
+
+    // check that we actually have some store files that were archived
+    Store store = region.getStore(storeName);
+    return HFileArchiveTestingUtil.getStoreArchivePath(util.getConfiguration(), region, store);
+  }
+}

Added: hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
URL: http://svn.apache.org/viewvc/hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java?rev=1364203&view=auto
==============================================================================
--- hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java (added)
+++ hbase/trunk/hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java Sun Jul 22 01:11:36 2012
@@ -0,0 +1,102 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.*;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.regionserver.HRegion;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test that the utility works as expected
+ */
+@Category(SmallTests.class)
+public class TestHFileArchiveUtil {
+
+  @Test
+  public void testGetConfiguredArchiveDir() {
+    assertEquals(HFileArchiveUtil.DEFAULT_HFILE_ARCHIVE_DIRECTORY,
+      HFileArchiveUtil.getConfiguredArchiveDirName(null));
+    Configuration conf = new Configuration();
+    assertEquals(HFileArchiveUtil.DEFAULT_HFILE_ARCHIVE_DIRECTORY,
+      HFileArchiveUtil.getConfiguredArchiveDirName(conf));
+    conf.set(HConstants.HFILE_ARCHIVE_DIRECTORY, "");
+    assertEquals("", HFileArchiveUtil.getConfiguredArchiveDirName(conf));
+    String archiveDir = "somearchive";
+    conf.set(HConstants.HFILE_ARCHIVE_DIRECTORY, archiveDir);
+    assertEquals(archiveDir, HFileArchiveUtil.getConfiguredArchiveDirName(conf));
+  }
+
+  @Test
+  public void testGetTableArchivePath() {
+    assertNotNull(HFileArchiveUtil.getTableArchivePath(null, new Path("table")));
+    Configuration conf = new Configuration();
+    conf.set(HConstants.HFILE_ARCHIVE_DIRECTORY, "");
+    assertNotNull(HFileArchiveUtil.getTableArchivePath(conf, new Path("root", new Path("table"))));
+  }
+
+  @Test
+  public void testGetArchivePath() throws Exception {
+    Configuration conf = new Configuration();
+    FSUtils.setRootDir(conf, new Path("root"));
+    assertNotNull(HFileArchiveUtil.getArchivePath(conf));
+    String archiveDir = "somearchive";
+    conf.set(HConstants.HFILE_ARCHIVE_DIRECTORY, archiveDir);
+    assertEquals(new Path(FSUtils.getRootDir(conf), archiveDir),
+      HFileArchiveUtil.getArchivePath(conf));
+  }
+  
+  @Test
+  public void testRegionArchiveDir() {
+    Path tableDir = new Path("table");
+    Path regionDir = new Path("region");
+    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(null, tableDir, regionDir));
+  }
+  
+  @Test
+  public void testGetStoreArchivePath(){
+      byte[] family = Bytes.toBytes("Family");
+    Path tabledir = new Path("table");
+    HRegionInfo region = new HRegionInfo(Bytes.toBytes("table"));
+    Configuration conf = null;
+    assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, region, tabledir, family));
+    conf = new Configuration();
+    assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, region, tabledir, family));
+    conf.set(HConstants.HFILE_ARCHIVE_DIRECTORY, "archiveDir");
+    assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, region, tabledir, family));
+
+    // do a little mocking of a region to get the same results
+    HRegion mockRegion = Mockito.mock(HRegion.class);
+    Mockito.when(mockRegion.getRegionInfo()).thenReturn(region);
+    Mockito.when(mockRegion.getTableDir()).thenReturn(tabledir);
+
+    assertNotNull(HFileArchiveUtil.getStoreArchivePath(null, mockRegion, family));
+    conf = new Configuration();
+    assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, mockRegion, family));
+    conf.set(HConstants.HFILE_ARCHIVE_DIRECTORY, "archiveDir");
+    assertNotNull(HFileArchiveUtil.getStoreArchivePath(conf, mockRegion, family));
+
+  }
+}