shuffler.hpp
1 
6 #pragma once
7 
8 #include <unordered_set>
9 
10 #include <rapidsmpf/shuffler/shuffler.hpp>
11 #include <rapidsmpf/streaming/core/actor.hpp>
12 #include <rapidsmpf/streaming/core/channel.hpp>
13 #include <rapidsmpf/streaming/core/context.hpp>
14 #include <rapidsmpf/streaming/cudf/partition.hpp>
15 
16 namespace rapidsmpf::streaming {
17 
51  public:
68  std::shared_ptr<Context> ctx,
69  std::shared_ptr<Communicator> comm,
70  OpID op_id,
74  );
75 
76  // Prevent copying
77  ShufflerAsync(ShufflerAsync const&) = delete;
78  ShufflerAsync& operator=(ShufflerAsync const&) = delete;
79 
80  ~ShufflerAsync() noexcept;
81 
87  [[nodiscard]] constexpr std::shared_ptr<Context> const& ctx() const {
88  return ctx_;
89  }
90 
96  [[nodiscard]] std::shared_ptr<Communicator> const& comm() const noexcept {
97  return shuffler_.comm();
98  }
99 
105  [[nodiscard]] constexpr shuffler::PartID total_num_partitions() const {
106  return shuffler_.total_num_partitions;
107  }
108 
114  [[nodiscard]] constexpr shuffler::Shuffler::PartitionOwner const&
115  partition_owner() const {
116  return shuffler_.partition_owner;
117  }
118 
120  [[nodiscard]] std::span<shuffler::PartID const> local_partitions() const;
121 
123  void insert(std::unordered_map<shuffler::PartID, PackedData>&& chunks);
124 
139  [[nodiscard]] Actor insert_finished();
140 
160  [[nodiscard]] coro::task<std::optional<std::vector<PackedData>>> extract_async(
161  shuffler::PartID pid
162  );
163 
169  using ExtractResult = std::pair<shuffler::PartID, std::vector<PackedData>>;
170 
186  [[nodiscard]] coro::task<std::optional<ExtractResult>> extract_any_async();
187 
188  private:
203  [[nodiscard]] Actor finished_drain();
204 
205  std::shared_ptr<Context> ctx_;
206  coro::task_group<coro::thread_pool>
207  notifications_;
208  Semaphore semaphore_{0};
209  coro::latch
210  latch_;
211  std::mutex mtx_;
212  shuffler::Shuffler shuffler_;
213 
221  std::unordered_set<shuffler::PartID> ready_pids_;
222  std::unordered_set<shuffler::PartID> extracted_pids_;
223 };
224 
225 namespace actor {
247 [[nodiscard]] Actor shuffler(
248  std::shared_ptr<Context> ctx,
249  std::shared_ptr<Communicator> comm,
250  std::shared_ptr<Channel> ch_in,
251  std::shared_ptr<Channel> ch_out,
252  OpID op_id,
253  shuffler::PartID total_num_partitions,
255 );
256 
257 } // namespace actor
258 
259 } // namespace rapidsmpf::streaming
Shuffle service for cuDF tables.
Definition: shuffler.hpp:44
std::function< Rank(std::shared_ptr< Communicator > const &, PartID, PartID)> PartitionOwner
Function that given a Communicator, PartID, and total partition count, returns the rapidsmpf::Rank of...
Definition: shuffler.hpp:51
PartID const total_num_partitions
Total number of partition in the shuffle.
Definition: shuffler.hpp:341
static Rank round_robin(std::shared_ptr< Communicator > const &comm, PartID pid, [[maybe_unused]] PartID total_num_partitions)
A PartitionOwner that distributes partitions using round robin assignment.
Definition: shuffler.hpp:61
std::shared_ptr< Communicator > const & comm() const noexcept
Gets the communicator associated with this Shuffler.
Definition: shuffler.hpp:159
PartitionOwner const partition_owner
Function to determine partition ownership.
Definition: shuffler.hpp:342
Context for actors (coroutines) in rapidsmpf.
Definition: context.hpp:41
An asynchronous shuffler that allows concurrent insertion and extraction of data.
Definition: shuffler.hpp:50
constexpr std::shared_ptr< Context > const & ctx() const
Gets the streaming context associated with this shuffler.
Definition: shuffler.hpp:87
std::shared_ptr< Communicator > const & comm() const noexcept
Gets the communicator associated with this shuffler.
Definition: shuffler.hpp:96
constexpr shuffler::Shuffler::PartitionOwner const & partition_owner() const
Gets the partition owner function used by this shuffler.
Definition: shuffler.hpp:115
ShufflerAsync(std::shared_ptr< Context > ctx, std::shared_ptr< Communicator > comm, OpID op_id, shuffler::PartID total_num_partitions, shuffler::Shuffler::PartitionOwner partition_owner=shuffler::Shuffler::round_robin)
Constructs a new ShufflerAsync instance.
coro::task< std::optional< ExtractResult > > extract_any_async()
Asynchronously extracts data for any ready partition.
Actor insert_finished()
Insert a finish mark for a list of partitions.
constexpr shuffler::PartID total_num_partitions() const
Gets the total number of partitions for this shuffle operation.
Definition: shuffler.hpp:105
void insert(std::unordered_map< shuffler::PartID, PackedData > &&chunks)
Insert a bunch of packed (serialized) chunks into the shuffle.
std::pair< shuffler::PartID, std::vector< PackedData > > ExtractResult
Result type for extract_any_async operations.
Definition: shuffler.hpp:169
std::span< shuffler::PartID const > local_partitions() const
Returns the local partition IDs owned by the current node.
coro::task< std::optional< std::vector< PackedData > > > extract_async(shuffler::PartID pid)
Asynchronously extracts all data for a specific partition.
std::uint32_t PartID
Partition ID, which goes from 0 to the total number of partitions.
Definition: chunk.hpp:22
Actor shuffler(std::shared_ptr< Context > ctx, std::shared_ptr< Communicator > comm, std::shared_ptr< Channel > ch_in, std::shared_ptr< Channel > ch_out, OpID op_id, shuffler::PartID total_num_partitions, shuffler::Shuffler::PartitionOwner partition_owner=shuffler::Shuffler::round_robin)
Launches a shuffler actor for a single shuffle operation.
coro::task< void > Actor
Alias for an actor in a streaming graph.
Definition: actor.hpp:18
coro::semaphore< std::numeric_limits< std::ptrdiff_t >::max()> Semaphore
An awaitable semaphore to manage acquisition and release of finite resources.
Definition: channel.hpp:31
std::int32_t OpID
Operation ID defined by the user. This allows users to concurrently execute multiple operations,...