HDFS-13592. TestNameNodePrunesMissingStorages#testNameNodePrunesUnreportedStorages does not shut down cluster properly. Contributed by Anbang Hu.
(cherry picked from commit 57b893de3d
)
This commit is contained in:
parent
472495ed1e
commit
da5eff8325
|
@ -383,51 +383,60 @@ public class TestNameNodePrunesMissingStorages {
|
||||||
.Builder(conf).numDataNodes(1)
|
.Builder(conf).numDataNodes(1)
|
||||||
.storagesPerDatanode(2)
|
.storagesPerDatanode(2)
|
||||||
.build();
|
.build();
|
||||||
// Create two files to ensure each storage has a block
|
try {
|
||||||
DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file1"),
|
cluster.waitActive();
|
||||||
102400, 102400, 102400, (short)1,
|
// Create two files to ensure each storage has a block
|
||||||
0x1BAD5EE);
|
DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file1"),
|
||||||
DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file2"),
|
102400, 102400, 102400, (short)1,
|
||||||
102400, 102400, 102400, (short)1,
|
0x1BAD5EE);
|
||||||
0x1BAD5EED);
|
DFSTestUtil.createFile(cluster.getFileSystem(), new Path("file2"),
|
||||||
// Get the datanode storages and data directories
|
102400, 102400, 102400, (short)1,
|
||||||
DataNode dn = cluster.getDataNodes().get(0);
|
0x1BAD5EED);
|
||||||
BlockManager bm = cluster.getNameNode().getNamesystem().getBlockManager();
|
// Get the datanode storages and data directories
|
||||||
DatanodeDescriptor dnDescriptor = bm.getDatanodeManager().
|
DataNode dn = cluster.getDataNodes().get(0);
|
||||||
getDatanode(cluster.getDataNodes().get(0).getDatanodeUuid());
|
BlockManager bm =
|
||||||
DatanodeStorageInfo[] dnStoragesInfosBeforeRestart =
|
cluster.getNameNode().getNamesystem().getBlockManager();
|
||||||
dnDescriptor.getStorageInfos();
|
DatanodeDescriptor dnDescriptor = bm.getDatanodeManager().
|
||||||
Collection<String> oldDirs = new ArrayList<String>(dn.getConf().
|
getDatanode(cluster.getDataNodes().get(0).getDatanodeUuid());
|
||||||
getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
|
DatanodeStorageInfo[] dnStoragesInfosBeforeRestart =
|
||||||
// Keep the first data directory and remove the second.
|
dnDescriptor.getStorageInfos();
|
||||||
String newDirs = oldDirs.iterator().next();
|
Collection<String> oldDirs = new ArrayList<String>(dn.getConf().
|
||||||
conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
|
getTrimmedStringCollection(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY));
|
||||||
// Restart the datanode with the new conf
|
// Keep the first data directory and remove the second.
|
||||||
cluster.stopDataNode(0);
|
String newDirs = oldDirs.iterator().next();
|
||||||
cluster.startDataNodes(conf, 1, false, null, null);
|
conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, newDirs);
|
||||||
dn = cluster.getDataNodes().get(0);
|
// Restart the datanode with the new conf
|
||||||
cluster.waitActive();
|
cluster.stopDataNode(0);
|
||||||
// Assert that the dnDescriptor has both the storages after restart
|
cluster.startDataNodes(conf, 1, false, null, null);
|
||||||
assertArrayEquals(dnStoragesInfosBeforeRestart,
|
dn = cluster.getDataNodes().get(0);
|
||||||
dnDescriptor.getStorageInfos());
|
cluster.waitActive();
|
||||||
// Assert that the removed storage is marked as FAILED
|
// Assert that the dnDescriptor has both the storages after restart
|
||||||
// when DN heartbeats to the NN
|
assertArrayEquals(dnStoragesInfosBeforeRestart,
|
||||||
int numFailedStoragesWithBlocks = 0;
|
dnDescriptor.getStorageInfos());
|
||||||
DatanodeStorageInfo failedStorageInfo = null;
|
// Assert that the removed storage is marked as FAILED
|
||||||
for (DatanodeStorageInfo dnStorageInfo: dnDescriptor.getStorageInfos()) {
|
// when DN heartbeats to the NN
|
||||||
if (dnStorageInfo.areBlocksOnFailedStorage()) {
|
int numFailedStoragesWithBlocks = 0;
|
||||||
numFailedStoragesWithBlocks++;
|
DatanodeStorageInfo failedStorageInfo = null;
|
||||||
failedStorageInfo = dnStorageInfo;
|
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);
|
||||||
|
} finally {
|
||||||
|
if (cluster != null) {
|
||||||
|
cluster.shutdown();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
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);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue