[120] | 1 | /** |
---|
| 2 | * Licensed to the Apache Software Foundation (ASF) under one |
---|
| 3 | * or more contributor license agreements. See the NOTICE file |
---|
| 4 | * distributed with this work for additional information |
---|
| 5 | * regarding copyright ownership. The ASF licenses this file |
---|
| 6 | * to you under the Apache License, Version 2.0 (the |
---|
| 7 | * "License"); you may not use this file except in compliance |
---|
| 8 | * with the License. You may obtain a copy of the License at |
---|
| 9 | * |
---|
| 10 | * http://www.apache.org/licenses/LICENSE-2.0 |
---|
| 11 | * |
---|
| 12 | * Unless required by applicable law or agreed to in writing, software |
---|
| 13 | * distributed under the License is distributed on an "AS IS" BASIS, |
---|
| 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
---|
| 15 | * See the License for the specific language governing permissions and |
---|
| 16 | * limitations under the License. |
---|
| 17 | */ |
---|
| 18 | package org.apache.hadoop.hdfs; |
---|
| 19 | |
---|
| 20 | import java.io.IOException; |
---|
| 21 | import java.net.InetSocketAddress; |
---|
| 22 | import java.net.URL; |
---|
| 23 | |
---|
| 24 | import org.apache.commons.logging.Log; |
---|
| 25 | import org.apache.commons.logging.LogFactory; |
---|
| 26 | import org.apache.hadoop.conf.Configuration; |
---|
| 27 | import org.apache.hadoop.fs.ChecksumException; |
---|
| 28 | import org.apache.hadoop.fs.FSDataInputStream; |
---|
| 29 | import org.apache.hadoop.fs.FileSystem; |
---|
| 30 | import org.apache.hadoop.fs.Path; |
---|
| 31 | |
---|
| 32 | import junit.framework.TestCase; |
---|
| 33 | |
---|
| 34 | /** |
---|
| 35 | * The test makes sure that NameNode detects presense blocks that do not have |
---|
| 36 | * any valid replicas. In addition, it verifies that HDFS front page displays |
---|
| 37 | * a warning in such a case. |
---|
| 38 | */ |
---|
| 39 | public class TestMissingBlocksAlert extends TestCase { |
---|
| 40 | |
---|
| 41 | private static final Log LOG = |
---|
| 42 | LogFactory.getLog(TestMissingBlocksAlert.class); |
---|
| 43 | |
---|
| 44 | public void testMissingBlocksAlert() throws IOException, |
---|
| 45 | InterruptedException { |
---|
| 46 | |
---|
| 47 | MiniDFSCluster cluster = null; |
---|
| 48 | |
---|
| 49 | try { |
---|
| 50 | Configuration conf = new Configuration(); |
---|
| 51 | //minimize test delay |
---|
| 52 | conf.setInt("dfs.replication.interval", 0); |
---|
| 53 | int fileLen = 10*1024; |
---|
| 54 | |
---|
| 55 | //start a cluster with single datanode |
---|
| 56 | cluster = new MiniDFSCluster(conf, 1, true, null); |
---|
| 57 | cluster.waitActive(); |
---|
| 58 | |
---|
| 59 | DistributedFileSystem dfs = |
---|
| 60 | (DistributedFileSystem) cluster.getFileSystem(); |
---|
| 61 | |
---|
| 62 | // create a normal file |
---|
| 63 | DFSTestUtil.createFile(dfs, new Path("/testMissingBlocksAlert/file1"), |
---|
| 64 | fileLen, (short)3, 0); |
---|
| 65 | |
---|
| 66 | Path corruptFile = new Path("/testMissingBlocks/corruptFile"); |
---|
| 67 | DFSTestUtil.createFile(dfs, corruptFile, fileLen, (short)3, 0); |
---|
| 68 | |
---|
| 69 | |
---|
| 70 | // Corrupt the block |
---|
| 71 | String block = DFSTestUtil.getFirstBlock(dfs, corruptFile).getBlockName(); |
---|
| 72 | TestDatanodeBlockScanner.corruptReplica(block, 0); |
---|
| 73 | |
---|
| 74 | // read the file so that the corrupt block is reported to NN |
---|
| 75 | FSDataInputStream in = dfs.open(corruptFile); |
---|
| 76 | try { |
---|
| 77 | in.readFully(new byte[fileLen]); |
---|
| 78 | } catch (ChecksumException ignored) { // checksum error is expected. |
---|
| 79 | } |
---|
| 80 | in.close(); |
---|
| 81 | |
---|
| 82 | LOG.info("Waiting for missing blocks count to increase..."); |
---|
| 83 | |
---|
| 84 | while (dfs.getMissingBlocksCount() <= 0) { |
---|
| 85 | Thread.sleep(100); |
---|
| 86 | } |
---|
| 87 | assertTrue(dfs.getMissingBlocksCount() == 1); |
---|
| 88 | |
---|
| 89 | |
---|
| 90 | // Now verify that it shows up on webui |
---|
| 91 | URL url = new URL("http://" + conf.get("dfs.http.address") + |
---|
| 92 | "/dfshealth.jsp"); |
---|
| 93 | String dfsFrontPage = DFSTestUtil.urlGet(url); |
---|
| 94 | String warnStr = "WARNING : There are about "; |
---|
| 95 | assertTrue("HDFS Front page does not contain expected warning", |
---|
| 96 | dfsFrontPage.contains(warnStr + "1 missing blocks")); |
---|
| 97 | |
---|
| 98 | // now do the reverse : remove the file expect the number of missing |
---|
| 99 | // blocks to go to zero |
---|
| 100 | |
---|
| 101 | dfs.delete(corruptFile, true); |
---|
| 102 | |
---|
| 103 | LOG.info("Waiting for missing blocks count to be zero..."); |
---|
| 104 | while (dfs.getMissingBlocksCount() > 0) { |
---|
| 105 | Thread.sleep(100); |
---|
| 106 | } |
---|
| 107 | |
---|
| 108 | // and make sure WARNING disappears |
---|
| 109 | // Now verify that it shows up on webui |
---|
| 110 | dfsFrontPage = DFSTestUtil.urlGet(url); |
---|
| 111 | assertFalse("HDFS Front page contains unexpected warning", |
---|
| 112 | dfsFrontPage.contains(warnStr)); |
---|
| 113 | } finally { |
---|
| 114 | if (cluster != null) { |
---|
| 115 | cluster.shutdown(); |
---|
| 116 | } |
---|
| 117 | } |
---|
| 118 | } |
---|
| 119 | } |
---|