Просмотр исходного кода

Merging changes r1035508:r1035718 from trunk to federation

git-svn-id: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/HDFS-1052@1078168 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 14 лет назад
Родитель
Сommit
12e1b66ab8

+ 12 - 0
CHANGES.txt

@@ -298,12 +298,16 @@ Release 0.22.0 - Unreleased
     HDFS-1500. TestOfflineImageViewer failing on trunk. (Todd Lipcon
     via hairong)
 
+    HDFS-1467. Append pipeline construction not succeeds with more than
+    one replica. (Todd Lipcon via hairong)
+
   IMPROVEMENTS
 
     HDFS-1304. Add a new unit test for HftpFileSystem.open(..).  (szetszwo)
 
     HDFS-1096. fix for prev. commit. (boryas)
 
+
     HDFS-1096. allow dfsadmin/mradmin refresh of superuser proxy group
      mappings (boryas)
 
@@ -433,6 +437,11 @@ Release 0.22.0 - Unreleased
 
     HDFS-697. Enable asserts for tests by default. (eli)
 
+    HDFS-1187. Modify fetchdt to allow renewing and canceling token.
+    (Owen O'Malley and Kan Zhang via jghoman)
+
+    HDFS-1387. Update HDFS permissions guide for security. (Todd Lipcon via eli)
+
   OPTIMIZATIONS
 
     HDFS-1140. Speedup INode.getPathComponents. (Dmytro Molkov via shv)
@@ -592,6 +601,9 @@ Release 0.22.0 - Unreleased
 
     HDFS-1466. TestFcHdfsSymlink relies on /tmp/test not existing. (eli)
 
+    HDFS-874. TestHDFSFileContextMainOperations fails on weirdly 
+    configured DNS hosts. (Todd Lipcon via eli)
+
 Release 0.21.1 - Unreleased
 
     HDFS-1411. Correct backup node startup command in hdfs user guide.

+ 31 - 31
src/docs/src/documentation/content/xdocs/hdfs_permissions_guide.xml

@@ -71,26 +71,41 @@
 
 <section><title>User Identity</title>
 <p>
-In this release of Hadoop the identity of a client process is just whatever the host operating system says it is. For Unix-like systems,
+As of Hadoop 0.22, Hadoop supports two different modes of operation to determine the user's identity, specified by the
+<code>hadoop.security.authentication</code> property:
 </p>
-<ul>
-<li>
-   The user name is the equivalent of <code>`whoami`</code>;
-</li>
-<li>
-   The group list is the equivalent of <code>`bash -c groups`</code>.
-</li>
-</ul>
+<dl>
+  <dt><code>simple</code></dt>
+  <dd>In this mode of operation, the identity of a client process is determined by the host operating system. On Unix-like systems,
+  the user name is the equivalent of <code>`whoami`</code>.</dd>
+  <dt><code>kerberos</code></dt>
+  <dd>In Kerberized operation, the identity of a client process is determined by its Kerberos credentials. For example, in a
+  Kerberized environment, a user may use the <code>kinit</code> utility to obtain a Kerberos ticket-granting-ticket (TGT) and
+  use <code>klist</code> to determine their current principal. When mapping a Kerberos principal to an HDFS username, all <em>components</em> except for the <em>primary</em> are dropped. For example, a principal <code>todd/foobar@CORP.COMPANY.COM</code> will act as the simple username <code>todd</code> on HDFS.
+  </dd>
+</dl>
+<p>
+Regardless of the mode of operation, the user identity mechanism is extrinsic to HDFS itself.
+There is no provision within HDFS for creating user identities, establishing groups, or processing user credentials.
+</p>
+</section>
 
+<section><title>Group Mapping</title>
+<p>
+Once a username has been determined as described above, the list of groups is determined by a <em>group mapping
+service</em>, configured by the <code>hadoop.security.group.mapping</code> property.
+The default implementation, <code>org.apache.hadoop.security.ShellBasedUnixGroupsMapping</code>, will shell out
+to the Unix <code>bash -c groups</code> command to resolve a list of groups for a user.
+</p>
 <p>
-In the future there will be other ways of establishing user identity (think Kerberos, LDAP, and others). There is no expectation that 
-this first method is secure in protecting one user from impersonating another. This user identity mechanism combined with the 
-permissions model allows a cooperative community to share file system resources in an organized fashion.
+For HDFS, the mapping of users to groups is performed on the NameNode. Thus, the host system configuration of
+the NameNode determines the group mappings for the users.
 </p>
 <p>
-In any case, the user identity mechanism is extrinsic to HDFS itself. There is no provision within HDFS for creating user identities, 
-establishing groups, or processing user credentials.
+Note that HDFS stores the user and group of a file or directory as strings; there is no conversion from user and
+group identity numbers as is conventional in Unix.
 </p>
+
 </section>
 
 <section> <title>Understanding the Implementation</title>
@@ -104,14 +119,6 @@ A second request made to find additional blocks may fail. On the other hand, del
 that already knows the blocks of the file. With the addition of permissions, a client's access to a file may be withdrawn between 
 requests. Again, changing permissions does not revoke the access of a client that already knows the file's blocks.
 </p>
-<p>
-The MapReduce framework delegates the user identity by passing strings without special concern for confidentiality. The owner 
-and group of a file or directory are stored as strings; there is no conversion from user and group identity numbers as is conventional in Unix.
-</p>
-<p>
-The permissions features of this release did not require any changes to the behavior of data nodes. Blocks on the data nodes 
-do not have any of the <em>Hadoop</em> ownership or permissions attributes associated with them.
-</p>
 </section>
      
 <section> <title>Changes to the File System API</title>
@@ -198,19 +205,12 @@ permission parameter <em>P</em>) is used, the mode of new directory is
 
 <section> <title>The Web Server</title>
 <p>
-The identity of the web server is a configuration parameter. That is, the name node has no notion of the identity of 
+By default, the identity of the web server is a configuration parameter. That is, the name node has no notion of the identity of 
 the <em>real</em> user, but the web server behaves as if it has the identity (user and groups) of a user chosen 
-by the administrator. Unless the chosen identity matches the super-user, parts of the name space may be invisible 
+by the administrator. Unless the chosen identity matches the super-user, parts of the name space may be inaccessible
 to the web server.</p>
 </section>
 
-<section> <title>On-line Upgrade</title>
-<p>
-If a cluster starts with a version 0.15 data set (<code>fsimage</code>), all files and directories will have 
-owner <em>O</em>, group <em>G</em>, and mode <em>M</em>, where <em>O</em> and <em>G</em> 
-are the user and group identity of the super-user, and <em>M</em> is a configuration parameter. </p>
-</section>
-
 <section> <title>Configuration Parameters</title>
 <ul>
 	<li><code>dfs.permissions = true </code>

+ 2 - 2
src/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -84,13 +84,13 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
   final private ReplicaInPipelineInterface replicaInfo;
   volatile private boolean mirrorError;
 
-  BlockReceiver(ExtendedBlock block, DataInputStream in, String inAddr,
+  BlockReceiver(ExtendedBlock inBlock, DataInputStream in, String inAddr,
                 String myAddr, BlockConstructionStage stage, 
                 long newGs, long minBytesRcvd, long maxBytesRcvd, 
                 String clientName, DatanodeInfo srcDataNode, DataNode datanode)
                 throws IOException {
     try{
-      this.block = block;
+      this.block = new ExtendedBlock(inBlock);
       this.in = in;
       this.inAddr = inAddr;
       this.myAddr = myAddr;

+ 148 - 106
src/java/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -18,37 +18,40 @@
 package org.apache.hadoop.hdfs.tools;
 
 import java.io.BufferedReader;
+import java.io.ByteArrayInputStream;
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.PrintStream;
 import java.net.HttpURLConnection;
-import java.net.InetAddress;
-import java.net.URI;
 import java.net.URL;
 import java.net.URLConnection;
 import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
+import java.util.Date;
 
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.namenode.CancelDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.GetDelegationTokenServlet;
 import org.apache.hadoop.hdfs.server.namenode.RenewDelegationTokenServlet;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.GenericOptionsParser;
 
 /**
  * Fetch a DelegationToken from the current Namenode and store it in the
@@ -56,99 +59,153 @@ import org.apache.hadoop.util.StringUtils;
  */
 @InterfaceAudience.Private
 public class DelegationTokenFetcher {
-  private static final String USAGE =
-    "fetchdt retrieves delegation tokens (optionally over http)\n" +
-    "and writes them to specified file.\n" +
-    "Usage: fetchdt [--webservice <namenode http addr>] <output filename>";
-  
-  private final DistributedFileSystem dfs;
-  private final UserGroupInformation ugi;
-  private final DataOutputStream out;
-  private final Configuration conf;
-  
   private static final Log LOG = 
     LogFactory.getLog(DelegationTokenFetcher.class);
+  private static final String WEBSERVICE = "webservice";
+  private static final String RENEWER = "renewer";
+  private static final String CANCEL = "cancel";
+  private static final String RENEW = "renew";
+  private static final String PRINT = "print";
 
   static {
     // Enable Kerberos sockets
     System.setProperty("https.cipherSuites", "TLS_KRB5_WITH_3DES_EDE_CBC_SHA");
   }
 
+  private static void printUsage(PrintStream err) throws IOException {
+    err.println("fetchdt retrieves delegation tokens from the NameNode");
+    err.println();
+    err.println("fetchdt <opts> <token file>");
+    err.println("Options:");
+    err.println("  --webservice <url>  Url to contact NN on");
+    err.println("  --renewer <name>    Name of the delegation token renewer");
+    err.println("  --cancel            Cancel the delegation token");
+    err.println("  --renew             Renew the delegation token");
+    err.println("  --print             Print the delegation token");
+    err.println();
+    GenericOptionsParser.printGenericCommandUsage(err);
+    System.exit(1);
+  }
+
+  private static Collection<Token<?>> readTokens(Path file, Configuration conf)
+      throws IOException {
+    Credentials creds = Credentials.readTokenStorageFile(file, conf);
+    return creds.getAllTokens();
+  }
+    
   /**
    * Command-line interface
    */
-  public static void main(final String [] args) throws Exception {
-    // Login the current user
-    UserGroupInformation.getCurrentUser().doAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        
-        if(args.length == 3 && "--webservice".equals(args[0])) {
-          getDTfromRemoteIntoFile(args[1], args[2]);
-          return null;
-        }
-        // avoid annoying mistake
-        if(args.length == 1 && "--webservice".equals(args[0])) {
-          System.out.println(USAGE);
-          return null;
-        }
-        if(args.length != 1 || args[0].isEmpty()) {
-          System.out.println(USAGE);
-          return null;
-        }
-        
-        DataOutputStream out = null;
-        
-        try {
-          Configuration conf = new HdfsConfiguration();
-          DistributedFileSystem dfs = (DistributedFileSystem) FileSystem.get(conf);
-          out = new DataOutputStream(new FileOutputStream(args[0]));
-          UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+  public static void main(final String[] args) throws Exception {
+    final Configuration conf = new HdfsConfiguration();
+    Options fetcherOptions = new Options();
+    fetcherOptions.addOption(WEBSERVICE, true,
+        "HTTPS url to reach the NameNode at");
+    fetcherOptions.addOption(RENEWER, true,
+        "Name of the delegation token renewer");
+    fetcherOptions.addOption(CANCEL, false, "cancel the token");
+    fetcherOptions.addOption(RENEW, false, "renew the token");
+    fetcherOptions.addOption(PRINT, false, "print the token");
+    GenericOptionsParser parser = new GenericOptionsParser(conf,
+        fetcherOptions, args);
+    CommandLine cmd = parser.getCommandLine();
+    
+    // get options
+    final String webUrl = cmd.hasOption(WEBSERVICE) ? cmd
+        .getOptionValue(WEBSERVICE) : null;
+    final String renewer = cmd.hasOption(RENEWER) ? 
+        cmd.getOptionValue(RENEWER) : null;
+    final boolean cancel = cmd.hasOption(CANCEL);
+    final boolean renew = cmd.hasOption(RENEW);
+    final boolean print = cmd.hasOption(PRINT);
+    String[] remaining = parser.getRemainingArgs();
 
-          new DelegationTokenFetcher(dfs, out, ugi, conf).go();
-          
-          out.flush();
-          System.out.println("Succesfully wrote token of size " + 
-              out.size() + " bytes to "+ args[0]);
-        } catch (IOException ioe) {
-          System.out.println("Exception encountered:\n" +
-              StringUtils.stringifyException(ioe));
-        } finally {
-          if(out != null) out.close();
-        }
-        return null;
-      }
-    });
+    // check option validity
+    if (cancel && renew || cancel && print || renew && print || cancel && renew
+        && print) {
+      System.err.println("ERROR: Only specify cancel, renew or print.");
+      printUsage(System.err);
+    }
+    if (remaining.length != 1 || remaining[0].charAt(0) == '-') {
+      System.err.println("ERROR: Must specify exacltly one token file");
+      printUsage(System.err);
+    }
+    // default to using the local file system
+    FileSystem local = FileSystem.getLocal(conf);
+    final Path tokenFile = new Path(local.getWorkingDirectory(), remaining[0]);
 
-  }
-  
-  public DelegationTokenFetcher(DistributedFileSystem dfs, 
-      DataOutputStream out, UserGroupInformation ugi, Configuration conf) {
-    checkNotNull("dfs", dfs); this.dfs = dfs;
-    checkNotNull("out", out); this.out = out;
-    checkNotNull("ugi", ugi); this.ugi = ugi;
-    checkNotNull("conf",conf); this.conf = conf;
-  }
-  
-  private void checkNotNull(String s, Object o) {
-    if(o == null) throw new IllegalArgumentException(s + " cannot be null.");
-  }
+    // Login the current user
+    UserGroupInformation.getCurrentUser().doAs(
+        new PrivilegedExceptionAction<Object>() {
+          @SuppressWarnings("unchecked")
+          @Override
+          public Object run() throws Exception {
 
-  public void go() throws IOException {
-    String fullName = ugi.getUserName();
-    String shortName = ugi.getShortUserName();
-    Token<DelegationTokenIdentifier> token = 
-      dfs.getDelegationToken(fullName);
-    
-    // Reconstruct the ip:port of the Namenode
-    URI uri = FileSystem.getDefaultUri(conf);
-    String nnAddress = 
-      InetAddress.getByName(uri.getHost()).getHostAddress() + ":" + uri.getPort();
-    token.setService(new Text(nnAddress));
-    
-    Credentials ts = new Credentials();
-    ts.addToken(new Text(shortName), token);
-    ts.writeTokenStorageToStream(out);
+            if (print) {
+              DelegationTokenIdentifier id = new DelegationTokenSecretManager(
+                  0, 0, 0, 0, null).createIdentifier();
+              for (Token<?> token : readTokens(tokenFile, conf)) {
+                DataInputStream in = new DataInputStream(
+                    new ByteArrayInputStream(token.getIdentifier()));
+                id.readFields(in);
+                System.out.println("Token (" + id + ") for " + token.getService());
+              }
+              return null;
+            }
+            
+            if (webUrl != null) {
+              if (renew) {
+                long result;
+                for (Token<?> token : readTokens(tokenFile, conf)) {
+                  result = renewDelegationToken(webUrl,
+                      (Token<DelegationTokenIdentifier>) token);
+                  System.out.println("Renewed token via " + webUrl + " for "
+                      + token.getService() + " until: " + new Date(result));
+                }
+              } else if (cancel) {
+                for (Token<?> token : readTokens(tokenFile, conf)) {
+                  cancelDelegationToken(webUrl,
+                      (Token<DelegationTokenIdentifier>) token);
+                  System.out.println("Cancelled token via " + webUrl + " for "
+                      + token.getService());
+                }
+              } else {
+                Credentials creds = getDTfromRemote(webUrl, renewer);
+                creds.writeTokenStorageFile(tokenFile, conf);
+                for (Token<?> token : creds.getAllTokens()) {
+                  System.out.println("Fetched token via " + webUrl + " for "
+                      + token.getService() + " into " + tokenFile);
+                }
+              }
+            } else {
+              FileSystem fs = FileSystem.get(conf);
+              if (cancel) {
+                for (Token<?> token : readTokens(tokenFile, conf)) {
+                  ((DistributedFileSystem) fs)
+                      .cancelDelegationToken((Token<DelegationTokenIdentifier>) token);
+                  System.out.println("Cancelled token for "
+                      + token.getService());
+                }
+              } else if (renew) {
+                long result;
+                for (Token<?> token : readTokens(tokenFile, conf)) {
+                  result = ((DistributedFileSystem) fs)
+                      .renewDelegationToken((Token<DelegationTokenIdentifier>) token);
+                  System.out.println("Renewed token for " + token.getService()
+                      + " until: " + new Date(result));
+                }
+              } else {
+                Token<?> token = fs.getDelegationToken(renewer);
+                Credentials cred = new Credentials();
+                cred.addToken(token.getService(), token);
+                cred.writeTokenStorageFile(tokenFile, conf);
+                System.out.println("Fetched token for " + token.getService()
+                    + " into " + tokenFile);
+              }
+            }
+            return null;
+          }
+        });
   }
   
   static public Credentials getDTfromRemote(String nnAddr, 
@@ -163,7 +220,6 @@ public class DelegationTokenFetcher {
       } else {
         url.append(nnAddr).append(GetDelegationTokenServlet.PATH_SPEC);
       }
-      System.out.println("Retrieving token from: " + url);
       URL remoteURL = new URL(url.toString());
       SecurityUtil.fetchServiceTicket(remoteURL);
       URLConnection connection = remoteURL.openConnection();
@@ -201,7 +257,11 @@ public class DelegationTokenFetcher {
     try {
       URL url = new URL(buf.toString());
       SecurityUtil.fetchServiceTicket(url);
-      URLConnection connection = url.openConnection();
+      HttpURLConnection connection = (HttpURLConnection) url.openConnection();
+      if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
+        throw new IOException("Error renewing token: " + 
+            connection.getResponseMessage());
+      }
       in = new BufferedReader(new InputStreamReader
           (connection.getInputStream()));
       long result = Long.parseLong(in.readLine());
@@ -235,7 +295,7 @@ public class DelegationTokenFetcher {
       SecurityUtil.fetchServiceTicket(url);
       HttpURLConnection connection = (HttpURLConnection) url.openConnection();
       if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
-        throw new IOException("Error cancelling token:" + 
+        throw new IOException("Error cancelling token: " + 
             connection.getResponseMessage());
       }
     } catch (IOException ie) {
@@ -243,22 +303,4 @@ public class DelegationTokenFetcher {
       throw ie;
     }
   }
-
-
-
-  /**
-   * Utility method to obtain a delegation token over http
-   * @param nnHttpAddr Namenode http addr, such as http://namenode:50070
-   * @param filename Name of file to store token in
-   */
-  static private void getDTfromRemoteIntoFile(String nnAddr, String filename) 
-  throws IOException {
-    Credentials ts = getDTfromRemote(nnAddr, null); 
-
-    DataOutputStream file = new DataOutputStream(new FileOutputStream(filename));
-    ts.writeTokenStorageToStream(file);
-    file.flush();
-    System.out.println("Successfully wrote token of " + file.size() 
-        + " bytes  to " + filename);
-  }
 }

+ 0 - 1
src/test/hdfs/org/apache/hadoop/hdfs/MiniDFSCluster.java

@@ -746,7 +746,6 @@ public class MiniDFSCluster {
     // If minicluster's name node is null assume that the conf has been
     // set with the right address:port of the name node.
     //
-    
     if (racks != null && numDataNodes > racks.length ) {
       throw new IllegalArgumentException( "The length of racks [" + racks.length
           + "] is less than the number of datanodes [" + numDataNodes + "].");

+ 1 - 1
src/test/hdfs/org/apache/hadoop/hdfs/TestPipelines.java

@@ -110,7 +110,7 @@ public class TestPipelines {
 
       assertTrue("Replica on DN " + dn + " shouldn't be null", r != null);
       assertEquals("Should be RBW replica on " + dn
-          + " after sequence of calls " + "append()/write()/hflush()",
+          + " after sequence of calls append()/write()/hflush()",
           HdfsConstants.ReplicaState.RBW, r.getState());
     }
     ofs.close();

+ 45 - 70
src/test/hdfs/org/apache/hadoop/tools/TestDelegationTokenFetcher.java

@@ -17,42 +17,43 @@
  */
 package org.apache.hadoop.tools;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
+import java.io.IOException;
 import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Iterator;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.junit.Before;
 import org.junit.Test;
 
 public class TestDelegationTokenFetcher {
   private DistributedFileSystem dfs;
-  private DataOutputStream out;
-  private UserGroupInformation ugi;
   private Configuration conf;
+  private URI uri;
+  private static final String SERVICE_VALUE = "localhost:2005";
+  private static String tokenFile = "file.dta";
 
   @Before 
-  public void init() {
+  public void init() throws URISyntaxException, IOException {
     dfs = mock(DistributedFileSystem.class);
-    out = mock(DataOutputStream.class);
-    ugi = mock(UserGroupInformation.class);
     conf = new Configuration();
+    uri = new URI("hdfs://" + SERVICE_VALUE);
+    FileSystemTestHelper.addFileSystemForTesting(uri, conf, dfs);
   }
   
   /**
@@ -61,68 +62,42 @@ public class TestDelegationTokenFetcher {
    */
   @Test
   public void expectedTokenIsRetrievedFromDFS() throws Exception {
-    final String LONG_NAME = "TheDoctor@TARDIS";
-    final String SHORT_NAME = "TheDoctor";
-    final String SERVICE_VALUE = "localhost:2005";
-    URI uri = new URI("hdfs://" + SERVICE_VALUE);
-    FileSystem.setDefaultUri(conf, uri);
-    
-    // Mock out the user's long and short names.
-    when(ugi.getUserName()).thenReturn(LONG_NAME);
-    when(ugi.getShortUserName()).thenReturn(SHORT_NAME);
-    
+    final byte[] ident = new DelegationTokenIdentifier(new Text("owner"),
+        new Text("renewer"), new Text("realuser")).getBytes();
+    final byte[] pw = new byte[] { 42 };
+    final Text kind = new Text("MY-KIND");
+    final Text service = new Text(uri.toString());
+
     // Create a token for the fetcher to fetch, wire NN to return it when asked
     // for this particular user.
-    Token<DelegationTokenIdentifier> t = new Token<DelegationTokenIdentifier>();
-    when(dfs.getDelegationToken(eq(LONG_NAME))).thenReturn(t);
-    
-    // Now, actually let the TokenFetcher go fetch the token.
-    final ByteArrayOutputStream baos = new ByteArrayOutputStream();
-    out = new DataOutputStream(baos);
-    new DelegationTokenFetcher(dfs, out, ugi, conf).go();
-    
-    // now read the data back in and verify correct values
-    Credentials ts = new Credentials();
-    DataInputStream dis = 
-      new DataInputStream(new ByteArrayInputStream(baos.toByteArray()));
-    
-    ts.readTokenStorageStream(dis);
-    Token<? extends TokenIdentifier> newToken = ts.getToken(new Text(SHORT_NAME));
-    
-    assertEquals("Should only be one token in storage", ts.numberOfTokens(), 1);
-    assertEquals("Service value should have survived", 
-        "127.0.0.1:2005", newToken.getService().toString());
-  }
+    Token<DelegationTokenIdentifier> t = new Token<DelegationTokenIdentifier>(
+        ident, pw, kind, service);
+    when(dfs.getDelegationToken((String) null)).thenReturn(t);
+    when(dfs.renewDelegationToken(eq(t))).thenReturn(1000L);
+    when(dfs.getUri()).thenReturn(uri);
 
-  private void checkWithNullParam(String s) {
+    FileSystem fileSys = FileSystem.getLocal(conf);
     try {
-      new DelegationTokenFetcher(dfs, out, ugi, conf);
-    } catch (IllegalArgumentException iae) {
-      assertEquals("Expected exception message not received", 
-          s + " cannot be null.", iae.getMessage());
-      return; // received expected exception. We're good.
-    }
-    fail("null parameter should have failed.");
-  }
-  
-  @Test
-  public void dfsCannotBeNull() {
-    dfs = null;
-    String s = "dfs";
-    checkWithNullParam(s);
-  }
+      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
+          tokenFile });
+      Path p = new Path(fileSys.getWorkingDirectory(), tokenFile);
+      Credentials creds = Credentials.readTokenStorageFile(p, conf);
+      Iterator<Token<?>> itr = creds.getAllTokens().iterator();
+      // make sure we got back exactly the 1 token we expected
+      assertTrue(itr.hasNext());
+      assertEquals(t, itr.next());
+      assertTrue(!itr.hasNext());
 
-  @Test
-  public void dosCannotBeNull() {
-    out = null;
-    String s = "out";
-    checkWithNullParam(s);
-  }
-  
-  @Test
-  public void ugiCannotBeNull() {
-    ugi = null;
-    String s = "ugi";
-    checkWithNullParam(s);
+      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
+          "--print", tokenFile });
+      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
+          "--renew", tokenFile });
+      DelegationTokenFetcher.main(new String[] { "-fs", uri.toString(),
+          "--cancel", tokenFile });
+      verify(dfs).renewDelegationToken(eq(t));
+      verify(dfs).cancelDelegationToken(eq(t));
+    } finally {
+      fileSys.delete(new Path(tokenFile), true);
+    }
   }
 }