Browse Source

svn merge -c 1195656 from trunk for HDFS-2385.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-0.23@1195660 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 13 years ago
parent
commit
611bb588f8

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

@@ -323,6 +323,9 @@ Release 0.23.0 - Unreleased
     HDFS-2348. Support getContentSummary and getFileChecksum in webhdfs.
     HDFS-2348. Support getContentSummary and getFileChecksum in webhdfs.
     (szetszwo)
     (szetszwo)
 
 
+    HDFS-2385. Support renew and cancel delegation tokens in webhdfs.
+    (szetszwo)
+
   IMPROVEMENTS
   IMPROVEMENTS
 
 
     HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost
     HDFS-1875. MiniDFSCluster hard-codes dfs.datanode.address to localhost

+ 6 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -63,8 +63,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final String  DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY = "dfs.datanode.drop.cache.behind.reads";
   public static final String  DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY = "dfs.datanode.drop.cache.behind.reads";
   public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false;
   public static final boolean DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT = false;
 
 
+  public static final String  DFS_NAMENODE_HTTP_PORT_KEY = "dfs.http.port";
+  public static final int     DFS_NAMENODE_HTTP_PORT_DEFAULT = 50070;
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
   public static final String  DFS_NAMENODE_HTTP_ADDRESS_KEY = "dfs.namenode.http-address";
-  public static final String  DFS_NAMENODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:50070";
+  public static final String  DFS_NAMENODE_HTTP_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTP_PORT_DEFAULT;
   public static final String  DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
   public static final String  DFS_NAMENODE_RPC_ADDRESS_KEY = "dfs.namenode.rpc-address";
   public static final String  DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.servicerpc-address";
   public static final String  DFS_NAMENODE_SERVICE_RPC_ADDRESS_KEY = "dfs.namenode.servicerpc-address";
   public static final String  DFS_NAMENODE_MAX_OBJECTS_KEY = "dfs.namenode.max.objects";
   public static final String  DFS_NAMENODE_MAX_OBJECTS_KEY = "dfs.namenode.max.objects";
@@ -151,8 +153,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
 
 
   //Following keys have no defaults
   //Following keys have no defaults
   public static final String  DFS_DATANODE_DATA_DIR_KEY = "dfs.datanode.data.dir";
   public static final String  DFS_DATANODE_DATA_DIR_KEY = "dfs.datanode.data.dir";
+  public static final String  DFS_NAMENODE_HTTPS_PORT_KEY = "dfs.https.port";
+  public static final int     DFS_NAMENODE_HTTPS_PORT_DEFAULT = 50470;
   public static final String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
   public static final String  DFS_NAMENODE_HTTPS_ADDRESS_KEY = "dfs.namenode.https-address";
-  public static final String  DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:50470";
+  public static final String  DFS_NAMENODE_HTTPS_ADDRESS_DEFAULT = "0.0.0.0:" + DFS_NAMENODE_HTTPS_PORT_DEFAULT;
   public static final String  DFS_NAMENODE_NAME_DIR_KEY = "dfs.namenode.name.dir";
   public static final String  DFS_NAMENODE_NAME_DIR_KEY = "dfs.namenode.name.dir";
   public static final String  DFS_NAMENODE_EDITS_DIR_KEY = "dfs.namenode.edits.dir";
   public static final String  DFS_NAMENODE_EDITS_DIR_KEY = "dfs.namenode.edits.dir";
   public static final String  DFS_CLIENT_READ_PREFETCH_SIZE_KEY = "dfs.client.read.prefetch.size"; 
   public static final String  DFS_CLIENT_READ_PREFETCH_SIZE_KEY = "dfs.client.read.prefetch.size"; 

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java

@@ -28,6 +28,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SERVICE_RPC_ADDR
 import java.io.IOException;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.io.UnsupportedEncodingException;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Comparator;
@@ -759,4 +761,14 @@ public class DFSUtil {
     }
     }
     return nameserviceId;
     return nameserviceId;
   }
   }
+
+  /** Create a URI from the scheme and address */
+  public static URI createUri(String scheme, InetSocketAddress address) {
+    try {
+      return new URI(scheme, null, address.getHostName(), address.getPort(),
+          null, null, null);
+    } catch (URISyntaxException ue) {
+      throw new IllegalArgumentException(ue);
+    }
+  }
 }
 }

+ 39 - 160
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HftpFileSystem.java

@@ -21,7 +21,6 @@ package org.apache.hadoop.hdfs;
 import java.io.FileNotFoundException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
-import java.lang.ref.WeakReference;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
 import java.net.InetSocketAddress;
 import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
@@ -32,9 +31,6 @@ import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.ArrayList;
 import java.util.TimeZone;
 import java.util.TimeZone;
-import java.util.concurrent.DelayQueue;
-import java.util.concurrent.Delayed;
-import java.util.concurrent.TimeUnit;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -49,6 +45,8 @@ import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenRenewer;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
 import org.apache.hadoop.hdfs.tools.DelegationTokenFetcher;
@@ -79,9 +77,14 @@ import org.xml.sax.helpers.XMLReaderFactory;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public class HftpFileSystem extends FileSystem {
+public class HftpFileSystem extends FileSystem
+    implements DelegationTokenRenewer.Renewable {
+  private static final DelegationTokenRenewer<HftpFileSystem> dtRenewer
+      = new DelegationTokenRenewer<HftpFileSystem>(HftpFileSystem.class);
+  
   static {
   static {
     HttpURLConnection.setFollowRedirects(true);
     HttpURLConnection.setFollowRedirects(true);
+    dtRenewer.start();
   }
   }
 
 
   public static final Text TOKEN_KIND = new Text("HFTP delegation");
   public static final Text TOKEN_KIND = new Text("HFTP delegation");
@@ -96,7 +99,6 @@ public class HftpFileSystem extends FileSystem {
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
   public static final String HFTP_DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ssZ";
   private Token<?> delegationToken;
   private Token<?> delegationToken;
   private Token<?> renewToken;
   private Token<?> renewToken;
-  public static final String HFTP_SERVICE_NAME_KEY = "hdfs.service.host_";
   
   
   public static final SimpleDateFormat getDateFormat() {
   public static final SimpleDateFormat getDateFormat() {
     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
     final SimpleDateFormat df = new SimpleDateFormat(HFTP_DATE_FORMAT);
@@ -111,14 +113,18 @@ public class HftpFileSystem extends FileSystem {
     }
     }
   };
   };
 
 
-  private static RenewerThread renewer = new RenewerThread();
-  static {
-    renewer.start();
-  }
-
   @Override
   @Override
   protected int getDefaultPort() {
   protected int getDefaultPort() {
-    return DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT;
+    return getDefaultSecurePort();
+
+    //TODO: un-comment the following once HDFS-7510 is committed. 
+//    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
+//        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
+  }
+
+  protected int getDefaultSecurePort() {
+    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_KEY,
+        DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT);
   }
   }
 
 
   @Override
   @Override
@@ -132,7 +138,6 @@ public class HftpFileSystem extends FileSystem {
   }
   }
 
 
 
 
-  @SuppressWarnings("unchecked")
   @Override
   @Override
   public void initialize(final URI name, final Configuration conf)
   public void initialize(final URI name, final Configuration conf)
   throws IOException {
   throws IOException {
@@ -161,7 +166,7 @@ public class HftpFileSystem extends FileSystem {
     // if one uses RPC port different from the Default one,  
     // if one uses RPC port different from the Default one,  
     // one should specify what is the setvice name for this delegation token
     // one should specify what is the setvice name for this delegation token
     // otherwise it is hostname:RPC_PORT
     // otherwise it is hostname:RPC_PORT
-    String key = HftpFileSystem.HFTP_SERVICE_NAME_KEY
+    String key = DelegationTokenSelector.SERVICE_NAME_KEY
         + SecurityUtil.buildDTServiceName(name,
         + SecurityUtil.buildDTServiceName(name,
             DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT);
             DFSConfigKeys.DFS_HTTPS_PORT_DEFAULT);
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {
@@ -205,7 +210,7 @@ public class HftpFileSystem extends FileSystem {
       //since we don't already have a token, go get one over https
       //since we don't already have a token, go get one over https
       if (delegationToken == null) {
       if (delegationToken == null) {
         setDelegationToken(getDelegationToken(null));
         setDelegationToken(getDelegationToken(null));
-        renewer.addTokenToRenew(this);
+        dtRenewer.addRenewAction(this);
       }
       }
     }
     }
   }
   }
@@ -222,7 +227,25 @@ public class HftpFileSystem extends FileSystem {
     return hostname + ":" + port;
     return hostname + ":" + port;
   }
   }
 
 
-  private <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
+  //TODO: un-comment the following once HDFS-7510 is committed. 
+//  protected Token<DelegationTokenIdentifier> selectHftpDelegationToken() {
+//    Text serviceName = SecurityUtil.buildTokenService(nnSecureAddr);
+//    return hftpTokenSelector.selectToken(serviceName, ugi.getTokens());      
+//  }
+  
+  protected Token<DelegationTokenIdentifier> selectHdfsDelegationToken() {
+    return  DelegationTokenSelector.selectHdfsDelegationToken(
+        nnAddr, ugi, getConf());
+  }
+  
+
+  @Override
+  public Token<?> getRenewToken() {
+    return renewToken;
+  }
+
+  @Override
+  public <T extends TokenIdentifier> void setDelegationToken(Token<T> token) {
     renewToken = token;
     renewToken = token;
     // emulate the 203 usage of the tokens
     // emulate the 203 usage of the tokens
     // by setting the kind and service as if they were hdfs tokens
     // by setting the kind and service as if they were hdfs tokens
@@ -628,150 +651,6 @@ public class HftpFileSystem extends FileSystem {
     return cs != null? cs: super.getContentSummary(f);
     return cs != null? cs: super.getContentSummary(f);
   }
   }
 
 
-
-  /**
-   * An action that will renew and replace the hftp file system's delegation 
-   * tokens automatically.
-   */
-  private static class RenewAction implements Delayed {
-    // when should the renew happen
-    private long timestamp;
-    // a weak reference to the file system so that it can be garbage collected
-    private final WeakReference<HftpFileSystem> weakFs;
-
-    RenewAction(long timestamp, HftpFileSystem fs) {
-      this.timestamp = timestamp;
-      this.weakFs = new WeakReference<HftpFileSystem>(fs);
-    }
-
-    /**
-     * Get the delay until this event should happen.
-     */
-    @Override
-    public long getDelay(TimeUnit unit) {
-      long millisLeft = timestamp - System.currentTimeMillis();
-      return unit.convert(millisLeft, TimeUnit.MILLISECONDS);
-    }
-
-    /**
-     * Compare two events in the same queue.
-     */
-    @Override
-    public int compareTo(Delayed o) {
-      if (o.getClass() != RenewAction.class) {
-        throw new IllegalArgumentException
-                  ("Illegal comparision to non-RenewAction");
-      }
-      RenewAction other = (RenewAction) o;
-      return timestamp < other.timestamp ? -1 :
-        (timestamp == other.timestamp ? 0 : 1);
-    }
-    
-    @Override
-    public int hashCode() {
-      assert false : "hashCode not designed";
-    return 33;  
-    }
-    /**
-     * equals
-     */
-    @Override
-    public boolean equals(Object o) {
-      if(!( o instanceof Delayed))
-        return false;
-      
-      return compareTo((Delayed) o) == 0;
-    }
-
-    /**
-     * Set a new time for the renewal. Can only be called when the action
-     * is not in the queue.
-     * @param newTime the new time
-     */
-    public void setNewTime(long newTime) {
-      timestamp = newTime;
-    }
-
-    /**
-     * Renew or replace the delegation token for this file system.
-     * @return
-     * @throws IOException
-     */
-    public boolean renew() throws IOException, InterruptedException {
-      final HftpFileSystem fs = weakFs.get();
-      if (fs != null) {
-        synchronized (fs) {
-          try {
-            fs.renewToken.renew(fs.getConf());
-          } catch (IOException ie) {
-            try {
-              fs.setDelegationToken(fs.getDelegationToken(null));
-            } catch (IOException ie2) {
-              throw new IOException("Can't renew or get new delegation "
-                  + "token ", ie);
-            }
-          }
-        }
-      }
-      return fs != null;
-    }
-
-    public String toString() {
-      StringBuilder result = new StringBuilder();
-      HftpFileSystem fs = weakFs.get();
-      if (fs == null) {
-        return "evaporated token renew";
-      }
-      synchronized (fs) {
-        result.append(fs.delegationToken);
-      }
-      result.append(" renew in ");
-      result.append(getDelay(TimeUnit.SECONDS));
-      result.append(" secs");
-      return result.toString();
-    }
-  }
-
-  /**
-   * A daemon thread that waits for the next file system to renew.
-   */
-  private static class RenewerThread extends Thread {
-    private DelayQueue<RenewAction> queue = new DelayQueue<RenewAction>();
-    // wait for 95% of a day between renewals
-    private static final int RENEW_CYCLE = (int) (0.95 * 24 * 60 * 60 * 1000);
-
-    public RenewerThread() {
-      super("HFTP Delegation Token Renewer");
-      setDaemon(true);
-    }
-
-    public void addTokenToRenew(HftpFileSystem fs) {
-      queue.add(new RenewAction(RENEW_CYCLE + System.currentTimeMillis(), fs));
-    }
-
-    public void run() {
-      RenewAction action = null;
-      while (true) {
-        try {
-          action = queue.take();
-          if (action.renew()) {
-            action.setNewTime(RENEW_CYCLE + System.currentTimeMillis());
-            queue.add(action);
-          }
-          action = null;
-        } catch (InterruptedException ie) {
-          return;
-        } catch (Exception ie) {
-          if (action != null) {
-            LOG.warn("Failure to renew token " + action, ie);
-          } else {
-            LOG.warn("Failure in renew queue", ie);
-          }
-        }
-      }
-    }
-  }
-  
   @InterfaceAudience.Private
   @InterfaceAudience.Private
   public static class TokenManager extends TokenRenewer {
   public static class TokenManager extends TokenRenewer {
 
 

+ 164 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenRenewer.java

@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.security.token.delegation;
+
+import java.io.IOException;
+import java.lang.ref.WeakReference;
+import java.util.concurrent.DelayQueue;
+import java.util.concurrent.Delayed;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+
+/**
+ * A daemon thread that waits for the next file system to renew.
+ */
+@InterfaceAudience.Private
+public class DelegationTokenRenewer<T extends FileSystem & DelegationTokenRenewer.Renewable>
+    extends Thread {
+  /** The renewable interface used by the renewer. */
+  public interface Renewable {
+    /** @return the renew token. */
+    public Token<?> getRenewToken();
+
+    /** Set delegation token. */
+    public <T extends TokenIdentifier> void setDelegationToken(Token<T> token);
+  }
+
+  /**
+   * An action that will renew and replace the file system's delegation 
+   * tokens automatically.
+   */
+  private static class RenewAction<T extends FileSystem & Renewable>
+      implements Delayed {
+    /** when should the renew happen */
+    private long renewalTime;
+    /** a weak reference to the file system so that it can be garbage collected */
+    private final WeakReference<T> weakFs;
+
+    private RenewAction(final T fs) {
+      this.weakFs = new WeakReference<T>(fs);
+      updateRenewalTime();
+    }
+ 
+    /** Get the delay until this event should happen. */
+    @Override
+    public long getDelay(final TimeUnit unit) {
+      final long millisLeft = renewalTime - System.currentTimeMillis();
+      return unit.convert(millisLeft, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public int compareTo(final Delayed delayed) {
+      final RenewAction<?> that = (RenewAction<?>)delayed;
+      return this.renewalTime < that.renewalTime? -1
+          : this.renewalTime == that.renewalTime? 0: 1;
+    }
+
+    @Override
+    public int hashCode() {
+      return (int)renewalTime ^ (int)(renewalTime >>> 32);
+    }
+
+    @Override
+    public boolean equals(final Object that) {
+      if (that == null || !(that instanceof RenewAction)) {
+        return false;
+      }
+      return compareTo((Delayed)that) == 0;
+    }
+
+    /**
+     * Set a new time for the renewal.
+     * It can only be called when the action is not in the queue.
+     * @param newTime the new time
+     */
+    private void updateRenewalTime() {
+      renewalTime = RENEW_CYCLE + System.currentTimeMillis();
+    }
+
+    /**
+     * Renew or replace the delegation token for this file system.
+     * @return
+     * @throws IOException
+     */
+    private boolean renew() throws IOException, InterruptedException {
+      final T fs = weakFs.get();
+      final boolean b = fs != null;
+      if (b) {
+        synchronized(fs) {
+          try {
+            fs.getRenewToken().renew(fs.getConf());
+          } catch (IOException ie) {
+            try {
+              fs.setDelegationToken(fs.getDelegationTokens(null).get(0));
+            } catch (IOException ie2) {
+              throw new IOException("Can't renew or get new delegation token ", ie);
+            }
+          }
+        }
+      }
+      return b;
+    }
+
+    @Override
+    public String toString() {
+      Renewable fs = weakFs.get();
+      return fs == null? "evaporated token renew"
+          : "The token will be renewed in " + getDelay(TimeUnit.SECONDS)
+            + " secs, renewToken=" + fs.getRenewToken();
+    }
+  }
+
+  /** Wait for 95% of a day between renewals */
+  private static final int RENEW_CYCLE = 24 * 60 * 60 * 950;
+
+  private DelayQueue<RenewAction<T>> queue = new DelayQueue<RenewAction<T>>();
+
+  public DelegationTokenRenewer(final Class<?> clazz) {
+    super(clazz.getSimpleName() + "-" + DelegationTokenRenewer.class.getSimpleName());
+    setDaemon(true);
+  }
+
+  /** Add a renew action to the queue. */
+  public void addRenewAction(final T fs) {
+    queue.add(new RenewAction<T>(fs));
+  }
+
+  @Override
+  public void run() {
+    for(;;) {
+      RenewAction<T> action = null;
+      try {
+        action = queue.take();
+        if (action.renew()) {
+          action.updateRenewalTime();
+          queue.add(action);
+        }
+      } catch (InterruptedException ie) {
+        return;
+      } catch (Exception ie) {
+        T.LOG.warn("Failed to renew token, action=" + action, ie);
+      }
+    }
+  }
+}

+ 33 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/security/token/delegation/DelegationTokenSelector.java

@@ -17,7 +17,16 @@
  */
  */
 package org.apache.hadoop.hdfs.security.token.delegation;
 package org.apache.hadoop.hdfs.security.token.delegation;
 
 
+import java.net.InetSocketAddress;
+
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.SecurityUtil;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 
 
 /**
 /**
@@ -26,6 +35,30 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelect
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 public class DelegationTokenSelector
 public class DelegationTokenSelector
     extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
     extends AbstractDelegationTokenSelector<DelegationTokenIdentifier>{
+  public static final String SERVICE_NAME_KEY = "hdfs.service.host_";
+
+  private static final DelegationTokenSelector INSTANCE = new DelegationTokenSelector();
+
+  /** Select the delegation token for hdfs from the ugi. */
+  public static Token<DelegationTokenIdentifier> selectHdfsDelegationToken(
+      final InetSocketAddress nnAddr, final UserGroupInformation ugi,
+      final Configuration conf) {
+    // this guesses the remote cluster's rpc service port.
+    // the current token design assumes it's the same as the local cluster's
+    // rpc port unless a config key is set.  there should be a way to automatic
+    // and correctly determine the value
+    final String key = SERVICE_NAME_KEY + SecurityUtil.buildTokenService(nnAddr);
+    final String nnServiceName = conf.get(key);
+    
+    int nnRpcPort = NameNode.DEFAULT_PORT;
+    if (nnServiceName != null) {
+      nnRpcPort = NetUtils.createSocketAddr(nnServiceName, nnRpcPort).getPort(); 
+    }
+    
+    final Text serviceName = SecurityUtil.buildTokenService(
+        new InetSocketAddress(nnAddr.getHostName(), nnRpcPort));
+    return INSTANCE.selectToken(serviceName, ugi.getTokens());
+  }
 
 
   public DelegationTokenSelector() {
   public DelegationTokenSelector() {
     super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
     super(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);

+ 19 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/web/resources/NamenodeWebHdfsMethods.java

@@ -43,8 +43,8 @@ import javax.ws.rs.core.Context;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.MediaType;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.ResponseBuilder;
 import javax.ws.rs.core.Response.ResponseBuilder;
-import javax.ws.rs.core.StreamingOutput;
 import javax.ws.rs.core.Response.Status;
 import javax.ws.rs.core.Response.Status;
+import javax.ws.rs.core.StreamingOutput;
 
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.commons.logging.LogFactory;
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.common.JspHelper;
@@ -87,7 +88,6 @@ import org.apache.hadoop.hdfs.web.resources.RenewerParam;
 import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
 import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.net.NodeBase;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.SecurityUtil;
@@ -155,9 +155,8 @@ public class NamenodeWebHdfsMethods {
         namenode, ugi,
         namenode, ugi,
         renewer != null? renewer: request.getUserPrincipal().getName());
         renewer != null? renewer: request.getUserPrincipal().getName());
     final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
     final Token<? extends TokenIdentifier> t = c.getAllTokens().iterator().next();
-    t.setService(new Text(SecurityUtil.buildDTServiceName(
-        NameNode.getUri(namenode.getNameNodeAddress()),
-        NameNode.DEFAULT_PORT)));
+    t.setKind(WebHdfsFileSystem.TOKEN_KIND);
+    SecurityUtil.setTokenService(t, namenode.getNameNodeAddress());
     return t;
     return t;
   }
   }
 
 
@@ -342,6 +341,21 @@ public class NamenodeWebHdfsMethods {
       np.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue());
       np.setTimes(fullpath, modificationTime.getValue(), accessTime.getValue());
       return Response.ok().type(MediaType.APPLICATION_JSON).build();
       return Response.ok().type(MediaType.APPLICATION_JSON).build();
     }
     }
+    case RENEWDELEGATIONTOKEN:
+    {
+      final Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
+      token.decodeFromUrlString(delegation.getValue());
+      final long expiryTime = np.renewDelegationToken(token);
+      final String js = JsonUtil.toJsonString("long", expiryTime);
+      return Response.ok(js).type(MediaType.APPLICATION_JSON).build();
+    }
+    case CANCELDELEGATIONTOKEN:
+    {
+      final Token<DelegationTokenIdentifier> token = new Token<DelegationTokenIdentifier>();
+      token.decodeFromUrlString(delegation.getValue());
+      np.cancelDelegationToken(token);
+      return Response.ok().type(MediaType.APPLICATION_JSON).build();
+    }
     default:
     default:
       throw new UnsupportedOperationException(op + " is not supported");
       throw new UnsupportedOperationException(op + " is not supported");
     }
     }

+ 201 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java

@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.io.InputStreamReader;
 import java.net.HttpURLConnection;
 import java.net.HttpURLConnection;
+import java.net.InetSocketAddress;
 import java.net.URI;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.URL;
@@ -40,6 +41,7 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileAlreadyExistsException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
@@ -48,12 +50,14 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.ByteRangeInputStream;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.HftpFileSystem;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenRenewer;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSelector;
+import org.apache.hadoop.hdfs.server.common.JspHelper;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
 import org.apache.hadoop.hdfs.web.resources.AccessTimeParam;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
 import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
@@ -79,16 +83,22 @@ import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.hdfs.web.resources.UserParam;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Progressable;
 import org.mortbay.util.ajax.JSON;
 import org.mortbay.util.ajax.JSON;
 
 
 /** A FileSystem for HDFS over the web. */
 /** A FileSystem for HDFS over the web. */
-public class WebHdfsFileSystem extends HftpFileSystem {
+public class WebHdfsFileSystem extends FileSystem
+    implements DelegationTokenRenewer.Renewable {
   public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
   public static final Log LOG = LogFactory.getLog(WebHdfsFileSystem.class);
   /** File System URI: {SCHEME}://namenode:port/path/to/file */
   /** File System URI: {SCHEME}://namenode:port/path/to/file */
   public static final String SCHEME = "webhdfs";
   public static final String SCHEME = "webhdfs";
@@ -97,11 +107,23 @@ public class WebHdfsFileSystem extends HftpFileSystem {
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
   /** Http URI: http://namenode:port/{PATH_PREFIX}/path/to/file */
   public static final String PATH_PREFIX = "/" + SCHEME + "/v" + VERSION;
   public static final String PATH_PREFIX = "/" + SCHEME + "/v" + VERSION;
 
 
+  /** SPNEGO authenticator */
   private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
   private static final KerberosUgiAuthenticator AUTH = new KerberosUgiAuthenticator();
+  /** Delegation token kind */
+  public static final Text TOKEN_KIND = new Text("WEBHDFS delegation");
+
+  private static final DelegationTokenRenewer<WebHdfsFileSystem> dtRenewer
+      = new DelegationTokenRenewer<WebHdfsFileSystem>(WebHdfsFileSystem.class);
+  static {
+    dtRenewer.start();
+  }
 
 
   private final UserGroupInformation ugi;
   private final UserGroupInformation ugi;
+  private InetSocketAddress nnAddr;
+  private Token<?> delegationToken;
+  private Token<?> renewToken;
   private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
   private final AuthenticatedURL.Token authToken = new AuthenticatedURL.Token();
-  protected Path workingDir;
+  private Path workingDir;
 
 
   {
   {
     try {
     try {
@@ -117,7 +139,47 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     super.initialize(uri, conf);
     super.initialize(uri, conf);
     setConf(conf);
     setConf(conf);
 
 
+    this.nnAddr = NetUtils.createSocketAddr(uri.toString());
     this.workingDir = getHomeDirectory();
     this.workingDir = getHomeDirectory();
+
+    if (UserGroupInformation.isSecurityEnabled()) {
+      initDelegationToken();
+    }
+  }
+
+  protected void initDelegationToken() throws IOException {
+    // look for webhdfs token, then try hdfs
+    final Text serviceName = SecurityUtil.buildTokenService(nnAddr);
+    Token<?> token = webhdfspTokenSelector.selectToken(
+        serviceName, ugi.getTokens());      
+    if (token == null) {
+      token = DelegationTokenSelector.selectHdfsDelegationToken(
+          nnAddr, ugi, getConf());
+    }
+
+    //since we don't already have a token, go get one
+    boolean createdToken = false;
+    if (token == null) {
+      token = getDelegationToken(null);
+      createdToken = (token != null);
+    }
+
+    // security might be disabled
+    if (token != null) {
+      setDelegationToken(token);
+      if (createdToken) {
+        dtRenewer.addRenewAction(this);
+        LOG.debug("Created new DT for " + token.getService());
+      } else {
+        LOG.debug("Found existing DT for " + token.getService());        
+      }
+    }
+  }
+
+  @Override
+  protected int getDefaultPort() {
+    return getConf().getInt(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_KEY,
+        DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT);
   }
   }
 
 
   @Override
   @Override
@@ -191,6 +253,35 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     return null;
     return null;
   }
   }
 
 
+  /**
+   * Return a URL pointing to given path on the namenode.
+   *
+   * @param path to obtain the URL for
+   * @param query string to append to the path
+   * @return namenode URL referring to the given path
+   * @throws IOException on error constructing the URL
+   */
+  private URL getNamenodeURL(String path, String query) throws IOException {
+    final URL url = new URL("http", nnAddr.getHostName(),
+          nnAddr.getPort(), path + '?' + query);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("url=" + url);
+    }
+    return url;
+  }
+  
+  private String addDt2Query(String query) throws IOException {
+    if (UserGroupInformation.isSecurityEnabled()) {
+      synchronized (this) {
+        if (delegationToken != null) {
+          final String encoded = delegationToken.encodeToUrlString();
+          return query + JspHelper.getDelegationTokenUrlParam(encoded);
+        } // else we are talking to an insecure cluster
+      }
+    }
+    return query;
+  }
+
   URL toUrl(final HttpOpParam.Op op, final Path fspath,
   URL toUrl(final HttpOpParam.Op op, final Path fspath,
       final Param<?,?>... parameters) throws IOException {
       final Param<?,?>... parameters) throws IOException {
     //initialize URI path and query
     //initialize URI path and query
@@ -199,7 +290,7 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     final String query = op.toQueryString()
     final String query = op.toQueryString()
         + '&' + new UserParam(ugi)
         + '&' + new UserParam(ugi)
         + Param.toSortedString("&", parameters);
         + Param.toSortedString("&", parameters);
-    final URL url = getNamenodeURL(path, addDelegationTokenParam(query));
+    final URL url = getNamenodeURL(path, addDt2Query(query));
     if (LOG.isTraceEnabled()) {
     if (LOG.isTraceEnabled()) {
       LOG.trace("url=" + url);
       LOG.trace("url=" + url);
     }
     }
@@ -403,6 +494,12 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     return write(op, conn, bufferSize);
     return write(op, conn, bufferSize);
   }
   }
 
 
+  @SuppressWarnings("deprecation")
+  @Override
+  public boolean delete(final Path f) throws IOException {
+    return delete(f, true);
+  }
+
   @Override
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
   public boolean delete(Path f, boolean recursive) throws IOException {
     final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
     final HttpOpParam.Op op = DeleteOpParam.Op.DELETE;
@@ -437,6 +534,7 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     return statuses;
     return statuses;
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   @Override
   @Override
   public Token<DelegationTokenIdentifier> getDelegationToken(final String renewer
   public Token<DelegationTokenIdentifier> getDelegationToken(final String renewer
       ) throws IOException {
       ) throws IOException {
@@ -454,6 +552,43 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     return Arrays.asList(t);
     return Arrays.asList(t);
   }
   }
 
 
+  @Override
+  public Token<?> getRenewToken() {
+    return renewToken;
+  }
+
+  @Override
+  public <T extends TokenIdentifier> void setDelegationToken(
+      final Token<T> token) {
+    synchronized(this) {
+      renewToken = token;
+      // emulate the 203 usage of the tokens
+      // by setting the kind and service as if they were hdfs tokens
+      delegationToken = new Token<T>(token);
+      // NOTE: the remote nn must be configured to use hdfs
+      delegationToken.setKind(DelegationTokenIdentifier.HDFS_DELEGATION_KIND);
+      // no need to change service because we aren't exactly sure what it
+      // should be.  we can guess, but it might be wrong if the local conf
+      // value is incorrect.  the service is a client side field, so the remote
+      // end does not care about the value
+    }
+  }
+
+  private synchronized long renewDelegationToken(final Token<?> token
+      ) throws IOException {
+    delegationToken = token;
+    final HttpOpParam.Op op = PutOpParam.Op.RENEWDELEGATIONTOKEN;
+    final Map<?, ?> m = run(op, null);
+    return (Long) m.get("long");
+  }
+
+  private synchronized void cancelDelegationToken(final Token<?> token
+      ) throws IOException {
+    delegationToken = token;
+    final HttpOpParam.Op op = PutOpParam.Op.CANCELDELEGATIONTOKEN;
+    run(op, null);
+  }
+  
   @Override
   @Override
   public BlockLocation[] getFileBlockLocations(final FileStatus status,
   public BlockLocation[] getFileBlockLocations(final FileStatus status,
       final long offset, final long length) throws IOException {
       final long offset, final long length) throws IOException {
@@ -492,4 +627,66 @@ public class WebHdfsFileSystem extends HftpFileSystem {
     final Map<?, ?> m = run(op, p);
     final Map<?, ?> m = run(op, p);
     return JsonUtil.toMD5MD5CRC32FileChecksum(m);
     return JsonUtil.toMD5MD5CRC32FileChecksum(m);
   }
   }
+
+  private static final DtSelector webhdfspTokenSelector = new DtSelector();
+
+  private static class DtSelector
+      extends AbstractDelegationTokenSelector<DelegationTokenIdentifier> {
+    private DtSelector() {
+      super(TOKEN_KIND);
+    }
+  }
+
+  /** Delegation token renewer. */
+  public static class DtRenewer extends TokenRenewer {
+    @Override
+    public boolean handleKind(Text kind) {
+      return kind.equals(TOKEN_KIND);
+    }
+  
+    @Override
+    public boolean isManaged(Token<?> token) throws IOException {
+      return true;
+    }
+
+    private static WebHdfsFileSystem getWebHdfs(
+        final Token<?> token, final Configuration conf
+        ) throws IOException, InterruptedException, URISyntaxException {
+      
+      final InetSocketAddress nnAddr =  NetUtils.createSocketAddr(
+          token.getService().toString());
+      final URI uri = DFSUtil.createUri(WebHdfsFileSystem.SCHEME, nnAddr);
+      return (WebHdfsFileSystem)FileSystem.get(uri, conf);
+    }
+
+    @Override
+    public long renew(final Token<?> token, final Configuration conf
+        ) throws IOException, InterruptedException {
+      final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      // update the kerberos credentials, if they are coming from a keytab
+      ugi.checkTGTAndReloginFromKeytab();
+
+      try {
+        WebHdfsFileSystem webhdfs = getWebHdfs(token, conf);
+        return webhdfs.renewDelegationToken(token);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+    }
+  
+    @Override
+    public void cancel(final Token<?> token, final Configuration conf
+        ) throws IOException, InterruptedException {
+      final UserGroupInformation ugi = UserGroupInformation.getLoginUser();
+      // update the kerberos credentials, if they are coming from a keytab
+      ugi.checkTGTAndReloginFromKeytab();
+
+      try {
+        final WebHdfsFileSystem webhdfs = getWebHdfs(token, conf);
+        webhdfs.cancelDelegationToken(token);
+      } catch (URISyntaxException e) {
+        throw new IOException(e);
+      }
+    }
+  }
 }
 }

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/resources/PutOpParam.java

@@ -33,6 +33,9 @@ public class PutOpParam extends HttpOpParam<PutOpParam.Op> {
     SETPERMISSION(false, HttpURLConnection.HTTP_OK),
     SETPERMISSION(false, HttpURLConnection.HTTP_OK),
     SETTIMES(false, HttpURLConnection.HTTP_OK),
     SETTIMES(false, HttpURLConnection.HTTP_OK),
     
     
+    RENEWDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
+    CANCELDELEGATIONTOKEN(false, HttpURLConnection.HTTP_OK),
+    
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
     NULL(false, HttpURLConnection.HTTP_NOT_IMPLEMENTED);
 
 
     final boolean doOutput;
     final boolean doOutput;

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenRenewer

@@ -1,3 +1,4 @@
 org.apache.hadoop.hdfs.DFSClient$Renewer
 org.apache.hadoop.hdfs.DFSClient$Renewer
 org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
 org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier$Renewer
-org.apache.hadoop.hdfs.HftpFileSystem$TokenManager
+org.apache.hadoop.hdfs.HftpFileSystem$TokenManager
+org.apache.hadoop.hdfs.web.WebHdfsFileSystem$DtRenewer