You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by mb...@apache.org on 2013/03/04 12:24:53 UTC

svn commit: r1452257 [14/14] - in /hbase/branches/0.94: security/src/main/java/org/apache/hadoop/hbase/security/access/ security/src/test/java/org/apache/hadoop/hbase/security/access/ src/main/jamon/org/apache/hadoop/hbase/tmpl/master/ src/main/java/or...

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java?rev=1452257&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotLogSplitter.java Mon Mar  4 11:24:50 2013
@@ -0,0 +1,176 @@
+/**
+ * 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.snapshot;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.TreeMap;
+
+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.HConstants;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
+import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.*;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test snapshot log splitter
+ */
+@Category(SmallTests.class)
+public class TestSnapshotLogSplitter {
+  final Log LOG = LogFactory.getLog(getClass());
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private byte[] TEST_QUALIFIER = Bytes.toBytes("q");
+  private byte[] TEST_FAMILY = Bytes.toBytes("f");
+
+  private Configuration conf;
+  private FileSystem fs;
+  private Path logFile;
+
+  @Before
+  public void setup() throws Exception {
+    conf = TEST_UTIL.getConfiguration();
+    fs = FileSystem.get(conf);
+    logFile = new Path(TEST_UTIL.getDataTestDir(), "test.log");
+    writeTestLog(logFile);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fs.delete(logFile, false);
+  }
+
+  @Test
+  public void testSplitLogs() throws IOException {
+    Map<byte[], byte[]> regionsMap = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
+    splitTestLogs(getTableName(5), regionsMap);
+  }
+
+  @Test
+  public void testSplitLogsOnDifferentTable() throws IOException {
+    byte[] tableName = getTableName(1);
+    Map<byte[], byte[]> regionsMap = new TreeMap<byte[], byte[]>(Bytes.BYTES_COMPARATOR);
+    for (int j = 0; j < 10; ++j) {
+      byte[] regionName = getRegionName(tableName, j);
+      byte[] newRegionName = getNewRegionName(tableName, j);
+      regionsMap.put(regionName, newRegionName);
+    }
+    splitTestLogs(tableName, regionsMap);
+  }
+
+  /*
+   * Split and verify test logs for the specified table
+   */
+  private void splitTestLogs(final byte[] tableName, final Map<byte[], byte[]> regionsMap)
+      throws IOException {
+    Path tableDir = new Path(TEST_UTIL.getDataTestDir(), Bytes.toString(tableName));
+    SnapshotLogSplitter logSplitter = new SnapshotLogSplitter(conf, fs, tableDir,
+      tableName, regionsMap);
+    try {
+      logSplitter.splitLog(logFile);
+    } finally {
+      logSplitter.close();
+    }
+    verifyRecoverEdits(tableDir, tableName, regionsMap);
+  }
+
+  /*
+   * Verify that every logs in the table directory has just the specified table and regions.
+   */
+  private void verifyRecoverEdits(final Path tableDir, final byte[] tableName,
+      final Map<byte[], byte[]> regionsMap) throws IOException {
+    for (FileStatus regionStatus: FSUtils.listStatus(fs, tableDir)) {
+      assertTrue(regionStatus.getPath().getName().startsWith(Bytes.toString(tableName)));
+      Path regionEdits = HLog.getRegionDirRecoveredEditsDir(regionStatus.getPath());
+      byte[] regionName = Bytes.toBytes(regionStatus.getPath().getName());
+      assertFalse(regionsMap.containsKey(regionName));
+      for (FileStatus logStatus: FSUtils.listStatus(fs, regionEdits)) {
+        HLog.Reader reader = HLog.getReader(fs, logStatus.getPath(), conf);
+        try {
+          HLog.Entry entry;
+          while ((entry = reader.next()) != null) {
+            HLogKey key = entry.getKey();
+            assertArrayEquals(tableName, key.getTablename());
+            assertArrayEquals(regionName, key.getEncodedRegionName());
+          }
+        } finally {
+          reader.close();
+        }
+      }
+    }
+  }
+
+  /*
+   * Write some entries in the log file.
+   * 7 different tables with name "testtb-%d"
+   * 10 region per table with name "tableName-region-%d"
+   * 50 entry with row key "row-%d"
+   */
+  private void writeTestLog(final Path logFile) throws IOException {
+    fs.mkdirs(logFile.getParent());
+    HLog.Writer writer = HLog.createWriter(fs, logFile, conf);
+    try {
+      for (int i = 0; i < 7; ++i) {
+        byte[] tableName = getTableName(i);
+        for (int j = 0; j < 10; ++j) {
+          byte[] regionName = getRegionName(tableName, j);
+          for (int k = 0; k < 50; ++k) {
+            byte[] rowkey = Bytes.toBytes("row-" + k);
+            HLogKey key = new HLogKey(regionName, tableName, (long)k,
+              System.currentTimeMillis(), HConstants.DEFAULT_CLUSTER_ID);
+            WALEdit edit = new WALEdit();
+            edit.add(new KeyValue(rowkey, TEST_FAMILY, TEST_QUALIFIER, rowkey));
+            writer.append(new HLog.Entry(key, edit));
+          }
+        }
+      }
+    } finally {
+      writer.close();
+    }
+  }
+
+  private byte[] getTableName(int tableId) {
+    return Bytes.toBytes("testtb-" + tableId);
+  }
+
+  private byte[] getRegionName(final byte[] tableName, int regionId) {
+    return Bytes.toBytes(Bytes.toString(tableName) + "-region-" + regionId);
+  }
+
+  private byte[] getNewRegionName(final byte[] tableName, int regionId) {
+    return Bytes.toBytes(Bytes.toString(tableName) + "-new-region-" + regionId);
+  }
+}

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java?rev=1452257&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestSnapshotTask.java Mon Mar  4 11:24:50 2013
@@ -0,0 +1,58 @@
+/**
+ * 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.snapshot;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.errorhandling.ForeignException;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.SnapshotTask;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+@Category(SmallTests.class)
+public class TestSnapshotTask {
+
+  /**
+   * Check that errors from running the task get propagated back to the error listener.
+   */
+  @Test
+  public void testErrorPropagation() throws Exception {
+    ForeignExceptionDispatcher error = mock(ForeignExceptionDispatcher.class);
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder().setName("snapshot")
+        .setTable("table").build();
+    final Exception thrown = new Exception("Failed!");
+    SnapshotTask fail = new SnapshotTask(snapshot, error) {
+      @Override
+      public Void call() {
+        snapshotFailure("Injected failure", thrown);
+        return null;
+      }
+    };
+    fail.call();
+
+    verify(error, Mockito.times(1)).receive(any(ForeignException.class));
+  }
+
+}

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestWALReferenceTask.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestWALReferenceTask.java?rev=1452257&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestWALReferenceTask.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/snapshot/TestWALReferenceTask.java Mon Mar  4 11:24:50 2013
@@ -0,0 +1,103 @@
+/**
+ * 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.snapshot;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.SmallTests;
+import org.apache.hadoop.hbase.errorhandling.ForeignExceptionDispatcher;
+import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
+import org.apache.hadoop.hbase.snapshot.ReferenceServerWALsTask;
+import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
+import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test that the WAL reference task works as expected
+ */
+@Category(SmallTests.class)
+public class TestWALReferenceTask {
+
+  private static final Log LOG = LogFactory.getLog(TestWALReferenceTask.class);
+  private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
+
+  @Test
+  public void testRun() throws IOException {
+    Configuration conf = UTIL.getConfiguration();
+    FileSystem fs = UTIL.getTestFileSystem();
+    // setup the log dir
+    Path testDir = UTIL.getDataTestDir();
+    Set<String> servers = new HashSet<String>();
+    Path logDir = new Path(testDir, ".logs");
+    Path server1Dir = new Path(logDir, "Server1");
+    servers.add(server1Dir.getName());
+    Path server2Dir = new Path(logDir, "me.hbase.com,56073,1348618509968");
+    servers.add(server2Dir.getName());
+    // logs under server 1
+    Path log1_1 = new Path(server1Dir, "me.hbase.com%2C56073%2C1348618509968.1348618520536");
+    Path log1_2 = new Path(server1Dir, "me.hbase.com%2C56073%2C1348618509968.1234567890123");
+    // logs under server 2
+    Path log2_1 = new Path(server2Dir, "me.hbase.com%2C56074%2C1348618509998.1348618515589");
+    Path log2_2 = new Path(server2Dir, "me.hbase.com%2C56073%2C1348618509968.1234567890123");
+
+    // create all the log files
+    fs.createNewFile(log1_1);
+    fs.createNewFile(log1_2);
+    fs.createNewFile(log2_1);
+    fs.createNewFile(log2_2);
+
+    FSUtils.logFileSystemState(fs, testDir, LOG);
+    FSUtils.setRootDir(conf, testDir);
+    SnapshotDescription snapshot = SnapshotDescription.newBuilder()
+        .setName("testWALReferenceSnapshot").build();
+    ForeignExceptionDispatcher listener = Mockito.mock(ForeignExceptionDispatcher.class);
+
+    // reference all the files in the first server directory
+    ReferenceServerWALsTask task = new ReferenceServerWALsTask(snapshot, listener, server1Dir,
+        conf, fs);
+    task.call();
+
+    // reference all the files in the first server directory
+    task = new ReferenceServerWALsTask(snapshot, listener, server2Dir, conf, fs);
+    task.call();
+
+    // verify that we got everything
+    FSUtils.logFileSystemState(fs, testDir, LOG);
+    Path workingDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(snapshot, testDir);
+    Path snapshotLogDir = new Path(workingDir, HConstants.HREGION_LOGDIR_NAME);
+
+    // make sure we reference the all the wal files
+    TakeSnapshotUtils.verifyAllLogsGotReferenced(fs, logDir, servers, snapshot, snapshotLogDir);
+
+    // make sure we never got an error
+    Mockito.verify(listener, Mockito.atLeastOnce()).rethrowException();
+    Mockito.verifyNoMoreInteractions(listener);
+  }
+}
\ No newline at end of file

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java?rev=1452257&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSVisitor.java Mon Mar  4 11:24:50 2013
@@ -0,0 +1,225 @@
+/**
+ *
+ * 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.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.UUID;
+import java.util.Set;
+import java.util.HashSet;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HDFSBlocksDistribution;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hbase.regionserver.wal.HLog;
+import org.apache.hadoop.hbase.util.MD5Hash;
+import org.apache.hadoop.hbase.util.FSUtils;
+import org.junit.*;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Test {@link FSUtils}.
+ */
+@Category(MediumTests.class)
+public class TestFSVisitor {
+  final Log LOG = LogFactory.getLog(getClass());
+
+  private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
+
+  private final String TABLE_NAME = "testtb";
+
+  private Set<String> tableFamilies;
+  private Set<String> tableRegions;
+  private Set<String> recoveredEdits;
+  private Set<String> tableHFiles;
+  private Set<String> regionServers;
+  private Set<String> serverLogs;
+
+  private FileSystem fs;
+  private Path tableDir;
+  private Path logsDir;
+  private Path rootDir;
+
+  @Before
+  public void setUp() throws Exception {
+    fs = FileSystem.get(TEST_UTIL.getConfiguration());
+    rootDir = TEST_UTIL.getDataTestDir("hbase");
+    logsDir = new Path(rootDir, HConstants.HREGION_LOGDIR_NAME);
+
+    tableFamilies = new HashSet<String>();
+    tableRegions = new HashSet<String>();
+    recoveredEdits = new HashSet<String>();
+    tableHFiles = new HashSet<String>();
+    regionServers = new HashSet<String>();
+    serverLogs = new HashSet<String>();
+    tableDir = createTableFiles(rootDir, TABLE_NAME, tableRegions, tableFamilies, tableHFiles);
+    createRecoverEdits(tableDir, tableRegions, recoveredEdits);
+    createLogs(logsDir, regionServers, serverLogs);
+    FSUtils.logFileSystemState(fs, rootDir, LOG);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    fs.delete(rootDir);
+  }
+
+  @Test
+  public void testVisitStoreFiles() throws IOException {
+    final Set<String> regions = new HashSet<String>();
+    final Set<String> families = new HashSet<String>();
+    final Set<String> hfiles = new HashSet<String>();
+    FSVisitor.visitTableStoreFiles(fs, tableDir, new FSVisitor.StoreFileVisitor() {
+      public void storeFile(final String region, final String family, final String hfileName)
+          throws IOException {
+        regions.add(region);
+        families.add(family);
+        hfiles.add(hfileName);
+      }
+    });
+    assertEquals(tableRegions, regions);
+    assertEquals(tableFamilies, families);
+    assertEquals(tableHFiles, hfiles);
+  }
+
+  @Test
+  public void testVisitRecoveredEdits() throws IOException {
+    final Set<String> regions = new HashSet<String>();
+    final Set<String> edits = new HashSet<String>();
+    FSVisitor.visitTableRecoveredEdits(fs, tableDir, new FSVisitor.RecoveredEditsVisitor() {
+      public void recoveredEdits (final String region, final String logfile)
+          throws IOException {
+        regions.add(region);
+        edits.add(logfile);
+      }
+    });
+    assertEquals(tableRegions, regions);
+    assertEquals(recoveredEdits, edits);
+  }
+
+  @Test
+  public void testVisitLogFiles() throws IOException {
+    final Set<String> servers = new HashSet<String>();
+    final Set<String> logs = new HashSet<String>();
+    FSVisitor.visitLogFiles(fs, rootDir, new FSVisitor.LogFileVisitor() {
+      public void logFile (final String server, final String logfile) throws IOException {
+        servers.add(server);
+        logs.add(logfile);
+      }
+    });
+    assertEquals(regionServers, servers);
+    assertEquals(serverLogs, logs);
+  }
+
+
+  /*
+   * |-testtb/
+   * |----f1d3ff8443297732862df21dc4e57262/
+   * |-------f1/
+   * |----------d0be84935ba84b66b1e866752ec5d663
+   * |----------9fc9d481718f4878b29aad0a597ecb94
+   * |-------f2/
+   * |----------4b0fe6068c564737946bcf4fd4ab8ae1
+   */
+  private Path createTableFiles(final Path rootDir, final String tableName,
+      final Set<String> tableRegions, final Set<String> tableFamilies,
+      final Set<String> tableHFiles) throws IOException {
+    Path tableDir = new Path(rootDir, tableName);
+    for (int r = 0; r < 10; ++r) {
+      String regionName = MD5Hash.getMD5AsHex(Bytes.toBytes(r));
+      tableRegions.add(regionName);
+      Path regionDir = new Path(tableDir, regionName);
+      for (int f = 0; f < 3; ++f) {
+        String familyName = "f" + f;
+        tableFamilies.add(familyName);
+        Path familyDir = new Path(regionDir, familyName);
+        fs.mkdirs(familyDir);
+        for (int h = 0; h < 5; ++h) {
+         String hfileName = UUID.randomUUID().toString().replaceAll("-", "");
+         tableHFiles.add(hfileName);
+         fs.createNewFile(new Path(familyDir, hfileName));
+        }
+      }
+    }
+    return tableDir;
+  }
+
+  /*
+   * |-testtb/
+   * |----f1d3ff8443297732862df21dc4e57262/
+   * |-------recovered.edits/
+   * |----------0000001351969633479
+   * |----------0000001351969633481
+   */
+  private void createRecoverEdits(final Path tableDir, final Set<String> tableRegions,
+      final Set<String> recoverEdits) throws IOException {
+    for (String region: tableRegions) {
+      Path regionEditsDir = HLog.getRegionDirRecoveredEditsDir(new Path(tableDir, region));
+      long seqId = System.currentTimeMillis();
+      for (int i = 0; i < 3; ++i) {
+        String editName = String.format("%019d", seqId + i);
+        recoverEdits.add(editName);
+        FSDataOutputStream stream = fs.create(new Path(regionEditsDir, editName));
+        stream.write(Bytes.toBytes("test"));
+        stream.close();
+      }
+    }
+  }
+
+  /*
+   * |-.logs/
+   * |----server5,5,1351969633508/
+   * |-------server5,5,1351969633508.0
+   * |----server6,6,1351969633512/
+   * |-------server6,6,1351969633512.0
+   * |-------server6,6,1351969633512.3
+   */
+  private void createLogs(final Path logDir, final Set<String> servers,
+      final Set<String> logs) throws IOException {
+    for (int s = 0; s < 7; ++s) {
+      String server = String.format("server%d,%d,%d", s, s, System.currentTimeMillis());
+      servers.add(server);
+      Path serverLogDir = new Path(logDir, server);
+      fs.mkdirs(serverLogDir);
+      for (int i = 0; i < 5; ++i) {
+        String logfile = server + '.' + i;
+        logs.add(logfile);
+        FSDataOutputStream stream = fs.create(new Path(serverLogDir, logfile));
+        stream.write(Bytes.toBytes("test"));
+        stream.close();
+      }
+    }
+  }
+}

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java?rev=1452257&r1=1452256&r2=1452257&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestHFileArchiveUtil.java Mon Mar  4 11:24:50 2013
@@ -50,9 +50,10 @@ public class TestHFileArchiveUtil {
   
   @Test
   public void testRegionArchiveDir() {
+    Configuration conf = null;
     Path tableDir = new Path("table");
     Path regionDir = new Path("region");
-    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(null, tableDir, regionDir));
+    assertNotNull(HFileArchiveUtil.getRegionArchiveDir(conf, tableDir, regionDir));
   }
   
   @Test