Author: wheat9
Date: Wed Apr  2 20:53:01 2014
New Revision: 1584174

URL: http://svn.apache.org/r1584174
Log:
HDFS-5570. Addendum commit for r1584100.

Added:
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
Removed:
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/CancelDelegationTokenServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileChecksumServlets.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileDataServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ListPathsServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RenewDelegationTokenServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/StreamFile.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HftpFileSystem.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/web/HsftpFileSystem.java
    hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/site/apt/Hftp.apt.vm
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/TestFiHftp.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestListPathServlet.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStreamFile.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestByteRangeInputStream.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpDelegationToken.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpFileSystem.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenFetcher.java
    
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestDelegationTokenRemoteFetcher.java

Added: 
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java?rev=1584174&view=auto
==============================================================================
--- 
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
 (added)
+++ 
hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDelegationTokenFetcher.java
 Wed Apr  2 20:53:01 2014
@@ -0,0 +1,93 @@
+/**
+ * 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.tools;
+
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import 
org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.tools.FakeRenewer;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestDelegationTokenFetcher {
+
+  private Configuration conf = new Configuration();
+
+  @Rule
+  public TemporaryFolder f = new TemporaryFolder();
+  private static final String tokenFile = "token";
+
+  /**
+   * try to fetch token without http server with IOException
+   */
+  @Test(expected = IOException.class)
+  public void testTokenFetchFail() throws Exception {
+    WebHdfsFileSystem fs = mock(WebHdfsFileSystem.class);
+    doThrow(new IOException()).when(fs).getDelegationToken(anyString());
+    Path p = new Path(f.getRoot().getAbsolutePath(), tokenFile);
+    DelegationTokenFetcher.saveDelegationToken(conf, fs, null, p);
+  }
+
+  /**
+   * Call fetch token using http server
+   */
+  @Test
+  public void expectedTokenIsRetrievedFromHttp() throws Exception {
+    final Token<DelegationTokenIdentifier> testToken = new 
Token<DelegationTokenIdentifier>(
+        "id".getBytes(), "pwd".getBytes(), FakeRenewer.KIND, new Text(
+            "127.0.0.1:1234"));
+
+    WebHdfsFileSystem fs = mock(WebHdfsFileSystem.class);
+
+    doReturn(testToken).when(fs).getDelegationToken(anyString());
+    Path p = new Path(f.getRoot().getAbsolutePath(), tokenFile);
+    DelegationTokenFetcher.saveDelegationToken(conf, fs, null, p);
+
+    Credentials creds = Credentials.readTokenStorageFile(p, conf);
+    Iterator<Token<?>> itr = creds.getAllTokens().iterator();
+    assertTrue("token not exist error", itr.hasNext());
+
+    Token<?> fetchedToken = itr.next();
+    Assert.assertArrayEquals("token wrong identifier error",
+        testToken.getIdentifier(), fetchedToken.getIdentifier());
+    Assert.assertArrayEquals("token wrong password error",
+        testToken.getPassword(), fetchedToken.getPassword());
+
+    DelegationTokenFetcher.renewTokens(conf, p);
+    Assert.assertEquals(testToken, FakeRenewer.getLastRenewed());
+
+    DelegationTokenFetcher.cancelTokens(conf, p);
+    Assert.assertEquals(testToken, FakeRenewer.getLastCanceled());
+  }
+}


Reply via email to