Browse Source

HDFS-7880. Remove the tests for legacy Web UI in branch-2. Contributed by Brahma Reddy Battula.

(cherry picked from commit 6c80a3d34a3abb9f5539741a0a62823cf1c23885)
Akira Ajisaka 10 years ago
parent
commit
718cd0f1f4

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

@@ -821,6 +821,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7830. DataNode does not release the volume lock when adding a volume
     fails. (Lei Xu via Colin P. Mccabe)
 
+    HDFS-7880. Remove the tests for legacy Web UI in branch-2.
+    (Brahma Reddy Battula via aajisaka)
+
     BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
 
       HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java

@@ -85,6 +85,7 @@ public class TestDecommission {
   static final int NAMENODE_REPLICATION_INTERVAL = 1; //replication interval
 
   final Random myrand = new Random();
+  Path dir;
   Path hostsFile;
   Path excludeFile;
   FileSystem localFileSys;
@@ -97,7 +98,7 @@ public class TestDecommission {
     // Set up the hosts/exclude files.
     localFileSys = FileSystem.getLocal(conf);
     Path workingDir = localFileSys.getWorkingDirectory();
-    Path dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission");
+    dir = new Path(workingDir, PathUtils.getTestDirName(getClass()) + "/work-dir/decommission");
     hostsFile = new Path(dir, "hosts");
     excludeFile = new Path(dir, "exclude");
     
@@ -118,7 +119,7 @@ public class TestDecommission {
   
   @After
   public void teardown() throws IOException {
-    cleanupFile(localFileSys, excludeFile.getParent());
+    cleanupFile(localFileSys, dir);
     if (cluster != null) {
       cluster.shutdown();
     }

+ 2 - 56
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/TestNNWithQJM.java

@@ -17,20 +17,17 @@
  */
 package org.apache.hadoop.hdfs.qjournal;
 
-import static org.junit.Assert.*;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.io.File;
 import java.io.IOException;
-import java.net.URL;
-import java.util.regex.Pattern;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -204,55 +201,4 @@ public class TestNNWithQJM {
           "Unable to start log segment 1: too few journals", ioe);
     }
   }
-
-  @Test (timeout = 30000)
-  public void testWebPageHasQjmInfo() throws Exception {
-    conf.set(DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY,
-        MiniDFSCluster.getBaseDirectory() + "/TestNNWithQJM/image");
-    conf.set(DFSConfigKeys.DFS_NAMENODE_EDITS_DIR_KEY,
-        mjc.getQuorumJournalURI("myjournal").toString());
-    // Speed up the test
-    conf.setInt(
-        CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 1);
-    
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-      .numDataNodes(0)
-      .manageNameDfsDirs(false)
-      .build();
-    try {
-      URL url = new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp");
-      
-      cluster.getFileSystem().mkdirs(TEST_PATH);
-      
-      String contents = DFSTestUtil.urlGet(url); 
-      assertTrue(contents.contains("QJM to ["));
-      assertTrue(contents.contains("Written txid 2"));
-
-      // Stop one JN, do another txn, and make sure it shows as behind
-      // stuck behind the others.
-      mjc.getJournalNode(0).stopAndJoin(0);
-      
-      cluster.getFileSystem().delete(TEST_PATH, true);
-      
-      contents = DFSTestUtil.urlGet(url); 
-      System.out.println(contents);
-      assertTrue(Pattern.compile("1 txns/\\d+ms behind").matcher(contents)
-          .find());
-
-      // Restart NN while JN0 is still down.
-      cluster.restartNameNode();
-
-      contents = DFSTestUtil.urlGet(url); 
-      System.out.println(contents);
-      assertTrue(Pattern.compile("never written").matcher(contents)
-          .find());
-      
-
-    } finally {
-      cluster.shutdown();
-    }
-
-  }
 }

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -169,11 +169,6 @@ public class TestJournalNode {
     assertTrue("Bad contents: " + pageContents,
         pageContents.contains(
             "Hadoop:service=JournalNode,name=JvmMetrics"));
-    
-    // Check JSP page.
-    pageContents = DFSTestUtil.urlGet(
-        new URL(urlRoot + "/journalstatus.jsp"));
-    assertTrue(pageContents.contains("JournalNode"));
 
     // Create some edits on server side
     byte[] EDITS_DATA = QJMTestUtil.createTxnData(1, 3);

+ 0 - 18
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/common/TestJspHelper.java

@@ -134,24 +134,6 @@ public class TestJspHelper {
     Assert.assertEquals(expected, tokenInUgi.getService().toString());
   }
   
-  
-  @Test
-  public void testDelegationTokenUrlParam() {
-    conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(conf);
-    String tokenString = "xyzabc";
-    String delegationTokenParam = JspHelper
-        .getDelegationTokenUrlParam(tokenString);
-    //Security is enabled
-    Assert.assertEquals(JspHelper.SET_DELEGATION + "xyzabc",
-        delegationTokenParam);
-    conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "simple");
-    UserGroupInformation.setConfiguration(conf);
-    delegationTokenParam = JspHelper
-        .getDelegationTokenUrlParam(tokenString);
-    //Empty string must be returned because security is disabled.
-    Assert.assertEquals("", delegationTokenParam);
-  }
 
   @Test
   public void testGetUgiFromToken() throws IOException {

+ 0 - 59
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestClusterJspHelper.java

@@ -1,59 +0,0 @@
-/**
- * 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.server.namenode;
-
-import static org.junit.Assert.*;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.ClusterStatus;
-import org.apache.hadoop.hdfs.server.namenode.ClusterJspHelper.DecommissionStatus;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-public class TestClusterJspHelper {
-
-  private MiniDFSCluster cluster;
-  private Configuration conf;
-    
-  @Before
-  public void setUp() throws Exception {
-    conf = new Configuration();  
-    cluster = new MiniDFSCluster.Builder(conf).build();
-    cluster.waitClusterUp();
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    if (cluster != null)
-      cluster.shutdown();    
-  }
-  
-  @Test(timeout = 15000)
-  public void testClusterJspHelperReports() {
-    ClusterJspHelper clusterJspHelper = new ClusterJspHelper();
-    ClusterStatus clusterStatus = clusterJspHelper
-     .generateClusterHealthReport();
-    assertNotNull("testClusterJspHelperReports ClusterStatus is null",
-        clusterStatus);       
-    DecommissionStatus decommissionStatus = clusterJspHelper
-        .generateDecommissioningReport();
-    assertNotNull("testClusterJspHelperReports DecommissionStatus is null",
-        decommissionStatus);    
-  }
-}

+ 10 - 17
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestHostsFiles.java

@@ -20,10 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode;
 import static org.junit.Assert.assertTrue;
 
 import java.lang.management.ManagementFactory;
-import java.net.InetSocketAddress;
-import java.net.URL;
 
-import org.apache.commons.lang.StringEscapeUtils;
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -37,9 +37,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.junit.Test;
 
-import javax.management.MBeanServer;
-import javax.management.ObjectName;
-
 /**
  * DFS_HOSTS and DFS_HOSTS_EXCLUDE tests
  * 
@@ -77,7 +74,7 @@ public class TestHostsFiles {
   }
 
   @Test
-  public void testHostsExcludeDfshealthJsp() throws Exception {
+  public void testHostsExcludeInUI() throws Exception {
     Configuration conf = getConf();
     short REPLICATION_FACTOR = 2;
     final Path filePath = new Path("/testFile");
@@ -122,16 +119,12 @@ public class TestHostsFiles {
       // Check the block still has sufficient # replicas across racks
       DFSTestUtil.waitForReplication(cluster, b, 2, REPLICATION_FACTOR, 0);
       
-      InetSocketAddress nnHttpAddress = cluster.getNameNode().getHttpAddress();
-      LOG.info("nnaddr = '" + nnHttpAddress + "'");
-      String nnHostName = nnHttpAddress.getHostName();
-      URL nnjsp = new URL("http://" + nnHostName + ":" + nnHttpAddress.getPort() + "/dfshealth.jsp");
-      LOG.info("fetching " + nnjsp);
-      String dfshealthPage = StringEscapeUtils.unescapeHtml(DFSTestUtil.urlGet(nnjsp));
-      LOG.info("got " + dfshealthPage);
-      assertTrue("dfshealth should contain " + nnHostName + ", got:" + dfshealthPage,
-          dfshealthPage.contains(nnHostName));
-
+      MBeanServer mbs = ManagementFactory.getPlatformMBeanServer();
+      ObjectName mxbeanName =
+          new ObjectName("Hadoop:service=NameNode,name=NameNodeInfo");
+      String nodes = (String) mbs.getAttribute(mxbeanName, "LiveNodes");
+      assertTrue("Live nodes should contain the decommissioned node",
+          nodes.contains("Decommissioned"));
     } finally {
       cluster.shutdown();
     }

+ 0 - 87
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeJspHelper.java

@@ -1,87 +0,0 @@
-/**
- * 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.server.namenode;
-
-import com.google.common.collect.ImmutableSet;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import javax.servlet.http.HttpServletRequest;
-import java.io.IOException;
-
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
-
-public class TestNameNodeJspHelper {
-
-  private static final int DATA_NODES_AMOUNT = 2;
-  
-  private static MiniDFSCluster cluster;
-  private static Configuration conf;
-  
-  @BeforeClass
-  public static void setUp() throws Exception {
-    conf = new HdfsConfiguration();
-    cluster = new MiniDFSCluster.Builder(conf)
-        .numDataNodes(DATA_NODES_AMOUNT).build();
-    cluster.waitClusterUp();
-  }
-
-  @AfterClass
-  public static void tearDown() throws Exception {
-    if (cluster != null)
-      cluster.shutdown();
-  }
-
-  @Test
-  public void testDelegationToken() throws IOException, InterruptedException {
-    NamenodeProtocols nn = cluster.getNameNodeRpc();
-    HttpServletRequest request = mock(HttpServletRequest.class);
-    UserGroupInformation ugi = UserGroupInformation.createRemoteUser("auser");
-    String tokenString = NamenodeJspHelper.getDelegationToken(nn, request,
-        conf, ugi);
-    // tokenString returned must be null because security is disabled
-    Assert.assertEquals(null, tokenString);
-  }
-
-  @Test(timeout = 15000)
-  public void testGetRandomDatanode() {
-    NameNode nameNode = cluster.getNameNode();
-    ImmutableSet.Builder<String> builder = ImmutableSet.builder();
-    for (DataNode dataNode : cluster.getDataNodes()) {
-      builder.add(dataNode.getDisplayName());
-    }
-    ImmutableSet<String> set = builder.build();
-
-    for (int i = 0; i < 10; i++) {
-      DatanodeDescriptor dnDescriptor = NamenodeJspHelper
-          .getRandomDatanode(nameNode);
-      assertTrue("testGetRandomDatanode error",
-          set.contains(dnDescriptor.toString()));
-    }
-  }
-}

+ 11 - 20
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java

@@ -19,20 +19,25 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
 
+import java.io.IOException;
+import java.lang.management.ManagementFactory;
+
+import javax.management.AttributeNotFoundException;
+import javax.management.InstanceNotFoundException;
+import javax.management.MBeanException;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import javax.management.*;
-import java.io.IOException;
-import java.lang.management.ManagementFactory;
-import java.net.URL;
-
 public class TestSecondaryWebUi {
   
   private static MiniDFSCluster cluster;
@@ -78,18 +83,4 @@ public class TestSecondaryWebUi {
     Assert.assertArrayEquals(checkpointEditlogDir,
             snn.getCheckpointEditlogDirectories());
   }
-
-  @Test
-  public void testSecondaryWebUiJsp()
-          throws IOException, MalformedObjectNameException,
-                 AttributeNotFoundException, MBeanException,
-                 ReflectionException, InstanceNotFoundException {
-    String pageContents = DFSTestUtil.urlGet(new URL("http://localhost:" +
-        SecondaryNameNode.getHttpAddress(conf).getPort() + "/status.jsp"));
-    Assert.assertTrue("Didn't find \"Last Checkpoint\"",
-        pageContents.contains("Last Checkpoint"));
-    Assert.assertTrue("Didn't find Checkpoint Transactions: 500",
-        pageContents.contains("Checkpoint Transactions: 500"));
-
-  }
 }

+ 0 - 74
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestHAWebUI.java

@@ -1,74 +0,0 @@
-/**
- * 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.server.namenode.ha;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import java.net.URL;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSNNTopology;
-import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.junit.Test;
-
-public class TestHAWebUI {
-
-  /**
-   * Tests that the web UI of the name node provides a link to browse the file
-   * system and summary of under-replicated blocks only in active state
-   * 
-   */
-  @Test
-  public void testLinkAndClusterSummary() throws Exception {
-    Configuration conf = new Configuration();
-
-    MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
-        .nnTopology(MiniDFSNNTopology.simpleHATopology()).numDataNodes(0)
-        .build();
-    try {
-      cluster.waitActive();
-
-      cluster.transitionToActive(0);
-      String pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp"));
-      assertTrue(pageContents.contains("Browse the filesystem"));
-      assertTrue(pageContents.contains("Number of Under-Replicated Blocks"));
-
-      cluster.transitionToStandby(0);
-      pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp"));
-      assertFalse(pageContents.contains("Browse the filesystem"));
-      assertFalse(pageContents.contains("Number of Under-Replicated Blocks"));
-
-      cluster.transitionToActive(0);
-      pageContents = DFSTestUtil.urlGet(new URL("http://localhost:"
-          + NameNode.getHttpAddress(cluster.getConfiguration(0)).getPort()
-          + "/dfshealth.jsp"));
-      assertTrue(pageContents.contains("Browse the filesystem"));
-      assertTrue(pageContents.contains("Number of Under-Replicated Blocks"));
-
-    } finally {
-      cluster.shutdown();
-    }
-  }
-}