HDFS-13588. Fix TestFsDatasetImpl test failures on Windows. Contributed by Xiao Liang.

This commit is contained in:
Inigo Goiri 2018-05-23 09:48:35 -07:00
parent cedc28d4ab
commit 411a2f609b

View File

@ -25,6 +25,7 @@
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys;
@ -260,7 +261,8 @@ public void testRemoveVolumes() throws IOException {
conf.get(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY).split(",");
final String volumePathToRemove = dataDirs[0];
Set<File> volumesToRemove = new HashSet<>();
volumesToRemove.add(StorageLocation.parse(volumePathToRemove).getFile());
volumesToRemove.add(StorageLocation.parse(volumePathToRemove).getFile()
.getAbsoluteFile());
dataset.removeVolumes(volumesToRemove, true);
int expectedNumVolumes = dataDirs.length - 1;
@ -313,7 +315,7 @@ public void testRemoveNewlyAddedVolume() throws IOException {
when(storage.getNumStorageDirs()).thenReturn(numExistingVolumes + 1);
when(storage.getStorageDir(numExistingVolumes)).thenReturn(sd);
Set<File> volumesToRemove = new HashSet<>();
volumesToRemove.add(loc.getFile());
volumesToRemove.add(loc.getFile().getAbsoluteFile());
dataset.removeVolumes(volumesToRemove, true);
assertEquals(numExistingVolumes, getNumVolumes());
}
@ -322,7 +324,7 @@ public void testRemoveNewlyAddedVolume() throws IOException {
public void testAddVolumeFailureReleasesInUseLock() throws IOException {
FsDatasetImpl spyDataset = spy(dataset);
FsVolumeImpl mockVolume = mock(FsVolumeImpl.class);
File badDir = new File(BASE_DIR, "bad");
File badDir = new File(BASE_DIR, "bad").getAbsoluteFile();
badDir.mkdirs();
doReturn(mockVolume).when(spyDataset)
.createFsVolume(anyString(), any(File.class), any(StorageType.class));
@ -578,7 +580,7 @@ public void run() {
Set<File> volumesToRemove = new HashSet<>();
try {
volumesToRemove.add(StorageLocation.parse(
dataset.getVolume(eb).getBasePath()).getFile());
dataset.getVolume(eb).getBasePath()).getFile().getAbsoluteFile());
} catch (Exception e) {
LOG.info("Problem preparing volumes to remove: ", e);
Assert.fail("Exception in remove volume thread, check log for " +
@ -638,7 +640,8 @@ public void testCleanShutdownOfVolume() throws Exception {
TimeUnit.MILLISECONDS);
config.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
cluster = new MiniDFSCluster.Builder(config,
GenericTestUtils.getRandomizedTestDir()).numDataNodes(1).build();
cluster.waitActive();
FileSystem fs = cluster.getFileSystem();
DataNode dataNode = cluster.getDataNodes().get(0);
@ -660,7 +663,7 @@ public void testCleanShutdownOfVolume() throws Exception {
// Remove write and execute access so that checkDiskErrorThread detects
// this volume is bad.
finalizedDir.setExecutable(false);
finalizedDir.setWritable(false);
assertTrue(FileUtil.setWritable(finalizedDir, false));
}
Assert.assertTrue("Reference count for the volume should be greater "
+ "than 0", volume.getReferenceCount() > 0);
@ -681,10 +684,10 @@ public void testCleanShutdownOfVolume() throws Exception {
} catch (IOException ioe) {
GenericTestUtils.assertExceptionContains(info.getXferAddr(), ioe);
}
finalizedDir.setWritable(true);
finalizedDir.setExecutable(true);
assertTrue(FileUtil.setWritable(finalizedDir, true));
} finally {
cluster.shutdown();
cluster.shutdown();
}
}