You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "westonpace (via GitHub)" <gi...@apache.org> on 2023/05/05 01:30:40 UTC

[GitHub] [arrow] westonpace opened a new pull request, #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

westonpace opened a new pull request, #35440:
URL: https://github.com/apache/arrow/pull/35440

   ### Rationale for this change
   
   The old model of "walk"ing the directory could lead to a large number of calls.  If someone is fully listing a bucket they will need to make one S3 API call for every single directory in the bucket.  With this approach there is only 1 call made for every 1000 files, regardless of how they are spread across directories.
   
   The only potential regression would be if max_recursion was set to something > 1.  For example, if a user had:
   
   ```
   bucket/foo/bar/<10000 files here>
   ```
   
   Then if they make a request for `bucket` with `max_recursion=2` the new approach will list all 10,000 files and then eliminate the files that don't match.
   
   However, I believe these cases (using max_recursion) to be rarer and less common than the typical case of listing all files (which dataset discovery does).
   
   ### What changes are included in this PR?
   
   The algorithm behind GetFileInfo and DeleteDirContents in S3FileSystem has changed.
   
   ### Are these changes tested?
   
   Yes, there should be no behavior change.  All of the existing filesystem tests will test this change.
   
   ### Are there any user-facing changes?
   
   No, other than (hopefully) better performance.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1535589911

   * Closes: #34213


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1267379629


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2312,35 +2287,137 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
       futures.push_back(std::move(fut));
     }
 
-    return AllComplete(futures);
+    return AllFinished(futures);
   }
 
   Status DeleteObjects(const std::string& bucket, const std::vector<std::string>& keys) {
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();
+    }
+    auto self = shared_from_this();
+    return DeferNotOk(SubmitIO(io_context_, [self, bucket, key]() mutable -> Status {
+      S3Model::HeadObjectRequest req;
+      req.SetBucket(ToAwsString(bucket));
+      req.SetKey(ToAwsString(key));
+
+      ARROW_ASSIGN_OR_RAISE(auto client_lock, self->holder_->Lock());
+      auto outcome = client_lock.Move()->HeadObject(req);
+      if (outcome.IsSuccess()) {
+        const auto& result = outcome.GetResult();
+        if (result.GetContentLength() > 0 || key[key.size() - 1] != '/') {
+          return Status::IOError("Cannot delete directory contents at ", bucket, kSep,
+                                 key, " because it is a file");
+        }

Review Comment:
   I changed the method to `EnsureIsDirAsync` and changed it so that it returns `Result<bool>` and moved the error message into `DeleteDirContentsAsync`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243016045


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));

Review Comment:
   I did and I put it in path_util.h



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth =
+            static_cast<int>(std::count(child_key.begin(), child_key.end(), kSep));
+        int depth = child_depth - base_depth;
+
+        if (depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
 
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      }
+    }
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,

Review Comment:
   Added comment here.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1267028411


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2312,35 +2287,137 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
       futures.push_back(std::move(fut));
     }
 
-    return AllComplete(futures);
+    return AllFinished(futures);
   }
 
   Status DeleteObjects(const std::string& bucket, const std::vector<std::string>& keys) {
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();
+    }
+    auto self = shared_from_this();
+    return DeferNotOk(SubmitIO(io_context_, [self, bucket, key]() mutable -> Status {
+      S3Model::HeadObjectRequest req;
+      req.SetBucket(ToAwsString(bucket));
+      req.SetKey(ToAwsString(key));
+
+      ARROW_ASSIGN_OR_RAISE(auto client_lock, self->holder_->Lock());
+      auto outcome = client_lock.Move()->HeadObject(req);
+      if (outcome.IsSuccess()) {
+        const auto& result = outcome.GetResult();
+        if (result.GetContentLength() > 0 || key[key.size() - 1] != '/') {
+          return Status::IOError("Cannot delete directory contents at ", bucket, kSep,
+                                 key, " because it is a file");
+        }

Review Comment:
   It seems a bit weird to have this code in a helper function named `EnsureFileAsync`.
   Perhaps make this a private helper inside `DoDeleteDirContentsAsync`, or have this return a `Result<FileType>` instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1535589933

   :warning: GitHub issue #34213 **has been automatically assigned in GitHub** to PR creator.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1607067492

   @westonpace Thanks for the analysis. I agree this looks very beneficial.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou merged pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou merged PR #35440:
URL: https://github.com/apache/arrow/pull/35440


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1241889516


##########
cpp/src/arrow/filesystem/path_util.cc:
##########
@@ -66,6 +67,28 @@ std::vector<std::string> SplitAbstractPath(const std::string& path, char sep) {
   return parts;
 }
 
+std::string SliceAbstractPath(const std::string& s, int offset, int length, char sep) {
+  if (offset < 0 || length < 0) {
+    return "";
+  }
+  std::vector<std::string> components = SplitAbstractPath(s, sep);
+  std::stringstream combined;
+  if (offset >= static_cast<int>(components.size())) {
+    return "";
+  }
+  int end = length;

Review Comment:
   Why not use  `length` directly?



##########
cpp/src/arrow/filesystem/path_util.h:
##########
@@ -38,9 +38,17 @@ constexpr char kSep = '/';
 ARROW_EXPORT
 std::vector<std::string> SplitAbstractPath(const std::string& path, char sep = kSep);
 
-// Return the extension of the file
+// Slice the individual components of an abstract path and combine them
+//
+// If offset or length are negative then an empty string is returned
+// If offset is >= the number of components then an empty string is returned
+// If offset + length is >= the number of components then length is truncated
 ARROW_EXPORT
-std::string GetAbstractPathExtension(const std::string& s);
+std::string SliceAbstractPath(const std::string& path, int offset, int length,

Review Comment:
   Can you add basic tests for this in `filesystem_test.cc`?



##########
cpp/src/arrow/util/async_util_test.cc:
##########
@@ -204,6 +204,29 @@ TEST(AsyncTaskScheduler, InitialTaskFails) {
   ASSERT_FINISHES_AND_RAISES(Invalid, finished);
 }
 
+TEST(AsyncTaskScheduler, TaskDestroyedBeforeSchedulerEnds) {
+  bool my_task_destroyed = false;

Review Comment:
   Should it be an atomic?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {

Review Comment:
   Either take a const-ref, or move the value below?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2116,28 +2083,86 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();

Review Comment:
   But this will also run the future continuation on this thread rather on the IO thread pool?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2285,6 +2301,11 @@ Result<FileInfo> S3FileSystem::GetFileInfo(const std::string& s) {
 
     auto outcome = impl_->client_->HeadObject(req);
     if (outcome.IsSuccess()) {
+      bool ends_in_slash = s[s.size() - 1] == '/';
+      if (outcome.GetResult().GetContentLength() == 0 && ends_in_slash) {
+        info.set_type(FileType::Directory);
+        return info;
+      }

Review Comment:
   Why is this change necessary?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2344,42 +2352,28 @@ FileInfoGenerator S3FileSystem::GetFileInfoGenerator(const FileSelector& select)
   }
   auto base_path = *std::move(maybe_base_path);
 
-  if (base_path.empty()) {
-    // List all buckets, then possibly recurse
-    PushGenerator<AsyncGenerator<FileInfoVector>> gen;
-    auto producer = gen.producer();
-
-    auto fut = impl_->ListBucketsAsync(io_context());
-    auto impl = impl_->shared_from_this();
-    fut.AddCallback(
-        [producer, select, impl](const Result<std::vector<std::string>>& res) mutable {
-          if (!res.ok()) {
-            producer.Push(res.status());
-            producer.Close();
-            return;
-          }
-          FileInfoVector buckets;
-          for (const auto& bucket : *res) {
-            buckets.push_back(FileInfo{bucket, FileType::Directory});
-          }
-          // Generate all bucket infos
-          auto buckets_fut = Future<FileInfoVector>::MakeFinished(std::move(buckets));
-          producer.Push(MakeSingleFutureGenerator(buckets_fut));
-          if (select.recursive) {
-            // Generate recursive walk for each bucket in turn
-            for (const auto& bucket : *buckets_fut.result()) {
-              producer.Push(impl->WalkAsync(select, bucket.path(), ""));
-            }
-          }
-          producer.Close();
-        });
+  PushGenerator<std::vector<FileInfo>> generator;
+  auto start_listing = [&, sink = generator.producer()](
+                           util::AsyncTaskScheduler* scheduler) {
+    if (base_path.empty()) {
+      bool should_recurse = select.recursive && select.max_recursion > 0;
+      impl_->FullListAsync(/*include_virtual=*/true, scheduler, sink, io_context(),
+                           should_recurse);
+    } else {
+      impl_->ListAsync(select, base_path.bucket, base_path.key,
+                       /*include_virtual=*/true, scheduler, sink, /*close_sink=*/false);
+    }
+    return Status::OK();
+  };
 
-    return MakeConcatenatedGenerator(
-        AsyncGenerator<AsyncGenerator<FileInfoVector>>{std::move(gen)});
-  }
+  Future<> all_done_fut = util::AsyncTaskScheduler::Make(
+      std::move(start_listing), [](const Status&) {}, StopToken::Unstoppable());

Review Comment:
   ```suggestion
     Future<> all_done_fut = util::AsyncTaskScheduler::Make(
         std::move(start_listing), /*abort_callback=*/ [](const Status&) {}, StopToken::Unstoppable());
   ```



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth =
+            static_cast<int>(std::count(child_key.begin(), child_key.end(), kSep));
+        int depth = child_depth - base_depth;
+
+        if (depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
 
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      }
+    }
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,

Review Comment:
   What is "include_virtual"?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible

Review Comment:
   ```suggestion
         // * If the key is not empty, then it's possible
   ```



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;

Review Comment:
   Can we make most of these members `const` to distinguish between settings and mutable state?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();

Review Comment:
   Does this mean we're in the top-level task? Why not do it from the task continuation?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and

Review Comment:
   What does "the file itself didn't exist and there were not files under it" mean?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;

Review Comment:
   Why not do this above?
   ```c++
   const auto parent_dir = internal::GetAbstractPathParent(current).first;
   ```



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth =
+            static_cast<int>(std::count(child_key.begin(), child_key.end(), kSep));
+        int depth = child_depth - base_depth;
+
+        if (depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
 
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      }
+    }
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink,
+                 bool close_sink) {
+    // We can only fetch kListObjectsMaxKeys files at a time and so we create a
+    // scheduler and schedule a task to grab the first batch.  Once that's done we
+    // schedule a new task for the next batch.  All of these tasks share the same
+    // FileListerState object but none of these tasks run in parallel so there is
+    // no need to worry about mutexes
+    auto state = std::make_shared<FileListerState>(
+        sink, select, bucket, key, include_virtual, io_context_, client_, close_sink);
 
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
+    // Create the first file lister task (it may spawn more)
+    auto file_lister_task = std::make_unique<FileListerTask>(state, scheduler);
+    scheduler->AddTask(std::move(file_lister_task));
   }
 
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
+  Future<std::vector<std::string>> ListBucketsAsync(io::IOContext ctx) {
     auto self = shared_from_this();
-
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
-      }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
-      }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
-      }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
+    return DeferNotOk(SubmitIO(ctx, [self]() { return self->client_->ListBuckets(); }))
+        // TODO(ARROW-12655) Change to Then(Impl::ProcessListBuckets)
+        .Then([](const Aws::S3::Model::ListBucketsOutcome& outcome) {
+          return Impl::ProcessListBuckets(outcome);
         });
-    return gen;
   }
 
-  struct WalkResult {
-    std::vector<std::string> file_keys;
-    std::vector<std::string> dir_keys;
-  };
-  Future<std::shared_ptr<WalkResult>> WalkForDeleteDirAsync(const std::string& bucket,
-                                                            const std::string& key) {
-    auto state = std::make_shared<WalkResult>();
-
-    auto handle_results = [state](const std::string& prefix,
-                                  const S3Model::ListObjectsV2Result& result) -> Status {
-      // Walk "files"
-      state->file_keys.reserve(state->file_keys.size() + result.GetContents().size());
-      for (const auto& obj : result.GetContents()) {
-        state->file_keys.emplace_back(FromAwsString(obj.GetKey()));
-      }
-      // Walk "directories"
-      state->dir_keys.reserve(state->dir_keys.size() + result.GetCommonPrefixes().size());
-      for (const auto& prefix : result.GetCommonPrefixes()) {
-        state->dir_keys.emplace_back(FromAwsString(prefix.GetPrefix()));
-      }
-      return Status::OK();
-    };
-
-    auto handle_error = [=](const AWSError<S3Errors>& error) -> Status {
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
+  // Fully list all files from all buckets
+  void FullListAsync(bool include_virtual, util::AsyncTaskScheduler* scheduler,
+                     FileInfoSink sink, io::IOContext io_context, bool recursive) {
     auto self = shared_from_this();
-    auto handle_recursion = [self](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return true;  // Recurse
-    };
-
-    return TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                 handle_results, handle_error, handle_recursion)
-        .Then([state]() { return state; });
+    scheduler->AddSimpleTask(
+        [self, scheduler, sink, io_context, include_virtual, recursive]() mutable {
+          return self->ListBucketsAsync(io_context)
+              .Then([self, scheduler, sink, include_virtual,
+                     recursive](const std::vector<std::string>& buckets) mutable {
+                // Return the buckets themselves as directories
+                std::vector<FileInfo> buckets_as_directories =
+                    MakeDirectoryInfos(buckets);
+                sink.Push(std::move(buckets_as_directories));
+
+                if (recursive) {
+                  // Recursively list each bucket (these will run in parallel but out_gen

Review Comment:
   ```suggestion
                     // Recursively list each bucket (these will run in parallel but sink
   ```



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;

Review Comment:
   What does "virtual" mean here?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));

Review Comment:
   Write a small helper for this?
   ```c++
   int GetAbstractPathDepth(util::string_view path) {
     if (path.empty()) { return 0; };
     return static_cast<int>(std::count(path.begin(), path.end(), kSep));
   }
   ```
   



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {

Review Comment:
   Please add a comment explaining what this does.



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2116,28 +2083,86 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();

Review Comment:
   I assume this is not a problem because our task functions will schedule the actual IO using `SubmitIO`, right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243013209


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {

Review Comment:
   I now move the value



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1535591025

   I'm leaving this in draft while I do more profiling.  I have already tested the worst case scenario (10k files spread across 10k directories) and it improves performance by 10-15x when testing from my desktop to S3.  I've also tested the flat scenario (10k files in the bucket with no directories) and there is no regression.
   
   I also want to test running from within EC2.  I expect the performance gains to be smaller since the request latency is smaller but there should still be some gain.
   
   Finally, I want to run some local perf tests with minio.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1629372805

   @pitrou I was hoping to get it in.  There are users that are waiting for this feature.  I've rebased the latest fix for the client holder so it should be ready for review still.  There is also a fix in here for a bug when deleting a large number of files though I don't know how critical that part is.
   
   However, I do know that things are a bit overwhelming at the moment with trying to stabilize for the release.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1641863240

   @westonpace Can you reintegrate the changes from https://github.com/apache/arrow/commit/c9ec4d3fed827072d1a913aac848ca661e68ac7e ? They were clobbered when you force-pushed.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243015023


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible

Review Comment:
   I've reworded this comment entirely.



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and

Review Comment:
   I've reworded the comment.  Please take a look.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1255075926


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,307 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_virtual;

Review Comment:
   I changed to `include_implicit_dirs` and updated the wording in comments from "virtual" to "implicit".



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,307 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_virtual;
+    const io::IOContext io_context;
+    const std::shared_ptr<Aws::S3::S3Client> client;
+
+    S3Model::ListObjectsV2Request req;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    void Finish() {
+      // `empty` means that we didn't get a single file info back from S3.  This may be
+      // a situation that we should consider as PathNotFound.
+      //
+      // * If the prefix is empty then we were querying the contents of an entire bucket
+      //   and this is not a PathNotFound case because if the bucket didn't exist then
+      //   we would have received an error and not an empty set of results.
+      //
+      // * If the prefix is not empty then we asked for all files under a particular
+      //   directory.  S3 will also return the directory itself, if it exists.  So if
+      //   we get zero results then we know that there are no files under the directory
+      //   and the directory itself doesn't exist.  This should be considered PathNotFound
+      if (empty && !allow_not_found && !req.GetPrefix().empty()) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+    }
+
+    // Given a path, iterate through all possible sub-paths and, if we haven't
+    // seen that sub-path before, return it.
+    //
+    // For example, given A/B/C we might return A/B and A if we have not seen
+    // those paths before.  This allows us to consider "virtual directories" which
+    // don't exist as objects in S3 but can be inferred.
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        const std::string parent_dir = internal::GetAbstractPathParent(current).first;
+        if (parent_dir.empty()) {
+          break;
+        }
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_dir));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth = internal::GetAbstractPathDepth(prefix);
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth = internal::GetAbstractPathDepth(child_key);
+        // Recursion depth is 1 smaller because a path with depth 1 (e.g. foo) is
+        // considered to have a "recursion" of 0
+        int recursion_depth = child_depth - base_depth - 1;
+        if (recursion_depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = recursion_depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      } else {
+        // Otherwise, we have finished listing all the files
+        state->Finish();
+      }
+    }
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
+  // Lists all file, potentially recursively, in a bucket
+  //
+  // include_virtual controls whether or not "virtual" files should be included.  These
+  // are files that are not actually file objects but instead are inferred from other
+  // objects.
+  //
+  // For example, if a file exists with path A/B/C then virtual directories A/ and A/B/
+  // will exist even if there are no file objects with these paths.
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink) {
+    // We can only fetch kListObjectsMaxKeys files at a time and so we create a
+    // scheduler and schedule a task to grab the first batch.  Once that's done we
+    // schedule a new task for the next batch.  All of these tasks share the same
+    // FileListerState object but none of these tasks run in parallel so there is
+    // no need to worry about mutexes
+    auto state = std::make_shared<FileListerState>(sink, select, bucket, key,
+                                                   include_virtual, io_context_, client_);
+
+    // Create the first file lister task (it may spawn more)
+    auto file_lister_task = std::make_unique<FileListerTask>(state, scheduler);
+    scheduler->AddTask(std::move(file_lister_task));
   }
 
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
+  Future<std::vector<std::string>> ListBucketsAsync(io::IOContext ctx) {
     auto self = shared_from_this();
-
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
-      }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
-      }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
-      }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
+    return DeferNotOk(SubmitIO(ctx, [self]() { return self->client_->ListBuckets(); }))
+        // TODO(ARROW-12655) Change to Then(Impl::ProcessListBuckets)

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1624487941

   I added a stress test for GetFileInfoGenerator and DeleteDirContents.  It was very useful (detected two bugs, one in this PR and one that existed before) however it is pretty slow (doubles the runtime of s3fs_test on my system).  The main slow part is I need to create more than 1000 files so that the `kListObjectsMaxKeys` and `kMultipleDeleteMaxKeys` limits are applied.
   
   @pitrou , I'd be curious to know if you think this test is worth leaving in or if I should remove it back out.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] dalbani commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "dalbani (via GitHub)" <gi...@apache.org>.
dalbani commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1577327996

   Thanks for the report @westonpace 👍 
   I can't wait to see this improvement in a future release, should the final tests confirm what you found so far.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1267379815


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2312,35 +2287,137 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
       futures.push_back(std::move(fut));
     }
 
-    return AllComplete(futures);
+    return AllFinished(futures);
   }
 
   Status DeleteObjects(const std::string& bucket, const std::vector<std::string>& keys) {
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();
+    }
+    auto self = shared_from_this();
+    return DeferNotOk(SubmitIO(io_context_, [self, bucket, key]() mutable -> Status {
+      S3Model::HeadObjectRequest req;
+      req.SetBucket(ToAwsString(bucket));
+      req.SetKey(ToAwsString(key));
+
+      ARROW_ASSIGN_OR_RAISE(auto client_lock, self->holder_->Lock());
+      auto outcome = client_lock.Move()->HeadObject(req);
+      if (outcome.IsSuccess()) {
+        const auto& result = outcome.GetResult();
+        if (result.GetContentLength() > 0 || key[key.size() - 1] != '/') {
+          return Status::IOError("Cannot delete directory contents at ", bucket, kSep,
+                                 key, " because it is a file");
+        }
+        return Status::OK();
+      }
+      if (IsNotFound(outcome.GetError())) {
+        // Might be ok, let DeleteDirContentsAsync worry about this
+        return Status::OK();
+      } else {
+        return ErrorToStatus(std::forward_as_tuple("When getting information for key '",
+                                                   key, "' in bucket '", bucket, "': "),
+                             "HeadObject", outcome.GetError());
+      }
+    }));
+  }
+
+  // Some operations require running multiple S3 calls, either in parallel or serially. We
+  // need to ensure that the S3 filesystem instance stays valid and that S3 isn't
+  // finalized.  We do this by wrapping all the tasks in a scheduler which keeps the
+  // resources alive
+  Future<> RunInScheduler(
+      std::function<Status(util::AsyncTaskScheduler*, S3FileSystem::Impl*)> callable) {
+    auto self = shared_from_this();
+    FnOnce<Status(util::AsyncTaskScheduler*)> initial_task =
+        [callable = std::move(callable),
+         this](util::AsyncTaskScheduler* scheduler) mutable {
+          return callable(scheduler, this);
+        };
+    Future<> scheduler_fut = util::AsyncTaskScheduler::Make(
+        std::move(initial_task),
+        /*abort_callback=*/
+        [](const Status& st) {
+          // No need for special abort logic.
+        },
+        StopToken::Unstoppable());

Review Comment:
   Done.  I also added a test for cancellation to make sure it works correctly.  There was one other spot I had to change to get it to pass correctly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1268939494


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2068,197 +1932,306 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
-      for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
-        const auto child_key =
-            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        FileInfo info;
-        info.set_path(child_path.str());
-        info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_implicit_dirs;
+    const io::IOContext io_context;
+    S3FileSystem::Impl* self;
+
+    S3Model::ListObjectsV2Request req;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_implicit_dirs,
+                    io::IOContext io_context, S3FileSystem::Impl* self)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_implicit_dirs(include_implicit_dirs),
+          io_context(io_context),
+          self(self) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
       }
-      // Walk "files"
-      for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
-          continue;
-        }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
       }
-      return Status::OK();
     }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
-        }
+    void Finish() {
+      // `empty` means that we didn't get a single file info back from S3.  This may be
+      // a situation that we should consider as PathNotFound.
+      //
+      // * If the prefix is empty then we were querying the contents of an entire bucket
+      //   and this is not a PathNotFound case because if the bucket didn't exist then
+      //   we would have received an error and not an empty set of results.
+      //
+      // * If the prefix is not empty then we asked for all files under a particular
+      //   directory.  S3 will also return the directory itself, if it exists.  So if
+      //   we get zero results then we know that there are no files under the directory
+      //   and the directory itself doesn't exist.  This should be considered PathNotFound
+      if (empty && !allow_not_found && !req.GetPrefix().empty()) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
       }
-      return Status::OK();
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
+    // Given a path, iterate through all possible sub-paths and, if we haven't
+    // seen that sub-path before, return it.
+    //
+    // For example, given A/B/C we might return A/B and A if we have not seen
+    // those paths before.  This allows us to consider "implicit" directories which
+    // don't exist as objects in S3 but can be inferred.
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        const std::string parent_dir = internal::GetAbstractPathParent(current).first;
+        if (parent_dir.empty()) {
+          break;
+        }
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_dir));
+        }
+      }
+      return new_directories;
+    }
   };
 
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    RETURN_NOT_OK(CheckS3Initialized());
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
 
-    FileInfoCollector collector(bucket, key, select);
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
 
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
+      for (const auto& child_prefix : result.GetCommonPrefixes()) {
+        const auto child_key =
+            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
+        FileInfo info;
+        info.set_path(child_path_ss.str());
+        info.set_type(FileType::Directory);
+        file_infos.push_back(std::move(info));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
-
-    RETURN_NOT_OK(TreeWalker::Walk(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
-
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
-  }
-
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
-    auto self = shared_from_this();
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth = internal::GetAbstractPathDepth(prefix);
+      for (const auto& obj : result.GetContents()) {
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
+          continue;
+        }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth = internal::GetAbstractPathDepth(child_key);
+        // Recursion depth is 1 smaller because a path with depth 1 (e.g. foo) is
+        // considered to have a "recursion" of 0
+        int recursion_depth = child_depth - base_depth - 1;
+        if (recursion_depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = recursion_depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+        if (state->include_implicit_dirs) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
+        }
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
+      if (file_infos.size() > 0) {
+        state->empty = false;
       }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      return file_infos;
+    }
 
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      Result<S3ClientLock> client_lock = state->self->holder_->Lock();
+      if (!client_lock.ok()) {
+        state->files_queue.Push(client_lock.status());
+        return;
       }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
-        });
-    return gen;
-  }
-
-  struct WalkResult {
-    std::vector<std::string> file_keys;
-    std::vector<std::string> dir_keys;
-  };
-  Future<std::shared_ptr<WalkResult>> WalkForDeleteDirAsync(const std::string& bucket,
-                                                            const std::string& key) {
-    auto state = std::make_shared<WalkResult>();
-
-    auto handle_results = [state](const std::string& prefix,
-                                  const S3Model::ListObjectsV2Result& result) -> Status {
-      // Walk "files"
-      state->file_keys.reserve(state->file_keys.size() + result.GetContents().size());
-      for (const auto& obj : result.GetContents()) {
-        state->file_keys.emplace_back(FromAwsString(obj.GetKey()));
+      S3Model::ListObjectsV2Outcome outcome =
+          client_lock->Move()->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
       }
-      // Walk "directories"
-      state->dir_keys.reserve(state->dir_keys.size() + result.GetCommonPrefixes().size());
-      for (const auto& prefix : result.GetCommonPrefixes()) {
-        state->dir_keys.emplace_back(FromAwsString(prefix.GetPrefix()));
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return Status::OK();
-    };
 
-    auto handle_error = [=](const AWSError<S3Errors>& error) -> Status {
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      } else {
+        // Otherwise, we have finished listing all the files
+        state->Finish();
+      }
+    }
 
-    auto self = shared_from_this();
-    auto handle_recursion = [self](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return true;  // Recurse
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    return TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                 handle_results, handle_error, handle_recursion)
-        .Then([state]() { return state; });
+  // Lists all file, potentially recursively, in a bucket
+  //
+  // include_implicit_dirs controls whether or not implicit directories should be
+  // included. These are directories that are not actually file objects but instead are
+  // inferred from other objects.
+  //
+  // For example, if a file exists with path A/B/C then implicit directories A/ and A/B/
+  // will exist even if there are no file objects with these paths.
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_implicit_dirs,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink,
+                 S3FileSystem::Impl* self) {

Review Comment:
   Yes.  Good catch.  I'm not sure what I was thinking.  I've removed these arguments.



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2068,197 +1932,306 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
-      for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
-        const auto child_key =
-            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        FileInfo info;
-        info.set_path(child_path.str());
-        info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_implicit_dirs;
+    const io::IOContext io_context;
+    S3FileSystem::Impl* self;
+
+    S3Model::ListObjectsV2Request req;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_implicit_dirs,
+                    io::IOContext io_context, S3FileSystem::Impl* self)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_implicit_dirs(include_implicit_dirs),
+          io_context(io_context),
+          self(self) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
       }
-      // Walk "files"
-      for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
-          continue;
-        }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
       }
-      return Status::OK();
     }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
-        }
+    void Finish() {
+      // `empty` means that we didn't get a single file info back from S3.  This may be
+      // a situation that we should consider as PathNotFound.
+      //
+      // * If the prefix is empty then we were querying the contents of an entire bucket
+      //   and this is not a PathNotFound case because if the bucket didn't exist then
+      //   we would have received an error and not an empty set of results.
+      //
+      // * If the prefix is not empty then we asked for all files under a particular
+      //   directory.  S3 will also return the directory itself, if it exists.  So if
+      //   we get zero results then we know that there are no files under the directory
+      //   and the directory itself doesn't exist.  This should be considered PathNotFound
+      if (empty && !allow_not_found && !req.GetPrefix().empty()) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
       }
-      return Status::OK();
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
+    // Given a path, iterate through all possible sub-paths and, if we haven't
+    // seen that sub-path before, return it.
+    //
+    // For example, given A/B/C we might return A/B and A if we have not seen
+    // those paths before.  This allows us to consider "implicit" directories which
+    // don't exist as objects in S3 but can be inferred.
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        const std::string parent_dir = internal::GetAbstractPathParent(current).first;
+        if (parent_dir.empty()) {
+          break;
+        }
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_dir));
+        }
+      }
+      return new_directories;
+    }
   };
 
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    RETURN_NOT_OK(CheckS3Initialized());
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
 
-    FileInfoCollector collector(bucket, key, select);
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
 
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
+      for (const auto& child_prefix : result.GetCommonPrefixes()) {
+        const auto child_key =
+            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
+        FileInfo info;
+        info.set_path(child_path_ss.str());
+        info.set_type(FileType::Directory);
+        file_infos.push_back(std::move(info));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
-
-    RETURN_NOT_OK(TreeWalker::Walk(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
-
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
-  }
-
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
-    auto self = shared_from_this();
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth = internal::GetAbstractPathDepth(prefix);
+      for (const auto& obj : result.GetContents()) {
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
+          continue;
+        }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth = internal::GetAbstractPathDepth(child_key);
+        // Recursion depth is 1 smaller because a path with depth 1 (e.g. foo) is
+        // considered to have a "recursion" of 0
+        int recursion_depth = child_depth - base_depth - 1;
+        if (recursion_depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = recursion_depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+        if (state->include_implicit_dirs) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
+        }
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
+      if (file_infos.size() > 0) {
+        state->empty = false;
       }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      return file_infos;
+    }
 
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      Result<S3ClientLock> client_lock = state->self->holder_->Lock();
+      if (!client_lock.ok()) {
+        state->files_queue.Push(client_lock.status());
+        return;
       }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
-        });
-    return gen;
-  }
-
-  struct WalkResult {
-    std::vector<std::string> file_keys;
-    std::vector<std::string> dir_keys;
-  };
-  Future<std::shared_ptr<WalkResult>> WalkForDeleteDirAsync(const std::string& bucket,
-                                                            const std::string& key) {
-    auto state = std::make_shared<WalkResult>();
-
-    auto handle_results = [state](const std::string& prefix,
-                                  const S3Model::ListObjectsV2Result& result) -> Status {
-      // Walk "files"
-      state->file_keys.reserve(state->file_keys.size() + result.GetContents().size());
-      for (const auto& obj : result.GetContents()) {
-        state->file_keys.emplace_back(FromAwsString(obj.GetKey()));
+      S3Model::ListObjectsV2Outcome outcome =
+          client_lock->Move()->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
       }
-      // Walk "directories"
-      state->dir_keys.reserve(state->dir_keys.size() + result.GetCommonPrefixes().size());
-      for (const auto& prefix : result.GetCommonPrefixes()) {
-        state->dir_keys.emplace_back(FromAwsString(prefix.GetPrefix()));
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return Status::OK();
-    };
 
-    auto handle_error = [=](const AWSError<S3Errors>& error) -> Status {
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      } else {
+        // Otherwise, we have finished listing all the files
+        state->Finish();
+      }
+    }
 
-    auto self = shared_from_this();
-    auto handle_recursion = [self](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return true;  // Recurse
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    return TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                 handle_results, handle_error, handle_recursion)
-        .Then([state]() { return state; });
+  // Lists all file, potentially recursively, in a bucket
+  //
+  // include_implicit_dirs controls whether or not implicit directories should be
+  // included. These are directories that are not actually file objects but instead are
+  // inferred from other objects.
+  //
+  // For example, if a file exists with path A/B/C then implicit directories A/ and A/B/
+  // will exist even if there are no file objects with these paths.
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_implicit_dirs,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink,
+                 S3FileSystem::Impl* self) {
+    // We can only fetch kListObjectsMaxKeys files at a time and so we create a
+    // scheduler and schedule a task to grab the first batch.  Once that's done we
+    // schedule a new task for the next batch.  All of these tasks share the same
+    // FileListerState object but none of these tasks run in parallel so there is
+    // no need to worry about mutexes
+    auto state = std::make_shared<FileListerState>(
+        sink, select, bucket, key, include_implicit_dirs, io_context_, self);
+
+    // Create the first file lister task (it may spawn more)
+    auto file_lister_task = std::make_unique<FileListerTask>(state, scheduler);
+    scheduler->AddTask(std::move(file_lister_task));
+  }
+
+  // Fully list all files from all buckets
+  void FullListAsync(bool include_implicit_dirs, util::AsyncTaskScheduler* scheduler,
+                     FileInfoSink sink, io::IOContext io_context, bool recursive,
+                     S3FileSystem::Impl* self) {

Review Comment:
   I've removed this argument



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243013062


##########
cpp/src/arrow/filesystem/path_util.cc:
##########
@@ -66,6 +67,28 @@ std::vector<std::string> SplitAbstractPath(const std::string& path, char sep) {
   return parts;
 }
 
+std::string SliceAbstractPath(const std::string& s, int offset, int length, char sep) {
+  if (offset < 0 || length < 0) {
+    return "";
+  }
+  std::vector<std::string> components = SplitAbstractPath(s, sep);
+  std::stringstream combined;
+  if (offset >= static_cast<int>(components.size())) {
+    return "";
+  }
+  int end = length;

Review Comment:
   Actually, this should be `length + offset`.  I've fixed it.



##########
cpp/src/arrow/filesystem/path_util.h:
##########
@@ -38,9 +38,17 @@ constexpr char kSep = '/';
 ARROW_EXPORT
 std::vector<std::string> SplitAbstractPath(const std::string& path, char sep = kSep);
 
-// Return the extension of the file
+// Slice the individual components of an abstract path and combine them
+//
+// If offset or length are negative then an empty string is returned
+// If offset is >= the number of components then an empty string is returned
+// If offset + length is >= the number of components then length is truncated
 ARROW_EXPORT
-std::string GetAbstractPathExtension(const std::string& s);
+std::string SliceAbstractPath(const std::string& path, int offset, int length,

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243017032


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2344,42 +2352,28 @@ FileInfoGenerator S3FileSystem::GetFileInfoGenerator(const FileSelector& select)
   }
   auto base_path = *std::move(maybe_base_path);
 
-  if (base_path.empty()) {
-    // List all buckets, then possibly recurse
-    PushGenerator<AsyncGenerator<FileInfoVector>> gen;
-    auto producer = gen.producer();
-
-    auto fut = impl_->ListBucketsAsync(io_context());
-    auto impl = impl_->shared_from_this();
-    fut.AddCallback(
-        [producer, select, impl](const Result<std::vector<std::string>>& res) mutable {
-          if (!res.ok()) {
-            producer.Push(res.status());
-            producer.Close();
-            return;
-          }
-          FileInfoVector buckets;
-          for (const auto& bucket : *res) {
-            buckets.push_back(FileInfo{bucket, FileType::Directory});
-          }
-          // Generate all bucket infos
-          auto buckets_fut = Future<FileInfoVector>::MakeFinished(std::move(buckets));
-          producer.Push(MakeSingleFutureGenerator(buckets_fut));
-          if (select.recursive) {
-            // Generate recursive walk for each bucket in turn
-            for (const auto& bucket : *buckets_fut.result()) {
-              producer.Push(impl->WalkAsync(select, bucket.path(), ""));
-            }
-          }
-          producer.Close();
-        });
+  PushGenerator<std::vector<FileInfo>> generator;
+  auto start_listing = [&, sink = generator.producer()](
+                           util::AsyncTaskScheduler* scheduler) {
+    if (base_path.empty()) {
+      bool should_recurse = select.recursive && select.max_recursion > 0;
+      impl_->FullListAsync(/*include_virtual=*/true, scheduler, sink, io_context(),
+                           should_recurse);
+    } else {
+      impl_->ListAsync(select, base_path.bucket, base_path.key,
+                       /*include_virtual=*/true, scheduler, sink, /*close_sink=*/false);
+    }
+    return Status::OK();
+  };
 
-    return MakeConcatenatedGenerator(
-        AsyncGenerator<AsyncGenerator<FileInfoVector>>{std::move(gen)});
-  }
+  Future<> all_done_fut = util::AsyncTaskScheduler::Make(
+      std::move(start_listing), [](const Status&) {}, StopToken::Unstoppable());

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243014753


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;

Review Comment:
   I've added a comment to `ListAsync`.  Imagine there is a file object at path `A/B/C`.  Sometimes there are empty file objects at `A/` and `A/B/`.  Sometimes there are not.  From what I've seen, S3 libs differ on whether these should be created or not.  Arrow-c++ does create these files (in MakeDirectory).  However, I don't want to depend on them existing.
   
   Virtual files are included when the user is listing files (`GetFileInfo`).  However, they are not included when the user is deleting files recursively (since there is no actual file object to delete).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1604505953

   I've gone through and done some more thorough testing now.  I was surprised to see we even have a big improvement when running in EC2, even if we're in the same datacenter (where the latency should be very low).  I tested with two test datasets.
   
   The first test dataset (s3://ursa-qa/wide-partition) was 10,000 files split into 10,000 folders nested two deep:
   
   ```
   x=0/y=0/file.parquet
   x=0/y=1/file.parquet
   ...
   x=0/y=99/file.parquet
   x=1/y=0/file.parquet
   ...
   x=99/y=99/file.parquet
   ```
   
   The second dataset (s3://ursa-qa/flat-partition) was 10,000 files in the root folder:
   
   ```
   file-0.parquet
   file-1.parquet
   ...
   file-9999.parquet
   ```
   
   For all of my tests I timed how long it took to recursively listed all files in the dataset.  I ran the tests on my local desktop (outside of EC2, on an EC2 server that was in a different region (us-west-2) and on an EC2 server that was in the same region (us-east-2).  All times are in seconds.
   
   There were (as hoped) significant performance improvements in the wide-partition dataset:
   
   ![Screenshot 2023-06-23 at 08-43-34 Online Graph Maker · Plotly Chart Studio](https://github.com/apache/arrow/assets/1696093/6819bdd9-30f8-48ec-b44d-792b335ee2ad)
   
   Regrettably, there might be a slight regression in the flat-partition dataset, although it is largely within the noise.  I have run the tests frequently enough that I feel it is stable:
   
   ![Screenshot 2023-06-23 at 08-44-36 Online Graph Maker · Plotly Chart Studio](https://github.com/apache/arrow/assets/1696093/53fcd6e0-1f3f-47ef-8f4e-f264ecd3770a)
   
   I've verified that, in both the old and new approach, we are sending the same number of HTTP messages to S3 and the content is very close (less than 300 bytes difference).  I don't think it is additional compute time (or else I'd expect to see a worse regression on the AWS servers).
   
   We could keep both styles (tree walking and recursive listing) but I don't think this regression is significant enough to justify the complexity.
   
   There is one other case that would likely regress.  That is the case where the data is deeply partitioned (e.g. each file is 4 or 5 folders deep) and the user specifies a low max recursion.  For example...
   
   ```
   a=0/b=0/c=0/d=0/e=0/file-0.parquet
   ...
   a=0/b=0/c=0/d=0/e=0/file-9999.parquet
   ```
   
   I would expect no regression if I fully listed the above dataset.  However, if I listed the above dataset with a max_recursion of 2 then the old approach would likely be much faster since it only needs to return 1 file info (the new approach would return all 10k file infos and then we would pare them down in memory).  I'm not aware of anyone using this use case (pyarrow doesn't even expose max_recursion) and so I'm not sure if it is worth the complexity of keeping both approaches.  Even in this case I suspect we would be looking at a difference of 0.3 vs 3 seconds which is better than our current worst case (~100 seconds).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1650102730

   @github-actions crossbow submit -g cpp


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] cboettig commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "cboettig (via GitHub)" <gi...@apache.org>.
cboettig commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1604667887

   Thanks @westonpace , this matches what we see as well.  We're really looking forward to seeing this merged, will be hugely beneficial on our current workflows.  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243015692


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();

Review Comment:
   Not necessarily the top-level task but it means that we've finished all tasks.  This happens as the final task is finishing.  However, I agree this makes more sense in the task continuation and I've moved it there (this code is no longer a destructor which was kind of confusing).



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243017657


##########
cpp/src/arrow/util/async_util_test.cc:
##########
@@ -204,6 +204,29 @@ TEST(AsyncTaskScheduler, InitialTaskFails) {
   ASSERT_FINISHES_AND_RAISES(Invalid, finished);
 }
 
+TEST(AsyncTaskScheduler, TaskDestroyedBeforeSchedulerEnds) {
+  bool my_task_destroyed = false;

Review Comment:
   No.  The AsyncTaskScheduler itself does not actually spawn thread tasks (in fact, it doesn't include executor.h or have any knowledge of thread pools).  So this test does not involve threads at all and is entirely synchronous.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243127374


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();

Review Comment:
   Also, in this process I ended up getting rid of `close_sink`and moved the logic higher up.  I think it has better symmetry as the close of the sink is at the same level as creating the sink.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1629123214

   @westonpace Is it ok if we defer this to 14.0.0?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] github-actions[bot] commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1650107642

   Revision: 7edea033cc09f7e2e97ccaffd9ae6464712a823c
   
   Submitted crossbow builds: [ursacomputing/crossbow @ actions-21347af314](https://github.com/ursacomputing/crossbow/branches/all?query=actions-21347af314)
   
   |Task|Status|
   |----|------|
   |test-alpine-linux-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-alpine-linux-cpp)](https://github.com/ursacomputing/crossbow/actions/runs/5659013545/job/15331612549)|
   |test-build-cpp-fuzz|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-build-cpp-fuzz)](https://github.com/ursacomputing/crossbow/actions/runs/5659012132/job/15331607950)|
   |test-conda-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-conda-cpp)](https://github.com/ursacomputing/crossbow/actions/runs/5659011796/job/15331607046)|
   |test-conda-cpp-valgrind|[![Azure](https://dev.azure.com/ursacomputing/crossbow/_apis/build/status/ursacomputing.crossbow?branchName=actions-21347af314-azure-test-conda-cpp-valgrind)](https://github.com/ursacomputing/crossbow/runs/15331613209)|
   |test-cuda-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-cuda-cpp)](https://github.com/ursacomputing/crossbow/actions/runs/5659014975/job/15331617885)|
   |test-debian-11-cpp-amd64|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-debian-11-cpp-amd64)](https://github.com/ursacomputing/crossbow/actions/runs/5659014492/job/15331616080)|
   |test-debian-11-cpp-i386|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-debian-11-cpp-i386)](https://github.com/ursacomputing/crossbow/actions/runs/5659012440/job/15331608920)|
   |test-fedora-35-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-fedora-35-cpp)](https://github.com/ursacomputing/crossbow/actions/runs/5659013000/job/15331610587)|
   |test-ubuntu-20.04-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-ubuntu-20.04-cpp)](https://github.com/ursacomputing/crossbow/actions/runs/5659015290/job/15331619195)|
   |test-ubuntu-20.04-cpp-bundled|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-ubuntu-20.04-cpp-bundled)](https://github.com/ursacomputing/crossbow/actions/runs/5659014121/job/15331614448)|
   |test-ubuntu-20.04-cpp-minimal-with-formats|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-ubuntu-20.04-cpp-minimal-with-formats)](https://github.com/ursacomputing/crossbow/actions/runs/5659011605/job/15331606437)|
   |test-ubuntu-20.04-cpp-thread-sanitizer|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-ubuntu-20.04-cpp-thread-sanitizer)](https://github.com/ursacomputing/crossbow/actions/runs/5659013797/job/15331613459)|
   |test-ubuntu-22.04-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-ubuntu-22.04-cpp)](https://github.com/ursacomputing/crossbow/actions/runs/5659012659/job/15331609768)|
   |test-ubuntu-22.04-cpp-20|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-21347af314-github-test-ubuntu-22.04-cpp-20)](https://github.com/ursacomputing/crossbow/actions/runs/5659014318/job/15331615323)|


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1245417508


##########
cpp/src/arrow/filesystem/path_util.cc:
##########
@@ -66,6 +67,42 @@ std::vector<std::string> SplitAbstractPath(const std::string& path, char sep) {
   return parts;
 }
 
+std::string SliceAbstractPath(const std::string& s, int offset, int length, char sep) {
+  if (offset < 0 || length < 0) {
+    return "";
+  }
+  std::vector<std::string> components = SplitAbstractPath(s, sep);
+  std::stringstream combined;
+  if (offset >= static_cast<int>(components.size())) {
+    return "";
+  }
+  int end = offset + length;
+  if (end > static_cast<int>(components.size())) {
+    end = static_cast<int>(components.size());
+  }
+  for (int i = offset; i < end; i++) {
+    combined << components[i];
+    if (i < end - 1) {
+      combined << sep;
+    }
+  }
+  return combined.str();
+}
+
+int GetAbstractPathDepth(std::string_view path) {
+  if (path.empty()) {
+    return 0;
+  }
+  int depth = static_cast<int>(std::count(path.begin(), path.end(), kSep)) + 1;
+  if (path[path.size() - 1] == kSep) {
+    depth -= 1;
+  }
+  if (path.size() > 0 && path[0] == kSep) {
+    depth -= 1;
+  }

Review Comment:
   Or just simply:
   ```suggestion
     if (path.back() == kSep) {
       depth -= 1;
     }
     if (path.front() == kSep) {
       depth -= 1;
     }
   ```



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,307 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_virtual;
+    const io::IOContext io_context;
+    const std::shared_ptr<Aws::S3::S3Client> client;
+
+    S3Model::ListObjectsV2Request req;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    void Finish() {
+      // `empty` means that we didn't get a single file info back from S3.  This may be
+      // a situation that we should consider as PathNotFound.
+      //
+      // * If the prefix is empty then we were querying the contents of an entire bucket
+      //   and this is not a PathNotFound case because if the bucket didn't exist then
+      //   we would have received an error and not an empty set of results.
+      //
+      // * If the prefix is not empty then we asked for all files under a particular
+      //   directory.  S3 will also return the directory itself, if it exists.  So if
+      //   we get zero results then we know that there are no files under the directory
+      //   and the directory itself doesn't exist.  This should be considered PathNotFound
+      if (empty && !allow_not_found && !req.GetPrefix().empty()) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+    }
+
+    // Given a path, iterate through all possible sub-paths and, if we haven't
+    // seen that sub-path before, return it.
+    //
+    // For example, given A/B/C we might return A/B and A if we have not seen
+    // those paths before.  This allows us to consider "virtual directories" which
+    // don't exist as objects in S3 but can be inferred.
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        const std::string parent_dir = internal::GetAbstractPathParent(current).first;
+        if (parent_dir.empty()) {
+          break;
+        }
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_dir));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth = internal::GetAbstractPathDepth(prefix);
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth = internal::GetAbstractPathDepth(child_key);
+        // Recursion depth is 1 smaller because a path with depth 1 (e.g. foo) is
+        // considered to have a "recursion" of 0
+        int recursion_depth = child_depth - base_depth - 1;
+        if (recursion_depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = recursion_depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      } else {
+        // Otherwise, we have finished listing all the files
+        state->Finish();
+      }
+    }
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
+  // Lists all file, potentially recursively, in a bucket
+  //
+  // include_virtual controls whether or not "virtual" files should be included.  These
+  // are files that are not actually file objects but instead are inferred from other
+  // objects.
+  //
+  // For example, if a file exists with path A/B/C then virtual directories A/ and A/B/
+  // will exist even if there are no file objects with these paths.
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink) {
+    // We can only fetch kListObjectsMaxKeys files at a time and so we create a
+    // scheduler and schedule a task to grab the first batch.  Once that's done we
+    // schedule a new task for the next batch.  All of these tasks share the same
+    // FileListerState object but none of these tasks run in parallel so there is
+    // no need to worry about mutexes
+    auto state = std::make_shared<FileListerState>(sink, select, bucket, key,
+                                                   include_virtual, io_context_, client_);
+
+    // Create the first file lister task (it may spawn more)
+    auto file_lister_task = std::make_unique<FileListerTask>(state, scheduler);
+    scheduler->AddTask(std::move(file_lister_task));
   }
 
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
+  Future<std::vector<std::string>> ListBucketsAsync(io::IOContext ctx) {
     auto self = shared_from_this();
-
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
-      }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
-      }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
-      }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
+    return DeferNotOk(SubmitIO(ctx, [self]() { return self->client_->ListBuckets(); }))
+        // TODO(ARROW-12655) Change to Then(Impl::ProcessListBuckets)

Review Comment:
   ```suggestion
           // TODO(GH-18652) Change to Then(Impl::ProcessListBuckets)
   ```



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth =
+            static_cast<int>(std::count(child_key.begin(), child_key.end(), kSep));
+        int depth = child_depth - base_depth;
+
+        if (depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
 
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      }
+    }
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,

Review Comment:
   Thanks for the explanation. Perhaps make the name more descriptive as already suggested above?



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,307 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_virtual;

Review Comment:
   Perhaps name this `include_implicit_directories`? It seems more descriptive than `include_virtual`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] rqthomas commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "rqthomas (via GitHub)" <gi...@apache.org>.
rqthomas commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1649975935

   I want to echo @cboettig about our excitement for this upgrade to arrow and hope to see it in the next release. It seems that it will greatly accelerate our data pipelines!  Thanks so much for your work developing and testing! 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1624850605

   > The main slow part is I need to create more than 1000 files so that the `kListObjectsMaxKeys` and `kMultipleDeleteMaxKeys` limits are applied.
   
   Have you tried creating those files in parallel using a ThreadPool?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1625619571

   > Have you tried creating those files in parallel using a ThreadPool?
   
   Yes, it is doing that.  Also, I believe this is ready for review again.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1634266659

   > @westonpace Did you rebase this on the latest S3 fixes?
   
   Yes, it should have the latest client_lock fixes (using `Move`).


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243015875


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;

Review Comment:
   I've switched to this.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243016244


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;
+    int max_recursion;
+    bool include_virtual;
+    S3Model::ListObjectsV2Request req;
+    io::IOContext io_context;
+    std::shared_ptr<Aws::S3::S3Client> client;
+    bool close_sink;
+    bool no_files_means_not_found;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_virtual,
+                    io::IOContext io_context, std::shared_ptr<Aws::S3::S3Client> client,
+                    bool close_sink)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_virtual(include_virtual),
+          io_context(io_context),
+          client(std::move(client)),
+          close_sink(close_sink),
+          no_files_means_not_found(!select.allow_not_found && !key.empty()) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
+      }
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
+      }
+    }
+
+    // The FileListerState is kept alive by the various FileListerTasks.  Once all the
+    // tasks are finished this will be destroyed and we can run some cleanup
+    ~FileListerState() {
+      // * If the bucket doesn't exist we will have already gotten an error from the
+      //     ListObjectsV2 call
+      // * If the key is empty, and the bucket exists, then there is
+      //     no way we can hit "not found"
+      // * If they key is not empty, then it's possible
+      //     that the file itself didn't exist and there
+      //     were not files under it.  In that case we will hit this if statement and
+      //     should treat this as a "not found" case.
+      if (empty && no_files_means_not_found) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
+      }
+      if (close_sink) {
+        files_queue.Close();
+      }
+    }
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        auto parent_base = internal::GetAbstractPathParent(current);
+        if (parent_base.first.empty()) {
+          break;
+        }
+        const std::string& parent_dir = parent_base.first;
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_base.first));
+        }
+      }
+      return new_directories;
+    }
+  };
+
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
+
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
+
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
       for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
         const auto child_key =
             internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
         FileInfo info;
-        info.set_path(child_path.str());
+        info.set_path(child_path_ss.str());
         info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+        file_infos.push_back(std::move(info));
       }
-      // Walk "files"
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth =
+          (prefix.empty())
+              ? 0
+              : static_cast<int>(std::count(prefix.begin(), prefix.end(), kSep));
       for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
           continue;
         }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
-      }
-      return Status::OK();
-    }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth =
+            static_cast<int>(std::count(child_key.begin(), child_key.end(), kSep));
+        int depth = child_depth - base_depth;
+
+        if (depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
+        if (state->include_virtual) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
         }
       }
-      return Status::OK();
+      if (file_infos.size() > 0) {
+        state->empty = false;
+      }
+      return file_infos;
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
-  };
-
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    FileInfoCollector collector(bucket, key, select);
-
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      S3Model::ListObjectsV2Outcome outcome = state->client->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
+      }
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
 
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      }
+    }
 
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    RETURN_NOT_OK(TreeWalker::Walk(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_virtual,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink,
+                 bool close_sink) {
+    // We can only fetch kListObjectsMaxKeys files at a time and so we create a
+    // scheduler and schedule a task to grab the first batch.  Once that's done we
+    // schedule a new task for the next batch.  All of these tasks share the same
+    // FileListerState object but none of these tasks run in parallel so there is
+    // no need to worry about mutexes
+    auto state = std::make_shared<FileListerState>(
+        sink, select, bucket, key, include_virtual, io_context_, client_, close_sink);
 
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
+    // Create the first file lister task (it may spawn more)
+    auto file_lister_task = std::make_unique<FileListerTask>(state, scheduler);
+    scheduler->AddTask(std::move(file_lister_task));
   }
 
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
+  Future<std::vector<std::string>> ListBucketsAsync(io::IOContext ctx) {
     auto self = shared_from_this();
-
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
-      }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
-      }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
-      }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
+    return DeferNotOk(SubmitIO(ctx, [self]() { return self->client_->ListBuckets(); }))
+        // TODO(ARROW-12655) Change to Then(Impl::ProcessListBuckets)
+        .Then([](const Aws::S3::Model::ListBucketsOutcome& outcome) {
+          return Impl::ProcessListBuckets(outcome);
         });
-    return gen;
   }
 
-  struct WalkResult {
-    std::vector<std::string> file_keys;
-    std::vector<std::string> dir_keys;
-  };
-  Future<std::shared_ptr<WalkResult>> WalkForDeleteDirAsync(const std::string& bucket,
-                                                            const std::string& key) {
-    auto state = std::make_shared<WalkResult>();
-
-    auto handle_results = [state](const std::string& prefix,
-                                  const S3Model::ListObjectsV2Result& result) -> Status {
-      // Walk "files"
-      state->file_keys.reserve(state->file_keys.size() + result.GetContents().size());
-      for (const auto& obj : result.GetContents()) {
-        state->file_keys.emplace_back(FromAwsString(obj.GetKey()));
-      }
-      // Walk "directories"
-      state->dir_keys.reserve(state->dir_keys.size() + result.GetCommonPrefixes().size());
-      for (const auto& prefix : result.GetCommonPrefixes()) {
-        state->dir_keys.emplace_back(FromAwsString(prefix.GetPrefix()));
-      }
-      return Status::OK();
-    };
-
-    auto handle_error = [=](const AWSError<S3Errors>& error) -> Status {
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
+  // Fully list all files from all buckets
+  void FullListAsync(bool include_virtual, util::AsyncTaskScheduler* scheduler,
+                     FileInfoSink sink, io::IOContext io_context, bool recursive) {
     auto self = shared_from_this();
-    auto handle_recursion = [self](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return true;  // Recurse
-    };
-
-    return TreeWalker::WalkAsync(client_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                 handle_results, handle_error, handle_recursion)
-        .Then([state]() { return state; });
+    scheduler->AddSimpleTask(
+        [self, scheduler, sink, io_context, include_virtual, recursive]() mutable {
+          return self->ListBucketsAsync(io_context)
+              .Then([self, scheduler, sink, include_virtual,
+                     recursive](const std::vector<std::string>& buckets) mutable {
+                // Return the buckets themselves as directories
+                std::vector<FileInfo> buckets_as_directories =
+                    MakeDirectoryInfos(buckets);
+                sink.Push(std::move(buckets_as_directories));
+
+                if (recursive) {
+                  // Recursively list each bucket (these will run in parallel but out_gen

Review Comment:
   Fixed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243016888


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2116,28 +2083,86 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();

Review Comment:
   This is only called in one place which then immediately calls `DoDeleteDirContentsAsync` which does put us on the IO thread pool.  However, I also don't think you can reasonable expect that an Async method is guaranteed to spawn a new task (regrettably).



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2285,6 +2301,11 @@ Result<FileInfo> S3FileSystem::GetFileInfo(const std::string& s) {
 
     auto outcome = impl_->client_->HeadObject(req);
     if (outcome.IsSuccess()) {
+      bool ends_in_slash = s[s.size() - 1] == '/';
+      if (outcome.GetResult().GetContentLength() == 0 && ends_in_slash) {
+        info.set_type(FileType::Directory);
+        return info;
+      }

Review Comment:
   It's not, I've removed it.  I think I was just trying to make sense of the way `GetFileInfo` works with directories while debugging failures.  We create these kinds of files when we create directories recursively.  For example, when we create `foo/bar/baz` recursively (in some bucket `my_bucket`) we will create empty files `foo/` and `foo/bar/` and I was concerned we were reading these as files.
   
   However, if `s` is `foo/bar/` here then the trailing slash will be removed in `S3Path::FromString`.  So this newly added branch is impossible to encounter.  Instead, we will look for `foo/bar` and fail to find it.  We will then eventually decide it is a directory because we search for `foo/bar/` as part of `IsNonEmptyDirectory`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243013384


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -1870,195 +1731,301 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(dirname);
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
+
+  struct FileListerState {
+    FileInfoSink files_queue;
+    bool allow_not_found;

Review Comment:
   Done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1243017657


##########
cpp/src/arrow/util/async_util_test.cc:
##########
@@ -204,6 +204,29 @@ TEST(AsyncTaskScheduler, InitialTaskFails) {
   ASSERT_FINISHES_AND_RAISES(Invalid, finished);
 }
 
+TEST(AsyncTaskScheduler, TaskDestroyedBeforeSchedulerEnds) {
+  bool my_task_destroyed = false;

Review Comment:
   No.  The AsyncTaskScheduler itself does not actually spawn thread tasks.  It relies on the tasks themselves to do this (in fact, it doesn't include executor.h or have any knowledge of thread pools).  So this test does not involve threads at all and is entirely synchronous.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1608560360

   @pitrou thanks for the review!  I believe I've addressed your feedback.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1634221936

   @westonpace Did you rebase this on the latest S3 fixes?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1267907137


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2068,197 +1932,306 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
-      for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
-        const auto child_key =
-            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        FileInfo info;
-        info.set_path(child_path.str());
-        info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_implicit_dirs;
+    const io::IOContext io_context;
+    S3FileSystem::Impl* self;
+
+    S3Model::ListObjectsV2Request req;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_implicit_dirs,
+                    io::IOContext io_context, S3FileSystem::Impl* self)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_implicit_dirs(include_implicit_dirs),
+          io_context(io_context),
+          self(self) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
       }
-      // Walk "files"
-      for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
-          continue;
-        }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
       }
-      return Status::OK();
     }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
-        }
+    void Finish() {
+      // `empty` means that we didn't get a single file info back from S3.  This may be
+      // a situation that we should consider as PathNotFound.
+      //
+      // * If the prefix is empty then we were querying the contents of an entire bucket
+      //   and this is not a PathNotFound case because if the bucket didn't exist then
+      //   we would have received an error and not an empty set of results.
+      //
+      // * If the prefix is not empty then we asked for all files under a particular
+      //   directory.  S3 will also return the directory itself, if it exists.  So if
+      //   we get zero results then we know that there are no files under the directory
+      //   and the directory itself doesn't exist.  This should be considered PathNotFound
+      if (empty && !allow_not_found && !req.GetPrefix().empty()) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
       }
-      return Status::OK();
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
+    // Given a path, iterate through all possible sub-paths and, if we haven't
+    // seen that sub-path before, return it.
+    //
+    // For example, given A/B/C we might return A/B and A if we have not seen
+    // those paths before.  This allows us to consider "implicit" directories which
+    // don't exist as objects in S3 but can be inferred.
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        const std::string parent_dir = internal::GetAbstractPathParent(current).first;
+        if (parent_dir.empty()) {
+          break;
+        }
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_dir));
+        }
+      }
+      return new_directories;
+    }
   };
 
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    RETURN_NOT_OK(CheckS3Initialized());
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
 
-    FileInfoCollector collector(bucket, key, select);
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
 
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
+      for (const auto& child_prefix : result.GetCommonPrefixes()) {
+        const auto child_key =
+            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
+        FileInfo info;
+        info.set_path(child_path_ss.str());
+        info.set_type(FileType::Directory);
+        file_infos.push_back(std::move(info));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
-
-    RETURN_NOT_OK(TreeWalker::Walk(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
-
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
-  }
-
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
-    auto self = shared_from_this();
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth = internal::GetAbstractPathDepth(prefix);
+      for (const auto& obj : result.GetContents()) {
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
+          continue;
+        }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth = internal::GetAbstractPathDepth(child_key);
+        // Recursion depth is 1 smaller because a path with depth 1 (e.g. foo) is
+        // considered to have a "recursion" of 0
+        int recursion_depth = child_depth - base_depth - 1;
+        if (recursion_depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = recursion_depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+        if (state->include_implicit_dirs) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
+        }
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
+      if (file_infos.size() > 0) {
+        state->empty = false;
       }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      return file_infos;
+    }
 
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      Result<S3ClientLock> client_lock = state->self->holder_->Lock();
+      if (!client_lock.ok()) {
+        state->files_queue.Push(client_lock.status());
+        return;
       }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
-        });
-    return gen;
-  }
-
-  struct WalkResult {
-    std::vector<std::string> file_keys;
-    std::vector<std::string> dir_keys;
-  };
-  Future<std::shared_ptr<WalkResult>> WalkForDeleteDirAsync(const std::string& bucket,
-                                                            const std::string& key) {
-    auto state = std::make_shared<WalkResult>();
-
-    auto handle_results = [state](const std::string& prefix,
-                                  const S3Model::ListObjectsV2Result& result) -> Status {
-      // Walk "files"
-      state->file_keys.reserve(state->file_keys.size() + result.GetContents().size());
-      for (const auto& obj : result.GetContents()) {
-        state->file_keys.emplace_back(FromAwsString(obj.GetKey()));
+      S3Model::ListObjectsV2Outcome outcome =
+          client_lock->Move()->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
       }
-      // Walk "directories"
-      state->dir_keys.reserve(state->dir_keys.size() + result.GetCommonPrefixes().size());
-      for (const auto& prefix : result.GetCommonPrefixes()) {
-        state->dir_keys.emplace_back(FromAwsString(prefix.GetPrefix()));
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return Status::OK();
-    };
 
-    auto handle_error = [=](const AWSError<S3Errors>& error) -> Status {
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      } else {
+        // Otherwise, we have finished listing all the files
+        state->Finish();
+      }
+    }
 
-    auto self = shared_from_this();
-    auto handle_recursion = [self](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return true;  // Recurse
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    return TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                 handle_results, handle_error, handle_recursion)
-        .Then([state]() { return state; });
+  // Lists all file, potentially recursively, in a bucket
+  //
+  // include_implicit_dirs controls whether or not implicit directories should be
+  // included. These are directories that are not actually file objects but instead are
+  // inferred from other objects.
+  //
+  // For example, if a file exists with path A/B/C then implicit directories A/ and A/B/
+  // will exist even if there are no file objects with these paths.
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_implicit_dirs,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink,
+                 S3FileSystem::Impl* self) {
+    // We can only fetch kListObjectsMaxKeys files at a time and so we create a
+    // scheduler and schedule a task to grab the first batch.  Once that's done we
+    // schedule a new task for the next batch.  All of these tasks share the same
+    // FileListerState object but none of these tasks run in parallel so there is
+    // no need to worry about mutexes
+    auto state = std::make_shared<FileListerState>(
+        sink, select, bucket, key, include_implicit_dirs, io_context_, self);
+
+    // Create the first file lister task (it may spawn more)
+    auto file_lister_task = std::make_unique<FileListerTask>(state, scheduler);
+    scheduler->AddTask(std::move(file_lister_task));
+  }
+
+  // Fully list all files from all buckets
+  void FullListAsync(bool include_implicit_dirs, util::AsyncTaskScheduler* scheduler,
+                     FileInfoSink sink, io::IOContext io_context, bool recursive,
+                     S3FileSystem::Impl* self) {

Review Comment:
   Same question. Also, the `IOContext` needn't be passed explicitly either (it's just `this->io_context_`).



##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2068,197 +1932,306 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
         "ListObjectsV2", outcome.GetError());
   }
 
-  Status CheckNestingDepth(int32_t nesting_depth) {
-    if (nesting_depth >= kMaxNestingDepth) {
-      return Status::IOError("S3 filesystem tree exceeds maximum nesting depth (",
-                             kMaxNestingDepth, ")");
+  static FileInfo MakeDirectoryInfo(std::string dirname) {
+    FileInfo dir;
+    dir.set_type(FileType::Directory);
+    dir.set_path(std::move(dirname));
+    return dir;
+  }
+
+  static std::vector<FileInfo> MakeDirectoryInfos(std::vector<std::string> dirnames) {
+    std::vector<FileInfo> dir_infos;
+    for (auto& dirname : dirnames) {
+      dir_infos.push_back(MakeDirectoryInfo(std::move(dirname)));
     }
-    return Status::OK();
+    return dir_infos;
   }
 
-  // A helper class for Walk and WalkAsync
-  struct FileInfoCollector {
-    FileInfoCollector(std::string bucket, std::string key, const FileSelector& select)
-        : bucket(std::move(bucket)),
-          key(std::move(key)),
-          allow_not_found(select.allow_not_found) {}
+  using FileInfoSink = PushGenerator<std::vector<FileInfo>>::Producer;
 
-    Status Collect(const std::string& prefix, const S3Model::ListObjectsV2Result& result,
-                   std::vector<FileInfo>* out) {
-      // Walk "directories"
-      for (const auto& child_prefix : result.GetCommonPrefixes()) {
-        is_empty = false;
-        const auto child_key =
-            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        FileInfo info;
-        info.set_path(child_path.str());
-        info.set_type(FileType::Directory);
-        out->push_back(std::move(info));
+  struct FileListerState {
+    FileInfoSink files_queue;
+    const bool allow_not_found;
+    const int max_recursion;
+
+    const bool include_implicit_dirs;
+    const io::IOContext io_context;
+    S3FileSystem::Impl* self;
+
+    S3Model::ListObjectsV2Request req;
+    std::unordered_set<std::string> directories;
+    bool empty = true;
+
+    FileListerState(PushGenerator<std::vector<FileInfo>>::Producer files_queue,
+                    FileSelector select, const std::string& bucket,
+                    const std::string& key, bool include_implicit_dirs,
+                    io::IOContext io_context, S3FileSystem::Impl* self)
+        : files_queue(std::move(files_queue)),
+          allow_not_found(select.allow_not_found),
+          max_recursion(select.max_recursion),
+          include_implicit_dirs(include_implicit_dirs),
+          io_context(io_context),
+          self(self) {
+      req.SetBucket(bucket);
+      req.SetMaxKeys(kListObjectsMaxKeys);
+      if (!key.empty()) {
+        req.SetPrefix(key + kSep);
       }
-      // Walk "files"
-      for (const auto& obj : result.GetContents()) {
-        is_empty = false;
-        FileInfo info;
-        const auto child_key = internal::RemoveTrailingSlash(FromAwsString(obj.GetKey()));
-        if (child_key == std::string_view(prefix)) {
-          // Amazon can return the "directory" key itself as part of the results, skip
-          continue;
-        }
-        std::stringstream child_path;
-        child_path << bucket << kSep << child_key;
-        info.set_path(child_path.str());
-        FileObjectToInfo(obj, &info);
-        out->push_back(std::move(info));
+      if (!select.recursive) {
+        req.SetDelimiter(Aws::String() + kSep);
       }
-      return Status::OK();
     }
 
-    Status Finish(Impl* impl) {
-      // If no contents were found, perhaps it's an empty "directory",
-      // or perhaps it's a nonexistent entry.  Check.
-      if (is_empty && !allow_not_found) {
-        ARROW_ASSIGN_OR_RAISE(bool is_actually_empty,
-                              impl->IsEmptyDirectory(bucket, key));
-        if (!is_actually_empty) {
-          return PathNotFound(bucket, key);
-        }
+    void Finish() {
+      // `empty` means that we didn't get a single file info back from S3.  This may be
+      // a situation that we should consider as PathNotFound.
+      //
+      // * If the prefix is empty then we were querying the contents of an entire bucket
+      //   and this is not a PathNotFound case because if the bucket didn't exist then
+      //   we would have received an error and not an empty set of results.
+      //
+      // * If the prefix is not empty then we asked for all files under a particular
+      //   directory.  S3 will also return the directory itself, if it exists.  So if
+      //   we get zero results then we know that there are no files under the directory
+      //   and the directory itself doesn't exist.  This should be considered PathNotFound
+      if (empty && !allow_not_found && !req.GetPrefix().empty()) {
+        files_queue.Push(PathNotFound(req.GetBucket(), req.GetPrefix()));
       }
-      return Status::OK();
     }
 
-    std::string bucket;
-    std::string key;
-    bool allow_not_found;
-    bool is_empty = true;
+    // Given a path, iterate through all possible sub-paths and, if we haven't
+    // seen that sub-path before, return it.
+    //
+    // For example, given A/B/C we might return A/B and A if we have not seen
+    // those paths before.  This allows us to consider "implicit" directories which
+    // don't exist as objects in S3 but can be inferred.
+    std::vector<std::string> GetNewDirectories(const std::string_view& path) {
+      std::string current(path);
+      std::string base = req.GetBucket();
+      if (!req.GetPrefix().empty()) {
+        base = base + kSep + std::string(internal::RemoveTrailingSlash(req.GetPrefix()));
+      }
+      std::vector<std::string> new_directories;
+      while (true) {
+        const std::string parent_dir = internal::GetAbstractPathParent(current).first;
+        if (parent_dir.empty()) {
+          break;
+        }
+        current = parent_dir;
+        if (current == base) {
+          break;
+        }
+        if (directories.insert(parent_dir).second) {
+          new_directories.push_back(std::move(parent_dir));
+        }
+      }
+      return new_directories;
+    }
   };
 
-  // Workhorse for GetFileInfo(FileSelector...)
-  Status Walk(const FileSelector& select, const std::string& bucket,
-              const std::string& key, std::vector<FileInfo>* out) {
-    RETURN_NOT_OK(CheckS3Initialized());
+  struct FileListerTask : public util::AsyncTaskScheduler::Task {
+    std::shared_ptr<FileListerState> state;
+    util::AsyncTaskScheduler* scheduler;
 
-    FileInfoCollector collector(bucket, key, select);
+    FileListerTask(std::shared_ptr<FileListerState> state,
+                   util::AsyncTaskScheduler* scheduler)
+        : state(state), scheduler(scheduler) {}
 
-    auto handle_error = [&](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+    std::vector<FileInfo> ToFileInfos(const std::string& bucket,
+                                      const std::string& prefix,
+                                      const S3Model::ListObjectsV2Result& result) {
+      std::vector<FileInfo> file_infos;
+      // If this is a non-recursive listing we may see "common prefixes" which represent
+      // directories we did not recurse into.  We will add those as directories.
+      for (const auto& child_prefix : result.GetCommonPrefixes()) {
+        const auto child_key =
+            internal::RemoveTrailingSlash(FromAwsString(child_prefix.GetPrefix()));
+        std::stringstream child_path_ss;
+        child_path_ss << bucket << kSep << child_key;
+        FileInfo info;
+        info.set_path(child_path_ss.str());
+        info.set_type(FileType::Directory);
+        file_infos.push_back(std::move(info));
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [&](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
-
-    auto handle_results = [&](const std::string& prefix,
-                              const S3Model::ListObjectsV2Result& result) -> Status {
-      return collector.Collect(prefix, result, out);
-    };
-
-    RETURN_NOT_OK(TreeWalker::Walk(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                   handle_results, handle_error, handle_recursion));
-
-    // If no contents were found, perhaps it's an empty "directory",
-    // or perhaps it's a nonexistent entry.  Check.
-    RETURN_NOT_OK(collector.Finish(this));
-    // Sort results for convenience, since they can come massively out of order
-    std::sort(out->begin(), out->end(), FileInfo::ByPath{});
-    return Status::OK();
-  }
-
-  // Workhorse for GetFileInfoGenerator(FileSelector...)
-  FileInfoGenerator WalkAsync(const FileSelector& select, const std::string& bucket,
-                              const std::string& key) {
-    PushGenerator<std::vector<FileInfo>> gen;
-    auto producer = gen.producer();
-    auto collector = std::make_shared<FileInfoCollector>(bucket, key, select);
-    auto self = shared_from_this();
+      // S3 doesn't have any concept of "max depth" and so we emulate it by counting the
+      // number of '/' characters.  E.g. if the user is searching bucket/subdirA/subdirB
+      // then the starting depth is 2.
+      // A file subdirA/subdirB/somefile will have a child depth of 2 and a "depth" of 0.
+      // A file subdirA/subdirB/subdirC/somefile will have a child depth of 3 and a
+      //   "depth" of 1
+      int base_depth = internal::GetAbstractPathDepth(prefix);
+      for (const auto& obj : result.GetContents()) {
+        if (obj.GetKey() == prefix) {
+          // S3 will return the basedir itself (if it is a file / empty file).  We don't
+          // want that.  But this is still considered "finding the basedir" and so we mark
+          // it "not empty".
+          state->empty = false;
+          continue;
+        }
+        std::string child_key =
+            std::string(internal::RemoveTrailingSlash(FromAwsString(obj.GetKey())));
+        bool had_trailing_slash = child_key.size() != obj.GetKey().size();
+        int child_depth = internal::GetAbstractPathDepth(child_key);
+        // Recursion depth is 1 smaller because a path with depth 1 (e.g. foo) is
+        // considered to have a "recursion" of 0
+        int recursion_depth = child_depth - base_depth - 1;
+        if (recursion_depth > state->max_recursion) {
+          // If we have A/B/C/D and max_recursion is 2 then we ignore this (don't add it
+          // to file_infos) but we still want to potentially add A and A/B as directories.
+          // So we "pretend" like we have a file A/B/C for the call to GetNewDirectories
+          // below
+          int to_trim = recursion_depth - state->max_recursion - 1;
+          if (to_trim > 0) {
+            child_key = bucket + kSep +
+                        internal::SliceAbstractPath(child_key, 0, child_depth - to_trim);
+          } else {
+            child_key = bucket + kSep + child_key;
+          }
+        } else {
+          // If the file isn't beyond our max recursion then count it as a file
+          // unless it's empty and then it depends on whether or not the file ends
+          // with a trailing slash
+          std::stringstream child_path_ss;
+          child_path_ss << bucket << kSep << child_key;
+          child_key = child_path_ss.str();
+          if (obj.GetSize() > 0 || !had_trailing_slash) {
+            // We found a real file
+            FileInfo info;
+            info.set_path(child_key);
+            FileObjectToInfo(obj, &info);
+            file_infos.push_back(std::move(info));
+          } else {
+            // We found an empty file and we want to treat it like a directory.  Only
+            // add it if we haven't seen this directory before.
+            if (state->directories.insert(child_key).second) {
+              file_infos.push_back(MakeDirectoryInfo(child_key));
+            }
+          }
+        }
 
-    auto handle_error = [select, bucket, key](const AWSError<S3Errors>& error) -> Status {
-      if (select.allow_not_found && IsNotFound(error)) {
-        return Status::OK();
+        if (state->include_implicit_dirs) {
+          // Now that we've dealt with the file itself we need to look at each of the
+          // parent paths and potentially add them as directories.  For example, after
+          // finding a file A/B/C/D we want to consider adding directories A, A/B, and
+          // A/B/C.
+          for (const auto& newdir : state->GetNewDirectories(child_key)) {
+            file_infos.push_back(MakeDirectoryInfo(newdir));
+          }
+        }
       }
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
-
-    auto handle_recursion = [producer, select,
-                             self](int32_t nesting_depth) -> Result<bool> {
-      if (producer.is_closed()) {
-        return false;
+      if (file_infos.size() > 0) {
+        state->empty = false;
       }
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return select.recursive && nesting_depth <= select.max_recursion;
-    };
+      return file_infos;
+    }
 
-    auto handle_results =
-        [collector, producer](
-            const std::string& prefix,
-            const S3Model::ListObjectsV2Result& result) mutable -> Status {
-      std::vector<FileInfo> out;
-      RETURN_NOT_OK(collector->Collect(prefix, result, &out));
-      if (!out.empty()) {
-        producer.Push(std::move(out));
+    void Run() {
+      // We are on an I/O thread now so just synchronously make the call and interpret the
+      // results.
+      Result<S3ClientLock> client_lock = state->self->holder_->Lock();
+      if (!client_lock.ok()) {
+        state->files_queue.Push(client_lock.status());
+        return;
       }
-      return Status::OK();
-    };
-
-    TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                          handle_results, handle_error, handle_recursion)
-        .AddCallback([collector, producer, self](const Status& status) mutable {
-          auto st = collector->Finish(self.get());
-          if (!st.ok()) {
-            producer.Push(st);
-          }
-          producer.Close();
-        });
-    return gen;
-  }
-
-  struct WalkResult {
-    std::vector<std::string> file_keys;
-    std::vector<std::string> dir_keys;
-  };
-  Future<std::shared_ptr<WalkResult>> WalkForDeleteDirAsync(const std::string& bucket,
-                                                            const std::string& key) {
-    auto state = std::make_shared<WalkResult>();
-
-    auto handle_results = [state](const std::string& prefix,
-                                  const S3Model::ListObjectsV2Result& result) -> Status {
-      // Walk "files"
-      state->file_keys.reserve(state->file_keys.size() + result.GetContents().size());
-      for (const auto& obj : result.GetContents()) {
-        state->file_keys.emplace_back(FromAwsString(obj.GetKey()));
+      S3Model::ListObjectsV2Outcome outcome =
+          client_lock->Move()->ListObjectsV2(state->req);
+      if (!outcome.IsSuccess()) {
+        const auto& err = outcome.GetError();
+        if (state->allow_not_found && IsNotFound(err)) {
+          return;
+        }
+        state->files_queue.Push(
+            ErrorToStatus(std::forward_as_tuple("When listing objects under key '",
+                                                state->req.GetPrefix(), "' in bucket '",
+                                                state->req.GetBucket(), "': "),
+                          "ListObjectsV2", err));
+        return;
       }
-      // Walk "directories"
-      state->dir_keys.reserve(state->dir_keys.size() + result.GetCommonPrefixes().size());
-      for (const auto& prefix : result.GetCommonPrefixes()) {
-        state->dir_keys.emplace_back(FromAwsString(prefix.GetPrefix()));
+      const S3Model::ListObjectsV2Result& result = outcome.GetResult();
+      // We could immediately schedule the continuation (if there are enough results to
+      // trigger paging) but that would introduce race condition complexity for arguably
+      // little benefit.
+      std::vector<FileInfo> file_infos =
+          ToFileInfos(state->req.GetBucket(), state->req.GetPrefix(), result);
+      if (file_infos.size() > 0) {
+        state->files_queue.Push(std::move(file_infos));
       }
-      return Status::OK();
-    };
 
-    auto handle_error = [=](const AWSError<S3Errors>& error) -> Status {
-      return ErrorToStatus(std::forward_as_tuple("When listing objects under key '", key,
-                                                 "' in bucket '", bucket, "': "),
-                           "ListObjectsV2", error);
-    };
+      // If there are enough files to warrant a continuation then go ahead and schedule
+      // that now.
+      if (result.GetIsTruncated()) {
+        DCHECK(!result.GetNextContinuationToken().empty());
+        state->req.SetContinuationToken(result.GetNextContinuationToken());
+        scheduler->AddTask(std::make_unique<FileListerTask>(state, scheduler));
+      } else {
+        // Otherwise, we have finished listing all the files
+        state->Finish();
+      }
+    }
 
-    auto self = shared_from_this();
-    auto handle_recursion = [self](int32_t nesting_depth) -> Result<bool> {
-      RETURN_NOT_OK(self->CheckNestingDepth(nesting_depth));
-      return true;  // Recurse
-    };
+    Result<Future<>> operator()() override {
+      return state->io_context.executor()->Submit([this] {
+        Run();
+        return Status::OK();
+      });
+    }
+    std::string_view name() const override { return "S3ListFiles"; }
+  };
 
-    return TreeWalker::WalkAsync(holder_, io_context_, bucket, key, kListObjectsMaxKeys,
-                                 handle_results, handle_error, handle_recursion)
-        .Then([state]() { return state; });
+  // Lists all file, potentially recursively, in a bucket
+  //
+  // include_implicit_dirs controls whether or not implicit directories should be
+  // included. These are directories that are not actually file objects but instead are
+  // inferred from other objects.
+  //
+  // For example, if a file exists with path A/B/C then implicit directories A/ and A/B/
+  // will exist even if there are no file objects with these paths.
+  void ListAsync(const FileSelector& select, const std::string& bucket,
+                 const std::string& key, bool include_implicit_dirs,
+                 util::AsyncTaskScheduler* scheduler, FileInfoSink sink,
+                 S3FileSystem::Impl* self) {

Review Comment:
   Why do we pass a `S3FileSystem::Impl*` explicitly here? It is just `this`, right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] pitrou commented on a diff in pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "pitrou (via GitHub)" <gi...@apache.org>.
pitrou commented on code in PR #35440:
URL: https://github.com/apache/arrow/pull/35440#discussion_r1267029510


##########
cpp/src/arrow/filesystem/s3fs.cc:
##########
@@ -2312,35 +2287,137 @@ class S3FileSystem::Impl : public std::enable_shared_from_this<S3FileSystem::Imp
       futures.push_back(std::move(fut));
     }
 
-    return AllComplete(futures);
+    return AllFinished(futures);
   }
 
   Status DeleteObjects(const std::string& bucket, const std::vector<std::string>& keys) {
     return DeleteObjectsAsync(bucket, keys).status();
   }
 
+  Future<> EnsureNotFileAsync(const std::string& bucket, const std::string& key) {
+    if (key.empty()) {
+      // There is no way for a bucket to be a file
+      return Future<>::MakeFinished();
+    }
+    auto self = shared_from_this();
+    return DeferNotOk(SubmitIO(io_context_, [self, bucket, key]() mutable -> Status {
+      S3Model::HeadObjectRequest req;
+      req.SetBucket(ToAwsString(bucket));
+      req.SetKey(ToAwsString(key));
+
+      ARROW_ASSIGN_OR_RAISE(auto client_lock, self->holder_->Lock());
+      auto outcome = client_lock.Move()->HeadObject(req);
+      if (outcome.IsSuccess()) {
+        const auto& result = outcome.GetResult();
+        if (result.GetContentLength() > 0 || key[key.size() - 1] != '/') {
+          return Status::IOError("Cannot delete directory contents at ", bucket, kSep,
+                                 key, " because it is a file");
+        }
+        return Status::OK();
+      }
+      if (IsNotFound(outcome.GetError())) {
+        // Might be ok, let DeleteDirContentsAsync worry about this
+        return Status::OK();
+      } else {
+        return ErrorToStatus(std::forward_as_tuple("When getting information for key '",
+                                                   key, "' in bucket '", bucket, "': "),
+                             "HeadObject", outcome.GetError());
+      }
+    }));
+  }
+
+  // Some operations require running multiple S3 calls, either in parallel or serially. We
+  // need to ensure that the S3 filesystem instance stays valid and that S3 isn't
+  // finalized.  We do this by wrapping all the tasks in a scheduler which keeps the
+  // resources alive
+  Future<> RunInScheduler(
+      std::function<Status(util::AsyncTaskScheduler*, S3FileSystem::Impl*)> callable) {
+    auto self = shared_from_this();
+    FnOnce<Status(util::AsyncTaskScheduler*)> initial_task =
+        [callable = std::move(callable),
+         this](util::AsyncTaskScheduler* scheduler) mutable {
+          return callable(scheduler, this);
+        };
+    Future<> scheduler_fut = util::AsyncTaskScheduler::Make(
+        std::move(initial_task),
+        /*abort_callback=*/
+        [](const Status& st) {
+          // No need for special abort logic.
+        },
+        StopToken::Unstoppable());

Review Comment:
   Pass `io_context().stop_token()` instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [arrow] westonpace commented on pull request #35440: GH-34213: [C++] Use recursive calls without a delimiter if the user is doing a recursive GetFileInfo

Posted by "westonpace (via GitHub)" <gi...@apache.org>.
westonpace commented on PR #35440:
URL: https://github.com/apache/arrow/pull/35440#issuecomment-1643204056

   > @westonpace Can you reintegrate the changes from https://github.com/apache/arrow/commit/c9ec4d3fed827072d1a913aac848ca661e68ac7e ? They were clobbered when you force-pushed.
   
   Sorry, I missed that when I rebased.  I've restored the commit.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org