Selaa lähdekoodia

Merge trunk into HA branch.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1245117 13f79535-47bb-0310-9956-ffa450edef68
Aaron Myers 13 vuotta sitten
vanhempi
commit
833e96534f

+ 6 - 0
hadoop-common-project/hadoop-common/CHANGES.txt

@@ -190,6 +190,12 @@ Release 0.23.2 - UNRELEASED
     Double.MAX_VALUE) to avoid making Ganglia's gmetad core. (Varun Kapoor
     Double.MAX_VALUE) to avoid making Ganglia's gmetad core. (Varun Kapoor
     via mattf)
     via mattf)
 
 
+    HADOOP-8074. Small bug in hadoop error message for unknown commands.
+    (Colin Patrick McCabe via eli)
+
+    HADOOP-8082 add hadoop-client and hadoop-minicluster to the 
+    dependency-management section. (tucu)
+
 Release 0.23.1 - 2012-02-08 
 Release 0.23.1 - 2012-02-08 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 9 - 1
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java

@@ -269,7 +269,15 @@ public class FsShell extends Configured implements Tool {
   
   
   private void displayError(String cmd, String message) {
   private void displayError(String cmd, String message) {
     for (String line : message.split("\n")) {
     for (String line : message.split("\n")) {
-      System.err.println(cmd.substring(1) + ": " + line);
+      System.err.println(cmd + ": " + line);
+      if (cmd.charAt(0) != '-') {
+        Command instance = null;
+        instance = commandFactory.getInstance("-" + cmd);
+        if (instance != null) {
+          System.err.println("Did you mean -" + cmd + "?  This command " +
+              "begins with a dash.");
+        }
+      }
     }
     }
   }
   }
   
   

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

@@ -251,6 +251,9 @@ Release 0.23.2 - UNRELEASED
     HDFS-2950. Secondary NN HTTPS address should be listed as a
     HDFS-2950. Secondary NN HTTPS address should be listed as a
     NAMESERVICE_SPECIFIC_KEY. (todd)
     NAMESERVICE_SPECIFIC_KEY. (todd)
 
 
+    HDFS-2938. Recursive delete of a large directory make namenode
+    unresponsive. (Hari Mankude via suresh)
+
 Release 0.23.1 - 2012-02-08 
 Release 0.23.1 - 2012-02-08 
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 14 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2387,15 +2387,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
     }
     }
-
-    getEditLog().logSync();
-
-    writeLock();
-    try {
-      removeBlocks(collectedBlocks); // Incremental deletion of blocks
-    } finally {
-      writeUnlock();
-    }
+    getEditLog().logSync(); 
+    removeBlocks(collectedBlocks); // Incremental deletion of blocks
     collectedBlocks.clear();
     collectedBlocks.clear();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
       NameNode.stateChangeLog.debug("DIR* Namesystem.delete: "
@@ -2404,16 +2397,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return true;
     return true;
   }
   }
 
 
-  /** From the given list, incrementally remove the blocks from blockManager */
+  /** 
+   * From the given list, incrementally remove the blocks from blockManager
+   * Writelock is dropped and reacquired every BLOCK_DELETION_INCREMENT to
+   * ensure that other waiters on the lock can get in. See HDFS-2938
+   */
   private void removeBlocks(List<Block> blocks) {
   private void removeBlocks(List<Block> blocks) {
-    assert hasWriteLock();
     int start = 0;
     int start = 0;
     int end = 0;
     int end = 0;
     while (start < blocks.size()) {
     while (start < blocks.size()) {
       end = BLOCK_DELETION_INCREMENT + start;
       end = BLOCK_DELETION_INCREMENT + start;
       end = end > blocks.size() ? blocks.size() : end;
       end = end > blocks.size() ? blocks.size() : end;
-      for (int i=start; i<end; i++) {
-        blockManager.removeBlock(blocks.get(i));
+      writeLock();
+      try {
+        for (int i = start; i < end; i++) {
+          blockManager.removeBlock(blocks.get(i));
+        }
+      } finally {
+        writeUnlock();
       }
       }
       start = end;
       start = end;
     }
     }

+ 4 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestLargeDirectoryDelete.java

@@ -117,8 +117,11 @@ public class TestLargeDirectoryDelete {
           try {
           try {
             int blockcount = getBlockCount();
             int blockcount = getBlockCount();
             if (blockcount < TOTAL_BLOCKS && blockcount > 0) {
             if (blockcount < TOTAL_BLOCKS && blockcount > 0) {
-              synchronized(mc.getNamesystem()) {
+              mc.getNamesystem().writeLock();
+              try {
                 lockOps++;
                 lockOps++;
+              } finally {
+                mc.getNamesystem().writeUnlock();
               }
               }
               Thread.sleep(1);
               Thread.sleep(1);
             }
             }

+ 6 - 0
hadoop-mapreduce-project/CHANGES.txt

@@ -94,6 +94,9 @@ Release 0.23.2 - UNRELEASED
 
 
   IMPROVEMENTS
   IMPROVEMENTS
 
 
+    MAPREDUCE-3849. Change TokenCache's reading of the binary token file
+    (Daryn Sharp via bobby)
+
     MAPREDUCE-3854. Fixed and reenabled tests related to MR child JVM's
     MAPREDUCE-3854. Fixed and reenabled tests related to MR child JVM's
     environmental variables in TestMiniMRChildTask. (Tom White via vinodkv)
     environmental variables in TestMiniMRChildTask. (Tom White via vinodkv)
 
 
@@ -109,6 +112,9 @@ Release 0.23.2 - UNRELEASED
 
 
     MAPREDUCE-3736. Variable substitution depth too large for fs.default.name 
     MAPREDUCE-3736. Variable substitution depth too large for fs.default.name 
     causes jobs to fail (ahmed via tucu).
     causes jobs to fail (ahmed via tucu).
+
+    MAPREDUCE-3864. Fix cluster setup docs for correct SecondaryNameNode
+    HTTPS parameters. (todd)
  
  
 Release 0.23.1 - 2012-02-08 
 Release 0.23.1 - 2012-02-08 
 
 

+ 17 - 22
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/security/TokenCache.java

@@ -114,31 +114,10 @@ public class TokenCache {
       throw new IOException(
       throw new IOException(
           "Can't get Master Kerberos principal for use as renewer");
           "Can't get Master Kerberos principal for use as renewer");
     }
     }
-    boolean readFile = true;
+    mergeBinaryTokens(credentials, conf);
 
 
     String fsName = fs.getCanonicalServiceName();
     String fsName = fs.getCanonicalServiceName();
     if (TokenCache.getDelegationToken(credentials, fsName) == null) {
     if (TokenCache.getDelegationToken(credentials, fsName) == null) {
-      //TODO: Need to come up with a better place to put
-      //this block of code to do with reading the file
-      if (readFile) {
-        readFile = false;
-        String binaryTokenFilename =
-          conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
-        if (binaryTokenFilename != null) {
-          Credentials binary;
-          try {
-            binary = Credentials.readTokenStorageFile(
-                new Path("file:///" +  binaryTokenFilename), conf);
-          } catch (IOException e) {
-            throw new RuntimeException(e);
-          }
-          credentials.addAll(binary);
-        }
-        if (TokenCache.getDelegationToken(credentials, fsName) != null) {
-          LOG.debug("DT for " + fsName  + " is already present");
-          return;
-        }
-      }
       List<Token<?>> tokens =
       List<Token<?>> tokens =
           fs.getDelegationTokens(delegTokenRenewer, credentials);
           fs.getDelegationTokens(delegTokenRenewer, credentials);
       if (tokens != null) {
       if (tokens != null) {
@@ -161,6 +140,22 @@ public class TokenCache {
     }
     }
   }
   }
 
 
+  private static void mergeBinaryTokens(Credentials creds, Configuration conf) {
+    String binaryTokenFilename =
+        conf.get(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY);
+    if (binaryTokenFilename != null) {
+      Credentials binary;
+      try {
+        binary = Credentials.readTokenStorageFile(
+            new Path("file:///" +  binaryTokenFilename), conf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      // supplement existing tokens with the tokens in the binary file
+      creds.mergeAll(binary);
+    }
+  }
+  
   /**
   /**
    * file name used on HDFS for generated job token
    * file name used on HDFS for generated job token
    */
    */

+ 89 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/security/TestTokenCache.java

@@ -21,23 +21,27 @@ package org.apache.hadoop.mapreduce.security;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 import static org.mockito.Mockito.when;
 
 
+import java.io.IOException;
 import java.net.URI;
 import java.net.URI;
 import java.util.LinkedList;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.List;
 
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.Master;
 import org.apache.hadoop.mapred.Master;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.junit.Test;
 import org.junit.Test;
 import org.mockito.invocation.InvocationOnMock;
 import org.mockito.invocation.InvocationOnMock;
@@ -162,6 +166,91 @@ public class TestTokenCache {
     return mockFs;
     return mockFs;
   }
   }
 
 
+  @Test
+  @SuppressWarnings("deprecation")
+  public void testBinaryCredentials() throws Exception {
+    Configuration conf = new Configuration();
+    conf.set(YarnConfiguration.RM_PRINCIPAL, "mapred/host@REALM");
+    String renewer = Master.getMasterPrincipal(conf);
+
+    Path TEST_ROOT_DIR =
+        new Path(System.getProperty("test.build.data","test/build/data"));
+    // ick, but need fq path minus file:/
+    String binaryTokenFile = FileSystem.getLocal(conf).makeQualified(
+        new Path(TEST_ROOT_DIR, "tokenFile")).toUri().getPath();
+
+    FileSystem fs1 = createFileSystemForService("service1");
+    FileSystem fs2 = createFileSystemForService("service2");
+    FileSystem fs3 = createFileSystemForService("service3");
+    
+    // get the tokens for fs1 & fs2 and write out to binary creds file
+    Credentials creds = new Credentials();
+    Token<?> token1 = fs1.getDelegationToken(renewer);
+    Token<?> token2 = fs2.getDelegationToken(renewer);
+    creds.addToken(token1.getService(), token1);
+    creds.addToken(token2.getService(), token2);
+    // wait to set, else the obtain tokens call above will fail with FNF
+    conf.set(MRJobConfig.MAPREDUCE_JOB_CREDENTIALS_BINARY, binaryTokenFile);
+    creds.writeTokenStorageFile(new Path(binaryTokenFile), conf);
+    
+    // re-init creds and add a newer token for fs1
+    creds = new Credentials();
+    Token<?> newerToken1 = fs1.getDelegationToken(renewer);
+    assertFalse(newerToken1.equals(token1));
+    creds.addToken(newerToken1.getService(), newerToken1);
+    checkToken(creds, newerToken1);
+    
+    // get token for fs1, see that fs2's token was loaded 
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
+    checkToken(creds, newerToken1, token2);
+    
+    // get token for fs2, nothing should change since already present
+    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf);
+    checkToken(creds, newerToken1, token2);
+    
+    // get token for fs3, should only add token for fs3
+    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf);
+    Token<?> token3 = creds.getToken(new Text(fs3.getCanonicalServiceName()));
+    assertTrue(token3 != null);
+    checkToken(creds, newerToken1, token2, token3);
+    
+    // be paranoid, check one last time that nothing changes
+    TokenCache.obtainTokensForNamenodesInternal(fs1, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs2, creds, conf);
+    TokenCache.obtainTokensForNamenodesInternal(fs3, creds, conf);
+    checkToken(creds, newerToken1, token2, token3);
+  }
+
+  private void checkToken(Credentials creds, Token<?> ... tokens) {
+    assertEquals(tokens.length, creds.getAllTokens().size());
+    for (Token<?> token : tokens) {
+      Token<?> credsToken = creds.getToken(token.getService());
+      assertTrue(credsToken != null);
+      assertEquals(token, credsToken);
+    }
+  }
+  
+  @SuppressWarnings("deprecation")
+  private FileSystem createFileSystemForService(final String service)
+      throws IOException {
+    FileSystem mockFs = mock(FileSystem.class);
+    when(mockFs.getCanonicalServiceName()).thenReturn(service);
+    when(mockFs.getDelegationToken(any(String.class))).thenAnswer(
+        new Answer<Token<?>>() {
+          int unique = 0;
+          @Override
+          public Token<?> answer(InvocationOnMock invocation) throws Throwable {
+            Token<?> token = new Token<TokenIdentifier>();
+            token.setService(new Text(service));
+            // use unique value so when we restore from token storage, we can
+            // tell if it's really the same token
+            token.setKind(new Text("token" + unique++));
+            return token;
+          }
+        });
+    return mockFs;
+  }
+
   @Test
   @Test
   public void testCleanUpTokenReferral() throws Exception {
   public void testCleanUpTokenReferral() throws Exception {
     Configuration conf = new Configuration();
     Configuration conf = new Configuration();

+ 2 - 2
hadoop-mapreduce-project/hadoop-yarn/hadoop-yarn-site/src/site/apt/ClusterSetup.apt.vm

@@ -738,9 +738,9 @@ KVNO Timestamp         Principal
 *-------------------------+-------------------------+------------------------+
 *-------------------------+-------------------------+------------------------+
 || Parameter              || Value                  || Notes                 |
 || Parameter              || Value                  || Notes                 |
 *-------------------------+-------------------------+------------------------+
 *-------------------------+-------------------------+------------------------+
-| <<<dfs.namenode.secondary.https-address>>> | <c_nn_host_fqdn:50090> | |
+| <<<dfs.namenode.secondary.http-address>>> | <c_nn_host_fqdn:50090> | |
 *-------------------------+-------------------------+------------------------+
 *-------------------------+-------------------------+------------------------+
-| <<<dfs.secondary.https.port>>> | <50090> | |
+| <<<dfs.namenode.secondary.https-port>>> | <50470> | |
 *-------------------------+-------------------------+------------------------+
 *-------------------------+-------------------------+------------------------+
 | <<<dfs.namenode.secondary.keytab.file>>> | | | 
 | <<<dfs.namenode.secondary.keytab.file>>> | | | 
 | | </etc/security/keytab/sn.service.keytab> | |
 | | </etc/security/keytab/sn.service.keytab> | |

+ 11 - 0
hadoop-project/pom.xml

@@ -239,6 +239,17 @@
         <version>${project.version}</version>
         <version>${project.version}</version>
       </dependency>
       </dependency>
 
 
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-client</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>org.apache.hadoop</groupId>
+        <artifactId>hadoop-minicluster</artifactId>
+        <version>${project.version}</version>
+      </dependency>
+
       <dependency>
       <dependency>
         <groupId>com.google.guava</groupId>
         <groupId>com.google.guava</groupId>
         <artifactId>guava</artifactId>
         <artifactId>guava</artifactId>