|
@@ -17,6 +17,7 @@
|
|
*/
|
|
*/
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
|
|
+import org.apache.hadoop.fs.BlockLocation;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.hdfs.DFSStripedOutputStream;
|
|
import org.apache.hadoop.hdfs.DFSStripedOutputStream;
|
|
@@ -65,6 +66,8 @@ import java.util.UUID;
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
+import static org.junit.Assert.assertFalse;
|
|
|
|
+import static org.junit.Assert.assertTrue;
|
|
|
|
|
|
public class TestAddStripedBlocks {
|
|
public class TestAddStripedBlocks {
|
|
private final ErasureCodingPolicy ecPolicy =
|
|
private final ErasureCodingPolicy ecPolicy =
|
|
@@ -476,4 +479,25 @@ public class TestAddStripedBlocks {
|
|
Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
|
|
Assert.assertEquals(3, bm.getCorruptReplicas(stored).size());
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ @Test
|
|
|
|
+ public void testStripedFlagInBlockLocation() throws IOException {
|
|
|
|
+ Path replicated = new Path("/blockLocation/replicated");
|
|
|
|
+ try (FSDataOutputStream out =
|
|
|
|
+ dfs.createFile(replicated).replicate().recursive().build()) {
|
|
|
|
+ out.write("this is a replicated file".getBytes());
|
|
|
|
+ }
|
|
|
|
+ BlockLocation[] locations = dfs.getFileBlockLocations(replicated, 0, 100);
|
|
|
|
+ assertEquals("There should be exactly one Block present",
|
|
|
|
+ 1, locations.length);
|
|
|
|
+ assertFalse("The file is Striped", locations[0].isStriped());
|
|
|
|
+
|
|
|
|
+ Path striped = new Path("/blockLocation/striped");
|
|
|
|
+ try (FSDataOutputStream out = dfs.createFile(striped).recursive().build()) {
|
|
|
|
+ out.write("this is a striped file".getBytes());
|
|
|
|
+ }
|
|
|
|
+ locations = dfs.getFileBlockLocations(striped, 0, 100);
|
|
|
|
+ assertEquals("There should be exactly one Block present",
|
|
|
|
+ 1, locations.length);
|
|
|
|
+ assertTrue("The file is not Striped", locations[0].isStriped());
|
|
|
|
+ }
|
|
}
|
|
}
|