Pārlūkot izejas kodu

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

Arpit Agarwal 10 gadi atpakaļ
vecāks
revīzija
ae6d273d63

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -858,6 +858,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

+ 7 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestFsDatasetImpl.java

@@ -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)