You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ap...@apache.org on 2022/04/20 07:12:22 UTC

[arrow] branch master updated: ARROW-16159: [C++][Python] Allow FileSystem::DeleteDirContents to succeed if the directory is missing

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

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


The following commit(s) were added to refs/heads/master by this push:
     new a5e45cecb2 ARROW-16159: [C++][Python] Allow FileSystem::DeleteDirContents to succeed if the directory is missing
a5e45cecb2 is described below

commit a5e45cecb24229433b825dac64e0ffd10d400e8c
Author: Weston Pace <we...@gmail.com>
AuthorDate: Wed Apr 20 09:12:16 2022 +0200

    ARROW-16159: [C++][Python] Allow FileSystem::DeleteDirContents to succeed if the directory is missing
    
    Also changes the dataset writer to use this new method (specifying `missing_dir_ok=true`).  This should address behavior seen in ARROW-12358
    
    Closes #12848 from westonpace/feature/ARROW-16159--missing-directory-delete-dir-contents
    
    Lead-authored-by: Weston Pace <we...@gmail.com>
    Co-authored-by: Antoine Pitrou <pi...@free.fr>
    Signed-off-by: Antoine Pitrou <an...@python.org>
---
 cpp/src/arrow/dataset/dataset_writer.cc      | 22 ++++++++++++----------
 cpp/src/arrow/dataset/dataset_writer_test.cc | 15 +++++++++++++++
 cpp/src/arrow/filesystem/filesystem.cc       | 19 +++++++++++--------
 cpp/src/arrow/filesystem/filesystem.h        | 10 ++++++----
 cpp/src/arrow/filesystem/gcsfs.cc            | 16 ++++++++++++----
 cpp/src/arrow/filesystem/gcsfs.h             |  2 +-
 cpp/src/arrow/filesystem/hdfs.cc             | 22 ++++++++++++++++------
 cpp/src/arrow/filesystem/hdfs.h              |  2 +-
 cpp/src/arrow/filesystem/hdfs_test.cc        |  3 +++
 cpp/src/arrow/filesystem/localfs.cc          |  4 ++--
 cpp/src/arrow/filesystem/localfs.h           |  2 +-
 cpp/src/arrow/filesystem/mockfs.cc           |  5 ++++-
 cpp/src/arrow/filesystem/mockfs.h            |  2 +-
 cpp/src/arrow/filesystem/s3fs.cc             | 23 ++++++++++++++++-------
 cpp/src/arrow/filesystem/s3fs.h              |  5 +++--
 cpp/src/arrow/filesystem/s3fs_test.cc        |  2 ++
 cpp/src/arrow/filesystem/test_util.cc        |  5 +++++
 cpp/src/arrow/python/filesystem.cc           |  4 ++--
 cpp/src/arrow/python/filesystem.h            |  4 ++--
 python/pyarrow/_fs.pyx                       | 19 ++++++++++++++-----
 python/pyarrow/fs.py                         | 14 ++++++++++----
 python/pyarrow/includes/libarrow_fs.pxd      |  4 ++--
 python/pyarrow/tests/test_fs.py              |  5 ++++-
 python/pyarrow/tests/util.py                 |  5 +++--
 24 files changed, 148 insertions(+), 66 deletions(-)

diff --git a/cpp/src/arrow/dataset/dataset_writer.cc b/cpp/src/arrow/dataset/dataset_writer.cc
index 5281de481d..a6fec5fb05 100644
--- a/cpp/src/arrow/dataset/dataset_writer.cc
+++ b/cpp/src/arrow/dataset/dataset_writer.cc
@@ -328,17 +328,19 @@ class DatasetWriterDirectoryQueue : public util::AsyncDestroyable {
   uint64_t rows_written() const { return rows_written_; }
 
   void PrepareDirectory() {
-    if (!directory_.empty()) {
-      init_future_ =
-          DeferNotOk(write_options_.filesystem->io_context().executor()->Submit(
-              [this]() { return write_options_.filesystem->CreateDir(directory_); }));
-    } else {
+    if (directory_.empty()) {
       init_future_ = Future<>::MakeFinished();
-    }
-    if (write_options_.existing_data_behavior ==
-        ExistingDataBehavior::kDeleteMatchingPartitions) {
-      init_future_ = init_future_.Then([this]() {
-        return write_options_.filesystem->DeleteDirContentsAsync(directory_);
+    } else {
+      if (write_options_.existing_data_behavior ==
+          ExistingDataBehavior::kDeleteMatchingPartitions) {
+        init_future_ = write_options_.filesystem->DeleteDirContentsAsync(
+            directory_, /*missing_dir_ok=*/true);
+      } else {
+        init_future_ = Future<>::MakeFinished();
+      }
+      init_future_ = init_future_.Then([this] {
+        return DeferNotOk(write_options_.filesystem->io_context().executor()->Submit(
+            [this]() { return write_options_.filesystem->CreateDir(directory_); }));
       });
     }
   }
diff --git a/cpp/src/arrow/dataset/dataset_writer_test.cc b/cpp/src/arrow/dataset/dataset_writer_test.cc
index 6e6b40b56e..2214954dd0 100644
--- a/cpp/src/arrow/dataset/dataset_writer_test.cc
+++ b/cpp/src/arrow/dataset/dataset_writer_test.cc
@@ -371,6 +371,21 @@ TEST_F(DatasetWriterTestFixture, MaxOpenFiles) {
                       "testdir/part1/chunk-0.arrow", "testdir/part2/chunk-0.arrow"});
 }
 
+TEST_F(DatasetWriterTestFixture, NoExistingDirectory) {
+  fs::TimePoint mock_now = std::chrono::system_clock::now();
+  ASSERT_OK_AND_ASSIGN(std::shared_ptr<fs::FileSystem> fs,
+                       MockFileSystem::Make(mock_now, {::arrow::fs::Dir("testdir")}));
+  filesystem_ = std::dynamic_pointer_cast<MockFileSystem>(fs);
+  write_options_.filesystem = filesystem_;
+  write_options_.existing_data_behavior = ExistingDataBehavior::kDeleteMatchingPartitions;
+  write_options_.base_dir = "testdir/subdir";
+  EXPECT_OK_AND_ASSIGN(auto dataset_writer, DatasetWriter::Make(write_options_));
+  Future<> queue_fut = dataset_writer->WriteRecordBatch(MakeBatch(100), "");
+  AssertFinished(queue_fut);
+  ASSERT_FINISHES_OK(dataset_writer->Finish());
+  AssertCreatedData({{"testdir/subdir/chunk-0.arrow", 0, 100}});
+}
+
 TEST_F(DatasetWriterTestFixture, DeleteExistingData) {
   fs::TimePoint mock_now = std::chrono::system_clock::now();
   ASSERT_OK_AND_ASSIGN(
diff --git a/cpp/src/arrow/filesystem/filesystem.cc b/cpp/src/arrow/filesystem/filesystem.cc
index 19d122b571..18b39125f5 100644
--- a/cpp/src/arrow/filesystem/filesystem.cc
+++ b/cpp/src/arrow/filesystem/filesystem.cc
@@ -193,10 +193,12 @@ Status ValidateInputFileInfo(const FileInfo& info) {
 
 }  // namespace
 
-Future<> FileSystem::DeleteDirContentsAsync(const std::string& path) {
-  return FileSystemDefer(
-      this, default_async_is_sync_,
-      [path](std::shared_ptr<FileSystem> self) { return self->DeleteDirContents(path); });
+Future<> FileSystem::DeleteDirContentsAsync(const std::string& path,
+                                            bool missing_dir_ok) {
+  return FileSystemDefer(this, default_async_is_sync_,
+                         [path, missing_dir_ok](std::shared_ptr<FileSystem> self) {
+                           return self->DeleteDirContents(path, missing_dir_ok);
+                         });
 }
 
 Result<std::shared_ptr<io::InputStream>> FileSystem::OpenInputStream(
@@ -379,12 +381,13 @@ Status SubTreeFileSystem::DeleteDir(const std::string& path) {
   return base_fs_->DeleteDir(real_path);
 }
 
-Status SubTreeFileSystem::DeleteDirContents(const std::string& path) {
+Status SubTreeFileSystem::DeleteDirContents(const std::string& path,
+                                            bool missing_dir_ok) {
   if (internal::IsEmptyPath(path)) {
     return internal::InvalidDeleteDirContents(path);
   }
   ARROW_ASSIGN_OR_RAISE(auto real_path, PrependBase(path));
-  return base_fs_->DeleteDirContents(real_path);
+  return base_fs_->DeleteDirContents(real_path, missing_dir_ok);
 }
 
 Status SubTreeFileSystem::DeleteRootDirContents() {
@@ -521,9 +524,9 @@ Status SlowFileSystem::DeleteDir(const std::string& path) {
   return base_fs_->DeleteDir(path);
 }
 
-Status SlowFileSystem::DeleteDirContents(const std::string& path) {
+Status SlowFileSystem::DeleteDirContents(const std::string& path, bool missing_dir_ok) {
   latencies_->Sleep();
-  return base_fs_->DeleteDirContents(path);
+  return base_fs_->DeleteDirContents(path, missing_dir_ok);
 }
 
 Status SlowFileSystem::DeleteRootDirContents() {
diff --git a/cpp/src/arrow/filesystem/filesystem.h b/cpp/src/arrow/filesystem/filesystem.h
index f74eaf98c3..dfa2b74008 100644
--- a/cpp/src/arrow/filesystem/filesystem.h
+++ b/cpp/src/arrow/filesystem/filesystem.h
@@ -214,10 +214,12 @@ class ARROW_EXPORT FileSystem : public std::enable_shared_from_this<FileSystem>
   ///
   /// Like DeleteDir, but doesn't delete the directory itself.
   /// Passing an empty path ("" or "/") is disallowed, see DeleteRootDirContents.
-  virtual Status DeleteDirContents(const std::string& path) = 0;
+  virtual Status DeleteDirContents(const std::string& path,
+                                   bool missing_dir_ok = false) = 0;
 
   /// Async version of DeleteDirContents.
-  virtual Future<> DeleteDirContentsAsync(const std::string& path);
+  virtual Future<> DeleteDirContentsAsync(const std::string& path,
+                                          bool missing_dir_ok = false);
 
   /// EXPERIMENTAL: Delete the root directory's contents, recursively.
   ///
@@ -348,7 +350,7 @@ class ARROW_EXPORT SubTreeFileSystem : public FileSystem {
   Status CreateDir(const std::string& path, bool recursive = true) override;
 
   Status DeleteDir(const std::string& path) override;
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
   Status DeleteRootDirContents() override;
 
   Status DeleteFile(const std::string& path) override;
@@ -416,7 +418,7 @@ class ARROW_EXPORT SlowFileSystem : public FileSystem {
   Status CreateDir(const std::string& path, bool recursive = true) override;
 
   Status DeleteDir(const std::string& path) override;
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
   Status DeleteRootDirContents() override;
 
   Status DeleteFile(const std::string& path) override;
diff --git a/cpp/src/arrow/filesystem/gcsfs.cc b/cpp/src/arrow/filesystem/gcsfs.cc
index 0b0bba6766..9bd1b15b99 100644
--- a/cpp/src/arrow/filesystem/gcsfs.cc
+++ b/cpp/src/arrow/filesystem/gcsfs.cc
@@ -449,14 +449,15 @@ class GcsFileSystem::Impl {
   }
 
   Status DeleteDir(const GcsPath& p, const io::IOContext& io_context) {
-    RETURN_NOT_OK(DeleteDirContents(p, io_context));
+    RETURN_NOT_OK(DeleteDirContents(p, /*missing_dir_ok=*/false, io_context));
     if (!p.object.empty()) {
       return internal::ToArrowStatus(client_.DeleteObject(p.bucket, p.object));
     }
     return internal::ToArrowStatus(client_.DeleteBucket(p.bucket));
   }
 
-  Status DeleteDirContents(const GcsPath& p, const io::IOContext& io_context) {
+  Status DeleteDirContents(const GcsPath& p, bool missing_dir_ok,
+                           const io::IOContext& io_context) {
     // If the directory marker exists, it better be a directory.
     auto dir = client_.GetObjectMetadata(p.bucket, p.object);
     if (dir && !IsDirectory(*dir)) return NotDirectoryError(*dir);
@@ -477,10 +478,17 @@ class GcsFileSystem::Impl {
     std::vector<Future<>> submitted;
     // This iterates over all the objects, and schedules parallel deletes.
     auto prefix = p.object.empty() ? gcs::Prefix() : gcs::Prefix(canonical);
+    bool at_least_one_obj = false;
     for (const auto& o : client_.ListObjects(p.bucket, prefix)) {
+      at_least_one_obj = true;
       submitted.push_back(DeferNotOk(io_context.executor()->Submit(async_delete, o)));
     }
 
+    if (!missing_dir_ok && !at_least_one_obj && !dir) {
+      // No files were found and no directory marker exists
+      return Status::IOError("No such directory: ", p.full_path);
+    }
+
     return AllFinished(submitted).status();
   }
 
@@ -750,9 +758,9 @@ Status GcsFileSystem::DeleteDir(const std::string& path) {
   return impl_->DeleteDir(p, io_context());
 }
 
-Status GcsFileSystem::DeleteDirContents(const std::string& path) {
+Status GcsFileSystem::DeleteDirContents(const std::string& path, bool missing_dir_ok) {
   ARROW_ASSIGN_OR_RAISE(auto p, GcsPath::FromString(path));
-  return impl_->DeleteDirContents(p, io_context());
+  return impl_->DeleteDirContents(p, missing_dir_ok, io_context());
 }
 
 Status GcsFileSystem::DeleteRootDirContents() {
diff --git a/cpp/src/arrow/filesystem/gcsfs.h b/cpp/src/arrow/filesystem/gcsfs.h
index d1e5e8ec99..c84374cdbb 100644
--- a/cpp/src/arrow/filesystem/gcsfs.h
+++ b/cpp/src/arrow/filesystem/gcsfs.h
@@ -151,7 +151,7 @@ class ARROW_EXPORT GcsFileSystem : public FileSystem {
 
   Status DeleteDir(const std::string& path) override;
 
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
 
   /// This is not implemented in GcsFileSystem, as it would be too dangerous.
   Status DeleteRootDirContents() override;
diff --git a/cpp/src/arrow/filesystem/hdfs.cc b/cpp/src/arrow/filesystem/hdfs.cc
index c6396deac0..59c5c0ef6c 100644
--- a/cpp/src/arrow/filesystem/hdfs.cc
+++ b/cpp/src/arrow/filesystem/hdfs.cc
@@ -26,6 +26,7 @@
 #include "arrow/io/hdfs.h"
 #include "arrow/io/hdfs_internal.h"
 #include "arrow/util/checked_cast.h"
+#include "arrow/util/io_util.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/value_parsing.h"
 #include "arrow/util/windows_fixup.h"
@@ -201,14 +202,21 @@ class HadoopFileSystem::Impl {
     return Status::OK();
   }
 
-  Status DeleteDirContents(const std::string& path) {
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok) {
     if (!IsDirectory(path)) {
       return Status::IOError("Cannot delete contents of directory '", path,
                              "': not a directory");
     }
     std::vector<std::string> file_list;
-    RETURN_NOT_OK(client_->GetChildren(path, &file_list));
-    for (auto file : file_list) {
+    Status get_children_st = client_->GetChildren(path, &file_list);
+    if (!get_children_st.ok()) {
+      if (missing_dir_ok &&
+          ::arrow::internal::ErrnoFromStatus(get_children_st) == ENOENT) {
+        return Status::OK();
+      }
+      return get_children_st;
+    }
+    for (const auto& file : file_list) {
       RETURN_NOT_OK(client_->Delete(file, /*recursive=*/true));
     }
     return Status::OK();
@@ -473,14 +481,16 @@ Status HadoopFileSystem::DeleteDir(const std::string& path) {
   return impl_->DeleteDir(path);
 }
 
-Status HadoopFileSystem::DeleteDirContents(const std::string& path) {
+Status HadoopFileSystem::DeleteDirContents(const std::string& path, bool missing_dir_ok) {
   if (internal::IsEmptyPath(path)) {
     return internal::InvalidDeleteDirContents(path);
   }
-  return impl_->DeleteDirContents(path);
+  return impl_->DeleteDirContents(path, missing_dir_ok);
 }
 
-Status HadoopFileSystem::DeleteRootDirContents() { return impl_->DeleteDirContents(""); }
+Status HadoopFileSystem::DeleteRootDirContents() {
+  return impl_->DeleteDirContents("", /*missing_dir_ok=*/false);
+}
 
 Status HadoopFileSystem::DeleteFile(const std::string& path) {
   return impl_->DeleteFile(path);
diff --git a/cpp/src/arrow/filesystem/hdfs.h b/cpp/src/arrow/filesystem/hdfs.h
index bc72e1cdc9..bed0ac4c61 100644
--- a/cpp/src/arrow/filesystem/hdfs.h
+++ b/cpp/src/arrow/filesystem/hdfs.h
@@ -77,7 +77,7 @@ class ARROW_EXPORT HadoopFileSystem : public FileSystem {
 
   Status DeleteDir(const std::string& path) override;
 
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
 
   Status DeleteRootDirContents() override;
 
diff --git a/cpp/src/arrow/filesystem/hdfs_test.cc b/cpp/src/arrow/filesystem/hdfs_test.cc
index bd670c4b9a..d8bb66a947 100644
--- a/cpp/src/arrow/filesystem/hdfs_test.cc
+++ b/cpp/src/arrow/filesystem/hdfs_test.cc
@@ -235,6 +235,9 @@ TEST_F(TestHadoopFileSystem, DeleteDirContents) {
   ASSERT_OK(this->fs_->DeleteDirContents("AB"));
   AssertFileInfo(this->fs_.get(), "AB", FileType::Directory);
   ASSERT_OK(this->fs_->DeleteDir("AB"));
+
+  ASSERT_OK(this->fs_->DeleteDirContents("DoesNotExist", /*missing_dir_ok=*/true));
+  ASSERT_RAISES(IOError, this->fs_->DeleteDirContents("DoesNotExist"));
 }
 
 TEST_F(TestHadoopFileSystem, WriteReadFile) {
diff --git a/cpp/src/arrow/filesystem/localfs.cc b/cpp/src/arrow/filesystem/localfs.cc
index 15d4678874..e459549109 100644
--- a/cpp/src/arrow/filesystem/localfs.cc
+++ b/cpp/src/arrow/filesystem/localfs.cc
@@ -332,13 +332,13 @@ Status LocalFileSystem::DeleteDir(const std::string& path) {
   return Status::OK();
 }
 
-Status LocalFileSystem::DeleteDirContents(const std::string& path) {
+Status LocalFileSystem::DeleteDirContents(const std::string& path, bool missing_dir_ok) {
   RETURN_NOT_OK(ValidatePath(path));
   if (internal::IsEmptyPath(path)) {
     return internal::InvalidDeleteDirContents(path);
   }
   ARROW_ASSIGN_OR_RAISE(auto fn, PlatformFilename::FromString(path));
-  auto st = ::arrow::internal::DeleteDirContents(fn, /*allow_not_found=*/false).status();
+  auto st = ::arrow::internal::DeleteDirContents(fn, missing_dir_ok).status();
   if (!st.ok()) {
     std::stringstream ss;
     ss << "Cannot delete directory contents in '" << path << "': " << st.message();
diff --git a/cpp/src/arrow/filesystem/localfs.h b/cpp/src/arrow/filesystem/localfs.h
index f8e77aee59..23c8196b05 100644
--- a/cpp/src/arrow/filesystem/localfs.h
+++ b/cpp/src/arrow/filesystem/localfs.h
@@ -77,7 +77,7 @@ class ARROW_EXPORT LocalFileSystem : public FileSystem {
   Status CreateDir(const std::string& path, bool recursive = true) override;
 
   Status DeleteDir(const std::string& path) override;
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
   Status DeleteRootDirContents() override;
 
   Status DeleteFile(const std::string& path) override;
diff --git a/cpp/src/arrow/filesystem/mockfs.cc b/cpp/src/arrow/filesystem/mockfs.cc
index 7306f3574b..31dba27326 100644
--- a/cpp/src/arrow/filesystem/mockfs.cc
+++ b/cpp/src/arrow/filesystem/mockfs.cc
@@ -489,7 +489,7 @@ Status MockFileSystem::DeleteDir(const std::string& path) {
   return Status::OK();
 }
 
-Status MockFileSystem::DeleteDirContents(const std::string& path) {
+Status MockFileSystem::DeleteDirContents(const std::string& path, bool missing_dir_ok) {
   RETURN_NOT_OK(ValidatePath(path));
   auto parts = SplitAbstractPath(path);
   RETURN_NOT_OK(ValidateAbstractPathParts(parts));
@@ -503,6 +503,9 @@ Status MockFileSystem::DeleteDirContents(const std::string& path) {
 
   Entry* entry = impl_->FindEntry(parts);
   if (entry == nullptr) {
+    if (missing_dir_ok) {
+      return Status::OK();
+    }
     return PathNotFound(path);
   }
   if (!entry->is_dir()) {
diff --git a/cpp/src/arrow/filesystem/mockfs.h b/cpp/src/arrow/filesystem/mockfs.h
index 378f30d295..2427d4a3bf 100644
--- a/cpp/src/arrow/filesystem/mockfs.h
+++ b/cpp/src/arrow/filesystem/mockfs.h
@@ -76,7 +76,7 @@ class ARROW_EXPORT MockFileSystem : public FileSystem {
   Status CreateDir(const std::string& path, bool recursive = true) override;
 
   Status DeleteDir(const std::string& path) override;
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
   Status DeleteRootDirContents() override;
 
   Status DeleteFile(const std::string& path) override;
diff --git a/cpp/src/arrow/filesystem/s3fs.cc b/cpp/src/arrow/filesystem/s3fs.cc
index 91d4359012..4336b78330 100644
--- a/cpp/src/arrow/filesystem/s3fs.cc
+++ b/cpp/src/arrow/filesystem/s3fs.cc
@@ -87,6 +87,7 @@
 #include "arrow/util/atomic_shared_ptr.h"
 #include "arrow/util/checked_cast.h"
 #include "arrow/util/future.h"
+#include "arrow/util/io_util.h"
 #include "arrow/util/key_value_metadata.h"
 #include "arrow/util/logging.h"
 #include "arrow/util/optional.h"
@@ -2387,21 +2388,29 @@ Status S3FileSystem::DeleteDir(const std::string& s) {
   }
 }
 
-Status S3FileSystem::DeleteDirContents(const std::string& s) {
-  return DeleteDirContentsAsync(s).status();
+Status S3FileSystem::DeleteDirContents(const std::string& s, bool missing_dir_ok) {
+  return DeleteDirContentsAsync(s, missing_dir_ok).status();
 }
 
-Future<> S3FileSystem::DeleteDirContentsAsync(const std::string& s) {
+Future<> S3FileSystem::DeleteDirContentsAsync(const std::string& s, bool missing_dir_ok) {
   ARROW_ASSIGN_OR_RAISE(auto path, S3Path::FromString(s));
 
   if (path.empty()) {
     return Status::NotImplemented("Cannot delete all S3 buckets");
   }
   auto self = impl_;
-  return impl_->DeleteDirContentsAsync(path.bucket, path.key).Then([path, self]() {
-    // Directory may be implicitly deleted, recreate it
-    return self->EnsureDirectoryExists(path);
-  });
+  return impl_->DeleteDirContentsAsync(path.bucket, path.key)
+      .Then(
+          [path, self]() {
+            // Directory may be implicitly deleted, recreate it
+            return self->EnsureDirectoryExists(path);
+          },
+          [missing_dir_ok](const Status& err) {
+            if (missing_dir_ok && ::arrow::internal::ErrnoFromStatus(err) == ENOENT) {
+              return Status::OK();
+            }
+            return err;
+          });
 }
 
 Status S3FileSystem::DeleteRootDirContents() {
diff --git a/cpp/src/arrow/filesystem/s3fs.h b/cpp/src/arrow/filesystem/s3fs.h
index de8933f8ac..c2c99e964e 100644
--- a/cpp/src/arrow/filesystem/s3fs.h
+++ b/cpp/src/arrow/filesystem/s3fs.h
@@ -231,8 +231,9 @@ class ARROW_EXPORT S3FileSystem : public FileSystem {
   Status CreateDir(const std::string& path, bool recursive = true) override;
 
   Status DeleteDir(const std::string& path) override;
-  Status DeleteDirContents(const std::string& path) override;
-  Future<> DeleteDirContentsAsync(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
+  Future<> DeleteDirContentsAsync(const std::string& path,
+                                  bool missing_dir_ok = false) override;
   Status DeleteRootDirContents() override;
 
   Status DeleteFile(const std::string& path) override;
diff --git a/cpp/src/arrow/filesystem/s3fs_test.cc b/cpp/src/arrow/filesystem/s3fs_test.cc
index 98ee30e848..b44bc7f50c 100644
--- a/cpp/src/arrow/filesystem/s3fs_test.cc
+++ b/cpp/src/arrow/filesystem/s3fs_test.cc
@@ -865,9 +865,11 @@ TEST_F(TestS3FS, DeleteDirContents) {
   select.base_dir = "bucket";
   std::vector<FileInfo> infos;
 
+  ASSERT_OK(fs_->DeleteDirContents("bucket/doesnotexist", /*missing_dir_ok=*/true));
   ASSERT_OK(fs_->DeleteDirContents("bucket/emptydir"));
   ASSERT_OK(fs_->DeleteDirContents("bucket/somedir"));
   ASSERT_RAISES(IOError, fs_->DeleteDirContents("bucket/somefile"));
+  ASSERT_RAISES(IOError, fs_->DeleteDirContents("bucket/doesnotexist"));
   ASSERT_OK_AND_ASSIGN(infos, fs_->GetFileInfo(select));
   ASSERT_EQ(infos.size(), 4);
   SortInfos(&infos);
diff --git a/cpp/src/arrow/filesystem/test_util.cc b/cpp/src/arrow/filesystem/test_util.cc
index a9a935af5e..3e197e110b 100644
--- a/cpp/src/arrow/filesystem/test_util.cc
+++ b/cpp/src/arrow/filesystem/test_util.cc
@@ -298,7 +298,12 @@ void GenericFileSystemTest::TestDeleteDirContents(FileSystem* fs) {
   // Not a directory
   CreateFile(fs, "abc", "");
   ASSERT_RAISES(IOError, fs->DeleteDirContents("abc"));
+  ASSERT_RAISES(IOError, fs->DeleteDirContents("abc", /*missing_dir_ok=*/true));
   AssertAllFiles(fs, {"AB/abc", "abc"});
+
+  // Missing directory
+  ASSERT_RAISES(IOError, fs->DeleteDirContents("missing"));
+  ASSERT_OK(fs->DeleteDirContents("missing", /*missing_dir_ok=*/true));
 }
 
 void GenericFileSystemTest::TestDeleteRootDirContents(FileSystem* fs) {
diff --git a/cpp/src/arrow/python/filesystem.cc b/cpp/src/arrow/python/filesystem.cc
index 8c12f05a0f..5e9b500a4f 100644
--- a/cpp/src/arrow/python/filesystem.cc
+++ b/cpp/src/arrow/python/filesystem.cc
@@ -112,9 +112,9 @@ Status PyFileSystem::DeleteDir(const std::string& path) {
   });
 }
 
-Status PyFileSystem::DeleteDirContents(const std::string& path) {
+Status PyFileSystem::DeleteDirContents(const std::string& path, bool missing_dir_ok) {
   return SafeCallIntoPython([&]() -> Status {
-    vtable_.delete_dir_contents(handler_.obj(), path);
+    vtable_.delete_dir_contents(handler_.obj(), path, missing_dir_ok);
     return CheckPyError();
   });
 }
diff --git a/cpp/src/arrow/python/filesystem.h b/cpp/src/arrow/python/filesystem.h
index e1235f8de5..003fd5cb80 100644
--- a/cpp/src/arrow/python/filesystem.h
+++ b/cpp/src/arrow/python/filesystem.h
@@ -46,7 +46,7 @@ class ARROW_PYTHON_EXPORT PyFileSystemVtable {
 
   std::function<void(PyObject*, const std::string& path, bool)> create_dir;
   std::function<void(PyObject*, const std::string& path)> delete_dir;
-  std::function<void(PyObject*, const std::string& path)> delete_dir_contents;
+  std::function<void(PyObject*, const std::string& path, bool)> delete_dir_contents;
   std::function<void(PyObject*)> delete_root_dir_contents;
   std::function<void(PyObject*, const std::string& path)> delete_file;
   std::function<void(PyObject*, const std::string& src, const std::string& dest)> move;
@@ -92,7 +92,7 @@ class ARROW_PYTHON_EXPORT PyFileSystem : public arrow::fs::FileSystem {
   Status CreateDir(const std::string& path, bool recursive = true) override;
 
   Status DeleteDir(const std::string& path) override;
-  Status DeleteDirContents(const std::string& path) override;
+  Status DeleteDirContents(const std::string& path, bool missing_dir_ok = false) override;
   Status DeleteRootDirContents() override;
 
   Status DeleteFile(const std::string& path) override;
diff --git a/python/pyarrow/_fs.pyx b/python/pyarrow/_fs.pyx
index 5f3f876872..75ad0ccd9b 100644
--- a/python/pyarrow/_fs.pyx
+++ b/python/pyarrow/_fs.pyx
@@ -475,7 +475,9 @@ cdef class FileSystem(_Weakrefable):
         with nogil:
             check_status(self.fs.DeleteDir(directory))
 
-    def delete_dir_contents(self, path, *, bint accept_root_dir=False):
+    def delete_dir_contents(self, path, *,
+                            bint accept_root_dir=False,
+                            bint missing_dir_ok=False):
         """
         Delete a directory's contents, recursively.
 
@@ -488,6 +490,9 @@ cdef class FileSystem(_Weakrefable):
         accept_root_dir : boolean, default False
             Allow deleting the root directory's contents
             (if path is empty or "/")
+        missing_dir_ok : boolean, default False
+            If False then an error is raised if path does
+            not exist
         """
         cdef c_string directory = _path_as_bytes(path)
         if accept_root_dir and directory.strip(b"/") == b"":
@@ -495,7 +500,8 @@ cdef class FileSystem(_Weakrefable):
                 check_status(self.fs.DeleteRootDirContents())
         else:
             with nogil:
-                check_status(self.fs.DeleteDirContents(directory))
+                check_status(self.fs.DeleteDirContents(directory,
+                             missing_dir_ok))
 
     def move(self, src, dest):
         """
@@ -988,7 +994,7 @@ class FileSystemHandler(ABC):
         """
 
     @abstractmethod
-    def delete_dir_contents(self, path):
+    def delete_dir_contents(self, path, missing_dir_ok=False):
         """
         Implement PyFileSystem.delete_dir_contents(...).
 
@@ -996,6 +1002,8 @@ class FileSystemHandler(ABC):
         ----------
         path : str
             path of the directory.
+        missing_dir_ok : bool
+            if False an error should be raised if path does not exist
         """
 
     @abstractmethod
@@ -1151,8 +1159,9 @@ cdef void _cb_create_dir(handler, const c_string& path,
 cdef void _cb_delete_dir(handler, const c_string& path) except *:
     handler.delete_dir(frombytes(path))
 
-cdef void _cb_delete_dir_contents(handler, const c_string& path) except *:
-    handler.delete_dir_contents(frombytes(path))
+cdef void _cb_delete_dir_contents(handler, const c_string& path,
+                                  c_bool missing_dir_ok) except *:
+    handler.delete_dir_contents(frombytes(path), missing_dir_ok)
 
 cdef void _cb_delete_root_dir_contents(handler) except *:
     handler.delete_root_dir_contents()
diff --git a/python/pyarrow/fs.py b/python/pyarrow/fs.py
index 64c88ee78f..e391e7d1ee 100644
--- a/python/pyarrow/fs.py
+++ b/python/pyarrow/fs.py
@@ -346,18 +346,24 @@ class FSSpecHandler(FileSystemHandler):
     def delete_dir(self, path):
         self.fs.rm(path, recursive=True)
 
-    def _delete_dir_contents(self, path):
-        for subpath in self.fs.listdir(path, detail=False):
+    def _delete_dir_contents(self, path, missing_dir_ok):
+        try:
+            subpaths = self.fs.listdir(path, detail=False)
+        except FileNotFoundError:
+            if missing_dir_ok:
+                return
+            raise
+        for subpath in subpaths:
             if self.fs.isdir(subpath):
                 self.fs.rm(subpath, recursive=True)
             elif self.fs.isfile(subpath):
                 self.fs.rm(subpath)
 
-    def delete_dir_contents(self, path):
+    def delete_dir_contents(self, path, missing_dir_ok):
         if path.strip("/") == "":
             raise ValueError(
                 "delete_dir_contents called on path '", path, "'")
-        self._delete_dir_contents(path)
+        self._delete_dir_contents(path, missing_dir_ok)
 
     def delete_root_dir_contents(self):
         self._delete_dir_contents("/")
diff --git a/python/pyarrow/includes/libarrow_fs.pxd b/python/pyarrow/includes/libarrow_fs.pxd
index 07a57a151c..e491233e88 100644
--- a/python/pyarrow/includes/libarrow_fs.pxd
+++ b/python/pyarrow/includes/libarrow_fs.pxd
@@ -67,7 +67,7 @@ cdef extern from "arrow/filesystem/api.h" namespace "arrow::fs" nogil:
         CResult[vector[CFileInfo]] GetFileInfo(const CFileSelector& select)
         CStatus CreateDir(const c_string& path, c_bool recursive)
         CStatus DeleteDir(const c_string& path)
-        CStatus DeleteDirContents(const c_string& path)
+        CStatus DeleteDirContents(const c_string& path, c_bool missing_dir_ok)
         CStatus DeleteRootDirContents()
         CStatus DeleteFile(const c_string& path)
         CStatus DeleteFiles(const vector[c_string]& paths)
@@ -254,7 +254,7 @@ ctypedef void CallbackGetFileInfoSelector(object, const CFileSelector&,
                                           vector[CFileInfo]*)
 ctypedef void CallbackCreateDir(object, const c_string&, c_bool)
 ctypedef void CallbackDeleteDir(object, const c_string&)
-ctypedef void CallbackDeleteDirContents(object, const c_string&)
+ctypedef void CallbackDeleteDirContents(object, const c_string&, c_bool)
 ctypedef void CallbackDeleteRootDirContents(object)
 ctypedef void CallbackDeleteFile(object, const c_string&)
 ctypedef void CallbackMove(object, const c_string&, const c_string&)
diff --git a/python/pyarrow/tests/test_fs.py b/python/pyarrow/tests/test_fs.py
index 4dcca99946..33c91933bf 100644
--- a/python/pyarrow/tests/test_fs.py
+++ b/python/pyarrow/tests/test_fs.py
@@ -101,7 +101,7 @@ class DummyHandler(FileSystemHandler):
     def delete_dir(self, path):
         assert path == "delete_dir"
 
-    def delete_dir_contents(self, path):
+    def delete_dir_contents(self, path, missing_dir_ok):
         if not path.strip("/"):
             raise ValueError
         assert path == "delete_dir_contents"
@@ -836,6 +836,9 @@ def test_delete_dir_contents(fs, pathfn):
     fs.delete_dir_contents(d)
     with pytest.raises(pa.ArrowIOError):
         fs.delete_dir(nd)
+    fs.delete_dir_contents(nd, missing_dir_ok=True)
+    with pytest.raises(pa.ArrowIOError):
+        fs.delete_dir_contents(nd)
     fs.delete_dir(d)
     with pytest.raises(pa.ArrowIOError):
         fs.delete_dir(d)
diff --git a/python/pyarrow/tests/util.py b/python/pyarrow/tests/util.py
index bd88ae4013..9d22e6e6a2 100644
--- a/python/pyarrow/tests/util.py
+++ b/python/pyarrow/tests/util.py
@@ -290,8 +290,9 @@ class ProxyHandler(pyarrow.fs.FileSystemHandler):
     def delete_dir(self, path):
         return self._fs.delete_dir(path)
 
-    def delete_dir_contents(self, path):
-        return self._fs.delete_dir_contents(path)
+    def delete_dir_contents(self, path, missing_dir_ok):
+        return self._fs.delete_dir_contents(path,
+                                            missing_dir_ok=missing_dir_ok)
 
     def delete_root_dir_contents(self):
         return self._fs.delete_dir_contents("", accept_root_dir=True)