HDFS-13588. Fix TestFsDatasetImpl test failures on Windows. Contributed by Xiao Liang.
(cherry picked from commit c0c9b7a8ef
)
This commit is contained in:
parent
6db710b9d8
commit
8d8ef081a2
|
@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.FileSystemTestHelper;
|
import org.apache.hadoop.fs.FileSystemTestHelper;
|
||||||
|
import org.apache.hadoop.fs.FileUtil;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.StorageType;
|
import org.apache.hadoop.fs.StorageType;
|
||||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||||
|
@ -654,7 +655,8 @@ public class TestFsDatasetImpl {
|
||||||
TimeUnit.MILLISECONDS);
|
TimeUnit.MILLISECONDS);
|
||||||
config.setInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, 1);
|
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();
|
cluster.waitActive();
|
||||||
FileSystem fs = cluster.getFileSystem();
|
FileSystem fs = cluster.getFileSystem();
|
||||||
DataNode dataNode = cluster.getDataNodes().get(0);
|
DataNode dataNode = cluster.getDataNodes().get(0);
|
||||||
|
@ -676,7 +678,7 @@ public class TestFsDatasetImpl {
|
||||||
// Remove write and execute access so that checkDiskErrorThread detects
|
// Remove write and execute access so that checkDiskErrorThread detects
|
||||||
// this volume is bad.
|
// this volume is bad.
|
||||||
finalizedDir.setExecutable(false);
|
finalizedDir.setExecutable(false);
|
||||||
finalizedDir.setWritable(false);
|
assertTrue(FileUtil.setWritable(finalizedDir, false));
|
||||||
}
|
}
|
||||||
Assert.assertTrue("Reference count for the volume should be greater "
|
Assert.assertTrue("Reference count for the volume should be greater "
|
||||||
+ "than 0", volume.getReferenceCount() > 0);
|
+ "than 0", volume.getReferenceCount() > 0);
|
||||||
|
@ -697,7 +699,7 @@ public class TestFsDatasetImpl {
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
GenericTestUtils.assertExceptionContains(info.getXferAddr(), ioe);
|
GenericTestUtils.assertExceptionContains(info.getXferAddr(), ioe);
|
||||||
}
|
}
|
||||||
finalizedDir.setWritable(true);
|
assertTrue(FileUtil.setWritable(finalizedDir, true));
|
||||||
finalizedDir.setExecutable(true);
|
finalizedDir.setExecutable(true);
|
||||||
} finally {
|
} finally {
|
||||||
cluster.shutdown();
|
cluster.shutdown();
|
||||||
|
|
Loading…
Reference in New Issue