HDFS-7950. Fix TestFsDatasetImpl#testAddVolumes failure on Windows. (Contributed by Xiaoyu Yao)

This commit is contained in:
Arpit Agarwal 2015-03-18 12:33:59 -07:00
parent 24d8c6f355
commit 97341a4a1d
2 changed files with 10 additions and 4 deletions

View File

@ -875,6 +875,9 @@ Release 2.7.0 - UNRELEASED
HDFS-7948. TestDataNodeHotSwapVolumes#testAddVolumeFailures failed on
Windows. (Xiaoyu Yao via Arpit Agarwal)
HDFS-7950. Fix TestFsDatasetImpl#testAddVolumes failure on Windows.
(Xiaoyu Yao via Arpit Agarwal)
BREAKDOWN OF HDFS-7584 SUBTASKS AND RELATED JIRAS
HDFS-7720. Quota by Storage Type API, tools and ClientNameNode

View File

@ -21,6 +21,7 @@ import com.google.common.collect.Lists;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystemTestHelper;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.HdfsConfiguration;
@ -111,7 +112,7 @@ public class TestFsDatasetImpl {
List<String> dirStrings = new ArrayList<String>();
for (int i = 0; i < numDirs; i++) {
File loc = new File(BASE_DIR + "/data" + i);
dirStrings.add(loc.toString());
dirStrings.add(new Path(loc.toString()).toUri().toString());
loc.mkdirs();
dirs.add(createStorageDirectory(loc));
when(storage.getStorageDir(i)).thenReturn(dirs.get(i));
@ -158,8 +159,9 @@ public class TestFsDatasetImpl {
}
for (int i = 0; i < numNewVolumes; i++) {
String path = BASE_DIR + "/newData" + i;
expectedVolumes.add(path);
StorageLocation loc = StorageLocation.parse(path);
String pathUri = new Path(path).toUri().toString();
expectedVolumes.add(new File(pathUri).toString());
StorageLocation loc = StorageLocation.parse(pathUri);
Storage.StorageDirectory sd = createStorageDirectory(new File(path));
DataStorage.VolumeBuilder builder =
new DataStorage.VolumeBuilder(storage, sd);
@ -178,7 +180,8 @@ public class TestFsDatasetImpl {
actualVolumes.add(
dataset.getVolumes().get(numExistingVolumes + i).getBasePath());
}
assertEquals(actualVolumes, expectedVolumes);
assertEquals(actualVolumes.size(), expectedVolumes.size());
assertTrue(actualVolumes.containsAll(expectedVolumes));
}
@Test(timeout = 30000)