[ 
https://issues.apache.org/jira/browse/HADOOP-19233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17914060#comment-17914060
 ] 

ASF GitHub Bot commented on HADOOP-19233:
-----------------------------------------

anmolanmol1234 commented on code in PR #7265:
URL: https://github.com/apache/hadoop/pull/7265#discussion_r1920024724


##########
hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java:
##########
@@ -192,10 +239,1304 @@ public void testRenameWithNoDestinationParentDir() 
throws Exception {
     // Verify that metadata was in an incomplete state after the rename
     // failure, and we retired the rename once more.
     IOStatistics ioStatistics = fs.getIOStatistics();
+    AbfsClient client = fs.getAbfsStore().getClient();
     IOStatisticAssertions.assertThatStatisticCounter(ioStatistics,
         RENAME_PATH_ATTEMPTS.getStatName())
-        .describedAs("There should be 2 rename attempts if metadata "
-            + "incomplete state failure is hit")
-        .isEqualTo(2);
+            .describedAs("For Dfs endpoint: There should be 2 rename "
+                    + "attempts if metadata incomplete state failure is hit."
+                    + "For Blob endpoint: There would be only one rename 
attempt which "
+                    + "would have a failed precheck.")
+            .isEqualTo(client instanceof AbfsDfsClient ? 2 : 1);
+  }
+
+  @Test
+  public void testRenameToRoot() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/src1/src2"));
+    assertTrue(fs.rename(new Path("/src1/src2"), new Path("/")));
+    assertTrue(fs.exists(new Path("/src2")));
+  }
+
+  @Test
+  public void testRenameNotFoundBlobToEmptyRoot() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assertFalse(fs.rename(new Path("/file"), new Path("/")));
+  }
+
+  private void assumeNonHnsAccountBlobEndpoint(final AzureBlobFileSystem fs) {
+    assertTrue(fs.getAbfsStore().getClient() instanceof AbfsBlobClient);
+  }
+
+  @Test(expected = IOException.class)
+  public void testRenameBlobToDstWithColonInPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeNonHnsAccountBlobEndpoint(fs);
+    fs.create(new Path("/src"));
+    fs.rename(new Path("/src"), new Path("/dst:file"));
+  }
+
+  @Test
+  public void testRenameBlobInSameDirectoryWithNoMarker() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    assumeNonHnsAccountBlobEndpoint(fs);
+    AbfsBlobClient client = (AbfsBlobClient) fs.getAbfsStore().getClient();
+    fs.create(new Path("/srcDir/dir/file"));
+    client.deleteBlobPath(new Path("/srcDir/dir"), null,
+            getTestTracingContext(fs, true));
+    assertTrue(fs.rename(new Path("/srcDir/dir"), new Path("/srcDir")));
+  }
+
+  /**
+   * <pre>
+   * Test to check behaviour of rename API if the destination directory is 
already
+   * there. The HNS call and the one for Blob endpoint should have same 
behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * </pre>
+   *
+   * Expectation for HNS / Blob endpoint:<ol>
+   * <li>Rename should fail</li>
+   * <li>No file should be transferred to destination directory</li>
+   * </ol>
+   */
+  @Test
+  public void testPosixRenameDirectoryWhereDirectoryAlreadyThereOnDestination()
+          throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testDir2/test1/test2/test3"));
+    fs.create(new Path("testDir2/test1/test2/test3/file"));
+    fs.mkdirs(new Path("testDir2/test4/test3"));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    assertFalse(fs.rename(new Path("testDir2/test1/test2/test3"),
+            new Path("testDir2/test4")));
+    assertTrue(fs.exists(new Path("testDir2")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2")));
+    assertTrue(fs.exists(new Path("testDir2/test4")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3")));
+    if (getIsNamespaceEnabled(fs)
+            || fs.getAbfsClient() instanceof AbfsBlobClient) {
+      assertFalse(fs.exists(new Path("testDir2/test4/test3/file")));
+      assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    } else {
+      assertTrue(fs.exists(new Path("testDir2/test4/test3/file")));
+      assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    }
+  }
+
+  /**
+   * <pre>
+   * Test to check behaviour of rename API if the destination directory is 
already
+   * there. The HNS call and the one for Blob endpoint should have same 
behaviour.
+   *
+   * /testDir2/test1/test2/test3 contains (/file)
+   * There is another path that exists: /testDir2/test4/test3
+   * On rename(/testDir2/test1/test2/test3, /testDir2/test4).
+   * </pre>
+   *
+   * Expectation for HNS / Blob endpoint:<ol>
+   * <li>Rename should fail</li>
+   * <li>No file should be transferred to destination directory</li>
+   * </ol>
+   */
+  @Test
+  public void testPosixRenameDirectoryWherePartAlreadyThereOnDestination()
+          throws Exception {
+    final AzureBlobFileSystem fs = this.getFileSystem();
+    fs.mkdirs(new Path("testDir2/test1/test2/test3"));
+    fs.create(new Path("testDir2/test1/test2/test3/file"));
+    fs.create(new Path("testDir2/test1/test2/test3/file1"));
+    fs.mkdirs(new Path("testDir2/test4/"));
+    fs.create(new Path("testDir2/test4/file1"));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2/test3/file1")));
+    Assert.assertTrue(fs.rename(new Path("testDir2/test1/test2/test3"),
+            new Path("testDir2/test4")));
+    assertTrue(fs.exists(new Path("testDir2")));
+    assertTrue(fs.exists(new Path("testDir2/test1/test2")));
+    assertTrue(fs.exists(new Path("testDir2/test4")));
+    assertFalse(fs.exists(new Path("testDir2/test1/test2/test3")));
+    assertFalse(fs.exists(new Path("testDir2/test4/file")));
+    assertTrue(fs.exists(new Path("testDir2/test4/file1")));
+    assertTrue(fs.exists(new Path("testDir2/test4/test3/file")));
+    assertTrue(fs.exists(new Path("testDir2/test4/test3/file1")));
+    assertTrue(fs.exists(new Path("testDir2/test4/file1")));
+    assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file")));
+    assertFalse(fs.exists(new Path("testDir2/test1/test2/test3/file1")));
+  }
+
+  /**
+   * Test that after completing rename for a directory which is enabled for
+   * AtomicRename, the RenamePending JSON file is deleted.
+   */
+  @Test
+  public void testRenamePendingJsonIsRemovedPostSuccessfulRename()
+          throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeNonHnsAccountBlobEndpoint(fs);
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    assumeNonHnsAccountBlobEndpoint(fs);
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(new Path("hbase/test1/test2/test3"));
+    fs.create(new Path("hbase/test1/test2/test3/file"));
+    fs.create(new Path("hbase/test1/test2/test3/file1"));
+    fs.mkdirs(new Path("hbase/test4/"));
+    fs.create(new Path("hbase/test4/file1"));
+    final Integer[] correctDeletePathCount = new Integer[1];
+    correctDeletePathCount[0] = 0;
+    Mockito.doAnswer(answer -> {
+              final String correctDeletePath = "/hbase/test1/test2/test3" + 
SUFFIX;
+              if (correctDeletePath.equals(
+                      ((Path) answer.getArgument(0)).toUri().getPath())) {
+                correctDeletePathCount[0] = 1;
+              }
+              return null;
+            })
+            .when(client)
+            .deleteBlobPath(Mockito.any(Path.class), 
Mockito.nullable(String.class),
+                    Mockito.any(TracingContext.class));
+    assertTrue(fs.rename(new Path("hbase/test1/test2/test3"),
+            new Path("hbase/test4")));
+    assertEquals("RenamePendingJson should be deleted",
+            1,
+            (int) correctDeletePathCount[0]);
+  }
+
+  private AbfsClient addSpyHooksOnClient(final AzureBlobFileSystem fs) {
+    AzureBlobFileSystemStore store = Mockito.spy(fs.getAbfsStore());
+    Mockito.doReturn(store).when(fs).getAbfsStore();
+    AbfsClient client = Mockito.spy(store.getClient());
+    Mockito.doReturn(client).when(store).getClient();
+    return client;
+  }
+
+  /**
+   * Test for a directory in /hbase directory. To simulate the crash of 
process,
+   * test will throw an exception with 403 on a copy of one of the blob.<br>
+   * ListStatus API will be called on the directory. Expectation is that the 
ListStatus
+   * API of {@link AzureBlobFileSystem} should recover the paused rename.
+   */
+  @Test
+  public void testHBaseHandlingForFailedRenameWithListRecovery()
+          throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeNonHnsAccountBlobEndpoint(fs);
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    String srcPath = "hbase/test1/test2";
+    final String failedCopyPath = srcPath + "/test3/file1";
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(new Path(srcPath));
+    fs.mkdirs(new Path(srcPath, "test3"));
+    fs.create(new Path(srcPath + "/test3/file"));
+    fs.create(new Path(failedCopyPath));
+    fs.mkdirs(new Path("hbase/test4/"));
+    fs.create(new Path("hbase/test4/file1"));
+    crashRenameAndRecover(fs, client, srcPath, (abfsFs) -> {
+      abfsFs.listStatus(new Path(srcPath).getParent());
+      return null;
+    });
+  }
+
+  /**
+   * Test for a directory in /hbase directory. To simulate the crash of 
process,
+   * test will throw an exception with 403 on a copy of one of the blob. The
+   * source directory is a nested directory.<br>
+   * GetFileStatus API will be called on the directory. Expectation is that the
+   * GetFileStatus API of {@link AzureBlobFileSystem} should recover the paused
+   * rename.
+   */
+  @Test
+  public void testHBaseHandlingForFailedRenameWithGetFileStatusRecovery()
+          throws Exception {
+    AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeNonHnsAccountBlobEndpoint(fs);
+    AbfsBlobClient client = (AbfsBlobClient) addSpyHooksOnClient(fs);
+    String srcPath = "hbase/test1/test2";
+    final String failedCopyPath = srcPath + "/test3/file1";
+    fs.setWorkingDirectory(new Path("/"));
+    fs.mkdirs(new Path(srcPath));
+    fs.mkdirs(new Path(srcPath, "test3"));
+    fs.create(new Path(srcPath + "/test3/file"));
+    fs.create(new Path(failedCopyPath));
+    fs.mkdirs(new Path("hbase/test4/"));
+    fs.create(new Path("hbase/test4/file1"));
+    crashRenameAndRecover(fs, client, srcPath, (abfsFs) -> {
+      abfsFs.exists(new Path(srcPath));
+      return null;
+    });
+  }
+
+  private void crashRenameAndRecover(final AzureBlobFileSystem fs,
+                                     AbfsBlobClient client,
+                                     final String srcPath,
+                                     final 
FunctionRaisingIOE<AzureBlobFileSystem, Void> recoveryCallable)
+          throws Exception {
+    crashRename(fs, client, srcPath);
+    AzureBlobFileSystem fs2 = Mockito.spy(getFileSystem());
+    fs2.setWorkingDirectory(new Path(ROOT_PATH));
+    client = (AbfsBlobClient) addSpyHooksOnClient(fs2);
+    int[] renameJsonDeleteCounter = new int[1];
+    Mockito.doAnswer(answer -> {
+              if ((ROOT_PATH + srcPath + SUFFIX)
+                      .equalsIgnoreCase(((Path) 
answer.getArgument(0)).toUri().getPath())) {
+                renameJsonDeleteCounter[0] = 1;
+              }
+              return answer.callRealMethod();
+            })
+            .when(client)
+            .deleteBlobPath(Mockito.any(Path.class), 
Mockito.nullable(String.class),
+                    Mockito.any(TracingContext.class));
+    recoveryCallable.apply(fs2);
+    Assertions.assertThat(renameJsonDeleteCounter[0])
+            .describedAs("RenamePendingJson should be deleted")
+            .isEqualTo(1);
+    //List would complete the rename orchestration.
+    assertFalse(fs2.exists(new Path("hbase/test1/test2")));
+    assertFalse(fs2.exists(new Path("hbase/test1/test2/test3")));
+    assertTrue(fs2.exists(new Path("hbase/test4/test2/test3")));
+    assertFalse(fs2.exists(new Path("hbase/test1/test2/test3/file")));
+    assertTrue(fs2.exists(new Path("hbase/test4/test2/test3/file")));
+    assertFalse(fs2.exists(new Path("hbase/test1/test2/test3/file1")));
+    assertTrue(fs2.exists(new Path("hbase/test4/test2/test3/file1")));
+  }
+
+  private void crashRename(final AzureBlobFileSystem fs,
+                           final AbfsBlobClient client,
+                           final String srcPath) throws Exception {
+    BlobRenameHandler[] blobRenameHandlers = new BlobRenameHandler[1];
+    AbfsClientTestUtil.mockGetRenameBlobHandler(client,
+            blobRenameHandler -> {
+              blobRenameHandlers[0] = blobRenameHandler;
+              return null;
+            });
+    //Fail rename orchestration on path hbase/test1/test2/test3/file1
+    Mockito.doThrow(new AbfsRestOperationException(HTTP_FORBIDDEN, "", "",
+                    new Exception()))
+            .when(client)
+            .copyBlob(Mockito.any(Path.class), Mockito.any(Path.class),
+                    Mockito.nullable(String.class),
+                    Mockito.any(TracingContext.class));
+    LambdaTestUtils.intercept(AccessDeniedException.class, () -> {
+      fs.rename(new Path(srcPath),
+              new Path("hbase/test4"));
+    });
+    //Release all the leases taken by atomic rename orchestration
+    List<AbfsLease> leases = new 
ArrayList<>(blobRenameHandlers[0].getLeases());
+    for (AbfsLease lease : leases) {
+      lease.free();
+    }
+  }
+
+  /**
+   * Simulates a scenario where HMaster in Hbase starts up and executes 
listStatus
+   * API on the directory that has to be renamed by some other 
executor-machine.
+   * The scenario is that RenamePending JSON is created but before it could be
+   * appended, it has been opened by the HMaster. The HMaster will delete it. 
The
+   * machine doing rename would have to recreate the JSON file.
+   * ref: <a 
href="https://issues.apache.org/jira/browse/HADOOP-12678";>issue</a>
+   */
+  @Test
+  public void 
testHbaseListStatusBeforeRenamePendingFileAppendedWithIngressOnBlob()
+          throws Exception {
+    final AzureBlobFileSystem fs = Mockito.spy(this.getFileSystem());
+    assumeNonHnsAccountBlobEndpoint(fs);
+    fs.setWorkingDirectory(new Path(ROOT_PATH));
+    testRenamePreRenameFailureResolution(fs);
+    testAtomicityRedoInvalidFile(fs);
+  }
+
+  private void testRenamePreRenameFailureResolution(final AzureBlobFileSystem 
fs)

Review Comment:
   comments for tests





> ABFS: [FnsOverBlob] Implementing Rename and Delete APIs over Blob Endpoint
> --------------------------------------------------------------------------
>
>                 Key: HADOOP-19233
>                 URL: https://issues.apache.org/jira/browse/HADOOP-19233
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/azure
>    Affects Versions: 3.4.0
>            Reporter: Anuj Modi
>            Assignee: Manish Bhatt
>            Priority: Major
>              Labels: pull-request-available
>
> Currently, we only support rename and delete operations on the DFS endpoint. 
> The reason for supporting rename and delete operations on the Blob endpoint 
> is that the Blob endpoint does not account for hierarchy. We need to ensure 
> that the HDFS contracts are maintained when performing rename and delete 
> operations. Renaming or deleting a directory over the Blob endpoint requires 
> the client to handle the orchestration and rename or delete all the blobs 
> within the specified directory.
>  
> The task outlines the considerations for implementing rename and delete 
> operations for the FNS-blob endpoint to ensure compatibility with HDFS 
> contracts.
>  * {*}Blob Endpoint Usage{*}: The task addresses the need for abstraction in 
> the code to maintain HDFS contracts while performing rename and delete 
> operations on the blob endpoint, which does not support hierarchy.
>  * {*}Rename Operations{*}: The {{AzureBlobFileSystem#rename()}} method will 
> use a {{RenameHandler}} instance to handle rename operations, with separate 
> handlers for the DFS and blob endpoints. This method includes prechecks, 
> destination adjustments, and orchestration of directory renaming for blobs.
>  * {*}Atomic Rename{*}: Atomic renaming is essential for blob endpoints, as 
> it requires orchestration to copy or delete each blob within the directory. A 
> configuration will allow developers to specify directories for atomic 
> renaming, with a JSON file to track the status of renames.
>  * {*}Delete Operations{*}: Delete operations are simpler than renames, 
> requiring fewer HDFS contract checks. For blob endpoints, the client must 
> handle orchestration, including managing orphaned directories created by 
> Az-copy.
>  * {*}Orchestration for Rename/Delete{*}: Orchestration for rename and delete 
> operations over blob endpoints involves listing blobs and performing actions 
> on each blob. The process must be optimized to handle large numbers of blobs 
> efficiently.
>  * {*}Need for Optimization{*}: Optimization is crucial because the 
> {{ListBlob}} API can return a maximum of 5000 blobs at once, necessitating 
> multiple calls for large directories. The task proposes a producer-consumer 
> model to handle blobs in parallel, thereby reducing processing time and 
> memory usage.
>  * {*}Producer-Consumer Design{*}: The proposed design includes a producer to 
> list blobs, a queue to store the blobs, and a consumer to process them in 
> parallel. This approach aims to improve efficiency and mitigate memory issues.
> More details will follow
> Perquisites for this Patch:
> 1. HADOOP-19187 ABFS: [FnsOverBlob]Making AbfsClient Abstract for supporting 
> both DFS and Blob Endpoint - ASF JIRA (apache.org)
> 2. HADOOP-19226 ABFS: [FnsOverBlob]Implementing Azure Rest APIs on Blob 
> Endpoint for AbfsBlobClient - ASF JIRA (apache.org)
> 3. HADOOP-19207 ABFS: [FnsOverBlob]Response Handling of Blob Endpoint APIs 
> and Metadata APIs - ASF JIRA (apache.org)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to