|
@@ -19,23 +19,22 @@
|
|
|
package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
import com.google.common.base.Supplier;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Collection;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileUtil;
|
|
|
-import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSTestUtil;
|
|
|
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
|
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|
|
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeID;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNode;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi.FsVolumeReferences;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
|
|
import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
|
|
@@ -46,7 +45,6 @@ import org.apache.log4j.Level;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
|
|
|
-import java.io.BufferedOutputStream;
|
|
|
import java.io.BufferedReader;
|
|
|
import java.io.BufferedWriter;
|
|
|
import java.io.File;
|
|
@@ -55,13 +53,11 @@ import java.io.FileReader;
|
|
|
import java.io.IOException;
|
|
|
import java.io.OutputStreamWriter;
|
|
|
import java.io.Writer;
|
|
|
-import java.util.Arrays;
|
|
|
-import java.util.EnumSet;
|
|
|
import java.util.Iterator;
|
|
|
-import java.util.List;
|
|
|
import java.util.UUID;
|
|
|
|
|
|
import static org.hamcrest.core.Is.is;
|
|
|
+import static org.junit.Assert.assertArrayEquals;
|
|
|
import static org.junit.Assert.assertThat;
|
|
|
import static org.junit.Assert.assertTrue;
|
|
|
import static org.junit.Assert.assertNotNull;
|
|
@@ -159,6 +155,8 @@ public class TestNameNodePrunesMissingStorages {
|
|
|
public void testRemovingStorageDoesNotProduceZombies() throws Exception {
|
|
|
Configuration conf = new HdfsConfiguration();
|
|
|
conf.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
|
|
|
+ conf.setInt(DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
+ 1000);
|
|
|
final int NUM_STORAGES_PER_DN = 2;
|
|
|
final MiniDFSCluster cluster = new MiniDFSCluster
|
|
|
.Builder(conf).numDataNodes(3)
|
|
@@ -257,7 +255,7 @@ public class TestNameNodePrunesMissingStorages {
|
|
|
assertEquals(NUM_STORAGES_PER_DN - 1, infos.length);
|
|
|
return true;
|
|
|
}
|
|
|
- }, 10, 30000);
|
|
|
+ }, 1000, 30000);
|
|
|
} finally {
|
|
|
if (cluster != null) {
|
|
|
cluster.shutdown();
|
|
@@ -365,4 +363,60 @@ public class TestNameNodePrunesMissingStorages {
|
|
|
cluster.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout=300000)
|
|
|
+ public void testNameNodePrunesUnreportedStorages() throws Exception {
|
|
|
+ Configuration conf = new HdfsConfiguration();
|
|
|
+ // Create a cluster with one datanode with two storages
|
|
|
+ MiniDFSCluster cluster = new MiniDFSCluster
|
|
|
+ .Builder(conf).numDataNodes(1)
|
|
|
+ .storagesPerDatanode(2)
|
|
|
+ .build();
|
|
|
+ // Create two files to ensure each storage has a block
|
|
|
+ DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file1"),
|
|
|
+ 102400, 102400, 102400, (short)1,
|
|
|
+ 0x1BAD5EE);
|
|
|
+ DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file2"),
|
|
|
+ 102400, 102400, 102400, (short)1,
|
|
|
+ 0x1BAD5EED);
|
|
|
+ // Get the datanode storages and data directories
|
|
|
+ DataNode dn = cluster.getDataNodes().get(0);
|
|
|
+ BlockManager bm = cluster.getNameNode().getNamesystem().getBlockManager();
|
|
|
+ DatanodeDescriptor dnDescriptor = bm.getDatanodeManager().
|
|
|
+ getDatanode(cluster.getDataNodes().get(0).getDatanodeUuid());
|
|
|
+ DatanodeStorageInfo[] dnStoragesInfosBeforeRestart =
|
|
|
+ dnDescriptor.getStorageInfos();
|
|
|
+ Collection<String> oldDirs = new ArrayList<String>(dn.getConf().
|
|
|
+ getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
|
|
|
+ // Keep the first data directory and remove the second.
|
|
|
+ String newDirs = oldDirs.iterator().next();
|
|
|
+ conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
|
|
|
+ // Restart the datanode with the new conf
|
|
|
+ cluster.stopDataNode(0);
|
|
|
+ cluster.startDataNodes(conf, 1, false, null, null);
|
|
|
+ dn = cluster.getDataNodes().get(0);
|
|
|
+ cluster.waitActive();
|
|
|
+ // Assert that the dnDescriptor has both the storages after restart
|
|
|
+ assertArrayEquals(dnStoragesInfosBeforeRestart,
|
|
|
+ dnDescriptor.getStorageInfos());
|
|
|
+ // Assert that the removed storage is marked as FAILED
|
|
|
+ // when DN heartbeats to the NN
|
|
|
+ int numFailedStoragesWithBlocks = 0;
|
|
|
+ DatanodeStorageInfo failedStorageInfo = null;
|
|
|
+ for (DatanodeStorageInfo dnStorageInfo: dnDescriptor.getStorageInfos()) {
|
|
|
+ if (dnStorageInfo.areBlocksOnFailedStorage()) {
|
|
|
+ numFailedStoragesWithBlocks++;
|
|
|
+ failedStorageInfo = dnStorageInfo;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ assertEquals(1, numFailedStoragesWithBlocks);
|
|
|
+ // Heartbeat manager removes the blocks associated with this failed storage
|
|
|
+ bm.getDatanodeManager().getHeartbeatManager().heartbeatCheck();
|
|
|
+ assertTrue(!failedStorageInfo.areBlocksOnFailedStorage());
|
|
|
+ // pruneStorageMap removes the unreported storage
|
|
|
+ cluster.triggerHeartbeats();
|
|
|
+ // Assert that the unreported storage is pruned
|
|
|
+ assertEquals(DataNode.getStorageLocations(dn.getConf()).size(),
|
|
|
+ dnDescriptor.getStorageInfos().length);
|
|
|
+ }
|
|
|
}
|