zanmato1984 commented on code in PR #45954:
URL: https://github.com/apache/arrow/pull/45954#discussion_r2020551248


##########
cpp/src/arrow/util/async_generator.h:
##########
@@ -772,20 +782,26 @@ class ReadaheadGenerator {
   Future<T> operator()() {
     if (state_->readahead_queue.empty()) {
       // This is the first request, let's pump the underlying queue
-      state_->num_running.store(state_->max_readahead);
+      {
+        auto guard = state_->mutex.Lock();
+        state_->num_running = state_->max_readahead;
+      }
       for (int i = 0; i < state_->max_readahead; i++) {
         auto next = state_->source_generator();
         auto next_after_check = AddMarkFinishedContinuation(std::move(next));
         state_->readahead_queue.push(std::move(next_after_check));
       }
     }
     // Pop one and add one
-    auto result = state_->readahead_queue.front();
+    auto result = std::move(state_->readahead_queue.front());
     state_->readahead_queue.pop();
-    if (state_->finished.load()) {
+    auto guard = state_->mutex.Lock();
+    if (state_->finished) {
+      guard.Unlock();
       state_->readahead_queue.push(AsyncGeneratorEnd<T>());
     } else {
-      state_->num_running.fetch_add(1);
+      ++state_->num_running;
+      guard.Unlock();

Review Comment:
   Maybe we can make the locking more compact and implicit (explicit unlocking 
in multiple branches could be somehow error-prone?), like:
   ```C++
   bool finished = false;
   {
     auto guard = state_->mutex.Lock();
     if (finished = state_->finished; !finished) {
       ++state_->num_running;
     }
   }
   if (finished) {
     // ...
   } else {
     // ...
   }
   ```



-- 
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]

Reply via email to