فهرست منبع

commit 31187ae59485ce849ac8cd4c650877596863ebbf
Author: Boris Shkolnik <borya@yahoo-inc.com>
Date: Thu Mar 11 23:06:25 2010 -0800

HDFS:1036 from https://issues.apache.org/jira/secure/attachment/12438585/HDFS-1036-BP20-1.patch

+++ b/YAHOO-CHANGES.txt
+ HDFS-1036. In DelegationTokenFetch pass Configuration object so getDefaultUri
+ will work correctly.
+


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.20-security-patches@1077317 13f79535-47bb-0310-9956-ffa450edef68

Owen O'Malley 14 سال پیش
والد
کامیت
51d2382f13

+ 5 - 3
src/hdfs/org/apache/hadoop/hdfs/tools/DelegationTokenFetcher.java

@@ -53,6 +53,7 @@ public class DelegationTokenFetcher {
   private final DistributedFileSystem dfs;
   private final DistributedFileSystem dfs;
   private final UserGroupInformation ugi;
   private final UserGroupInformation ugi;
   private final DataOutputStream out;
   private final DataOutputStream out;
+  private final Configuration conf;
 
 
   /**
   /**
    * Command-line interface
    * Command-line interface
@@ -85,7 +86,7 @@ public class DelegationTokenFetcher {
           out = new DataOutputStream(new FileOutputStream(args[0]));
           out = new DataOutputStream(new FileOutputStream(args[0]));
           UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
           UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
 
 
-          new DelegationTokenFetcher(dfs, out, ugi).go();
+          new DelegationTokenFetcher(dfs, out, ugi, conf).go();
           
           
           out.flush();
           out.flush();
           System.out.println("Succesfully wrote token of size " + 
           System.out.println("Succesfully wrote token of size " + 
@@ -103,10 +104,11 @@ public class DelegationTokenFetcher {
   }
   }
   
   
   public DelegationTokenFetcher(DistributedFileSystem dfs, 
   public DelegationTokenFetcher(DistributedFileSystem dfs, 
-      DataOutputStream out, UserGroupInformation ugi) {
+      DataOutputStream out, UserGroupInformation ugi, Configuration conf) {
     checkNotNull("dfs", dfs); this.dfs = dfs;
     checkNotNull("dfs", dfs); this.dfs = dfs;
     checkNotNull("out", out); this.out = out;
     checkNotNull("out", out); this.out = out;
     checkNotNull("ugi", ugi); this.ugi = ugi;
     checkNotNull("ugi", ugi); this.ugi = ugi;
+    checkNotNull("conf",conf); this.conf = conf;
   }
   }
   
   
   private void checkNotNull(String s, Object o) {
   private void checkNotNull(String s, Object o) {
@@ -120,7 +122,7 @@ public class DelegationTokenFetcher {
       dfs.getDelegationToken(new Text(fullName));
       dfs.getDelegationToken(new Text(fullName));
     
     
     // Reconstruct the ip:port of the Namenode
     // Reconstruct the ip:port of the Namenode
-    URI uri = dfs.getDefaultUri(dfs.getConf());
+    URI uri = FileSystem.getDefaultUri(conf);
     String nnAddress = 
     String nnAddress = 
       InetAddress.getByName(uri.getHost()).getHostAddress() + ":" + uri.getPort();
       InetAddress.getByName(uri.getHost()).getHostAddress() + ":" + uri.getPort();
     token.setService(new Text(nnAddress));
     token.setService(new Text(nnAddress));

+ 9 - 7
src/test/org/apache/hadoop/tools/TestDelegationTokenFetcher.java

@@ -28,6 +28,8 @@ import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.DataOutputStream;
 import java.net.URI;
 import java.net.URI;
 
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
@@ -43,12 +45,14 @@ public class TestDelegationTokenFetcher {
   private DistributedFileSystem dfs;
   private DistributedFileSystem dfs;
   private DataOutputStream out;
   private DataOutputStream out;
   private UserGroupInformation ugi;
   private UserGroupInformation ugi;
+  private Configuration conf;
 
 
   @Before 
   @Before 
   public void init() {
   public void init() {
     dfs = mock(DistributedFileSystem.class);
     dfs = mock(DistributedFileSystem.class);
     out = mock(DataOutputStream.class);
     out = mock(DataOutputStream.class);
     ugi = mock(UserGroupInformation.class);
     ugi = mock(UserGroupInformation.class);
+    conf = new Configuration();
   }
   }
   
   
   /**
   /**
@@ -60,6 +64,8 @@ public class TestDelegationTokenFetcher {
     final String LONG_NAME = "TheDoctor@TARDIS";
     final String LONG_NAME = "TheDoctor@TARDIS";
     final String SHORT_NAME = "TheDoctor";
     final String SHORT_NAME = "TheDoctor";
     final String SERVICE_VALUE = "localhost:2005";
     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.
     // Mock out the user's long and short names.
     when(ugi.getUserName()).thenReturn(LONG_NAME);
     when(ugi.getUserName()).thenReturn(LONG_NAME);
@@ -69,15 +75,11 @@ public class TestDelegationTokenFetcher {
     // for this particular user.
     // for this particular user.
     Token<DelegationTokenIdentifier> t = new Token<DelegationTokenIdentifier>();
     Token<DelegationTokenIdentifier> t = new Token<DelegationTokenIdentifier>();
     when(dfs.getDelegationToken(eq(new Text(LONG_NAME)))).thenReturn(t);
     when(dfs.getDelegationToken(eq(new Text(LONG_NAME)))).thenReturn(t);
-
-    // Mock the NN's URI, which is stored as the service value
-    URI uri = new URI("hdfs://" + SERVICE_VALUE);
-    when(dfs.getUri()).thenReturn(uri);
-    
+ 
     // Now, actually let the TokenFetcher go fetch the token.
     // Now, actually let the TokenFetcher go fetch the token.
     final ByteArrayOutputStream baos = new ByteArrayOutputStream();
     final ByteArrayOutputStream baos = new ByteArrayOutputStream();
     out = new DataOutputStream(baos);
     out = new DataOutputStream(baos);
-    new DelegationTokenFetcher(dfs, out, ugi).go();
+    new DelegationTokenFetcher(dfs, out, ugi, conf).go();
     
     
     // now read the data back in and verify correct values
     // now read the data back in and verify correct values
     Credentials ts = new Credentials();
     Credentials ts = new Credentials();
@@ -94,7 +96,7 @@ public class TestDelegationTokenFetcher {
 
 
   private void checkWithNullParam(String s) {
   private void checkWithNullParam(String s) {
     try {
     try {
-      new DelegationTokenFetcher(dfs, out, ugi);
+      new DelegationTokenFetcher(dfs, out, ugi, conf);
     } catch (IllegalArgumentException iae) {
     } catch (IllegalArgumentException iae) {
       assertEquals("Expected exception message not received", 
       assertEquals("Expected exception message not received", 
           s + " cannot be null.", iae.getMessage());
           s + " cannot be null.", iae.getMessage());