|
@@ -24,7 +24,9 @@ import java.net.SocketException;
|
|
|
import java.net.SocketTimeoutException;
|
|
|
import java.net.URL;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
+import java.util.Set;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Executors;
|
|
@@ -90,6 +92,8 @@ public class ITestAzureBlobFileSystemListStatus extends
|
|
|
AbstractAbfsIntegrationTest {
|
|
|
private static final int TEST_FILES_NUMBER = 6000;
|
|
|
public static final String TEST_CONTINUATION_TOKEN = "continuation";
|
|
|
+ private static final int TOTAL_NUMBER_OF_PATHS = 11;
|
|
|
+ private static final int NUMBER_OF_UNIQUE_PATHS = 7;
|
|
|
|
|
|
public ITestAzureBlobFileSystemListStatus() throws Exception {
|
|
|
super();
|
|
@@ -197,7 +201,7 @@ public class ITestAzureBlobFileSystemListStatus extends
|
|
|
Mockito.doReturn(spiedStore).when(spiedFs).getAbfsStore();
|
|
|
Mockito.doReturn(spiedClient).when(spiedStore).getClient();
|
|
|
|
|
|
- Mockito.doThrow(new SocketException(CONNECTION_RESET_MESSAGE)).when(spiedClient).filterDuplicateEntriesAndRenamePendingFiles(any(), any());
|
|
|
+ Mockito.doThrow(new SocketException(CONNECTION_RESET_MESSAGE)).when(spiedClient).filterRenamePendingFiles(any(), any());
|
|
|
List<FileStatus> fileStatuses = new ArrayList<>();
|
|
|
AbfsDriverException ex = intercept(AbfsDriverException.class,
|
|
|
() -> {
|
|
@@ -532,6 +536,87 @@ public class ITestAzureBlobFileSystemListStatus extends
|
|
|
.describedAs("Listing Size Not as expected").hasSize(1);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Test to verify that listStatus returns the correct file status
|
|
|
+ * after removing duplicates across multiple iterations of list blobs.
|
|
|
+ * Also verifies that in case of non-empty explicit dir,
|
|
|
+ * entry corresponding to marker blob is returned.
|
|
|
+ * @throws Exception if test fails.
|
|
|
+ */
|
|
|
+ @Test
|
|
|
+ public void testDuplicateEntriesAcrossListBlobIterations() throws Exception {
|
|
|
+ AzureBlobFileSystem fs = Mockito.spy(getFileSystem());
|
|
|
+ AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
|
|
|
+ store.getAbfsConfiguration().setListMaxResults(1);
|
|
|
+ AbfsClient client = Mockito.spy(store.getClient());
|
|
|
+
|
|
|
+ Mockito.doReturn(store).when(fs).getAbfsStore();
|
|
|
+ Mockito.doReturn(client).when(store).getClient();
|
|
|
+
|
|
|
+ /*
|
|
|
+ * Following entries will be created inside the root path.
|
|
|
+ * 0. /A - implicit directory without any marker blob
|
|
|
+ * 1. /a - marker file for explicit directory
|
|
|
+ * 2. /a/file1 - normal file inside explicit directory
|
|
|
+ * 3. /b - normal file inside root
|
|
|
+ * 4. /c - marker file for explicit directory
|
|
|
+ * 5. /c.bak - marker file for explicit directory
|
|
|
+ * 6. /c.bak/file2 - normal file inside explicit directory
|
|
|
+ * 7. /c/file3 - normal file inside explicit directory
|
|
|
+ * 8. /d - implicit directory
|
|
|
+ * 9. /e - marker file for explicit directory
|
|
|
+ * 10. /e/file4 - normal file inside explicit directory
|
|
|
+ */
|
|
|
+ // Create Path 0
|
|
|
+ createAzCopyFolder(new Path("/A"));
|
|
|
+
|
|
|
+ // Create Path 1 and 2.
|
|
|
+ fs.create(new Path("/a/file1"));
|
|
|
+
|
|
|
+ // Create Path 3
|
|
|
+ fs.create(new Path("/b"));
|
|
|
+
|
|
|
+ // Create Path 4 and 7
|
|
|
+ fs.create(new Path("/c/file3"));
|
|
|
+
|
|
|
+ // Create Path 5 and 6
|
|
|
+ fs.create(new Path("/c.bak/file2"));
|
|
|
+
|
|
|
+ // Create Path 8
|
|
|
+ createAzCopyFolder(new Path("/d"));
|
|
|
+
|
|
|
+ // Create Path 9 and 10
|
|
|
+ fs.create(new Path("/e/file4"));
|
|
|
+
|
|
|
+ FileStatus[] fileStatuses = fs.listStatus(new Path(ROOT_PATH));
|
|
|
+
|
|
|
+ // Assert that client.listPath was called 11 times.
|
|
|
+ // This will assert server returned 11 entries in total.
|
|
|
+ Mockito.verify(client, Mockito.times(TOTAL_NUMBER_OF_PATHS))
|
|
|
+ .listPath(eq(ROOT_PATH), eq(false), eq(1), any(), any(), any());
|
|
|
+
|
|
|
+ // Assert that after duplicate removal, only 7 unique entries are returned.
|
|
|
+ Assertions.assertThat(fileStatuses.length)
|
|
|
+ .describedAs("List size is not expected").isEqualTo(NUMBER_OF_UNIQUE_PATHS);
|
|
|
+
|
|
|
+ // Assert that for duplicates, entry corresponding to marker blob is returned.
|
|
|
+ assertImplicitDirectoryFileStatus(fileStatuses[0], fs.makeQualified(new Path("/A")));
|
|
|
+ assertExplicitDirectoryFileStatus(fileStatuses[1], fs.makeQualified(new Path("/a")));
|
|
|
+ assertFilePathFileStatus(fileStatuses[2], fs.makeQualified(new Path("/b")));
|
|
|
+ assertExplicitDirectoryFileStatus(fileStatuses[3], fs.makeQualified(new Path("/c")));
|
|
|
+ assertExplicitDirectoryFileStatus(fileStatuses[4], fs.makeQualified(new Path("/c.bak")));
|
|
|
+ assertImplicitDirectoryFileStatus(fileStatuses[5], fs.makeQualified(new Path("/d")));
|
|
|
+ assertExplicitDirectoryFileStatus(fileStatuses[6], fs.makeQualified(new Path("/e")));
|
|
|
+
|
|
|
+ // Assert that there are no duplicates in the returned file statuses.
|
|
|
+ Set<Path> uniquePaths = new HashSet<>();
|
|
|
+ for (FileStatus fileStatus : fileStatuses) {
|
|
|
+ Assertions.assertThat(uniquePaths.add(fileStatus.getPath()))
|
|
|
+ .describedAs("Duplicate Entries found")
|
|
|
+ .isTrue();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
private void assertFilePathFileStatus(final FileStatus fileStatus,
|
|
|
final Path qualifiedPath) {
|
|
|
Assertions.assertThat(fileStatus.getPath())
|