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 tu...@apache.org on 2014/08/08 07:01:13 UTC

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

Author: tucu
Date: Fri Aug  8 05:01:12 2014
New Revision: 1616674

URL: http://svn.apache.org/r1616674
Log:
HADOOP-10771. Refactor HTTP delegation support out of httpfs to common, PART 2. (tucu)

Added:
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
      - copied unchanged from r1616672, hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSAuthenticationFilter.java
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/resources/httpfs-default.xml
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandlerForTesting.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java
    hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSWithKerberos.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=1616674&r1=1616673&r2=1616674&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  8 05:01:12 2014
@@ -39,12 +39,14 @@ import org.apache.hadoop.fs.permission.A
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
-import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
-import org.apache.hadoop.security.authentication.client.Authenticator;
+import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticatedURL;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticator;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.util.StringUtils;
@@ -67,7 +69,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.net.HttpURLConnection;
-import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
@@ -75,7 +76,6 @@ import java.security.PrivilegedException
 import java.text.MessageFormat;
 import java.util.HashMap;
 import java.util.Map;
-import java.util.concurrent.Callable;
 
 /**
  * HttpFSServer implementation of the FileSystemAccess FileSystem.
@@ -217,34 +217,15 @@ public class HttpFSFileSystem extends Fi
 
   }
 
-
-  private AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
+  private DelegationTokenAuthenticatedURL authURL;
+  private DelegationTokenAuthenticatedURL.Token authToken =
+      new DelegationTokenAuthenticatedURL.Token();
   private URI uri;
-  private InetSocketAddress httpFSAddr;
   private Path workingDir;
   private UserGroupInformation realUser;
   private String doAs;
-  private Token<?> delegationToken;
 
-  //This method enables handling UGI doAs with SPNEGO, we have to
-  //fallback to the realuser who logged in with Kerberos credentials
-  private <T> T doAsRealUserIfNecessary(final Callable<T> callable)
-    throws IOException {
-    try {
-      if (realUser.getShortUserName().equals(doAs)) {
-        return callable.call();
-      } else {
-        return realUser.doAs(new PrivilegedExceptionAction<T>() {
-          @Override
-          public T run() throws Exception {
-            return callable.call();
-          }
-        });
-      }
-    } catch (Exception ex) {
-      throw new IOException(ex.toString(), ex);
-    }
-  }
+
 
   /**
    * Convenience method that creates a <code>HttpURLConnection</code> for the
@@ -291,20 +272,26 @@ public class HttpFSFileSystem extends Fi
   private HttpURLConnection getConnection(final String method,
       Map<String, String> params, Map<String, List<String>> multiValuedParams,
       Path path, boolean makeQualified) throws IOException {
-    if (!realUser.getShortUserName().equals(doAs)) {
-      params.put(DO_AS_PARAM, doAs);
-    }
-    HttpFSKerberosAuthenticator.injectDelegationToken(params, delegationToken);
     if (makeQualified) {
       path = makeQualified(path);
     }
     final URL url = HttpFSUtils.createURL(path, params, multiValuedParams);
-    return doAsRealUserIfNecessary(new Callable<HttpURLConnection>() {
-      @Override
-      public HttpURLConnection call() throws Exception {
-        return getConnection(url, method);
+    try {
+      return UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<HttpURLConnection>() {
+            @Override
+            public HttpURLConnection run() throws Exception {
+              return getConnection(url, method);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
       }
-    });
+    }
   }
 
   /**
@@ -321,12 +308,8 @@ public class HttpFSFileSystem extends Fi
    * @throws IOException thrown if an IO error occurrs.
    */
   private HttpURLConnection getConnection(URL url, String method) throws IOException {
-    Class<? extends Authenticator> klass =
-      getConf().getClass("httpfs.authenticator.class",
-                         HttpFSKerberosAuthenticator.class, Authenticator.class);
-    Authenticator authenticator = ReflectionUtils.newInstance(klass, getConf());
     try {
-      HttpURLConnection conn = new AuthenticatedURL(authenticator).openConnection(url, authToken);
+      HttpURLConnection conn = authURL.openConnection(url, authToken);
       conn.setRequestMethod(method);
       if (method.equals(HTTP_POST) || method.equals(HTTP_PUT)) {
         conn.setDoOutput(true);
@@ -357,10 +340,17 @@ public class HttpFSFileSystem extends Fi
     super.initialize(name, conf);
     try {
       uri = new URI(name.getScheme() + "://" + name.getAuthority());
-      httpFSAddr = NetUtils.createSocketAddr(getCanonicalUri().toString());
     } catch (URISyntaxException ex) {
       throw new IOException(ex);
     }
+
+    Class<? extends DelegationTokenAuthenticator> klass =
+        getConf().getClass("httpfs.authenticator.class",
+            KerberosDelegationTokenAuthenticator.class,
+            DelegationTokenAuthenticator.class);
+    DelegationTokenAuthenticator authenticator =
+        ReflectionUtils.newInstance(klass, getConf());
+    authURL = new DelegationTokenAuthenticatedURL(authenticator);
   }
 
   @Override
@@ -1060,38 +1050,57 @@ public class HttpFSFileSystem extends Fi
   @Override
   public Token<?> getDelegationToken(final String renewer)
     throws IOException {
-    return doAsRealUserIfNecessary(new Callable<Token<?>>() {
-      @Override
-      public Token<?> call() throws Exception {
-        return HttpFSKerberosAuthenticator.
-          getDelegationToken(uri, httpFSAddr, authToken, renewer);
+    try {
+      return UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<Token<?>>() {
+            @Override
+            public Token<?> run() throws Exception {
+              return authURL.getDelegationToken(uri.toURL(), authToken,
+                  renewer);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
       }
-    });
+    }
   }
 
   public long renewDelegationToken(final Token<?> token) throws IOException {
-    return doAsRealUserIfNecessary(new Callable<Long>() {
-      @Override
-      public Long call() throws Exception {
-        return HttpFSKerberosAuthenticator.
-          renewDelegationToken(uri,  authToken, token);
+    try {
+      return UserGroupInformation.getCurrentUser().doAs(
+          new PrivilegedExceptionAction<Long>() {
+            @Override
+            public Long run() throws Exception {
+              return authURL.renewDelegationToken(uri.toURL(), authToken);
+            }
+          }
+      );
+    } catch (Exception ex) {
+      if (ex instanceof IOException) {
+        throw (IOException) ex;
+      } else {
+        throw new IOException(ex);
       }
-    });
+    }
   }
 
   public void cancelDelegationToken(final Token<?> token) throws IOException {
-    HttpFSKerberosAuthenticator.
-      cancelDelegationToken(uri, authToken, token);
+    authURL.cancelDelegationToken(uri.toURL(), authToken);
   }
 
   @Override
   public Token<?> getRenewToken() {
-    return delegationToken;
+    return null; //TODO : for renewer
   }
 
   @Override
+  @SuppressWarnings("unchecked")
   public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
-    delegationToken = token;
+    //TODO : for renewer
   }
 
   @Override

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml?rev=1616674&r1=1616673&r2=1616674&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/resources/httpfs-default.xml Fri Aug  8 05:01:12 2014
@@ -35,7 +35,6 @@
       org.apache.hadoop.lib.service.scheduler.SchedulerService,
       org.apache.hadoop.lib.service.security.GroupsService,
       org.apache.hadoop.lib.service.security.ProxyUserService,
-      org.apache.hadoop.lib.service.security.DelegationTokenManagerService,
       org.apache.hadoop.lib.service.hadoop.FileSystemAccessService
     </value>
     <description>
@@ -226,12 +225,4 @@
     </description>
   </property>
 
-  <property>
-    <name>httpfs.user.provider.user.pattern</name>
-    <value>^[A-Za-z_][A-Za-z0-9._-]*[$]?$</value>
-    <description>
-      Valid pattern for user and group names, it must be a valid java regex.
-    </description>
-  </property>
-
 </configuration>

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandlerForTesting.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/HttpFSKerberosAuthenticationHandlerForTesting.java?rev=1616674&r1=1616673&r2=1616674&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandlerForTesting.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/HttpFSKerberosAuthenticationHandlerForTesting.java Fri Aug  8 05:01:12 2014
@@ -17,15 +17,19 @@
  */
 package org.apache.hadoop.fs.http.server;
 
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
+
 import javax.servlet.ServletException;
 import java.util.Properties;
 
 public class HttpFSKerberosAuthenticationHandlerForTesting
-  extends HttpFSKerberosAuthenticationHandler {
+  extends KerberosDelegationTokenAuthenticationHandler {
 
   @Override
   public void init(Properties config) throws ServletException {
     //NOP overwrite to avoid Kerberos initialization
+    config.setProperty(TOKEN_KIND, "t");
+    initTokenManager(config);
   }
 
   @Override

Modified: hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.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/TestHttpFSServer.java?rev=1616674&r1=1616673&r2=1616674&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java (original)
+++ hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java Fri Aug  8 05:01:12 2014
@@ -18,6 +18,8 @@
 package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
+import org.apache.hadoop.security.token.delegation.web.KerberosDelegationTokenAuthenticationHandler;
 import org.json.simple.JSONArray;
 import org.junit.Assert;
 
@@ -43,7 +45,6 @@ import org.apache.hadoop.fs.CommonConfig
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.XAttrCodec;
-import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.lib.server.Service;
 import org.apache.hadoop.lib.server.ServiceException;
 import org.apache.hadoop.lib.service.Groups;
@@ -682,7 +683,7 @@ public class TestHttpFSServer extends HF
 
     AuthenticationToken token =
       new AuthenticationToken("u", "p",
-        HttpFSKerberosAuthenticationHandlerForTesting.TYPE);
+          new KerberosDelegationTokenAuthenticationHandler().getType());
     token.setExpires(System.currentTimeMillis() + 100000000);
     Signer signer = new Signer(new StringSignerSecretProvider("secret"));
     String tokenSigned = signer.sign(token.toString());
@@ -706,9 +707,9 @@ public class TestHttpFSServer extends HF
     JSONObject json = (JSONObject)
       new JSONParser().parse(new InputStreamReader(conn.getInputStream()));
     json = (JSONObject)
-      json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON);
+      json.get(DelegationTokenAuthenticator.DELEGATION_TOKEN_JSON);
     String tokenStr = (String)
-        json.get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
+        json.get(DelegationTokenAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
 
     url = new URL(TestJettyHelper.getJettyURL(),
                   "/webhdfs/v1/?op=GETHOMEDIRECTORY&delegation=" + tokenStr);

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=1616674&r1=1616673&r2=1616674&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  8 05:01:12 2014
@@ -23,11 +23,11 @@ import org.apache.hadoop.fs.DelegationTo
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
-import org.apache.hadoop.fs.http.client.HttpFSKerberosAuthenticator;
 import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticator;
 import org.apache.hadoop.test.HFSTestCase;
 import org.apache.hadoop.test.KerberosTestUtils;
 import org.apache.hadoop.test.TestDir;
@@ -166,9 +166,9 @@ public class TestHttpFSWithKerberos exte
           .parse(new InputStreamReader(conn.getInputStream()));
         json =
           (JSONObject) json
-            .get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_JSON);
+            .get(DelegationTokenAuthenticator.DELEGATION_TOKEN_JSON);
         String tokenStr = (String) json
-          .get(HttpFSKerberosAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
+          .get(DelegationTokenAuthenticator.DELEGATION_TOKEN_URL_STRING_JSON);
 
         //access httpfs using the delegation token
         url = new URL(TestJettyHelper.getJettyURL(),