You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/07/19 17:10:50 UTC

[GitHub] [arrow] westonpace opened a new pull request, #13651: ARROW-17087: [C++] Race condition in scanner test

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

   The ExecPlan::AddTask variant that takes a future was problematic because a future is not created until a task has already been started and, if the task group was already ended, it wasn't tracking the future.  So work would be started that would not be tracked anywhere and the plan was ending prematurely.  As a fix I changed that AddTask variant to BeginExternalTask.  This method will request permission to start the task before the future is created.  At some point the scanner should be moved to an exec plan instead of async generators and I think BeginExternalTask can go away.


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925300980


##########
cpp/src/arrow/compute/exec/exec_plan.h:
##########
@@ -69,17 +69,16 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this<ExecPlan> {
   /// e.g. make an array of thread-locals off this.
   size_t max_concurrency() const;
 
-  /// \brief Add a future to the plan's task group.
+  /// \brief Starts an external task

Review Comment:
   ```suggestion
     /// \brief Start an external task
   ```



##########
cpp/src/arrow/util/async_util.h:
##########
@@ -120,8 +120,12 @@ class ARROW_EXPORT AsyncTaskGroup {
   Status AddTaskIfNotEnded(std::function<Result<Future<>>()> task);
   /// Add a task that has already been started
   Status AddTask(const Future<>& task);
-  /// Same as AddTask but doesn't add the task if End() has been called.
-  Status AddTaskIfNotEnded(const Future<>& task);
+  /// \brief Attempts to add a task that has already been started to this group's tracking

Review Comment:
   ```suggestion
     /// \brief Attempt to add a task that has already been started to this group's tracking
   ```



##########
cpp/src/arrow/util/future.cc:
##########
@@ -339,8 +339,8 @@ class ConcreteFutureImpl : public FutureImpl {
       if (waiter_ != nullptr) {
         waiter_->MarkFutureFinishedUnlocked(waiter_arg_, state);
       }
+      cv_.notify_all();

Review Comment:
   Can you add a comment so we remember why we chose to notify while locked?



##########
cpp/src/arrow/util/future_test.cc:
##########
@@ -526,6 +527,23 @@ TEST(FutureStressTest, TryAddCallback) {
   }
 }
 
+TEST(FutureStressTest, DeleteAfterWait) {
+  constexpr int kNumTasks = 100;
+  for (int i = 0; i < kNumTasks; i++) {
+    {
+      std::unique_ptr<Future<>> future =
+          internal::make_unique<Future<>>(Future<>::Make());
+      std::thread t([&]() {
+        SleepABit();
+        future->MarkFinished();
+      });
+      ASSERT_TRUE(future->Wait(100));

Review Comment:
   100 seconds sounds a bit long... ?



##########
cpp/src/arrow/compute/exec/source_node.cc:
##########
@@ -139,13 +147,18 @@ struct SourceNode : ExecNode {
                      options);
                })
                    .Then(
-                       [=](int total_batches) {
+                       [this, scan_task](int total_batches) mutable {
+                         std::unique_lock<std::mutex> lock(mutex_);
+                         bool should_mark_finished = !finished_.is_finished();
+                         lock.unlock();

Review Comment:
   I'm curious, why is the locking necessary? Inspecting a Future should be thread-safe.



##########
cpp/src/arrow/compute/exec/exec_plan.h:
##########
@@ -69,17 +69,16 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this<ExecPlan> {
   /// e.g. make an array of thread-locals off this.
   size_t max_concurrency() const;
 
-  /// \brief Add a future to the plan's task group.
+  /// \brief Starts an external task
   ///
-  /// \param fut The future to add
+  /// This should be avoided if possible.  It is kept in for now for legacy
+  /// purposes.  This should be called before the external task is started.  If
+  /// a future is returned then it should be marked complete when the external
+  /// task has finished.
   ///
-  /// Use this when interfacing with anything that returns a future (such as IO), but
-  /// prefer ScheduleTask/StartTaskGroup inside of ExecNodes.
-  /// The below API interfaces with the scheduler to add tasks to the task group. Tasks
-  /// should be added sparingly! Prefer just doing the work immediately rather than adding
-  /// a task for it. Tasks are used in pipeline breakers that may output many more rows
-  /// than they received (such as a full outer join).
-  Status AddFuture(Future<> fut);
+  /// \return nullopt if the plan has already ended, otherwise this returns
+  ///         a future that must be completed when the external task finishes
+  Result<util::optional<Future<>>> BeginExternalTask();

Review Comment:
   I'm not sure the `optional` is necessary since you could also return an invalid (default-initialized) Future.



##########
cpp/src/arrow/util/future_test.cc:
##########
@@ -526,6 +527,23 @@ TEST(FutureStressTest, TryAddCallback) {
   }
 }
 
+TEST(FutureStressTest, DeleteAfterWait) {

Review Comment:
   There are a couple timeouts on CI in `threading-utility-test`, I'm assuming this is because of this test? Can you try to find out the issue?



-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925639943


##########
cpp/src/arrow/util/future_test.cc:
##########
@@ -526,6 +527,23 @@ TEST(FutureStressTest, TryAddCallback) {
   }
 }
 
+TEST(FutureStressTest, DeleteAfterWait) {
+  constexpr int kNumTasks = 100;
+  for (int i = 0; i < kNumTasks; i++) {
+    {
+      std::unique_ptr<Future<>> future =
+          internal::make_unique<Future<>>(Future<>::Make());
+      std::thread t([&]() {
+        SleepABit();
+        future->MarkFinished();
+      });
+      ASSERT_TRUE(future->Wait(100));

Review Comment:
   In most cases this wait will only be micro/milli seconds.  However, the CI machines have a tendency to hang for seconds at a time without yielding and I don't see any real advantage in cutting it close.  I will change it to `arrow::kDefaultAssertFinishesWaitSeconds` which is 64 but maybe a bit more clearly named.



-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190703453

   > Looks like it's stuck after this error:
   
   Ah, thanks!  These tests were disabled because I was testing everything in TSAN so I missed it.  I can reproduce it now.


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1191290652

   Thanks a lot @westonpace !


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190534933

   Looks like it's stuck after this error:
   ```
   [----------] 3 tests from TestThreadPoolForkSafety
   [ RUN      ] TestThreadPoolForkSafety.Basics
   /arrow/cpp/src/arrow/util/thread_pool_test.cc:780: Failure
   Value of: _fut41.Wait(::arrow::kDefaultAssertFinishesWaitSeconds)
     Actual: false
   Expected: true
   ```
   https://github.com/apache/arrow/runs/7433803880?check_suite_focus=true#step:6:3837


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925696912


##########
cpp/src/arrow/compute/exec/exec_plan.h:
##########
@@ -69,17 +69,16 @@ class ARROW_EXPORT ExecPlan : public std::enable_shared_from_this<ExecPlan> {
   /// e.g. make an array of thread-locals off this.
   size_t max_concurrency() const;
 
-  /// \brief Add a future to the plan's task group.
+  /// \brief Starts an external task
   ///
-  /// \param fut The future to add
+  /// This should be avoided if possible.  It is kept in for now for legacy
+  /// purposes.  This should be called before the external task is started.  If
+  /// a future is returned then it should be marked complete when the external
+  /// task has finished.
   ///
-  /// Use this when interfacing with anything that returns a future (such as IO), but
-  /// prefer ScheduleTask/StartTaskGroup inside of ExecNodes.
-  /// The below API interfaces with the scheduler to add tasks to the task group. Tasks
-  /// should be added sparingly! Prefer just doing the work immediately rather than adding
-  /// a task for it. Tasks are used in pipeline breakers that may output many more rows
-  /// than they received (such as a full outer join).
-  Status AddFuture(Future<> fut);
+  /// \return nullopt if the plan has already ended, otherwise this returns
+  ///         a future that must be completed when the external task finishes
+  Result<util::optional<Future<>>> BeginExternalTask();

Review Comment:
   Good idea.  I removed `optional`.



-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189581093

   Revision: 1611c84f3e98e7b3cd58332c041e3e532b5a0f75
   
   Submitted crossbow builds: [ursacomputing/crossbow @ actions-43fa840b7d](https://github.com/ursacomputing/crossbow/branches/all?query=actions-43fa840b7d)
   
   |Task|Status|
   |----|------|
   |test-ubuntu-20.04-cpp-thread-sanitizer|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-43fa840b7d-github-test-ubuntu-20.04-cpp-thread-sanitizer)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-43fa840b7d-github-test-ubuntu-20.04-cpp-thread-sanitizer)|


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189346813

   https://issues.apache.org/jira/browse/ARROW-17087


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
pitrou merged PR #13651:
URL: https://github.com/apache/arrow/pull/13651


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1191184234

   Revision: 490528381786657988711cba85a4dc96c3824926
   
   Submitted crossbow builds: [ursacomputing/crossbow @ actions-c54ceaacae](https://github.com/ursacomputing/crossbow/branches/all?query=actions-c54ceaacae)
   
   |Task|Status|
   |----|------|
   |test-build-cpp-fuzz|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-build-cpp-fuzz)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-build-cpp-fuzz)|
   |test-conda-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-conda-cpp)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-conda-cpp)|
   |test-conda-cpp-valgrind|[![Azure](https://dev.azure.com/ursacomputing/crossbow/_apis/build/status/ursacomputing.crossbow?branchName=actions-c54ceaacae-azure-test-conda-cpp-valgrind)](https://dev.azure.com/ursacomputing/crossbow/_build/latest?definitionId=1&branchName=actions-c54ceaacae-azure-test-conda-cpp-valgrind)|
   |test-debian-10-cpp-amd64|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-debian-10-cpp-amd64)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-debian-10-cpp-amd64)|
   |test-debian-10-cpp-i386|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-debian-10-cpp-i386)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-debian-10-cpp-i386)|
   |test-debian-11-cpp-amd64|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-debian-11-cpp-amd64)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-debian-11-cpp-amd64)|
   |test-debian-11-cpp-i386|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-debian-11-cpp-i386)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-debian-11-cpp-i386)|
   |test-fedora-35-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-fedora-35-cpp)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-fedora-35-cpp)|
   |test-ubuntu-18.04-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-18.04-cpp)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-18.04-cpp)|
   |test-ubuntu-18.04-cpp-release|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-18.04-cpp-release)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-18.04-cpp-release)|
   |test-ubuntu-18.04-cpp-static|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-18.04-cpp-static)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-18.04-cpp-static)|
   |test-ubuntu-20.04-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-20.04-cpp)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-20.04-cpp)|
   |test-ubuntu-20.04-cpp-14|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-20.04-cpp-14)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-20.04-cpp-14)|
   |test-ubuntu-20.04-cpp-17|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-20.04-cpp-17)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-20.04-cpp-17)|
   |test-ubuntu-20.04-cpp-bundled|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-20.04-cpp-bundled)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-20.04-cpp-bundled)|
   |test-ubuntu-20.04-cpp-thread-sanitizer|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-20.04-cpp-thread-sanitizer)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-20.04-cpp-thread-sanitizer)|
   |test-ubuntu-22.04-cpp|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-c54ceaacae-github-test-ubuntu-22.04-cpp)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-c54ceaacae-github-test-ubuntu-22.04-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] cyb70289 commented on pull request #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
cyb70289 commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1191183703

   Tested on my local host, just FYI.
   Without this PR, I see failures (sigsegv, future already finished) repeatedly in about 15 minutes.
   With this PR, the test has run 6 hours without error.


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190876126

   Conda timed out on caching the docker image so I'm rerunning


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r926097680


##########
cpp/src/arrow/util/async_util.cc:
##########
@@ -105,15 +105,16 @@ Status AsyncTaskGroup::AddTask(const Future<>& task_fut) {
   return AddTaskUnlocked(task_fut, std::move(guard));
 }
 
-Status AsyncTaskGroup::AddTaskIfNotEnded(const Future<>& task_fut) {
+Result<bool> AsyncTaskGroup::AddTaskIfNotEnded(const Future<>& task_fut) {

Review Comment:
   I went ahead and added this for consistency but the return bool here can generally be ignored because the method doesn't actually start the work if it returns false.



-- 
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] save-buffer commented on pull request #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
save-buffer commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190816106

   WOW that's a lot of deleted code 😍 


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189394181

   @github-actions crossbow submit test-ubuntu-20.04-cpp-thread-sanitizer


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189691168

   Looks like there is still one deadlock remaining (https://github.com/apache/arrow/runs/7419479233?check_suite_focus=true) but I believe this is considerably better than what is in master at the moment.  I'll continue to search for the remaining issue tomorrow.


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189648047

   Revision: 4c303d70020c3d4131005ea7b0669d229139861a
   
   Submitted crossbow builds: [ursacomputing/crossbow @ actions-0fdb0f74cd](https://github.com/ursacomputing/crossbow/branches/all?query=actions-0fdb0f74cd)
   
   |Task|Status|
   |----|------|
   |test-ubuntu-20.04-cpp-thread-sanitizer|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-0fdb0f74cd-github-test-ubuntu-20.04-cpp-thread-sanitizer)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-0fdb0f74cd-github-test-ubuntu-20.04-cpp-thread-sanitizer)|


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190820224

   @github-actions crossbow submit test-ubuntu-20.04-cpp-thread-sanitizer


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
pitrou commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1191169197

   @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] westonpace commented on a diff in pull request #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925697917


##########
cpp/src/arrow/compute/exec/source_node.cc:
##########
@@ -139,13 +147,18 @@ struct SourceNode : ExecNode {
                      options);
                })
                    .Then(
-                       [=](int total_batches) {
+                       [this, scan_task](int total_batches) mutable {
+                         std::unique_lock<std::mutex> lock(mutex_);
+                         bool should_mark_finished = !finished_.is_finished();
+                         lock.unlock();

Review Comment:
   Good catch.  This lock was obviously not needed and it turns out I could get rid of locking above too.  The code was perhaps a bit more paranoid than it needed to be.



##########
cpp/src/arrow/util/future.cc:
##########
@@ -339,8 +339,8 @@ class ConcreteFutureImpl : public FutureImpl {
       if (waiter_ != nullptr) {
         waiter_->MarkFutureFinishedUnlocked(waiter_arg_, state);
       }
+      cv_.notify_all();

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] github-actions[bot] commented on pull request #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190820817

   Revision: 490528381786657988711cba85a4dc96c3824926
   
   Submitted crossbow builds: [ursacomputing/crossbow @ actions-986eaa68ab](https://github.com/ursacomputing/crossbow/branches/all?query=actions-986eaa68ab)
   
   |Task|Status|
   |----|------|
   |test-ubuntu-20.04-cpp-thread-sanitizer|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-986eaa68ab-github-test-ubuntu-20.04-cpp-thread-sanitizer)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-986eaa68ab-github-test-ubuntu-20.04-cpp-thread-sanitizer)|


-- 
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] ursabot commented on pull request #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
ursabot commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1192763162

   Benchmark runs are scheduled for baseline = 9c422a2011404ee0c5c01eeb2a6a1d5333816cad and contender = 1ae8dc747c6835069c652874124dae07c50a3bf4. 1ae8dc747c6835069c652874124dae07c50a3bf4 is a master commit associated with this PR. Results will be available as each benchmark for each run completes.
   Conbench compare runs links:
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ec2-t3-xlarge-us-east-2](https://conbench.ursa.dev/compare/runs/041dcaf2c7bf47b59833a47a3602d629...6f321d012cae4b43b08d885290d3e8af/)
   [Failed :arrow_down:0.73% :arrow_up:0.21%] [test-mac-arm](https://conbench.ursa.dev/compare/runs/b41f659adcda4edeba06230dc1c048bf...832cf1a3dea143be95ca69e7b450bfb3/)
   [Failed :arrow_down:0.0% :arrow_up:0.0%] [ursa-i9-9960x](https://conbench.ursa.dev/compare/runs/d19d48a3d3924f1c8068421574529b56...335629f871fe4aa2983e227a574949e7/)
   [Finished :arrow_down:0.93% :arrow_up:0.61%] [ursa-thinkcentre-m75q](https://conbench.ursa.dev/compare/runs/7c458ca0a67d477d9cc513023881e859...417c2743dee64406949e9e63a729afe2/)
   Buildkite builds:
   [Failed] [`1ae8dc74` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1168)
   [Failed] [`1ae8dc74` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1173)
   [Failed] [`1ae8dc74` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1155)
   [Finished] [`1ae8dc74` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1172)
   [Failed] [`9c422a20` ec2-t3-xlarge-us-east-2](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ec2-t3-xlarge-us-east-2/builds/1167)
   [Failed] [`9c422a20` test-mac-arm](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-test-mac-arm/builds/1178)
   [Failed] [`9c422a20` ursa-i9-9960x](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-i9-9960x/builds/1163)
   [Finished] [`9c422a20` ursa-thinkcentre-m75q](https://buildkite.com/apache-arrow/arrow-bci-benchmark-on-ursa-thinkcentre-m75q/builds/1184)
   Supported benchmarks:
   ec2-t3-xlarge-us-east-2: Supported benchmark langs: Python, R. Runs only benchmarks with cloud = True
   test-mac-arm: Supported benchmark langs: C++, Python, R
   ursa-i9-9960x: Supported benchmark langs: Python, R, JavaScript
   ursa-thinkcentre-m75q: Supported benchmark langs: C++, Java
   


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1190817974

   The root cause ended up being the global future waiter mutex.  This mutex was not reset after a fork.  So if a fork happened while this mutex was held then it would lead to deadlock in the child.
   
   We weren't encountering this in tests before because we would have released the mutex before we signaled the condition variable.  Now that we signal the cv inside the mutex there is a small chance we could fork while the mutex was still being released.
   
   Since we wanted to remove the future waiter stuff anyways I just ripped it out instead of trying to find a way to properly reset the mutex.


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189580165

   @github-actions crossbow submit test-ubuntu-20.04-cpp-thread-sanitizer


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189346854

   :warning: Ticket **has not been started in JIRA**, please click 'Start Progress'.


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189361884

   ```
   Unable to match any tasks for `ubuntu-cpp-thread-sanitizer`
   The Archery job run can be found at: https://github.com/apache/arrow/actions/runs/2699361302
   ```


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189697726

   CC @save-buffer can you please review?


-- 
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] save-buffer commented on a diff in pull request #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
save-buffer commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925959617


##########
cpp/src/arrow/util/async_util.cc:
##########
@@ -105,15 +105,16 @@ Status AsyncTaskGroup::AddTask(const Future<>& task_fut) {
   return AddTaskUnlocked(task_fut, std::move(guard));
 }
 
-Status AsyncTaskGroup::AddTaskIfNotEnded(const Future<>& task_fut) {
+Result<bool> AsyncTaskGroup::AddTaskIfNotEnded(const Future<>& task_fut) {

Review Comment:
   Can you also update the interface of the overload of `AddTaskIfNotEnded` to return a bool?



-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925700449


##########
cpp/src/arrow/util/future_test.cc:
##########
@@ -526,6 +527,23 @@ TEST(FutureStressTest, TryAddCallback) {
   }
 }
 
+TEST(FutureStressTest, DeleteAfterWait) {

Review Comment:
   I've stressed this test in a variety of ways.  It's also possible the change to future introduced a deadlock somewhere.  However, I've so far been unable to reproduce, with or without TSAN.



-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189348771

   @github-actions crossbow submit ubuntu-cpp-thread-sanitizer


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189395411

   Revision: ed29949ec3316a97a9c629433c3f9bac96e00a9e
   
   Submitted crossbow builds: [ursacomputing/crossbow @ actions-105d119606](https://github.com/ursacomputing/crossbow/branches/all?query=actions-105d119606)
   
   |Task|Status|
   |----|------|
   |test-ubuntu-20.04-cpp-thread-sanitizer|[![Github Actions](https://github.com/ursacomputing/crossbow/workflows/Crossbow/badge.svg?branch=actions-105d119606-github-test-ubuntu-20.04-cpp-thread-sanitizer)](https://github.com/ursacomputing/crossbow/actions?query=branch:actions-105d119606-github-test-ubuntu-20.04-cpp-thread-sanitizer)|


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
westonpace commented on PR #13651:
URL: https://github.com/apache/arrow/pull/13651#issuecomment-1189647559

   @github-actions crossbow submit test-ubuntu-20.04-cpp-thread-sanitizer


-- 
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 #13651: ARROW-17087: [C++] Race condition in scanner test

Posted by GitBox <gi...@apache.org>.
pitrou commented on code in PR #13651:
URL: https://github.com/apache/arrow/pull/13651#discussion_r925308941


##########
cpp/src/arrow/util/future_test.cc:
##########
@@ -526,6 +527,23 @@ TEST(FutureStressTest, TryAddCallback) {
   }
 }
 
+TEST(FutureStressTest, DeleteAfterWait) {

Review Comment:
   Perhaps running with TSAN can help.



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