You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by fe...@apache.org on 2024/02/21 01:04:28 UTC

(arrow) branch main updated: GH-40074: [C++][FS][Azure] Implement `DeleteFile()` for flat-namespace storage accounts (#40075)

This is an automated email from the ASF dual-hosted git repository.

felipecrv pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow.git


The following commit(s) were added to refs/heads/main by this push:
     new a2d072929f GH-40074: [C++][FS][Azure] Implement `DeleteFile()` for flat-namespace storage accounts (#40075)
a2d072929f is described below

commit a2d072929fea956a77775671e2eaf20f9d0ed5fa
Author: Felipe Oliveira Carvalho <fe...@gmail.com>
AuthorDate: Tue Feb 20 22:04:21 2024 -0300

    GH-40074: [C++][FS][Azure] Implement `DeleteFile()` for flat-namespace storage accounts (#40075)
    
    ### Rationale for this change
    
    It was not implemented yet.
    
    ### What changes are included in this PR?
    
     - An implementation of `DeleteFile()` that is specialized to storage accounts that don't have HNS support enabled
     - This fixes a semantic issue: deleting a file should not delete the parent directory when the file deleted was the last one
     - Increased test coverage
     - Fix of a bug in the version that deletes files in HNS-enabled accounts (we shouldn't let `DeleteFile` delete directories even if they are empty)
    
    ### Are these changes tested?
    
    Yes. Tests were re-written and moved to `TestAzureFileSystemOnAllScenarios`.
    * Closes: #40074
    
    Lead-authored-by: Felipe Oliveira Carvalho <fe...@gmail.com>
    Co-authored-by: jerry.adair <Je...@sas.com>
    Signed-off-by: Felipe Oliveira Carvalho <fe...@gmail.com>
---
 cpp/src/arrow/filesystem/azurefs.cc      | 180 ++++++++++++++++++++++++++----
 cpp/src/arrow/filesystem/azurefs_test.cc | 184 +++++++++++++++++++++----------
 2 files changed, 283 insertions(+), 81 deletions(-)

diff --git a/cpp/src/arrow/filesystem/azurefs.cc b/cpp/src/arrow/filesystem/azurefs.cc
index 23af67a33d..de7cdba245 100644
--- a/cpp/src/arrow/filesystem/azurefs.cc
+++ b/cpp/src/arrow/filesystem/azurefs.cc
@@ -1085,7 +1085,11 @@ class LeaseGuard {
     return Status::OK();
   }
 
-  /// \brief Break the lease before deleting or renaming the resource.
+  /// \brief Break the lease before deleting or renaming the resource via the
+  /// DataLakeFileSystemClient API.
+  ///
+  /// NOTE: When using the Blobs API, this is not necessary -- you can release a
+  /// lease on a path after it's deleted with a lease on it.
   ///
   /// Calling this is recommended when the resource for which the lease was acquired is
   /// about to be deleted as there is no way of releasing the lease after that, we can
@@ -1926,26 +1930,6 @@ class AzureFileSystem::Impl {
     }
   }
 
-  Status DeleteFile(const AzureLocation& location) {
-    RETURN_NOT_OK(ValidateFileLocation(location));
-    auto file_client = datalake_service_client_->GetFileSystemClient(location.container)
-                           .GetFileClient(location.path);
-    try {
-      auto response = file_client.Delete();
-      // Only the "*IfExists" functions ever set Deleted to false.
-      // All the others either succeed or throw an exception.
-      DCHECK(response.Value.Deleted);
-    } catch (const Storage::StorageException& exception) {
-      if (exception.ErrorCode == "FilesystemNotFound" ||
-          exception.ErrorCode == "PathNotFound") {
-        return PathNotFound(location);
-      }
-      return ExceptionToStatus(exception, "Failed to delete a file: ", location.path,
-                               ": ", file_client.GetUrl());
-    }
-    return Status::OK();
-  }
-
  private:
   /// \brief Create a BlobLeaseClient and acquire a lease on the container.
   ///
@@ -1994,7 +1978,7 @@ class AzureFileSystem::Impl {
   /// optional (nullptr denotes blob not found)
   Result<std::unique_ptr<Blobs::BlobLeaseClient>> AcquireBlobLease(
       const AzureLocation& location, std::chrono::seconds lease_duration,
-      bool allow_missing = false, bool retry_allowed = true) {
+      bool allow_missing, bool retry_allowed = true) {
     DCHECK(!location.container.empty() && !location.path.empty());
     auto path = std::string{internal::RemoveTrailingSlash(location.path)};
     auto blob_client = GetBlobClient(location.container, std::move(path));
@@ -2057,6 +2041,131 @@ class AzureFileSystem::Impl {
   static constexpr auto kTimeNeededForFileOrDirectoryRename = std::chrono::seconds{3};
 
  public:
+  /// \pre location.container is not empty.
+  /// \pre location.path is not empty.
+  Status DeleteFileOnFileSystem(const DataLake::DataLakeFileSystemClient& adlfs_client,
+                                const AzureLocation& location,
+                                bool require_file_to_exist) {
+    DCHECK(!location.container.empty());
+    DCHECK(!location.path.empty());
+    auto path_no_trailing_slash =
+        std::string{internal::RemoveTrailingSlash(location.path)};
+    auto file_client = adlfs_client.GetFileClient(path_no_trailing_slash);
+    try {
+      // This is necessary to avoid deletion of directories via DeleteFile.
+      auto properties = file_client.GetProperties();
+      if (properties.Value.IsDirectory) {
+        return internal::NotAFile(location.all);
+      }
+      if (internal::HasTrailingSlash(location.path)) {
+        return internal::NotADir(location.all);
+      }
+      auto response = file_client.Delete();
+      // Only the "*IfExists" functions ever set Deleted to false.
+      // All the others either succeed or throw an exception.
+      DCHECK(response.Value.Deleted);
+    } catch (const Storage::StorageException& exception) {
+      if (exception.StatusCode == Http::HttpStatusCode::NotFound) {
+        // ErrorCode can be "FilesystemNotFound", "PathNotFound"...
+        if (require_file_to_exist) {
+          return PathNotFound(location);
+        }
+        return Status::OK();
+      }
+      return ExceptionToStatus(exception, "Failed to delete a file: ", location.path,
+                               ": ", file_client.GetUrl());
+    }
+    return Status::OK();
+  }
+
+  /// \pre location.container is not empty.
+  /// \pre location.path is not empty.
+  Status DeleteFileOnContainer(const Blobs::BlobContainerClient& container_client,
+                               const AzureLocation& location, bool require_file_to_exist,
+                               const char* operation) {
+    DCHECK(!location.container.empty());
+    DCHECK(!location.path.empty());
+    constexpr auto kFileBlobLeaseTime = std::chrono::seconds{15};
+
+    // When it's known that the blob doesn't exist as a file, check if it exists as a
+    // directory to generate the appropriate error message.
+    auto check_if_location_exists_as_dir = [&]() -> Status {
+      auto no_trailing_slash = location;
+      no_trailing_slash.path = internal::RemoveTrailingSlash(location.path);
+      no_trailing_slash.all = internal::RemoveTrailingSlash(location.all);
+      ARROW_ASSIGN_OR_RAISE(auto file_info,
+                            GetFileInfo(container_client, no_trailing_slash));
+      if (file_info.type() == FileType::NotFound) {
+        return require_file_to_exist ? PathNotFound(location) : Status::OK();
+      }
+      if (file_info.type() == FileType::Directory) {
+        return internal::NotAFile(location.all);
+      }
+      return internal::HasTrailingSlash(location.path) ? internal::NotADir(location.all)
+                                                       : internal::NotAFile(location.all);
+    };
+
+    // Paths ending with trailing slashes are never leading to a deletion,
+    // but the correct error message requires a check of the path.
+    if (internal::HasTrailingSlash(location.path)) {
+      return check_if_location_exists_as_dir();
+    }
+
+    // If the parent directory of a file is not the container itself, there is a
+    // risk that deleting the file also deletes the *implied directory* -- the
+    // directory that is implied by the existence of the file path.
+    //
+    // In this case, we must ensure that the deletion is not semantically
+    // equivalent to also deleting the directory. This is done by ensuring the
+    // directory marker blob exists before the file is deleted.
+    std::optional<LeaseGuard> file_blob_lease_guard;
+    const auto parent = location.parent();
+    if (!parent.path.empty()) {
+      // We have to check the existence of the file before checking the
+      // existence of the parent directory marker, so we acquire a lease on the
+      // file first.
+      ARROW_ASSIGN_OR_RAISE(auto file_blob_lease_client,
+                            AcquireBlobLease(location, kFileBlobLeaseTime,
+                                             /*allow_missing=*/true));
+      if (file_blob_lease_client) {
+        file_blob_lease_guard.emplace(std::move(file_blob_lease_client),
+                                      kFileBlobLeaseTime);
+        // Ensure the empty directory marker blob of the parent exists before the file is
+        // deleted.
+        //
+        // There is not need to hold a lease on the directory marker because if
+        // a concurrent client deletes the directory marker right after we
+        // create it, the file deletion itself won't be the cause of the directory
+        // deletion. Additionally, the fact that a lease is held on the blob path
+        // semantically preserves the directory -- its existence is implied
+        // until the blob representing the file is deleted -- even if another
+        // client deletes the directory marker.
+        RETURN_NOT_OK(EnsureEmptyDirExists(container_client, parent, operation));
+      } else {
+        return check_if_location_exists_as_dir();
+      }
+    }
+
+    auto blob_client = container_client.GetBlobClient(location.path);
+    Blobs::DeleteBlobOptions options;
+    if (file_blob_lease_guard) {
+      options.AccessConditions.LeaseId = file_blob_lease_guard->LeaseId();
+    }
+    try {
+      auto response = blob_client.Delete(options);
+      // Only the "*IfExists" functions ever set Deleted to false.
+      // All the others either succeed or throw an exception.
+      DCHECK(response.Value.Deleted);
+    } catch (const Storage::StorageException& exception) {
+      if (exception.StatusCode == Http::HttpStatusCode::NotFound) {
+        return check_if_location_exists_as_dir();
+      }
+      return ExceptionToStatus(exception, "Failed to delete a file: ", location.all, ": ",
+                               blob_client.GetUrl());
+    }
+    return Status::OK();
+  }
+
   /// The conditions for a successful container rename are derived from the
   /// conditions for a successful `Move("/$src.container", "/$dest.container")`.
   /// The numbers here match the list in `Move`.
@@ -2238,7 +2347,8 @@ class AzureFileSystem::Impl {
     const auto dest_path = std::string{internal::RemoveTrailingSlash(dest.path)};
 
     // Ensure that src exists and, if path has a trailing slash, that it's a directory.
-    ARROW_ASSIGN_OR_RAISE(auto src_lease_client, AcquireBlobLease(src, kLeaseDuration));
+    ARROW_ASSIGN_OR_RAISE(auto src_lease_client,
+                          AcquireBlobLease(src, kLeaseDuration, /*allow_missing=*/false));
     LeaseGuard src_lease_guard{std::move(src_lease_client), kLeaseDuration};
     // It might be necessary to check src is a directory 0-3 times in this function,
     // so we use a lazy evaluation function to avoid redundant calls to GetFileInfo().
@@ -2551,7 +2661,29 @@ Status AzureFileSystem::DeleteRootDirContents() {
 
 Status AzureFileSystem::DeleteFile(const std::string& path) {
   ARROW_ASSIGN_OR_RAISE(auto location, AzureLocation::FromString(path));
-  return impl_->DeleteFile(location);
+  if (location.container.empty()) {
+    return Status::Invalid("DeleteFile requires a non-empty path.");
+  }
+  auto container_client = impl_->GetBlobContainerClient(location.container);
+  if (location.path.empty()) {
+    // Container paths (locations w/o path) are either not found or represent directories.
+    ARROW_ASSIGN_OR_RAISE(auto container_info,
+                          GetContainerPropsAsFileInfo(location, container_client));
+    return container_info.IsDirectory() ? NotAFile(location) : PathNotFound(location);
+  }
+  auto adlfs_client = impl_->GetFileSystemClient(location.container);
+  ARROW_ASSIGN_OR_RAISE(auto hns_support,
+                        impl_->HierarchicalNamespaceSupport(adlfs_client));
+  if (hns_support == HNSSupport::kContainerNotFound) {
+    return PathNotFound(location);
+  }
+  if (hns_support == HNSSupport::kEnabled) {
+    return impl_->DeleteFileOnFileSystem(adlfs_client, location,
+                                         /*require_file_to_exist=*/true);
+  }
+  return impl_->DeleteFileOnContainer(container_client, location,
+                                      /*require_file_to_exist=*/true,
+                                      /*operation=*/"DeleteFile");
 }
 
 Status AzureFileSystem::Move(const std::string& src, const std::string& dest) {
diff --git a/cpp/src/arrow/filesystem/azurefs_test.cc b/cpp/src/arrow/filesystem/azurefs_test.cc
index e6bd80d1d2..7f5cd247a8 100644
--- a/cpp/src/arrow/filesystem/azurefs_test.cc
+++ b/cpp/src/arrow/filesystem/azurefs_test.cc
@@ -641,6 +641,18 @@ class TestAzureFileSystem : public ::testing::Test {
 #endif
   }
 
+  static bool WithErrno(const Status& status, int expected_errno) {
+    auto* detail = status.detail().get();
+    return detail &&
+           arrow::internal::ErrnoFromStatusDetail(*detail).value_or(-1) == expected_errno;
+  }
+
+#define ASSERT_RAISES_ERRNO(expr, expected_errno)                                     \
+  for (::arrow::Status _st = ::arrow::internal::GenericToStatus((expr));              \
+       !WithErrno(_st, (expected_errno));)                                            \
+  FAIL() << "'" ARROW_STRINGIFY(expr) "' did not fail with errno=" << #expected_errno \
+         << ": " << _st.ToString()
+
   // Tests that are called from more than one implementation of TestAzureFileSystem
 
   void TestDetectHierarchicalNamespace(bool trip_up_azurite);
@@ -935,6 +947,106 @@ class TestAzureFileSystem : public ::testing::Test {
     ASSERT_RAISES(IOError, fs()->DeleteDirContents(directory_path, false));
   }
 
+  void TestDeleteFileAtRoot() {
+    ASSERT_RAISES_ERRNO(fs()->DeleteFile("file0"), ENOENT);
+    ASSERT_RAISES_ERRNO(fs()->DeleteFile("file1/"), ENOENT);
+    const auto container_name = PreexistingData::RandomContainerName(rng_);
+    if (WithHierarchicalNamespace()) {
+      ARROW_UNUSED(CreateFilesystem(container_name));
+    } else {
+      ARROW_UNUSED(CreateContainer(container_name));
+    }
+    arrow::fs::AssertFileInfo(fs(), container_name, FileType::Directory);
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError, ::testing::HasSubstr("Not a regular file: '" + container_name + "'"),
+        fs()->DeleteFile(container_name));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError, ::testing::HasSubstr("Not a regular file: '" + container_name + "/'"),
+        fs()->DeleteFile(container_name + "/"));
+  }
+
+  void TestDeleteFileAtContainerRoot() {
+    auto data = SetUpPreexistingData();
+
+    ASSERT_RAISES_ERRNO(fs()->DeleteFile(data.Path("nonexistent-path")), ENOENT);
+    ASSERT_RAISES_ERRNO(fs()->DeleteFile(data.Path("nonexistent-path/")), ENOENT);
+
+    arrow::fs::AssertFileInfo(fs(), data.ObjectPath(), FileType::File);
+    ASSERT_OK(fs()->DeleteFile(data.ObjectPath()));
+    arrow::fs::AssertFileInfo(fs(), data.ObjectPath(), FileType::NotFound);
+
+    if (WithHierarchicalNamespace()) {
+      auto adlfs_client =
+          datalake_service_client_->GetFileSystemClient(data.container_name);
+      CreateFile(adlfs_client, data.kObjectName, PreexistingData::kLoremIpsum);
+    } else {
+      auto container_client = CreateContainer(data.container_name);
+      CreateBlob(container_client, data.kObjectName, PreexistingData::kLoremIpsum);
+    }
+    arrow::fs::AssertFileInfo(fs(), data.ObjectPath(), FileType::File);
+
+    ASSERT_RAISES_ERRNO(fs()->DeleteFile(data.ObjectPath() + "/"), ENOTDIR);
+    ASSERT_OK(fs()->DeleteFile(data.ObjectPath()));
+    arrow::fs::AssertFileInfo(fs(), data.ObjectPath(), FileType::NotFound);
+  }
+
+  void TestDeleteFileAtSubdirectory(bool create_empty_dir_marker_first) {
+    auto data = SetUpPreexistingData();
+
+    auto setup_dir_file0 = [this, create_empty_dir_marker_first, &data]() {
+      if (WithHierarchicalNamespace()) {
+        ASSERT_FALSE(create_empty_dir_marker_first);
+        auto adlfs_client =
+            datalake_service_client_->GetFileSystemClient(data.container_name);
+        CreateFile(adlfs_client, "dir/file0", PreexistingData::kLoremIpsum);
+      } else {
+        auto container_client = CreateContainer(data.container_name);
+        if (create_empty_dir_marker_first) {
+          CreateBlob(container_client, "dir/", "");
+        }
+        CreateBlob(container_client, "dir/file0", PreexistingData::kLoremIpsum);
+      }
+    };
+    setup_dir_file0();
+
+    // Trying to delete a non-existing file in an existing directory should fail
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError,
+        ::testing::HasSubstr("Path does not exist '" + data.Path("dir/nonexistent-path") +
+                             "'"),
+        fs()->DeleteFile(data.Path("dir/nonexistent-path")));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError,
+        ::testing::HasSubstr("Path does not exist '" +
+                             data.Path("dir/nonexistent-path/") + "'"),
+        fs()->DeleteFile(data.Path("dir/nonexistent-path/")));
+
+    // Trying to delete the directory with DeleteFile should fail
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError, ::testing::HasSubstr("Not a regular file: '" + data.Path("dir") + "'"),
+        fs()->DeleteFile(data.Path("dir")));
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError, ::testing::HasSubstr("Not a regular file: '" + data.Path("dir/") + "'"),
+        fs()->DeleteFile(data.Path("dir/")));
+
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir"), FileType::Directory);
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir/"), FileType::Directory);
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir/file0"), FileType::File);
+    ASSERT_OK(fs()->DeleteFile(data.Path("dir/file0")));
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir"), FileType::Directory);
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir/"), FileType::Directory);
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir/file0"), FileType::NotFound);
+
+    // Recreating the file on the same path gurantees leases were properly released/broken
+    setup_dir_file0();
+
+    EXPECT_RAISES_WITH_MESSAGE_THAT(
+        IOError,
+        ::testing::HasSubstr("Not a directory: '" + data.Path("dir/file0/") + "'"),
+        fs()->DeleteFile(data.Path("dir/file0/")));
+    arrow::fs::AssertFileInfo(fs(), data.Path("dir/file0"), FileType::File);
+  }
+
  private:
   using StringMatcher =
       ::testing::PolymorphicMatcher<::testing::internal::HasSubstrMatcher<std::string>>;
@@ -1092,12 +1204,6 @@ class TestAzureFileSystem : public ::testing::Test {
     AssertFileInfo(fs(), dest, type);
   }
 
-  static bool WithErrno(const Status& status, int expected_errno) {
-    auto* detail = status.detail().get();
-    return detail &&
-           arrow::internal::ErrnoFromStatusDetail(*detail).value_or(-1) == expected_errno;
-  }
-
   std::optional<StringMatcher> MoveErrorMessageMatcher(const FileInfo& src_info,
                                                        const std::string& src,
                                                        const std::string& dest,
@@ -1596,6 +1702,21 @@ TYPED_TEST(TestAzureFileSystemOnAllScenarios, DeleteDirContentsFailureNonexisten
   this->TestDeleteDirContentsFailureNonexistent();
 }
 
+TYPED_TEST(TestAzureFileSystemOnAllScenarios, DeleteFileAtRoot) {
+  this->TestDeleteFileAtRoot();
+}
+
+TYPED_TEST(TestAzureFileSystemOnAllScenarios, DeleteFileAtContainerRoot) {
+  this->TestDeleteFileAtContainerRoot();
+}
+
+TYPED_TEST(TestAzureFileSystemOnAllScenarios, DeleteFileAtSubdirectory) {
+  this->TestDeleteFileAtSubdirectory(/*create_empty_dir_marker_first=*/false);
+  if (!this->WithHierarchicalNamespace()) {
+    this->TestDeleteFileAtSubdirectory(/*create_empty_dir_marker_first=*/true);
+  }
+}
+
 TYPED_TEST(TestAzureFileSystemOnAllScenarios, RenameContainer) {
   this->TestRenameContainer();
 }
@@ -1884,57 +2005,6 @@ TEST_F(TestAzuriteFileSystem, DeleteDirContentsFailureNonexistent) {
   this->TestDeleteDirContentsFailureNonexistent();
 }
 
-TEST_F(TestAzuriteFileSystem, DeleteFileSuccess) {
-  const auto container_name = PreexistingData::RandomContainerName(rng_);
-  const auto file_name = ConcatAbstractPath(container_name, "filename");
-  if (WithHierarchicalNamespace()) {
-    auto adlfs_client = CreateFilesystem(container_name);
-    CreateFile(adlfs_client, "filename", "data");
-  } else {
-    auto container = CreateContainer(container_name);
-    CreateBlob(container, "filename", "data");
-  }
-  arrow::fs::AssertFileInfo(fs(), file_name, FileType::File);
-  ASSERT_OK(fs()->DeleteFile(file_name));
-  arrow::fs::AssertFileInfo(fs(), file_name, FileType::NotFound);
-}
-
-TEST_F(TestAzuriteFileSystem, DeleteFileFailureNonexistent) {
-  const auto container_name = PreexistingData::RandomContainerName(rng_);
-  const auto nonexistent_file_name = ConcatAbstractPath(container_name, "nonexistent");
-  if (WithHierarchicalNamespace()) {
-    ARROW_UNUSED(CreateFilesystem(container_name));
-  } else {
-    ARROW_UNUSED(CreateContainer(container_name));
-  }
-  ASSERT_RAISES(IOError, fs()->DeleteFile(nonexistent_file_name));
-}
-
-TEST_F(TestAzuriteFileSystem, DeleteFileFailureContainer) {
-  const auto container_name = PreexistingData::RandomContainerName(rng_);
-  if (WithHierarchicalNamespace()) {
-    ARROW_UNUSED(CreateFilesystem(container_name));
-  } else {
-    ARROW_UNUSED(CreateContainer(container_name));
-  }
-  arrow::fs::AssertFileInfo(fs(), container_name, FileType::Directory);
-  ASSERT_RAISES(IOError, fs()->DeleteFile(container_name));
-}
-
-TEST_F(TestAzuriteFileSystem, DeleteFileFailureDirectory) {
-  auto container_name = PreexistingData::RandomContainerName(rng_);
-  if (WithHierarchicalNamespace()) {
-    auto adlfs_client = CreateFilesystem(container_name);
-    CreateDirectory(adlfs_client, "directory");
-  } else {
-    auto container = CreateContainer(container_name);
-    CreateBlob(container, "directory/");
-  }
-  auto directory_path = ConcatAbstractPath(container_name, "directory");
-  arrow::fs::AssertFileInfo(fs(), directory_path, FileType::Directory);
-  ASSERT_RAISES(IOError, fs()->DeleteFile(directory_path));
-}
-
 TEST_F(TestAzuriteFileSystem, CopyFileSuccessDestinationNonexistent) {
   auto data = SetUpPreexistingData();
   const auto destination_path = data.ContainerPath("copy-destionation");