diff --git a/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java b/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java index f15bf04..dbbc874 100644 --- a/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java +++ b/hadoop-swiftfs/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java @@ -133,7 +133,7 @@ public class SwiftNativeFileSystemStore { InputStream inputStream, long length) throws IOException { - String stringPath = path.toUri().toString(); + String stringPath = path.toUri().getPath(); String partitionFilename = SwiftUtils.partitionFilenameFromNumber( partNumber); if (stringPath.endsWith("/")) { diff --git a/hadoop-swiftfs/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java b/hadoop-swiftfs/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java index 9b223f8..6fe039c 100644 --- a/hadoop-swiftfs/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java +++ b/hadoop-swiftfs/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java @@ -439,4 +439,40 @@ public class TestSwiftFileSystemPartitionedUploads extends } + /** + * Test writes partitioned file writing that path is qualified. + * @throws Throwable + */ + @Test(timeout = SWIFT_BULK_IO_TEST_TIMEOUT) + public void testQualifiedPath() throws Throwable { + final Path path = path("/test/qualifiedPath"); + int len = PART_SIZE_BYTES * 4; + final byte[] src = SwiftTestUtils.dataset(len, 32, 144); + FSDataOutputStream out = fs.create(path, + false, + getBufferSize(), + (short) 1, + BLOCK_SIZE); + + out.write(src, 0, src.length); + int expected = + getExpectedPartitionsWritten(len, PART_SIZE_BYTES, true); + out.close(); + assertPartitionsWritten("write completed", out, expected); + assertEquals("too few bytes written", len, + SwiftNativeFileSystem.getBytesWritten(out)); + assertEquals("too few bytes uploaded", len, + SwiftNativeFileSystem.getBytesUploaded(out)); + //now we verify that the data comes back. If it + //doesn't, it means that the ordering of the partitions + //isn't right + byte[] dest = readDataset(fs, path, len); + //compare data + SwiftTestUtils.compareByteArrays(src, dest, len); + //finally, check the data + FileStatus[] stats = fs.listStatus(path); + assertEquals("wrong entry count in " + + SwiftTestUtils.dumpStats(path.toString(), stats), + expected, stats.length); + } }