wgtmac commented on code in PR #687:
URL: https://github.com/apache/iceberg-cpp/pull/687#discussion_r3321987627


##########
src/iceberg/manifest/manifest_group.cc:
##########
@@ -376,57 +399,73 @@ ManifestGroup::ReadEntries() {
         Evaluator::Make(*DataFileFilterSchema(), file_filter_, 
case_sensitive_));
   }
 
-  std::unordered_map<int32_t, std::vector<ManifestEntry>> result;
+  std::vector<std::unordered_map<int32_t, std::vector<ManifestEntry>>> 
manifest_results(
+      data_manifests_.size());
 
-  // TODO(gangwu): Parallelize reading manifests
-  for (const auto& manifest : data_manifests_) {
-    const int32_t spec_id = manifest.partition_spec_id;
+  auto read_tasks = TaskGroup().SetExecutor(executor_);
+  for (auto&& [manifest, manifest_result] :
+       std::views::zip(data_manifests_, manifest_results)) {
+    read_tasks.Submit([&]() -> Status {
+      const int32_t spec_id = manifest.partition_spec_id;
 
-    ICEBERG_ASSIGN_OR_RAISE(auto manifest_evaluator, 
get_manifest_evaluator(spec_id));
-    ICEBERG_ASSIGN_OR_RAISE(bool should_match, 
manifest_evaluator->Evaluate(manifest));
-    if (!should_match) {
-      // Skip this manifest because it doesn't match partition filter
-      continue;
-    }
+      ICEBERG_ASSIGN_OR_RAISE(auto manifest_evaluator, 
get_manifest_evaluator(spec_id));
+      ICEBERG_ASSIGN_OR_RAISE(bool should_match, 
manifest_evaluator->Evaluate(manifest));
+      if (!should_match) {
+        // Skip this manifest because it doesn't match partition filter
+        return {};
+      }
 
-    if (ignore_deleted_) {
-      // only scan manifests that have entries other than deletes
-      if (!manifest.has_added_files() && !manifest.has_existing_files()) {
-        continue;
+      if (ignore_deleted_) {
+        // only scan manifests that have entries other than deletes
+        if (!manifest.has_added_files() && !manifest.has_existing_files()) {
+          return {};
+        }
       }
-    }
 
-    if (ignore_existing_) {
-      // only scan manifests that have entries other than existing
-      if (!manifest.has_added_files() && !manifest.has_deleted_files()) {
-        continue;
+      if (ignore_existing_) {
+        // only scan manifests that have entries other than existing
+        if (!manifest.has_added_files() && !manifest.has_deleted_files()) {
+          return {};
+        }
       }
-    }
 
-    // Read manifest entries
-    ICEBERG_ASSIGN_OR_RAISE(auto reader, MakeReader(manifest));
-    ICEBERG_ASSIGN_OR_RAISE(auto entries,
-                            ignore_deleted_ ? reader->LiveEntries() : 
reader->Entries());
+      // Read manifest entries
+      ICEBERG_ASSIGN_OR_RAISE(auto reader, MakeReader(manifest));
+      ICEBERG_ASSIGN_OR_RAISE(
+          auto entries, ignore_deleted_ ? reader->LiveEntries() : 
reader->Entries());
 
-    for (auto& entry : entries) {
-      if (ignore_existing_ && entry.status == ManifestStatus::kExisting) {
-        continue;
-      }
+      for (auto& entry : entries) {
+        if (ignore_existing_ && entry.status == ManifestStatus::kExisting) {
+          continue;
+        }
 
-      if (data_file_evaluator != nullptr) {
-        DataFileStructLike data_file(*entry.data_file);
-        ICEBERG_ASSIGN_OR_RAISE(bool should_match,
-                                data_file_evaluator->Evaluate(data_file));
-        if (!should_match) {
+        if (data_file_evaluator != nullptr) {
+          DataFileStructLike data_file(*entry.data_file);
+          ICEBERG_ASSIGN_OR_RAISE(bool should_match,
+                                  data_file_evaluator->Evaluate(data_file));
+          if (!should_match) {
+            continue;
+          }
+        }
+
+        if (!manifest_entry_predicate_(entry)) {
           continue;
         }
-      }
 
-      if (!manifest_entry_predicate_(entry)) {
-        continue;
+        manifest_result[spec_id].push_back(std::move(entry));
       }
+      return {};
+    });
+  }
+  ICEBERG_RETURN_UNEXPECTED(std::move(read_tasks).Run());
 
-      result[spec_id].push_back(std::move(entry));
+  std::unordered_map<int32_t, std::vector<ManifestEntry>> result;
+  for (auto& manifest_result : manifest_results) {
+    result.merge(manifest_result);

Review Comment:
   This depends on `merge()` moving out the non-conflicting nodes and leaving 
only the conflicting keys, which are then appended. It's correct, but takes a 
moment to parse. A plain loop doing `result[spec_id].insert(...)` for every 
entry reads more directly, at negligible cost.



##########
src/iceberg/util/task_group.h:
##########
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#pragma once
+
+#include <concepts>
+#include <functional>
+#include <optional>
+#include <type_traits>
+#include <utility>
+#include <vector>
+
+#include "iceberg/iceberg_export.h"
+#include "iceberg/result.h"
+#include "iceberg/util/executor.h"
+#include "iceberg/util/functional.h"
+#include "iceberg/util/retry_util.h"
+
+namespace iceberg {
+
+namespace internal {
+
+template <typename F>
+concept OnceStatusTask = RvalueInvocable<Status, F>;
+
+template <typename T>
+concept RepeatableStatusTask =
+    std::is_invocable_r_v<Status, const T&> ||
+    (std::copy_constructible<T> && std::is_invocable_r_v<Status, T&>);
+
+template <typename F>
+concept RetryableStatusTask = std::constructible_from<std::remove_cvref_t<F>, 
F> &&
+                              RepeatableStatusTask<std::remove_cvref_t<F>>;
+
+ICEBERG_EXPORT Status RunTasksSingleThreaded(std::vector<FnOnce<Status()>> 
tasks);
+
+ICEBERG_EXPORT Status RunTasksParallel(Executor& executor,
+                                       std::vector<FnOnce<Status()>> tasks);
+
+}  // namespace internal
+
+template <retry::Policy RetryPolicy = retry::NoRetry>
+class ICEBERG_TEMPLATE_CLASS_EXPORT TaskGroup {
+ private:
+  static constexpr bool kRetryEnabled = !std::same_as<RetryPolicy, 
retry::NoRetry>;
+
+  struct Empty {};
+
+  using RetryConfigStorage = std::conditional_t<kRetryEnabled, RetryConfig, 
Empty>;
+
+ public:
+  TaskGroup() = default;
+
+  explicit TaskGroup(RetryConfig retry_config)
+    requires(kRetryEnabled)
+      : retry_config_(std::move(retry_config)) {}
+
+  auto&& SetExecutor(this auto&& self, OptionalExecutor executor) {
+    self.executor_ = std::move(executor);
+    return std::forward<decltype(self)>(self);
+  }
+
+  template <typename F>
+    requires((!kRetryEnabled && internal::OnceStatusTask<F>) ||
+             (kRetryEnabled && internal::RetryableStatusTask<F>))
+  auto&& Submit(this auto&& self, F&& task) {
+    self.tasks_.emplace_back([&] {
+      if constexpr (!kRetryEnabled) {
+        return std::forward<F>(task);
+      } else {
+        return [retry_config = self.retry_config_,
+                task = std::forward<F>(task)]() mutable -> Status {
+          return RetryRunner<RetryPolicy>(retry_config).Run(task);
+        };
+      }
+    }());
+    return std::forward<decltype(self)>(self);
+  }
+
+  Status Run() && {
+    if (!executor_.has_value()) {
+      return internal::RunTasksSingleThreaded(std::move(tasks_));
+    }
+    return internal::RunTasksParallel(executor_->get(), std::move(tasks_));

Review Comment:
   `Run()` blocks on `future.get()` from the calling thread. That's fine for 
the current sequential call sites, but `PlanWith` is now public API. If someone 
drives a `TaskGroup` from a worker thread of the same bounded executor, it can 
deadlock (pool saturated while waiting on a task queued behind it). Worth 
documenting that the driving thread must not be one of the executor's own 
workers.



##########
src/iceberg/util/task_group.cc:
##########
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+#include "iceberg/util/task_group.h"
+
+#include <format>
+#include <future>
+#include <string>
+#include <utility>
+
+#include "iceberg/util/macros.h"
+
+namespace iceberg::internal {
+
+namespace {
+
+Status AggregateTaskErrors(std::vector<Error> errors) {
+  if (errors.empty()) {
+    return {};
+  }
+  if (errors.size() == 1) {
+    return std::unexpected(std::move(errors.front()));
+  }
+
+  ErrorKind kind = errors.front().kind;
+  std::string message = std::format("Task group failed with {} errors:", 
errors.size());
+  for (const auto& error : errors) {
+    message += std::format("\n  - {}", error.message);
+  }
+  return std::unexpected(Error{.kind = kind, .message = std::move(message)});
+}
+
+Result<std::future<Status>> SubmitTask(Executor& executor, FnOnce<Status()> 
task) {
+  std::promise<Status> promise;
+  auto future = promise.get_future();
+
+  ExecutorTask executor_task(
+      [promise = std::move(promise), task = std::move(task)]() mutable {
+        promise.set_value(std::move(task)());

Review Comment:
   If a task throws instead of returning a `Status`, the promise is never set 
and the exception escapes on the worker thread, while `future.get()` sees a 
broken promise. The repo uses `Result`/`Status` rather than exceptions, so this 
is unlikely in our own code, but `Executor` is a public extension point and 
users may plug in pools where it matters. Wrapping the call in try/catch and 
turning the exception into a `Status` would be safer.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


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

Reply via email to