Browse Source

HDFS-5695. Merge change r1555164 from trunk.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1555166 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 years ago
parent
commit
1d6d0b823f

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -184,6 +184,9 @@ Release 2.4.0 - UNRELEASED
     HDFS-5540. Fix intermittent failure in TestBlocksWithNotEnoughRacks.
     (Binglin Chang via junping_du)
 
+    HDFS-5695. Clean up TestOfflineEditsViewer and OfflineEditsViewerHelper.
+    (Haohui Mai via jing9)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

+ 3 - 129
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java

@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import java.io.File;
 import java.io.IOException;
-import java.security.PrivilegedExceptionAction;
 import java.util.Iterator;
 
 import org.apache.commons.logging.Log;
@@ -29,23 +28,13 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.Options.Rename;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.DFSClientAdapter;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.common.Util;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.Token;
 
 /**
  * OfflineEditsViewerHelper is a helper class for TestOfflineEditsViewer,
@@ -133,126 +122,11 @@ public class OfflineEditsViewerHelper {
    * OP_CLEAR_NS_QUOTA  (12)
    */
   private CheckpointSignature runOperations() throws IOException {
-
     LOG.info("Creating edits by performing fs operations");
     // no check, if it's not it throws an exception which is what we want
-    DistributedFileSystem dfs =
-      (DistributedFileSystem)cluster.getFileSystem();
-    FileContext fc = FileContext.getFileContext(cluster.getURI(0), config);
-    // OP_ADD 0
-    Path pathFileCreate = new Path("/file_create_u\1F431");
-    FSDataOutputStream s = dfs.create(pathFileCreate);
-    // OP_CLOSE 9
-    s.close();
-    // OP_RENAME_OLD 1
-    Path pathFileMoved = new Path("/file_moved");
-    dfs.rename(pathFileCreate, pathFileMoved);
-    // OP_DELETE 2
-    dfs.delete(pathFileMoved, false);
-    // OP_MKDIR 3
-    Path pathDirectoryMkdir = new Path("/directory_mkdir");
-    dfs.mkdirs(pathDirectoryMkdir);
-    // OP_ALLOW_SNAPSHOT 29
-    dfs.allowSnapshot(pathDirectoryMkdir);
-    // OP_DISALLOW_SNAPSHOT 30
-    dfs.disallowSnapshot(pathDirectoryMkdir);
-    // OP_CREATE_SNAPSHOT 26
-    String ssName = "snapshot1";
-    dfs.allowSnapshot(pathDirectoryMkdir);
-    dfs.createSnapshot(pathDirectoryMkdir, ssName);
-    // OP_RENAME_SNAPSHOT 28
-    String ssNewName = "snapshot2";
-    dfs.renameSnapshot(pathDirectoryMkdir, ssName, ssNewName);
-    // OP_DELETE_SNAPSHOT 27
-    dfs.deleteSnapshot(pathDirectoryMkdir, ssNewName);
-    // OP_SET_REPLICATION 4
-    s = dfs.create(pathFileCreate);
-    s.close();
-    dfs.setReplication(pathFileCreate, (short)1);
-    // OP_SET_PERMISSIONS 7
-    Short permission = 0777;
-    dfs.setPermission(pathFileCreate, new FsPermission(permission));
-    // OP_SET_OWNER 8
-    dfs.setOwner(pathFileCreate, new String("newOwner"), null);
-    // OP_CLOSE 9 see above
-    // OP_SET_GENSTAMP 10 see above
-    // OP_SET_NS_QUOTA 11 obsolete
-    // OP_CLEAR_NS_QUOTA 12 obsolete
-    // OP_TIMES 13
-    long mtime = 1285195527000L; // Wed, 22 Sep 2010 22:45:27 GMT
-    long atime = mtime;
-    dfs.setTimes(pathFileCreate, mtime, atime);
-    // OP_SET_QUOTA 14
-    dfs.setQuota(pathDirectoryMkdir, 1000L, HdfsConstants.QUOTA_DONT_SET);
-    // OP_RENAME 15
-    fc.rename(pathFileCreate, pathFileMoved, Rename.NONE);
-    // OP_CONCAT_DELETE 16
-    Path   pathConcatTarget = new Path("/file_concat_target");
-    Path[] pathConcatFiles  = new Path[2];
-    pathConcatFiles[0]      = new Path("/file_concat_0");
-    pathConcatFiles[1]      = new Path("/file_concat_1");
-
-    long  length      = blockSize * 3; // multiple of blocksize for concat
-    short replication = 1;
-    long  seed        = 1;
-
-    DFSTestUtil.createFile(dfs, pathConcatTarget, length, replication, seed);
-    DFSTestUtil.createFile(dfs, pathConcatFiles[0], length, replication, seed);
-    DFSTestUtil.createFile(dfs, pathConcatFiles[1], length, replication, seed);
-    dfs.concat(pathConcatTarget, pathConcatFiles);
-    // OP_SYMLINK 17
-    Path pathSymlink = new Path("/file_symlink");
-    fc.createSymlink(pathConcatTarget, pathSymlink, false);
-    // OP_GET_DELEGATION_TOKEN 18
-    // OP_RENEW_DELEGATION_TOKEN 19
-    // OP_CANCEL_DELEGATION_TOKEN 20
-    // see TestDelegationToken.java
-    // fake the user to renew token for
-    final Token<?>[] tokens = dfs.addDelegationTokens("JobTracker", null);
-    UserGroupInformation longUgi = UserGroupInformation.createRemoteUser(
-      "JobTracker/foo.com@FOO.COM");
-    try {
-      longUgi.doAs(new PrivilegedExceptionAction<Object>() {
-        @Override
-        public Object run() throws IOException, InterruptedException {
-          for (Token<?> token : tokens) {
-            token.renew(config);
-            token.cancel(config);
-          }
-          return null;
-        }
-      });
-    } catch(InterruptedException e) {
-      throw new IOException(
-        "renewDelegationToken threw InterruptedException", e);
-    }
-    // OP_UPDATE_MASTER_KEY 21
-    //   done by getDelegationTokenSecretManager().startThreads();
-
-    // sync to disk, otherwise we parse partial edits
-    cluster.getNameNode().getFSImage().getEditLog().logSync();
-    
-    // OP_REASSIGN_LEASE 22
-    String filePath = "/hard-lease-recovery-test";
-    byte[] bytes = "foo-bar-baz".getBytes();
-    DFSClientAdapter.stopLeaseRenewer(dfs);
-    FSDataOutputStream leaseRecoveryPath = dfs.create(new Path(filePath));
-    leaseRecoveryPath.write(bytes);
-    leaseRecoveryPath.hflush();
-    // Set the hard lease timeout to 1 second.
-    cluster.setLeasePeriod(60 * 1000, 1000);
-    // wait for lease recovery to complete
-    LocatedBlocks locatedBlocks;
-    do {
-      try {
-        Thread.sleep(1000);
-      } catch (InterruptedException e) {
-        LOG.info("Innocuous exception", e);
-      }
-      locatedBlocks = DFSClientAdapter.callGetBlockLocations(
-          cluster.getNameNodeRpc(), filePath, 0L, bytes.length);
-    } while (locatedBlocks.isUnderConstruction());
-
+    DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
+    DFSTestUtil.runOperations(cluster, dfs, cluster.getConfiguration(0),
+        dfs.getDefaultBlockSize(), 0);
     // Force a roll so we get an OP_END_LOG_SEGMENT txn
     return cluster.getNameNodeRpc().rollEditLog();
   }

+ 93 - 124
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineEditsViewer/TestOfflineEditsViewer.java

@@ -26,8 +26,6 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
-import java.util.HashMap;
-import java.util.Map;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -36,168 +34,142 @@ import org.apache.hadoop.hdfs.server.namenode.FSEditLogOpCodes;
 import org.apache.hadoop.hdfs.server.namenode.OfflineEditsViewerHelper;
 import org.apache.hadoop.hdfs.tools.offlineEditsViewer.OfflineEditsViewer.Flags;
 import org.apache.hadoop.test.PathUtils;
+import org.junit.After;
 import org.junit.Before;
+import org.junit.Rule;
 import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
 
-public class TestOfflineEditsViewer {
-  private static final Log LOG = LogFactory.getLog(TestOfflineEditsViewer.class);
-
-  private static final Map<FSEditLogOpCodes, Boolean> obsoleteOpCodes =
-    new HashMap<FSEditLogOpCodes, Boolean>();
-
-  private static final Map<FSEditLogOpCodes, Boolean> missingOpCodes =
-      new HashMap<FSEditLogOpCodes, Boolean>();
+import com.google.common.collect.ImmutableSet;
 
-  static {
-    initializeObsoleteOpCodes();
-    initializeMissingOpCodes();
-  }
-
-  private static String buildDir =
-    PathUtils.getTestDirName(TestOfflineEditsViewer.class);
+public class TestOfflineEditsViewer {
+  private static final Log LOG = LogFactory
+      .getLog(TestOfflineEditsViewer.class);
 
-  private static String cacheDir =
-    System.getProperty("test.cache.data", "build/test/cache");
+  private static String buildDir = PathUtils
+      .getTestDirName(TestOfflineEditsViewer.class);
 
   // to create edits and get edits filename
-  private static final OfflineEditsViewerHelper nnHelper 
-    = new OfflineEditsViewerHelper();
+  private static final OfflineEditsViewerHelper nnHelper = new OfflineEditsViewerHelper();
+  private static final ImmutableSet<FSEditLogOpCodes> skippedOps = skippedOps();
 
-  /**
-   * Initialize obsoleteOpCodes
-   *
-   * Reason for suppressing "deprecation" warnings:
-   *
-   * These are the opcodes that are not used anymore, some
-   * are marked deprecated, we need to include them here to make
-   * sure we exclude them when checking for completeness of testing,
-   * that's why the "deprecation" warnings are suppressed.
-   */
   @SuppressWarnings("deprecation")
-  private static void initializeObsoleteOpCodes() {
-    obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_ADD, true);
-    obsoleteOpCodes.put(FSEditLogOpCodes.OP_DATANODE_REMOVE, true);
-    obsoleteOpCodes.put(FSEditLogOpCodes.OP_SET_NS_QUOTA, true);
-    obsoleteOpCodes.put(FSEditLogOpCodes.OP_CLEAR_NS_QUOTA, true);
+  private static ImmutableSet<FSEditLogOpCodes> skippedOps() {
+    ImmutableSet.Builder<FSEditLogOpCodes> b = ImmutableSet
+        .<FSEditLogOpCodes> builder();
+
+    // Deprecated opcodes
+    b.add(FSEditLogOpCodes.OP_DATANODE_ADD)
+        .add(FSEditLogOpCodes.OP_DATANODE_REMOVE)
+        .add(FSEditLogOpCodes.OP_SET_NS_QUOTA)
+        .add(FSEditLogOpCodes.OP_CLEAR_NS_QUOTA)
+        .add(FSEditLogOpCodes.OP_SET_GENSTAMP_V1);
+
+    // Cannot test delegation token related code in insecure set up
+    b.add(FSEditLogOpCodes.OP_GET_DELEGATION_TOKEN)
+        .add(FSEditLogOpCodes.OP_RENEW_DELEGATION_TOKEN)
+        .add(FSEditLogOpCodes.OP_CANCEL_DELEGATION_TOKEN);
+
+    // Skip invalid opcode
+    b.add(FSEditLogOpCodes.OP_INVALID);
+    return b.build();
   }
 
-  /**
-   * Initialize missingOpcodes
-   *
-   *  Opcodes that are not available except after uprade from
-   *  an older version. We don't test these here.
-   */
-  private static void initializeMissingOpCodes() {
-    obsoleteOpCodes.put(FSEditLogOpCodes.OP_SET_GENSTAMP_V1, true);
-  }
+  @Rule
+  public final TemporaryFolder folder = new TemporaryFolder();
 
   @Before
-  public void setup() {
-    new File(cacheDir).mkdirs();
+  public void setUp() throws IOException {
+    nnHelper.startCluster(buildDir + "/dfs/");
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    nnHelper.shutdownCluster();
   }
-  
+
   /**
    * Test the OfflineEditsViewer
    */
   @Test
   public void testGenerated() throws IOException {
-
-    LOG.info("START - testing with generated edits");
-
-    nnHelper.startCluster(buildDir + "/dfs/");
-
     // edits generated by nnHelper (MiniDFSCluster), should have all op codes
     // binary, XML, reparsed binary
-    String edits          = nnHelper.generateEdits();
-    String editsParsedXml = cacheDir + "/editsParsed.xml";
-    String editsReparsed  = cacheDir + "/editsReparsed";
+    String edits = nnHelper.generateEdits();
+    String editsParsedXml = folder.newFile("editsParsed.xml").getAbsolutePath();
+    String editsReparsed = folder.newFile("editsParsed").getAbsolutePath();
 
     // parse to XML then back to binary
     assertEquals(0, runOev(edits, editsParsedXml, "xml", false));
     assertEquals(0, runOev(editsParsedXml, editsReparsed, "binary", false));
 
     // judgment time
+    assertTrue("Edits " + edits + " should have all op codes",
+        hasAllOpCodes(edits));
+    LOG.info("Comparing generated file " + editsReparsed
+        + " with reference file " + edits);
     assertTrue(
-      "Edits " + edits + " should have all op codes",
-      hasAllOpCodes(edits));
-    LOG.info("Comparing generated file " + editsReparsed +
-             " with reference file " + edits);
-    assertTrue(
-      "Generated edits and reparsed (bin to XML to bin) should be same",
-      filesEqualIgnoreTrailingZeros(edits, editsReparsed));
-
-    // removes edits so do this at the end
-    nnHelper.shutdownCluster();
-
-    LOG.info("END");
+        "Generated edits and reparsed (bin to XML to bin) should be same",
+        filesEqualIgnoreTrailingZeros(edits, editsReparsed));
   }
 
   @Test
   public void testRecoveryMode() throws IOException {
-    LOG.info("START - testing with generated edits");
-
-    nnHelper.startCluster(buildDir + "/dfs/");
-
     // edits generated by nnHelper (MiniDFSCluster), should have all op codes
     // binary, XML, reparsed binary
-    String edits          = nnHelper.generateEdits();
-    
+    String edits = nnHelper.generateEdits();
+    FileOutputStream os = new FileOutputStream(edits, true);
     // Corrupt the file by truncating the end
-    FileChannel editsFile = new FileOutputStream(edits, true).getChannel();
+    FileChannel editsFile = os.getChannel();
     editsFile.truncate(editsFile.size() - 5);
-    
-    String editsParsedXml = cacheDir + "/editsRecoveredParsed.xml";
-    String editsReparsed  = cacheDir + "/editsRecoveredReparsed";
-    String editsParsedXml2 = cacheDir + "/editsRecoveredParsed2.xml";
+
+    String editsParsedXml = folder.newFile("editsRecoveredParsed.xml")
+        .getAbsolutePath();
+    String editsReparsed = folder.newFile("editsRecoveredReparsed")
+        .getAbsolutePath();
+    String editsParsedXml2 = folder.newFile("editsRecoveredParsed2.xml")
+        .getAbsolutePath();
 
     // Can't read the corrupted file without recovery mode
     assertEquals(-1, runOev(edits, editsParsedXml, "xml", false));
-    
+
     // parse to XML then back to binary
     assertEquals(0, runOev(edits, editsParsedXml, "xml", true));
-    assertEquals(0, runOev(editsParsedXml, editsReparsed,  "binary", false));
+    assertEquals(0, runOev(editsParsedXml, editsReparsed, "binary", false));
     assertEquals(0, runOev(editsReparsed, editsParsedXml2, "xml", false));
 
     // judgment time
     assertTrue("Test round trip",
-      filesEqualIgnoreTrailingZeros(editsParsedXml, editsParsedXml2));
+        filesEqualIgnoreTrailingZeros(editsParsedXml, editsParsedXml2));
 
-    // removes edits so do this at the end
-    nnHelper.shutdownCluster();
-
-    LOG.info("END");
+    os.close();
   }
 
   @Test
   public void testStored() throws IOException {
-
-    LOG.info("START - testing with stored reference edits");
-
     // reference edits stored with source code (see build.xml)
+    final String cacheDir = System.getProperty("test.cache.data",
+        "build/test/cache");
     // binary, XML, reparsed binary
-    String editsStored             = cacheDir + "/editsStored";
-    String editsStoredParsedXml    = cacheDir + "/editsStoredParsed.xml";
-    String editsStoredReparsed     = cacheDir + "/editsStoredReparsed";
+    String editsStored = cacheDir + "/editsStored";
+    String editsStoredParsedXml = cacheDir + "/editsStoredParsed.xml";
+    String editsStoredReparsed = cacheDir + "/editsStoredReparsed";
     // reference XML version of editsStored (see build.xml)
-    String editsStoredXml          = cacheDir + "/editsStored.xml";
-      
+    String editsStoredXml = cacheDir + "/editsStored.xml";
+
     // parse to XML then back to binary
     assertEquals(0, runOev(editsStored, editsStoredParsedXml, "xml", false));
-    assertEquals(0, runOev(editsStoredParsedXml, editsStoredReparsed,
-        "binary", false));
+    assertEquals(0,
+        runOev(editsStoredParsedXml, editsStoredReparsed, "binary", false));
 
     // judgement time
+    assertTrue("Edits " + editsStored + " should have all op codes",
+        hasAllOpCodes(editsStored));
+    assertTrue("Reference XML edits and parsed to XML should be same",
+        filesEqual(editsStoredXml, editsStoredParsedXml));
     assertTrue(
-      "Edits " + editsStored + " should have all op codes",
-      hasAllOpCodes(editsStored));
-    assertTrue(
-      "Reference XML edits and parsed to XML should be same",
-      filesEqual(editsStoredXml, editsStoredParsedXml));
-    assertTrue(
-      "Reference edits and reparsed (bin to XML to bin) should be same",
-      filesEqualIgnoreTrailingZeros(editsStored, editsStoredReparsed));
-
-    LOG.info("END");
+        "Reference edits and reparsed (bin to XML to bin) should be same",
+        filesEqualIgnoreTrailingZeros(editsStored, editsStoredReparsed));
   }
 
   /**
@@ -233,22 +205,17 @@ public class TestOfflineEditsViewer {
     OfflineEditsViewer oev = new OfflineEditsViewer();
     if (oev.go(inFilename, outFilename, "stats", new Flags(), visitor) != 0)
       return false;
-    LOG.info("Statistics for " + inFilename + "\n" +
-      visitor.getStatisticsString());
-    
+    LOG.info("Statistics for " + inFilename + "\n"
+        + visitor.getStatisticsString());
+
     boolean hasAllOpCodes = true;
-    for(FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
+    for (FSEditLogOpCodes opCode : FSEditLogOpCodes.values()) {
       // don't need to test obsolete opCodes
-      if(obsoleteOpCodes.containsKey(opCode)) {
+      if (skippedOps.contains(opCode))
         continue;
-      } else if (missingOpCodes.containsKey(opCode)) {
-        continue;
-      } else if (opCode == FSEditLogOpCodes.OP_INVALID) {
-        continue;
-      }
 
       Long count = visitor.getStatistics().get(opCode);
-      if((count == null) || (count == 0)) {
+      if ((count == null) || (count == 0)) {
         hasAllOpCodes = false;
         LOG.info("Opcode " + opCode + " not tested in " + inFilename);
       }
@@ -257,9 +224,9 @@ public class TestOfflineEditsViewer {
   }
 
   /**
-   * Compare two files, ignore trailing zeros at the end,
-   * for edits log the trailing zeros do not make any difference,
-   * throw exception is the files are not same
+   * Compare two files, ignore trailing zeros at the end, for edits log the
+   * trailing zeros do not make any difference, throw exception is the files are
+   * not same
    *
    * @param filenameSmall first file to compare (doesn't have to be smaller)
    * @param filenameLarge second file to compare (doesn't have to be larger)
@@ -271,7 +238,7 @@ public class TestOfflineEditsViewer {
     ByteBuffer large = ByteBuffer.wrap(DFSTestUtil.loadFile(filenameLarge));
 
     // now correct if it's otherwise
-    if(small.capacity() > large.capacity()) {
+    if (small.capacity() > large.capacity()) {
       ByteBuffer tmpByteBuffer = small;
       small = large;
       large = tmpByteBuffer;
@@ -288,13 +255,15 @@ public class TestOfflineEditsViewer {
     large.limit(small.capacity());
 
     // compares position to limit
-    if(!small.equals(large)) { return false; }
+    if (!small.equals(large)) {
+      return false;
+    }
 
     // everything after limit should be 0xFF
     int i = large.limit();
     large.clear();
-    for(; i < large.capacity(); i++) {
-      if(large.get(i) != FSEditLogOpCodes.OP_INVALID.getOpCode()) {
+    for (; i < large.capacity(); i++) {
+      if (large.get(i) != FSEditLogOpCodes.OP_INVALID.getOpCode()) {
         return false;
       }
     }