Skip to content

Commit 141b711

Browse files
committed
remvoing shuffle nb from header
Signed-off-by: niranda perera <niranda.perera@gmail.com>
1 parent 2e64204 commit 141b711

File tree

3 files changed

+149
-161
lines changed

3 files changed

+149
-161
lines changed

cpp/include/rapidsmpf/streaming/cudf/shuffler.hpp

Lines changed: 0 additions & 31 deletions
Original file line numberDiff line numberDiff line change
@@ -47,35 +47,4 @@ Node shuffler(
4747
shuffler::Shuffler::PartitionOwner partition_owner = shuffler::Shuffler::round_robin
4848
);
4949

50-
/**
51-
* @brief Launches a non-blocking shuffler node for a single shuffle operation.
52-
*
53-
* This is a non-blocking version of `shuffler` that returns a pair of nodes. The first
54-
* node inserts the partition map chunks into the shuffler and the second node extracts
55-
* the packed chunks from the shuffler and sends them to the output channel.
56-
*
57-
* @param ctx The streaming context providing communication, memory, stream, and execution
58-
* resources.
59-
* @param stream The CUDA stream on which to perform the shuffling. If chunks from the
60-
* input channel aren't created on `stream`, the streams are all synchronized.
61-
* @param ch_in Input channel providing packed partition chunks to be shuffled.
62-
* @param ch_out Output channel where the shuffled results are sent.
63-
* @param op_id Unique operation ID for this shuffle. Must not be reused until all nodes
64-
* have called `Shuffler::shutdown()`.
65-
* @param total_num_partitions Total number of partitions to shuffle the data into.
66-
* @param partition_owner Function that maps a partition ID to its owning rank/node.
67-
*
68-
* @return A pair of nodes that complete when the shuffling has finished and the output
69-
* channel is drained.
70-
*/
71-
std::pair<Node, Node> shuffler_nb(
72-
std::shared_ptr<Context> ctx,
73-
rmm::cuda_stream_view stream,
74-
std::shared_ptr<Channel> ch_in,
75-
std::shared_ptr<Channel> ch_out,
76-
OpID op_id,
77-
shuffler::PartID total_num_partitions,
78-
shuffler::Shuffler::PartitionOwner partition_owner = shuffler::Shuffler::round_robin
79-
);
80-
8150
} // namespace rapidsmpf::streaming::node

cpp/src/streaming/cudf/shuffler.cpp

Lines changed: 0 additions & 109 deletions
Original file line numberDiff line numberDiff line change
@@ -99,113 +99,4 @@ Node shuffler(
9999
co_await ch_out->drain(ctx->executor());
100100
}
101101

102-
std::pair<Node, Node> shuffler_nb(
103-
std::shared_ptr<Context> ctx,
104-
rmm::cuda_stream_view stream,
105-
std::shared_ptr<Channel> ch_in,
106-
std::shared_ptr<Channel> ch_out,
107-
OpID op_id,
108-
shuffler::PartID total_num_partitions,
109-
shuffler::Shuffler::PartitionOwner partition_owner
110-
) {
111-
// make a shared_ptr to the shuffler so that it can be passed into multiple coroutines
112-
auto shuffler = std::make_shared<rapidsmpf::shuffler::Shuffler>(
113-
ctx->comm(),
114-
ctx->progress_thread(),
115-
op_id,
116-
total_num_partitions,
117-
stream,
118-
ctx->br(),
119-
ctx->statistics(),
120-
std::move(partition_owner)
121-
);
122-
123-
// insert task: insert the partition map chunks into the shuffler
124-
auto insert_task =
125-
[](
126-
auto shuffler, auto ctx, auto total_num_partitions, auto stream, auto ch_in
127-
) -> Node {
128-
ShutdownAtExit c{ch_in};
129-
co_await ctx->executor()->schedule();
130-
CudaEvent event;
131-
132-
while (true) {
133-
auto msg = co_await ch_in->receive();
134-
if (msg.empty()) {
135-
break;
136-
}
137-
auto partition_map = msg.template release<PartitionMapChunk>();
138-
139-
// Make sure that the input chunk's stream is in sync with shuffler's stream.
140-
sync_streams(stream, partition_map.stream, event);
141-
142-
shuffler->insert(std::move(partition_map.data));
143-
}
144-
145-
// Tell the shuffler that we have no more input data.
146-
std::vector<rapidsmpf::shuffler::PartID> finished(total_num_partitions);
147-
std::iota(finished.begin(), finished.end(), 0);
148-
shuffler->insert_finished(std::move(finished));
149-
co_return;
150-
};
151-
152-
// extract task: extract the packed chunks from the shuffler and send them to the
153-
// output channel
154-
auto extract_task = [](auto shuffler, auto ctx, auto ch_out) -> Node {
155-
ShutdownAtExit c{ch_out};
156-
co_await ctx->executor()->schedule();
157-
158-
coro::mutex mtx{};
159-
coro::condition_variable cv{};
160-
bool finished{false};
161-
162-
shuffler->register_finished_callback(
163-
[shuffler, ctx, ch_out, &mtx, &cv, &finished](auto pid) {
164-
// task to extract and send each finished partition
165-
auto extract_and_send = [](auto shuffler,
166-
auto ctx,
167-
auto ch_out,
168-
auto pid,
169-
coro::condition_variable& cv,
170-
coro::mutex& mtx,
171-
bool& finished) -> Node {
172-
co_await ctx->executor()->schedule();
173-
auto packed_chunks = shuffler->extract(pid);
174-
co_await ch_out->send(
175-
std::make_unique<PartitionVectorChunk>(
176-
pid, std::move(packed_chunks)
177-
)
178-
);
179-
180-
// signal that all partitions have been finished
181-
if (shuffler->finished()) {
182-
{
183-
auto lock = co_await mtx.scoped_lock();
184-
finished = true;
185-
}
186-
co_await cv.notify_one();
187-
}
188-
};
189-
// schedule a detached task to extract and send the packed chunks
190-
ctx->executor()->spawn(
191-
extract_and_send(shuffler, ctx, ch_out, pid, cv, mtx, finished)
192-
);
193-
}
194-
);
195-
196-
// wait for all partitions to be finished
197-
{
198-
auto lock = co_await mtx.scoped_lock();
199-
co_await cv.wait(lock, [&finished]() { return finished; });
200-
}
201-
202-
co_await ch_out->drain(ctx->executor());
203-
};
204-
205-
return {
206-
insert_task(shuffler, ctx, total_num_partitions, stream, std::move(ch_in)),
207-
extract_task(std::move(shuffler), std::move(ctx), std::move(ch_out))
208-
};
209-
}
210-
211102
} // namespace rapidsmpf::streaming::node

cpp/tests/streaming/test_shuffler.cpp

Lines changed: 149 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -98,31 +98,159 @@ class StreamingShuffler : public BaseStreamingFixture {
9898
};
9999

100100
TEST_F(StreamingShuffler, Basic) {
101-
run_test([&](auto ctx, auto ch_in, auto ch_out, std::vector<Node>& nodes) {
102-
nodes.emplace_back(
103-
node::shuffler(
101+
EXPECT_NO_FATAL_FAILURE(
102+
run_test([&](auto ctx, auto ch_in, auto ch_out, std::vector<Node>& nodes) {
103+
nodes.emplace_back(
104+
node::shuffler(
105+
std::move(ctx),
106+
stream,
107+
std::move(ch_in),
108+
std::move(ch_out),
109+
op_id,
110+
num_partitions
111+
)
112+
);
113+
})
114+
);
115+
}
116+
117+
namespace {
118+
119+
void sync_streams(
120+
rmm::cuda_stream_view primary,
121+
rmm::cuda_stream_view secondary,
122+
cudaEvent_t const& event
123+
) {
124+
if (primary.value() != secondary.value()) {
125+
RAPIDSMPF_CUDA_TRY(cudaEventRecord(event, secondary));
126+
RAPIDSMPF_CUDA_TRY(cudaStreamWaitEvent(primary, event));
127+
}
128+
}
129+
130+
// emulate shuffler node with callbacks
131+
std::pair<Node, Node> shuffler_nb(
132+
std::shared_ptr<Context> ctx,
133+
rmm::cuda_stream_view stream,
134+
std::shared_ptr<Channel> ch_in,
135+
std::shared_ptr<Channel> ch_out,
136+
OpID op_id,
137+
shuffler::PartID total_num_partitions
138+
) {
139+
// make a shared_ptr to the shuffler so that it can be passed into multiple coroutines
140+
auto shuffler = std::make_shared<rapidsmpf::shuffler::Shuffler>(
141+
ctx->comm(),
142+
ctx->progress_thread(),
143+
op_id,
144+
total_num_partitions,
145+
stream,
146+
ctx->br(),
147+
ctx->statistics(),
148+
shuffler::Shuffler::round_robin
149+
);
150+
151+
// insert task: insert the partition map chunks into the shuffler
152+
auto insert_task =
153+
[](
154+
auto shuffler, auto ctx, auto total_num_partitions, auto stream, auto ch_in
155+
) -> Node {
156+
ShutdownAtExit c{ch_in};
157+
co_await ctx->executor()->schedule();
158+
CudaEvent event;
159+
160+
while (true) {
161+
auto msg = co_await ch_in->receive();
162+
if (msg.empty()) {
163+
break;
164+
}
165+
auto partition_map = msg.template release<PartitionMapChunk>();
166+
167+
// Make sure that the input chunk's stream is in sync with shuffler's stream.
168+
sync_streams(stream, partition_map.stream, event);
169+
170+
shuffler->insert(std::move(partition_map.data));
171+
}
172+
173+
// Tell the shuffler that we have no more input data.
174+
std::vector<rapidsmpf::shuffler::PartID> finished(total_num_partitions);
175+
std::iota(finished.begin(), finished.end(), 0);
176+
shuffler->insert_finished(std::move(finished));
177+
co_return;
178+
};
179+
180+
// extract task: extract the packed chunks from the shuffler and send them to the
181+
// output channel
182+
auto extract_task = [](auto shuffler, auto ctx, auto ch_out) -> Node {
183+
ShutdownAtExit c{ch_out};
184+
co_await ctx->executor()->schedule();
185+
186+
coro::mutex mtx{};
187+
coro::condition_variable cv{};
188+
bool finished{false};
189+
190+
shuffler->register_finished_callback(
191+
[shuffler, ctx, ch_out, &mtx, &cv, &finished](auto pid) {
192+
// task to extract and send each finished partition
193+
auto extract_and_send = [](auto shuffler,
194+
auto ctx,
195+
auto ch_out,
196+
auto pid,
197+
coro::condition_variable& cv,
198+
coro::mutex& mtx,
199+
bool& finished) -> Node {
200+
co_await ctx->executor()->schedule();
201+
auto packed_chunks = shuffler->extract(pid);
202+
co_await ch_out->send(
203+
std::make_unique<PartitionVectorChunk>(
204+
pid, std::move(packed_chunks)
205+
)
206+
);
207+
208+
// signal that all partitions have been finished
209+
if (shuffler->finished()) {
210+
{
211+
auto lock = co_await mtx.scoped_lock();
212+
finished = true;
213+
}
214+
co_await cv.notify_one();
215+
}
216+
};
217+
// schedule a detached task to extract and send the packed chunks
218+
ctx->executor()->spawn(
219+
extract_and_send(shuffler, ctx, ch_out, pid, cv, mtx, finished)
220+
);
221+
}
222+
);
223+
224+
// wait for all partitions to be finished
225+
{
226+
auto lock = co_await mtx.scoped_lock();
227+
co_await cv.wait(lock, [&finished]() { return finished; });
228+
}
229+
230+
co_await ch_out->drain(ctx->executor());
231+
};
232+
233+
return {
234+
insert_task(shuffler, ctx, total_num_partitions, stream, std::move(ch_in)),
235+
extract_task(std::move(shuffler), std::move(ctx), std::move(ch_out))
236+
};
237+
}
238+
239+
} // namespace
240+
241+
TEST_F(StreamingShuffler, callbacks) {
242+
EXPECT_NO_FATAL_FAILURE(
243+
run_test([&](auto ctx, auto ch_in, auto ch_out, std::vector<Node>& nodes) {
244+
auto [insert_node, extract_node] = shuffler_nb(
104245
std::move(ctx),
105246
stream,
106247
std::move(ch_in),
107248
std::move(ch_out),
108249
op_id,
109250
num_partitions
110-
)
111-
);
112-
});
113-
}
114-
115-
TEST_F(StreamingShuffler, callbacks) {
116-
run_test([&](auto ctx, auto ch_in, auto ch_out, std::vector<Node>& nodes) {
117-
auto [insert_node, extract_node] = node::shuffler_nb(
118-
std::move(ctx),
119-
stream,
120-
std::move(ch_in),
121-
std::move(ch_out),
122-
op_id,
123-
num_partitions
124-
);
125-
nodes.emplace_back(std::move(insert_node));
126-
nodes.emplace_back(std::move(extract_node));
127-
});
251+
);
252+
nodes.emplace_back(std::move(insert_node));
253+
nodes.emplace_back(std::move(extract_node));
254+
})
255+
);
128256
}

0 commit comments

Comments
 (0)