Skip to content

Commit

Permalink
fix atomics now that we have multiple thread reads...
Browse files Browse the repository at this point in the history
Signed-off-by: Ruiyang Wang <rywang014@gmail.com>
  • Loading branch information
rynewang committed Sep 24, 2024
1 parent 36fc808 commit a87c39d
Showing 1 changed file with 6 additions and 3 deletions.
9 changes: 6 additions & 3 deletions src/ray/gcs/gcs_server/gcs_job_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -247,11 +247,14 @@ void GcsJobManager::HandleGetAllJobInfo(rpc::GetAllJobInfoRequest request,
// entrypoint script calls ray.init() multiple times).
std::unordered_map<std::string, std::vector<int>> job_data_key_to_indices;

// Create a shared counter for the number of jobs processed
std::shared_ptr<int> num_processed_jobs = std::make_shared<int>(0);
// Create a shared counter for the number of jobs processed.
// This is written in internal_kv_'s thread and read in the main thread.
std::shared_ptr<std::atomic<size_t>> num_processed_jobs =
std::make_shared<std::atomic<size_t>>(0);

// Create a shared boolean flag for the internal KV callback completion
std::shared_ptr<bool> kv_callback_done = std::make_shared<bool>(false);
std::shared_ptr<std::atomic<bool>> kv_callback_done =
std::make_shared<std::atomic<bool>>(false);

// Function to send the reply once all jobs have been processed and KV callback
// completed
Expand Down

0 comments on commit a87c39d

Please sign in to comment.