You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by da...@apache.org on 2012/08/17 18:26:03 UTC

svn commit: r1374346 - in /hadoop/common/branches/branch-2/hadoop-hdfs-project: hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/ hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/ hadoop-hdfs-httpfs/src/test/java/org/a...

Author: daryn
Date: Fri Aug 17 16:26:01 2012
New Revision: 1374346

URL: http://svn.apache.org/viewvc?rev=1374346&view=rev
Log:
svn merge -c 1374271 FIXES: HADOOP-7967. Need generalized multi-token filesystem support (daryn)

Modified:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java Fri Aug 17 16:26:01 2012
@@ -863,7 +863,6 @@ public class HttpFSFileSystem extends Fi
 
 
   @Override
-  @SuppressWarnings("deprecation")
   public Token<?> getDelegationToken(final String renewer)
     throws IOException {
     return doAsRealUserIfNecessary(new Callable<Token<?>>() {
@@ -875,19 +874,6 @@ public class HttpFSFileSystem extends Fi
     });
   }
 
-
-  @Override
-  public List<Token<?>> getDelegationTokens(final String renewer)
-    throws IOException {
-    return doAsRealUserIfNecessary(new Callable<List<Token<?>>>() {
-      @Override
-      public List<Token<?>> call() throws Exception {
-        return HttpFSKerberosAuthenticator.
-          getDelegationTokens(uri, httpFSAddr, authToken, renewer);
-      }
-    });
-  }
-
   public long renewDelegationToken(final Token<?> token) throws IOException {
     return doAsRealUserIfNecessary(new Callable<Long>() {
       @Override

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSKerberosAuthenticator.java Fri Aug 17 16:26:01 2012
@@ -66,7 +66,6 @@ public class HttpFSKerberosAuthenticator
   public static final String RENEWER_PARAM = "renewer";
   public static final String TOKEN_KIND = "HTTPFS_DELEGATION_TOKEN";
   public static final String DELEGATION_TOKEN_JSON = "Token";
-  public static final String DELEGATION_TOKENS_JSON = "Tokens";
   public static final String DELEGATION_TOKEN_URL_STRING_JSON = "urlString";
   public static final String RENEW_DELEGATION_TOKEN_JSON = "long";
 
@@ -76,7 +75,6 @@ public class HttpFSKerberosAuthenticator
   @InterfaceAudience.Private
   public static enum DelegationTokenOperation {
     GETDELEGATIONTOKEN(HTTP_GET, true),
-    GETDELEGATIONTOKENS(HTTP_GET, true),
     RENEWDELEGATIONTOKEN(HTTP_PUT, true),
     CANCELDELEGATIONTOKEN(HTTP_PUT, false);
 
@@ -121,10 +119,11 @@ public class HttpFSKerberosAuthenticator
 
   public static final String OP_PARAM = "op";
 
-  private static List<Token<?>> getDelegationTokens(URI fsURI,
-    InetSocketAddress httpFSAddr, DelegationTokenOperation op,
-    AuthenticatedURL.Token token, String renewer)
-    throws IOException {
+  public static Token<?> getDelegationToken(URI fsURI,
+    InetSocketAddress httpFSAddr, AuthenticatedURL.Token token,
+    String renewer) throws IOException {
+    DelegationTokenOperation op = 
+      DelegationTokenOperation.GETDELEGATIONTOKEN;
     Map<String, String> params = new HashMap<String, String>();
     params.put(OP_PARAM, op.toString());
     params.put(RENEWER_PARAM,renewer);
@@ -135,56 +134,20 @@ public class HttpFSKerberosAuthenticator
       HttpURLConnection conn = aUrl.openConnection(url, token);
       conn.setRequestMethod(op.getHttpMethod());
       HttpFSUtils.validateResponse(conn, HttpURLConnection.HTTP_OK);
-      List<String> list = new ArrayList<String>();
-      if (op == DelegationTokenOperation.GETDELEGATIONTOKEN) {
-        JSONObject json = (JSONObject) ((JSONObject)
-          HttpFSUtils.jsonParse(conn)).get(DELEGATION_TOKEN_JSON);
-        String tokenStr = (String)
-          json.get(DELEGATION_TOKEN_URL_STRING_JSON);
-        list.add(tokenStr);
-      }
-      else if (op == DelegationTokenOperation.GETDELEGATIONTOKENS) {
-        JSONObject json = (JSONObject) ((JSONObject)
-          HttpFSUtils.jsonParse(conn)).get(DELEGATION_TOKENS_JSON);
-        JSONArray array = (JSONArray) json.get(DELEGATION_TOKEN_JSON);
-        for (Object element : array) {
-          String tokenStr = (String)
-            ((Map) element).get(DELEGATION_TOKEN_URL_STRING_JSON);
-          list.add(tokenStr);
-        }
-
-      } else {
-        throw new IllegalArgumentException("Invalid operation: " +
-                                           op.toString());
-      }
-      List<Token<?>> dTokens = new ArrayList<Token<?>>();
-      for (String tokenStr : list) {
-        Token<AbstractDelegationTokenIdentifier> dToken =
-          new Token<AbstractDelegationTokenIdentifier>();
-        dToken.decodeFromUrlString(tokenStr);
-        dTokens.add(dToken);
-        SecurityUtil.setTokenService(dToken, httpFSAddr);
-      }
-      return dTokens;
+      JSONObject json = (JSONObject) ((JSONObject)
+        HttpFSUtils.jsonParse(conn)).get(DELEGATION_TOKEN_JSON);
+      String tokenStr = (String)
+        json.get(DELEGATION_TOKEN_URL_STRING_JSON);
+      Token<AbstractDelegationTokenIdentifier> dToken =
+        new Token<AbstractDelegationTokenIdentifier>();
+      dToken.decodeFromUrlString(tokenStr);
+      SecurityUtil.setTokenService(dToken, httpFSAddr);
+      return dToken;
     } catch (AuthenticationException ex) {
       throw new IOException(ex.toString(), ex);
     }
   }
 
-  public static List<Token<?>> getDelegationTokens(URI fsURI,
-    InetSocketAddress httpFSAddr, AuthenticatedURL.Token token,
-    String renewer) throws IOException {
-    return getDelegationTokens(fsURI, httpFSAddr,
-      DelegationTokenOperation.GETDELEGATIONTOKENS, token, renewer);
-  }
-
-  public static Token<?> getDelegationToken(URI fsURI,
-    InetSocketAddress httpFSAddr, AuthenticatedURL.Token token,
-    String renewer) throws IOException {
-    return getDelegationTokens(fsURI, httpFSAddr,
-      DelegationTokenOperation.GETDELEGATIONTOKENS, token, renewer).get(0);
-  }
-
   public static long renewDelegationToken(URI fsURI,
     AuthenticatedURL.Token token, Token<?> dToken) throws IOException {
     Map<String, String> params = new HashMap<String, String>();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandler.java Fri Aug 17 16:26:01 2012
@@ -64,8 +64,6 @@ public class HttpFSKerberosAuthenticatio
     DELEGATION_TOKEN_OPS.add(
       DelegationTokenOperation.GETDELEGATIONTOKEN.toString());
     DELEGATION_TOKEN_OPS.add(
-      DelegationTokenOperation.GETDELEGATIONTOKENS.toString());
-    DELEGATION_TOKEN_OPS.add(
       DelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
     DELEGATION_TOKEN_OPS.add(
       DelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
@@ -111,7 +109,6 @@ public class HttpFSKerberosAuthenticatio
             Map map = null;
             switch (dtOp) {
               case GETDELEGATIONTOKEN:
-              case GETDELEGATIONTOKENS:
                 String renewerParam =
                   request.getParameter(HttpFSKerberosAuthenticator.RENEWER_PARAM);
                 if (renewerParam == null) {
@@ -119,11 +116,7 @@ public class HttpFSKerberosAuthenticatio
                 }
                 Token<?> dToken = tokenManager.createToken(
                   UserGroupInformation.getCurrentUser(), renewerParam);
-                if (dtOp == DelegationTokenOperation.GETDELEGATIONTOKEN) {
-                  map = delegationTokenToJSON(dToken);
-                } else {
-                  map = delegationTokensToJSON(Arrays.asList((Token)dToken));
-                }
+                map = delegationTokenToJSON(dToken);
                 break;
               case RENEWDELEGATIONTOKEN:
               case CANCELDELEGATIONTOKEN:
@@ -191,23 +184,6 @@ public class HttpFSKerberosAuthenticatio
     return response;
   }
   
-  @SuppressWarnings("unchecked")
-  private static Map delegationTokensToJSON(List<Token> tokens)
-    throws IOException {
-    List list = new ArrayList();
-    for (Token token : tokens) {
-      Map map = new HashMap();
-      map.put(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON,
-              token.encodeToUrlString());
-      list.add(map);
-    }
-    Map map = new HashMap();
-    map.put(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON, list);
-    Map response = new LinkedHashMap();
-    response.put(HttpFSKerberosAuthenticator.DELEGATION_TOKENS_JSON, map);
-    return response;
-  }
-
   /**
    * Authenticates a request looking for the <code>delegation</code>
    * query-string parameter and verifying it is a valid token. If there is not

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSKerberosAuthenticationHandler.java Fri Aug 17 16:26:01 2012
@@ -68,10 +68,8 @@ public class TestHttpFSKerberosAuthentic
 
       testNonManagementOperation(handler);
       testManagementOperationErrors(handler);
-      testGetToken(handler, false, null);
-      testGetToken(handler, true, null);
-      testGetToken(handler, false, "foo");
-      testGetToken(handler, true, "foo");
+      testGetToken(handler, null);
+      testGetToken(handler, "foo");
       testCancelToken(handler);
       testRenewToken(handler);
 
@@ -115,12 +113,9 @@ public class TestHttpFSKerberosAuthentic
       Mockito.contains("requires SPNEGO"));
   }
 
-  private void testGetToken(AuthenticationHandler handler, boolean tokens,
-                            String renewer)
+  private void testGetToken(AuthenticationHandler handler, String renewer)
     throws Exception {
-    DelegationTokenOperation op =
-      (tokens) ? DelegationTokenOperation.GETDELEGATIONTOKENS
-               : DelegationTokenOperation.GETDELEGATIONTOKEN;
+    DelegationTokenOperation op = DelegationTokenOperation.GETDELEGATIONTOKEN;
     HttpServletRequest request = Mockito.mock(HttpServletRequest.class);
     HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
     Mockito.when(request.getParameter(HttpFSFileSystem.OP_PARAM)).
@@ -148,23 +143,13 @@ public class TestHttpFSKerberosAuthentic
     Mockito.verify(response).setContentType(MediaType.APPLICATION_JSON);
     pwriter.close();
     String responseOutput = writer.toString();
-    String tokenLabel = (tokens)
-                        ? HttpFSKerberosAuthenticator.DELEGATION_TOKENS_JSON
-                        : HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON;
-    if (tokens) {
-      Assert.assertTrue(responseOutput.contains(tokenLabel));
-    } else {
-      Assert.assertTrue(responseOutput.contains(tokenLabel));
-    }
+    String tokenLabel = HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON;
+    Assert.assertTrue(responseOutput.contains(tokenLabel));
     Assert.assertTrue(responseOutput.contains(
       HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON));
     JSONObject json = (JSONObject) new JSONParser().parse(responseOutput);
     json = (JSONObject) json.get(tokenLabel);
     String tokenStr;
-    if (tokens) {
-      json = (JSONObject) ((JSONArray)
-        json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON)).get(0);
-    }
     tokenStr = (String)
       json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
     Token<DelegationTokenIdentifier> dt = new Token<DelegationTokenIdentifier>();

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.java Fri Aug 17 16:26:01 2012
@@ -222,10 +222,11 @@ public class TestHttpFSWithKerberos exte
     URI uri = new URI( "webhdfs://" +
                        TestJettyHelper.getJettyURL().toURI().getAuthority());
     FileSystem fs = FileSystem.get(uri, conf);
-    Token<?> token = fs.getDelegationToken("foo");
+    Token<?> tokens[] = fs.addDelegationTokens("foo", null);
     fs.close();
+    Assert.assertEquals(1, tokens.length);
     fs = FileSystem.get(uri, conf);
-    ((DelegationTokenRenewer.Renewable) fs).setDelegationToken(token);
+    ((DelegationTokenRenewer.Renewable) fs).setDelegationToken(tokens[0]);
     fs.listStatus(new Path("/"));
     fs.close();
   }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java Fri Aug 17 16:26:01 2012
@@ -24,7 +24,6 @@ import java.net.InetSocketAddress;
 import java.net.URI;
 import java.util.ArrayList;
 import java.util.EnumSet;
-import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -809,14 +808,6 @@ public class DistributedFileSystem exten
     return getDelegationToken(renewer.toString());
   }
   
-  @Override // FileSystem
-  public List<Token<?>> getDelegationTokens(String renewer) throws IOException {
-    List<Token<?>> tokenList = new ArrayList<Token<?>>();
-    Token<DelegationTokenIdentifier> token = this.getDelegationToken(renewer);
-    tokenList.add(token);
-    return tokenList;
-  }
-
   /**
    * Renew an existing delegation token.
    * 

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java Fri Aug 17 16:26:01 2012
@@ -669,17 +669,6 @@ public class NamenodeWebHdfsMethods {
       final String js = JsonUtil.toJsonString(token);
       return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
     }
-    case GETDELEGATIONTOKENS:
-    {
-      if (delegation.getValue() != null) {
-        throw new IllegalArgumentException(delegation.getName()
-            + " parameter is not null.");
-      }
-      final Token<? extends TokenIdentifier>[] tokens = new Token<?>[1];
-      tokens[0] = generateDelegationToken(namenode, ugi, renewer.getValue());
-      final String js = JsonUtil.toJsonString(tokens);
-      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
-    }
     case GETHOMEDIRECTORY:
     {
       final String js = JsonUtil.toJsonString(

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java Fri Aug 17 16:26:01 2012
@@ -188,12 +188,13 @@ public class DelegationTokenFetcher {
                 }
               } else {
                 FileSystem fs = FileSystem.get(conf);
-                Token<?> token = fs.getDelegationToken(renewer);
                 Credentials cred = new Credentials();
-                cred.addToken(token.getService(), token);
+                Token<?> tokens[] = fs.addDelegationTokens(renewer, cred);
                 cred.writeTokenStorageFile(tokenFile, conf);
-                System.out.println("Fetched token for " + token.getService()
-                    + " into " + tokenFile);
+                for (Token<?> token : tokens) {
+                  System.out.println("Fetched token for " + token.getService()
+                      + " into " + tokenFile);
+                }
               }
             }
             return null;

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java Fri Aug 17 16:26:01 2012
@@ -30,7 +30,6 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.StringTokenizer;
 
@@ -376,8 +375,7 @@ public class WebHdfsFileSystem extends F
         + Param.toSortedString("&", parameters);
     final URL url;
     if (op == PutOpParam.Op.RENEWDELEGATIONTOKEN
-        || op == GetOpParam.Op.GETDELEGATIONTOKEN
-        || op == GetOpParam.Op.GETDELEGATIONTOKENS) {
+        || op == GetOpParam.Op.GETDELEGATIONTOKEN) {
       // Skip adding delegation token for getting or renewing delegation token,
       // because these operations require kerberos authentication.
       url = getNamenodeURL(path, query);
@@ -840,10 +838,9 @@ public class WebHdfsFileSystem extends F
     return statuses;
   }
 
-  @SuppressWarnings("deprecation")
   @Override
-  public Token<DelegationTokenIdentifier> getDelegationToken(final String renewer
-      ) throws IOException {
+  public Token<DelegationTokenIdentifier> getDelegationToken(
+      final String renewer) throws IOException {
     final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKEN;
     final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
     final Token<DelegationTokenIdentifier> token = JsonUtil.toDelegationToken(m); 
@@ -852,18 +849,6 @@ public class WebHdfsFileSystem extends F
   }
 
   @Override
-  public List<Token<?>> getDelegationTokens(final String renewer
-      ) throws IOException {
-    final HttpOpParam.Op op = GetOpParam.Op.GETDELEGATIONTOKENS;
-    final Map<?, ?> m = run(op, null, new RenewerParam(renewer));
-    final List<Token<?>> tokens = JsonUtil.toTokenList(m);
-    for(Token<?> t : tokens) {
-      SecurityUtil.setTokenService(t, nnAddr);
-    }
-    return tokens;
-  }
-
-  @Override
   public Token<?> getRenewToken() {
     return delegationToken;
   }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/GetOpParam.java Fri Aug 17 16:26:01 2012
@@ -32,7 +32,6 @@ public class GetOpParam extends HttpOpPa
 
     GETHOMEDIRECTORY(false, HttpURLConnection.HTTP_OK),
     GETDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
-    GETDELEGATIONTOKENS(false, HttpURLConnection.HTTP_OK),
 
     /** GET_BLOCK_LOCATIONS is a private unstable op. */
     GET_BLOCK_LOCATIONS(false, HttpURLConnection.HTTP_OK),

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemAtHdfsRoot.java Fri Aug 17 16:26:01 2012
@@ -59,7 +59,9 @@ public class TestViewFileSystemAtHdfsRoo
       
   @AfterClass
   public static void clusterShutdownAtEnd() throws Exception {
-    cluster.shutdown();   
+    if (cluster != null) {
+      cluster.shutdown();
+    }
   }
 
   @Override
@@ -84,7 +86,7 @@ public class TestViewFileSystemAtHdfsRoo
 
   @Override
   int getExpectedDelegationTokenCount() {
-    return 8;
+    return 1; // all point to the same fs so 1 unique token
   }
 
   @Override

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java Fri Aug 17 16:26:01 2012
@@ -117,7 +117,7 @@ public class TestViewFileSystemHdfs exte
 
   @Override
   int getExpectedDelegationTokenCount() {
-    return 9;
+    return 2; // Mount points to 2 unique hdfs 
   }
 
   @Override

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationToken.java Fri Aug 17 16:26:01 2012
@@ -28,8 +28,6 @@ import java.io.DataInputStream;
 import java.io.IOException;
 import java.net.URI;
 import java.security.PrivilegedExceptionAction;
-import java.util.List;
-
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -50,6 +48,7 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
@@ -154,25 +153,18 @@ public class TestDelegationToken {
   }
   
   @Test
-  public void testDelegationTokenDFSApi() throws Exception {
+  public void testAddDelegationTokensDFSApi() throws Exception {
+    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("JobTracker");
     DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
-    final Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
-    DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
-    byte[] tokenId = token.getIdentifier();
-    identifier.readFields(new DataInputStream(
-             new ByteArrayInputStream(tokenId)));
-    LOG.info("A valid token should have non-null password, and should be renewed successfully");
-    Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
-    dtSecretManager.renewToken(token, "JobTracker");
-    UserGroupInformation.createRemoteUser("JobTracker").doAs(
-        new PrivilegedExceptionAction<Object>() {
-          @Override
-          public Object run() throws Exception {
-            token.renew(config);
-            token.cancel(config);
-            return null;
-          }
-        });
+    Credentials creds = new Credentials();
+    final Token<?> tokens[] = dfs.addDelegationTokens("JobTracker", creds);
+    Assert.assertEquals(1, tokens.length);
+    Assert.assertEquals(1, creds.numberOfTokens());
+    checkTokenIdentifier(ugi, tokens[0]);
+
+    final Token<?> tokens2[] = dfs.addDelegationTokens("JobTracker", creds);
+    Assert.assertEquals(0, tokens2.length); // already have token
+    Assert.assertEquals(1, creds.numberOfTokens());
   }
   
   @SuppressWarnings("deprecation")
@@ -192,51 +184,27 @@ public class TestDelegationToken {
       }
     });
 
-    { //test getDelegationToken(..)
-      final Token<DelegationTokenIdentifier> token = webhdfs
-          .getDelegationToken("JobTracker");
-      DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
-      byte[] tokenId = token.getIdentifier();
-      identifier.readFields(new DataInputStream(new ByteArrayInputStream(tokenId)));
-      LOG.info("A valid token should have non-null password, and should be renewed successfully");
-      Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
-      dtSecretManager.renewToken(token, "JobTracker");
-      ugi.doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          token.renew(config);
-          token.cancel(config);
-          return null;
-        }
-      });
-    }
-
-    { //test getDelegationTokens(..)
-      final List<Token<?>> tokenlist = webhdfs.getDelegationTokens("JobTracker");
-      DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
-      @SuppressWarnings("unchecked")
-      final Token<DelegationTokenIdentifier> token = (Token<DelegationTokenIdentifier>)tokenlist.get(0);
-      byte[] tokenId = token.getIdentifier();
-      identifier.readFields(new DataInputStream(new ByteArrayInputStream(tokenId)));
-      LOG.info("A valid token should have non-null password, and should be renewed successfully");
-      Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
-      dtSecretManager.renewToken(token, "JobTracker");
-      ugi.doAs(new PrivilegedExceptionAction<Void>() {
-        @Override
-        public Void run() throws Exception {
-          token.renew(config);
-          token.cancel(config);
-          return null;
-        }
-      });
+    { //test addDelegationTokens(..)
+      Credentials creds = new Credentials();
+      final Token<?> tokens[] = webhdfs.addDelegationTokens("JobTracker", creds);
+      Assert.assertEquals(1, tokens.length);
+      Assert.assertEquals(1, creds.numberOfTokens());
+      Assert.assertSame(tokens[0], creds.getAllTokens().iterator().next());
+      checkTokenIdentifier(ugi, tokens[0]);
+      final Token<?> tokens2[] = webhdfs.addDelegationTokens("JobTracker", creds);
+      Assert.assertEquals(0, tokens2.length);
     }
   }
 
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
     final DistributedFileSystem dfs = (DistributedFileSystem) cluster.getFileSystem();
-    final Token<DelegationTokenIdentifier> token = 
-      dfs.getDelegationToken("JobTracker");
+    final Credentials creds = new Credentials();
+    final Token<?> tokens[] = dfs.addDelegationTokens("JobTracker", creds);
+    Assert.assertEquals(1, tokens.length);
+    @SuppressWarnings("unchecked")
+    final Token<DelegationTokenIdentifier> token =
+        (Token<DelegationTokenIdentifier>) tokens[0];
     final UserGroupInformation longUgi = UserGroupInformation
         .createRemoteUser("JobTracker/foo.com@FOO.COM");
     final UserGroupInformation shortUgi = UserGroupInformation
@@ -326,4 +294,33 @@ public class TestDelegationToken {
     assertFalse(nn.isInSafeMode());
     assertTrue(sm.isRunning());
   }
+  
+  @SuppressWarnings("unchecked")
+  private void checkTokenIdentifier(UserGroupInformation ugi, final Token<?> token)
+      throws Exception {
+    Assert.assertNotNull(token);
+    // should be able to use token.decodeIdentifier() but webhdfs isn't
+    // registered with the service loader for token decoding
+    DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
+    byte[] tokenId = token.getIdentifier();
+    DataInputStream in = new DataInputStream(new ByteArrayInputStream(tokenId));
+    try {
+      identifier.readFields(in);
+    } finally {
+      in.close();
+    }
+    Assert.assertNotNull(identifier);
+    LOG.info("A valid token should have non-null password, and should be renewed successfully");
+    Assert.assertTrue(null != dtSecretManager.retrievePassword(identifier));
+    dtSecretManager.renewToken((Token<DelegationTokenIdentifier>) token, "JobTracker");
+    ugi.doAs(
+        new PrivilegedExceptionAction<Object>() {
+          @Override
+          public Object run() throws Exception {
+            token.renew(config);
+            token.cancel(config);
+            return null;
+          }
+        });
+  }
 }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/security/TestDelegationTokenForProxyUser.java Fri Aug 17 16:26:01 2012
@@ -135,15 +135,15 @@ public class TestDelegationTokenForProxy
     final UserGroupInformation proxyUgi = UserGroupInformation
         .createProxyUserForTesting(PROXY_USER, ugi, GROUP_NAMES);
     try {
-      Token<DelegationTokenIdentifier> token = proxyUgi
-          .doAs(new PrivilegedExceptionAction<Token<DelegationTokenIdentifier>>() {
+      Token<?>[] tokens = proxyUgi
+          .doAs(new PrivilegedExceptionAction<Token<?>[]>() {
             @Override
-            public Token<DelegationTokenIdentifier> run() throws IOException {
-              return cluster.getFileSystem().getDelegationToken("RenewerUser");
+            public Token<?>[] run() throws IOException {
+              return cluster.getFileSystem().addDelegationTokens("RenewerUser", null);
             }
           });
       DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
-      byte[] tokenId = token.getIdentifier();
+      byte[] tokenId = tokens[0].getIdentifier();
       identifier.readFields(new DataInputStream(new ByteArrayInputStream(
           tokenId)));
       Assert.assertEquals(identifier.getUser().getUserName(), PROXY_USER);

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/OfflineEditsViewerHelper.java Fri Aug 17 16:26:01 2012
@@ -41,7 +41,6 @@ import org.apache.hadoop.hdfs.Distribute
 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.security.token.delegation.DelegationTokenIdentifier;
 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;
@@ -195,20 +194,21 @@ public class OfflineEditsViewerHelper {
     Path pathSymlink = new Path("/file_symlink");
     fc.createSymlink(pathConcatTarget, pathSymlink, false);
     // OP_GET_DELEGATION_TOKEN 18
-    final Token<DelegationTokenIdentifier> token =
-      dfs.getDelegationToken("JobTracker");
     // 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 {
-          token.renew(config);
-          token.cancel(config);
+          for (Token<?> token : tokens) {
+            token.renew(config);
+            token.cancel(config);
+          }
           return null;
         }
       });

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java Fri Aug 17 16:26:01 2012
@@ -116,7 +116,8 @@ public class TestDelegationTokensWithHA 
   
   @Test
   public void testDelegationTokenDFSApi() throws Exception {
-    Token<DelegationTokenIdentifier> token = dfs.getDelegationToken("JobTracker");
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
     DelegationTokenIdentifier identifier = new DelegationTokenIdentifier();
     byte[] tokenId = token.getIdentifier();
     identifier.readFields(new DataInputStream(
@@ -157,8 +158,8 @@ public class TestDelegationTokensWithHA 
   @SuppressWarnings("deprecation")
   @Test
   public void testDelegationTokenWithDoAs() throws Exception {
-    final Token<DelegationTokenIdentifier> token = 
-        dfs.getDelegationToken("JobTracker");
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
     final UserGroupInformation longUgi = UserGroupInformation
         .createRemoteUser("JobTracker/foo.com@FOO.COM");
     final UserGroupInformation shortUgi = UserGroupInformation
@@ -196,8 +197,8 @@ public class TestDelegationTokensWithHA 
   
   @Test
   public void testHAUtilClonesDelegationTokens() throws Exception {
-    final Token<DelegationTokenIdentifier> token = 
-      dfs.getDelegationToken("test");
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(fs, "JobTracker");
 
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser("test");
     
@@ -258,8 +259,9 @@ public class TestDelegationTokensWithHA 
     URI hAUri = HATestUtil.getLogicalUri(cluster);
     String haService = HAUtil.buildTokenServiceForLogicalUri(hAUri).toString();
     assertEquals(haService, dfs.getCanonicalServiceName());
-    Token<?> token = dfs.getDelegationToken(
-        UserGroupInformation.getCurrentUser().getShortUserName());
+    final String renewer = UserGroupInformation.getCurrentUser().getShortUserName();
+    final Token<DelegationTokenIdentifier> token =
+        getDelegationToken(dfs, renewer);
     assertEquals(haService, token.getService().toString());
     // make sure the logical uri is handled correctly
     token.renew(dfs.getConf());
@@ -281,6 +283,13 @@ public class TestDelegationTokensWithHA 
     token.cancel(conf);
   }
   
+  @SuppressWarnings("unchecked")
+  private Token<DelegationTokenIdentifier> getDelegationToken(FileSystem fs,
+      String renewer) throws IOException {
+    final Token<?> tokens[] = fs.addDelegationTokens(renewer, null);
+    assertEquals(1, tokens.length);
+    return (Token<DelegationTokenIdentifier>) tokens[0];
+  }
   enum TokenTestAction {
     RENEW, CANCEL;
   }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewer.java Fri Aug 17 16:26:01 2012
@@ -126,8 +126,8 @@ public class TestOfflineImageViewer {
       }
 
       // Get delegation tokens so we log the delegation token op
-      List<Token<?>> delegationTokens = 
-          hdfs.getDelegationTokens(TEST_RENEWER);
+      Token<?>[] delegationTokens = 
+          hdfs.addDelegationTokens(TEST_RENEWER, null);
       for (Token<?> t : delegationTokens) {
         LOG.debug("got token " + t);
       }

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java?rev=1374346&r1=1374345&r2=1374346&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java Fri Aug 17 16:26:01 2012
@@ -41,6 +41,9 @@ import org.apache.hadoop.security.token.
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import static org.mockito.Matchers.*;
 
 public class TestDelegationTokenFetcher {
   private DistributedFileSystem dfs;
@@ -105,9 +108,17 @@ public class TestDelegationTokenFetcher 
 
     // Create a token for the fetcher to fetch, wire NN to return it when asked
     // for this particular user.
-    Token<DelegationTokenIdentifier> t = 
+    final Token<DelegationTokenIdentifier> t = 
       new Token<DelegationTokenIdentifier>(ident, pw, KIND, service);
-    when(dfs.getDelegationToken(eq((String) null))).thenReturn(t);
+    when(dfs.addDelegationTokens(eq((String) null), any(Credentials.class))).thenAnswer(
+        new Answer<Token<?>[]>() {
+          @Override
+          public Token<?>[] answer(InvocationOnMock invocation) {
+            Credentials creds = (Credentials)invocation.getArguments()[1];
+            creds.addToken(service, t);
+            return new Token<?>[]{t};
+          }
+        });
     when(dfs.renewDelegationToken(eq(t))).thenReturn(1000L);
     when(dfs.getUri()).thenReturn(uri);
     FakeRenewer.reset();