Browse Source

HDFS-6168. Remove a deprecated constructor and the deprecated methods reportChecksumFailure, getDelegationToken(Text), renewDelegationToken and cancelDelegationToken from DistributedFileSystem.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1582856 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 years ago
parent
commit
3f7aa79e58

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

@@ -249,6 +249,10 @@ Release 2.5.0 - UNRELEASED
 
   INCOMPATIBLE CHANGES
 
+    HDFS-6168. Remove a deprecated constructor and the deprecated methods reportChecksumFailure,
+    getDelegationToken(Text), renewDelegationToken and cancelDelegationToken from
+    DistributedFileSystem.  (szetszwo)
+
   NEW FEATURES
 
   IMPROVEMENTS

+ 2 - 110
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -124,12 +124,6 @@ public class DistributedFileSystem extends FileSystem {
     return HdfsConstants.HDFS_URI_SCHEME;
   }
 
-  @Deprecated
-  public DistributedFileSystem(InetSocketAddress namenode,
-    Configuration conf) throws IOException {
-    initialize(NameNode.getUri(namenode), conf);
-  }
-
   @Override
   public URI getUri() { return uri; }
 
@@ -1016,55 +1010,6 @@ public class DistributedFileSystem extends FileSystem {
     return dfs.getServerDefaults();
   }
 
-  /**
-   * We need to find the blocks that didn't match.  Likely only one 
-   * is corrupt but we will report both to the namenode.  In the future,
-   * we can consider figuring out exactly which block is corrupt.
-   */
-  // We do not see a need for user to report block checksum errors and do not  
-  // want to rely on user to report block corruptions.
-  @Deprecated
-  public boolean reportChecksumFailure(Path f, 
-    FSDataInputStream in, long inPos, 
-    FSDataInputStream sums, long sumsPos) {
-    
-    if(!(in instanceof HdfsDataInputStream && sums instanceof HdfsDataInputStream))
-      throw new IllegalArgumentException(
-          "Input streams must be types of HdfsDataInputStream");
-    
-    LocatedBlock lblocks[] = new LocatedBlock[2];
-
-    // Find block in data stream.
-    HdfsDataInputStream dfsIn = (HdfsDataInputStream) in;
-    ExtendedBlock dataBlock = dfsIn.getCurrentBlock();
-    if (dataBlock == null) {
-      LOG.error("Error: Current block in data stream is null! ");
-      return false;
-    }
-    DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; 
-    lblocks[0] = new LocatedBlock(dataBlock, dataNode);
-    LOG.info("Found checksum error in data stream at "
-        + dataBlock + " on datanode="
-        + dataNode[0]);
-
-    // Find block in checksum stream
-    HdfsDataInputStream dfsSums = (HdfsDataInputStream) sums;
-    ExtendedBlock sumsBlock = dfsSums.getCurrentBlock();
-    if (sumsBlock == null) {
-      LOG.error("Error: Current block in checksum stream is null! ");
-      return false;
-    }
-    DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; 
-    lblocks[1] = new LocatedBlock(sumsBlock, sumsNode);
-    LOG.info("Found checksum error in checksum stream at "
-        + sumsBlock + " on datanode=" + sumsNode[0]);
-
-    // Ask client to delete blocks.
-    dfs.reportChecksumFailure(f.toString(), lblocks);
-
-    return true;
-  }
-
   /**
    * Returns the stat information about the file.
    * @throws FileNotFoundException if the file does not exist.
@@ -1282,66 +1227,13 @@ public class DistributedFileSystem extends FileSystem {
   }
 
   @Override
-  public 
-  Token<DelegationTokenIdentifier> getDelegationToken(String renewer
-  ) throws IOException {
+  public Token<DelegationTokenIdentifier> getDelegationToken(String renewer)
+      throws IOException {
     Token<DelegationTokenIdentifier> result =
       dfs.getDelegationToken(renewer == null ? null : new Text(renewer));
     return result;
   }
 
-  /*
-   * Delegation Token Operations
-   * These are DFS only operations.
-   */
-  
-  /**
-   * Get a valid Delegation Token.
-   * 
-   * @param renewer Name of the designated renewer for the token
-   * @return Token<DelegationTokenIdentifier>
-   * @throws IOException
-   * @deprecated use {@link #getDelegationToken(String)}
-   */
-  @Deprecated
-  public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
-      throws IOException {
-    return getDelegationToken(renewer.toString());
-  }
-  
-  /**
-   * Renew an existing delegation token.
-   * 
-   * @param token delegation token obtained earlier
-   * @return the new expiration time
-   * @throws IOException
-   * @deprecated Use Token.renew instead.
-   */
-  public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws InvalidToken, IOException {
-    try {
-      return token.renew(getConf());
-    } catch (InterruptedException ie) {
-      throw new RuntimeException("Caught interrupted", ie);
-    }
-  }
-
-  /**
-   * Cancel an existing delegation token.
-   * 
-   * @param token delegation token
-   * @throws IOException
-   * @deprecated Use Token.cancel instead.
-   */
-  public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
-      throws IOException {
-    try {
-      token.cancel(getConf());
-    } catch (InterruptedException ie) {
-      throw new RuntimeException("Caught interrupted", ie);
-    }
-  }
-
   /**
    * Requests the namenode to tell all datanodes to use a new, non-persistent
    * bandwidth value for dfs.balance.bandwidthPerSec.

+ 6 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java

@@ -195,7 +195,6 @@ public class TestDelegationToken {
     }
   }
 
-  @SuppressWarnings("deprecation")
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
     final DistributedFileSystem dfs = cluster.getFileSystem();
@@ -212,11 +211,9 @@ public class TestDelegationToken {
     longUgi.doAs(new PrivilegedExceptionAction<Object>() {
       @Override
       public Object run() throws IOException {
-        final DistributedFileSystem dfs = cluster.getFileSystem();
         try {
-          //try renew with long name
-          dfs.renewDelegationToken(token);
-        } catch (IOException e) {
+          token.renew(config);
+        } catch (Exception e) {
           Assert.fail("Could not renew delegation token for user "+longUgi);
         }
         return null;
@@ -224,20 +221,17 @@ public class TestDelegationToken {
     });
     shortUgi.doAs(new PrivilegedExceptionAction<Object>() {
       @Override
-      public Object run() throws IOException {
-        final DistributedFileSystem dfs = cluster.getFileSystem();
-        dfs.renewDelegationToken(token);
+      public Object run() throws Exception {
+        token.renew(config);
         return null;
       }
     });
     longUgi.doAs(new PrivilegedExceptionAction<Object>() {
       @Override
       public Object run() throws IOException {
-        final DistributedFileSystem dfs = cluster.getFileSystem();
         try {
-          //try cancel with long name
-          dfs.cancelDelegationToken(token);
-        } catch (IOException e) {
+          token.cancel(config);
+        } catch (Exception e) {
           Assert.fail("Could not cancel delegation token for user "+longUgi);
         }
         return null;

+ 7 - 12
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java

@@ -99,6 +99,9 @@ public class TestDelegationTokensWithHA {
       .build();
     cluster.waitActive();
     
+    String logicalName = HATestUtil.getLogicalHostname(cluster);
+    HATestUtil.setFailoverConfigurations(cluster, conf, logicalName, 0);
+
     nn0 = cluster.getNameNode(0);
     nn1 = cluster.getNameNode(1);
     fs = HATestUtil.configureFailoverFs(cluster, conf);
@@ -246,8 +249,7 @@ public class TestDelegationTokensWithHA {
     doRenewOrCancel(token, clientConf, TokenTestAction.RENEW);
     doRenewOrCancel(token, clientConf, TokenTestAction.CANCEL);
   }
-  
-  @SuppressWarnings("deprecation")
+
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
     final Token<DelegationTokenIdentifier> token =
@@ -259,29 +261,22 @@ public class TestDelegationTokensWithHA {
     longUgi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        DistributedFileSystem dfs = (DistributedFileSystem)
-            HATestUtil.configureFailoverFs(cluster, conf);
         // try renew with long name
-        dfs.renewDelegationToken(token);
+        token.renew(conf);
         return null;
       }
     });
     shortUgi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        DistributedFileSystem dfs = (DistributedFileSystem)
-            HATestUtil.configureFailoverFs(cluster, conf);
-        dfs.renewDelegationToken(token);
+        token.renew(conf);
         return null;
       }
     });
     longUgi.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws Exception {
-        DistributedFileSystem dfs = (DistributedFileSystem)
-            HATestUtil.configureFailoverFs(cluster, conf);
-        // try cancel with long name
-        dfs.cancelDelegationToken(token);
+        token.cancel(conf);;
         return null;
       }
     });

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java

@@ -62,7 +62,6 @@ public class TestDelegationTokenFetcher {
    * Verify that when the DelegationTokenFetcher runs, it talks to the Namenode,
    * pulls out the correct user's token and successfully serializes it to disk.
    */
-  @SuppressWarnings("deprecation")
   @Test
   public void expectedTokenIsRetrievedFromDFS() throws Exception {
     final byte[] ident = new DelegationTokenIdentifier(new Text("owner"),
@@ -83,7 +82,6 @@ public class TestDelegationTokenFetcher {
             return new Token<?>[]{t};
           }
         });
-    when(dfs.renewDelegationToken(eq(t))).thenReturn(1000L);
     when(dfs.getUri()).thenReturn(uri);
     FakeRenewer.reset();
 

+ 10 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestDelegationTokenRenewer.java

@@ -271,8 +271,8 @@ public class TestDelegationTokenRenewer {
     public void initialize(URI uri, Configuration conf) throws IOException {}
     
     @Override 
-    public MyToken getDelegationToken(Text renewer) throws IOException {
-      MyToken result = createTokens(renewer);
+    public MyToken getDelegationToken(String renewer) throws IOException {
+      MyToken result = createTokens(new Text(renewer));
       LOG.info("Called MYDFS.getdelegationtoken " + result);
       return result;
     }
@@ -329,9 +329,9 @@ public class TestDelegationTokenRenewer {
     
     // get the delegation tokens
     MyToken token1, token2, token3;
-    token1 = dfs.getDelegationToken(new Text("user1"));
-    token2 = dfs.getDelegationToken(new Text("user2"));
-    token3 = dfs.getDelegationToken(new Text("user3"));
+    token1 = dfs.getDelegationToken("user1");
+    token2 = dfs.getDelegationToken("user2");
+    token3 = dfs.getDelegationToken("user3");
 
     //to cause this one to be set for renew in 2 secs
     Renewer.tokenToRenewIn2Sec = token1;
@@ -381,7 +381,7 @@ public class TestDelegationTokenRenewer {
     // time is up.
     // Wait for 3 secs , and make sure no renews were called
     ts = new Credentials();
-    MyToken token4 = dfs.getDelegationToken(new Text("user4"));
+    MyToken token4 = dfs.getDelegationToken("user4");
     
     //to cause this one to be set for renew in 2 secs
     Renewer.tokenToRenewIn2Sec = token4; 
@@ -420,7 +420,7 @@ public class TestDelegationTokenRenewer {
     MyFS dfs = (MyFS)FileSystem.get(conf);
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
 
-    MyToken token = dfs.getDelegationToken(new Text("user1"));
+    MyToken token = dfs.getDelegationToken("user1");
     token.cancelToken();
 
     Credentials ts = new Credentials();
@@ -461,7 +461,7 @@ public class TestDelegationTokenRenewer {
     LOG.info("dfs="+(Object)dfs.hashCode() + ";conf="+conf.hashCode());
 
     Credentials ts = new Credentials();
-    MyToken token1 = dfs.getDelegationToken(new Text("user1"));
+    MyToken token1 = dfs.getDelegationToken("user1");
 
     //to cause this one to be set for renew in 2 secs
     Renewer.tokenToRenewIn2Sec = token1; 
@@ -532,7 +532,7 @@ public class TestDelegationTokenRenewer {
     
     Credentials ts = new Credentials();
     // get the delegation tokens
-    MyToken token1 = dfs.getDelegationToken(new Text("user1"));
+    MyToken token1 = dfs.getDelegationToken("user1");
 
     String nn1 = DelegationTokenRenewer.SCHEME + "://host1:0";
     ts.addToken(new Text(nn1), token1);
@@ -609,7 +609,7 @@ public class TestDelegationTokenRenewer {
 
     Credentials ts = new Credentials();
     // get the delegation tokens
-    MyToken token1 = dfs.getDelegationToken(new Text("user1"));
+    MyToken token1 = dfs.getDelegationToken("user1");
     
     String nn1 = DelegationTokenRenewer.SCHEME + "://host1:0";
     ts.addToken(new Text(nn1), token1);