Skip to content

Commit 7fe72db

Browse files
[SYCL] Lower queue::wait() to piQueueFinish when possible (#4044)
This patch changes the logic of queue::wait() from waiting on each individual event in order of submission of their tasks to checking if each event's task has been enqueued, waiting for those that haven't been and calling piQueueFinish to take care of the rest. Notable exceptions to this new behaviour are host queues, queues that emulate out-of-order execution by creating multiple queues underneath and host task events, which are run on host regardless of the queue they are bound to.
1 parent 76051cc commit 7fe72db

File tree

10 files changed

+330
-29
lines changed

10 files changed

+330
-29
lines changed

sycl/source/detail/event_impl.cpp

Lines changed: 7 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -200,8 +200,7 @@ void event_impl::wait(
200200
waitInternal();
201201
else if (MCommand)
202202
detail::Scheduler::getInstance().waitForEvent(Self);
203-
if (MCommand && !SYCLConfig<SYCL_DISABLE_EXECUTION_GRAPH_CLEANUP>::get())
204-
detail::Scheduler::getInstance().cleanupFinishedCommands(std::move(Self));
203+
cleanupCommand(std::move(Self));
205204

206205
#ifdef XPTI_ENABLE_INSTRUMENTATION
207206
instrumentationEpilog(TelemetryEvent, Name, StreamID, IId);
@@ -222,6 +221,12 @@ void event_impl::wait_and_throw(
222221
Cmd->getQueue()->throw_asynchronous();
223222
}
224223

224+
void event_impl::cleanupCommand(
225+
std::shared_ptr<cl::sycl::detail::event_impl> Self) const {
226+
if (MCommand && !SYCLConfig<SYCL_DISABLE_EXECUTION_GRAPH_CLEANUP>::get())
227+
detail::Scheduler::getInstance().cleanupFinishedCommands(std::move(Self));
228+
}
229+
225230
template <>
226231
cl_ulong
227232
event_impl::get_profiling_info<info::event_profiling::command_submit>() const {

sycl/source/detail/event_impl.hpp

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -74,6 +74,12 @@ class event_impl {
7474
/// \param Self is a pointer to this event.
7575
void wait_and_throw(std::shared_ptr<cl::sycl::detail::event_impl> Self);
7676

77+
/// Clean up the command associated with the event. Assumes that the task this
78+
/// event is associated with has been completed.
79+
///
80+
/// \param Self is a pointer to this event.
81+
void cleanupCommand(std::shared_ptr<cl::sycl::detail::event_impl> Self) const;
82+
7783
/// Queries this event for profiling information.
7884
///
7985
/// If the requested info is not available when this member function is

sycl/source/detail/queue_impl.cpp

Lines changed: 68 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -61,7 +61,11 @@ event queue_impl::memset(const std::shared_ptr<detail::queue_impl> &Self,
6161
return event();
6262

6363
event ResEvent = prepareUSMEvent(Self, NativeEvent);
64-
addSharedEvent(ResEvent);
64+
// Track only if we won't be able to handle it with piQueueFinish.
65+
// FIXME these events are stored for level zero until as a workaround, remove
66+
// once piEventRelease no longer calls wait on the event in the plugin.
67+
if (!MSupportOOO || getPlugin().getBackend() == backend::level_zero)
68+
addSharedEvent(ResEvent);
6569
return ResEvent;
6670
}
6771

@@ -76,7 +80,11 @@ event queue_impl::memcpy(const std::shared_ptr<detail::queue_impl> &Self,
7680
return event();
7781

7882
event ResEvent = prepareUSMEvent(Self, NativeEvent);
79-
addSharedEvent(ResEvent);
83+
// Track only if we won't be able to handle it with piQueueFinish.
84+
// FIXME these events are stored for level zero until as a workaround, remove
85+
// once piEventRelease no longer calls wait on the event in the plugin.
86+
if (!MSupportOOO || getPlugin().getBackend() == backend::level_zero)
87+
addSharedEvent(ResEvent);
8088
return ResEvent;
8189
}
8290

@@ -92,7 +100,11 @@ event queue_impl::mem_advise(const std::shared_ptr<detail::queue_impl> &Self,
92100
return event();
93101

94102
event ResEvent = prepareUSMEvent(Self, NativeEvent);
95-
addSharedEvent(ResEvent);
103+
// Track only if we won't be able to handle it with piQueueFinish.
104+
// FIXME these events are stored for level zero until as a workaround, remove
105+
// once piEventRelease no longer calls wait on the event in the plugin.
106+
if (!MSupportOOO || getPlugin().getBackend() == backend::level_zero)
107+
addSharedEvent(ResEvent);
96108
return ResEvent;
97109
}
98110

@@ -101,8 +113,14 @@ void queue_impl::addEvent(const event &Event) {
101113
Command *Cmd = (Command *)(Eimpl->getCommand());
102114
if (!Cmd) {
103115
// if there is no command on the event, we cannot track it with MEventsWeak
104-
// as that will leave it with no owner. Track in MEventsShared
105-
addSharedEvent(Event);
116+
// as that will leave it with no owner. Track in MEventsShared only if we're
117+
// unable to call piQueueFinish during wait.
118+
// FIXME these events are stored for level zero until as a workaround,
119+
// remove once piEventRelease no longer calls wait on the event in the
120+
// plugin.
121+
if (is_host() || !MSupportOOO ||
122+
getPlugin().getBackend() == backend::level_zero)
123+
addSharedEvent(Event);
106124
} else {
107125
std::weak_ptr<event_impl> EventWeakPtr{Eimpl};
108126
std::lock_guard<std::mutex> Lock{MMutex};
@@ -114,6 +132,10 @@ void queue_impl::addEvent(const event &Event) {
114132
/// but some events have no other owner. In this case,
115133
/// addSharedEvent will have the queue track the events via a shared pointer.
116134
void queue_impl::addSharedEvent(const event &Event) {
135+
// FIXME The assertion should be corrected once the Level Zero workaround is
136+
// removed.
137+
assert(is_host() || !MSupportOOO ||
138+
getPlugin().getBackend() == backend::level_zero);
117139
std::lock_guard<std::mutex> Lock(MMutex);
118140
// Events stored in MEventsShared are not released anywhere else aside from
119141
// calls to queue::wait/wait_and_throw, which a user application might not
@@ -234,21 +256,49 @@ void queue_impl::wait(const detail::code_location &CodeLoc) {
234256
TelemetryEvent = instrumentationProlog(CodeLoc, Name, StreamID, IId);
235257
#endif
236258

237-
std::vector<std::weak_ptr<event_impl>> Events;
238-
std::vector<event> USMEvents;
259+
std::vector<std::weak_ptr<event_impl>> WeakEvents;
260+
std::vector<event> SharedEvents;
239261
{
240-
std::lock_guard<std::mutex> Lock(MMutex);
241-
Events.swap(MEventsWeak);
242-
USMEvents.swap(MEventsShared);
262+
std::lock_guard<mutex_class> Lock(MMutex);
263+
WeakEvents.swap(MEventsWeak);
264+
SharedEvents.swap(MEventsShared);
265+
}
266+
// If the queue is either a host one or does not support OOO (and we use
267+
// multiple in-order queues as a result of that), wait for each event
268+
// directly. Otherwise, only wait for unenqueued or host task events, starting
269+
// from the latest submitted task in order to minimize total amount of calls,
270+
// then handle the rest with piQueueFinish.
271+
bool SupportsPiFinish = !is_host() && MSupportOOO;
272+
for (auto EventImplWeakPtrIt = WeakEvents.rbegin();
273+
EventImplWeakPtrIt != WeakEvents.rend(); ++EventImplWeakPtrIt) {
274+
if (std::shared_ptr<event_impl> EventImplSharedPtr =
275+
EventImplWeakPtrIt->lock()) {
276+
// A nullptr PI event indicates that piQueueFinish will not cover it,
277+
// either because it's a host task event or an unenqueued one.
278+
if (!SupportsPiFinish || nullptr == EventImplSharedPtr->getHandleRef()) {
279+
EventImplSharedPtr->wait(EventImplSharedPtr);
280+
}
281+
}
282+
}
283+
if (SupportsPiFinish) {
284+
const detail::plugin &Plugin = getPlugin();
285+
Plugin.call<detail::PiApiKind::piQueueFinish>(getHandleRef());
286+
for (std::weak_ptr<event_impl> &EventImplWeakPtr : WeakEvents)
287+
if (std::shared_ptr<event_impl> EventImplSharedPtr =
288+
EventImplWeakPtr.lock())
289+
EventImplSharedPtr->cleanupCommand(EventImplSharedPtr);
290+
// FIXME these events are stored for level zero until as a workaround,
291+
// remove once piEventRelease no longer calls wait on the event in the
292+
// plugin.
293+
if (Plugin.getBackend() == backend::level_zero) {
294+
SharedEvents.clear();
295+
}
296+
assert(SharedEvents.empty() && "Queues that support calling piQueueFinish "
297+
"shouldn't have shared events");
298+
} else {
299+
for (event &Event : SharedEvents)
300+
Event.wait();
243301
}
244-
245-
for (std::weak_ptr<event_impl> &EventImplWeakPtr : Events)
246-
if (std::shared_ptr<event_impl> EventImplPtr = EventImplWeakPtr.lock())
247-
EventImplPtr->wait(EventImplPtr);
248-
249-
for (event &Event : USMEvents)
250-
Event.wait();
251-
252302
#ifdef XPTI_ENABLE_INSTRUMENTATION
253303
instrumentationEpilog(TelemetryEvent, Name, StreamID, IId);
254304
#endif

sycl/source/detail/scheduler/commands.cpp

Lines changed: 20 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -486,10 +486,14 @@ Command *Command::processDepEvent(EventImplPtr DepEvent, const DepDesc &Dep) {
486486
const ContextImplPtr &WorkerContext = WorkerQueue->getContextImplPtr();
487487

488488
// 1. Async work is not supported for host device.
489-
// 2. The event handle can be null in case of, for example, alloca command,
490-
// which is currently synchronous, so don't generate OpenCL event.
491-
// Though, this event isn't host one as it's context isn't host one.
492-
if (DepEvent->is_host() || DepEvent->getHandleRef() == nullptr) {
489+
// 2. Some types of commands do not produce PI events after they are enqueued
490+
// (e.g. alloca). Note that we can't check the pi event to make that
491+
// distinction since the command might still be unenqueued at this point.
492+
bool PiEventExpected = !DepEvent->is_host();
493+
if (auto *DepCmd = static_cast<Command *>(DepEvent->getCommand()))
494+
PiEventExpected &= DepCmd->producesPiEvent();
495+
496+
if (!PiEventExpected) {
493497
// call to waitInternal() is in waitForPreparedHostEvents() as it's called
494498
// from enqueue process functions
495499
MPreparedHostDepsEvents.push_back(DepEvent);
@@ -520,6 +524,8 @@ const ContextImplPtr &Command::getWorkerContext() const {
520524

521525
const QueueImplPtr &Command::getWorkerQueue() const { return MQueue; }
522526

527+
bool Command::producesPiEvent() const { return true; }
528+
523529
Command *Command::addDep(DepDesc NewDep) {
524530
Command *ConnectionCmd = nullptr;
525531

@@ -731,6 +737,8 @@ void AllocaCommandBase::emitInstrumentationData() {
731737
#endif
732738
}
733739

740+
bool AllocaCommandBase::producesPiEvent() const { return false; }
741+
734742
AllocaCommand::AllocaCommand(QueueImplPtr Queue, Requirement Req,
735743
bool InitFromUserData,
736744
AllocaCommandBase *LinkedAllocaCmd)
@@ -998,6 +1006,8 @@ void ReleaseCommand::printDot(std::ostream &Stream) const {
9981006
}
9991007
}
10001008

1009+
bool ReleaseCommand::producesPiEvent() const { return false; }
1010+
10011011
MapMemObject::MapMemObject(AllocaCommandBase *SrcAllocaCmd, Requirement Req,
10021012
void **DstPtr, QueueImplPtr Queue,
10031013
access::mode MapMode)
@@ -1392,6 +1402,8 @@ void EmptyCommand::printDot(std::ostream &Stream) const {
13921402
}
13931403
}
13941404

1405+
bool EmptyCommand::producesPiEvent() const { return false; }
1406+
13951407
void MemCpyCommandHost::printDot(std::ostream &Stream) const {
13961408
Stream << "\"" << this << "\" [style=filled, fillcolor=\"#B6A2EB\", label=\"";
13971409

@@ -2193,6 +2205,10 @@ cl_int ExecCGCommand::enqueueImp() {
21932205
return PI_INVALID_OPERATION;
21942206
}
21952207

2208+
bool ExecCGCommand::producesPiEvent() const {
2209+
return MCommandGroup->getType() != CG::CGTYPE::CODEPLAY_HOST_TASK;
2210+
}
2211+
21962212
} // namespace detail
21972213
} // namespace sycl
21982214
} // __SYCL_INLINE_NAMESPACE(cl)

sycl/source/detail/scheduler/commands.hpp

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,9 @@ class Command {
189189
/// for memory copy commands.
190190
virtual const QueueImplPtr &getWorkerQueue() const;
191191

192+
/// Returns true iff the command produces a PI event on non-host devices.
193+
virtual bool producesPiEvent() const;
194+
192195
protected:
193196
EventImplPtr MEvent;
194197
QueueImplPtr MQueue;
@@ -306,6 +309,8 @@ class EmptyCommand : public Command {
306309

307310
void emitInstrumentationData() override;
308311

312+
bool producesPiEvent() const final;
313+
309314
private:
310315
cl_int enqueueImp() final;
311316

@@ -323,6 +328,7 @@ class ReleaseCommand : public Command {
323328

324329
void printDot(std::ostream &Stream) const final;
325330
void emitInstrumentationData() override;
331+
bool producesPiEvent() const final;
326332

327333
private:
328334
cl_int enqueueImp() final;
@@ -347,6 +353,8 @@ class AllocaCommandBase : public Command {
347353

348354
void emitInstrumentationData() override;
349355

356+
bool producesPiEvent() const final;
357+
350358
void *MMemAllocation = nullptr;
351359

352360
/// Alloca command linked with current command.
@@ -518,6 +526,8 @@ class ExecCGCommand : public Command {
518526
MCommandGroup.release();
519527
}
520528

529+
bool producesPiEvent() const final;
530+
521531
private:
522532
cl_int enqueueImp() final;
523533

sycl/test/on-device/plugins/level_zero_batch_event_status.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -25,10 +25,10 @@
2525
// CHECK: ZE ---> zeCommandListClose
2626
// CHECK: ZE ---> zeCommandQueueExecuteCommandLists
2727
// CHECK: ---> piEventGetInfo
28-
// CHECK-NOT: piEventsWait
28+
// CHECK-NOT: piQueueFinish
2929
// CHECK: ---> piEnqueueKernelLaunch
3030
// CHECK: ZE ---> zeCommandListAppendLaunchKernel
31-
// CHECK: ---> piEventsWait
31+
// CHECK: ---> piQueueFinish
3232
// Look for close and Execute after piEventsWait
3333
// CHECK: ZE ---> zeCommandListClose
3434
// CHECK: ZE ---> zeCommandQueueExecuteCommandLists

sycl/test/on-device/plugins/level_zero_batch_test.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -86,7 +86,7 @@
8686
// CKB4: ZE ---> zeCommandQueueExecuteCommandLists(
8787
// CKB8: ZE ---> zeCommandListClose(
8888
// CKB8: ZE ---> zeCommandQueueExecuteCommandLists(
89-
// CKALL: ---> piEventsWait(
89+
// CKALL: ---> piQueueFinish(
9090
// CKB3: ZE ---> zeCommandListClose(
9191
// CKB3: ZE ---> zeCommandQueueExecuteCommandLists(
9292
// CKB5: ZE ---> zeCommandListClose(
@@ -142,7 +142,7 @@
142142
// CKB4: ZE ---> zeCommandQueueExecuteCommandLists(
143143
// CKB8: ZE ---> zeCommandListClose(
144144
// CKB8: ZE ---> zeCommandQueueExecuteCommandLists(
145-
// CKALL: ---> piEventsWait(
145+
// CKALL: ---> piQueueFinish(
146146
// CKB3: ZE ---> zeCommandListClose(
147147
// CKB3: ZE ---> zeCommandQueueExecuteCommandLists(
148148
// CKB5: ZE ---> zeCommandListClose(
@@ -198,7 +198,7 @@
198198
// CKB4: ZE ---> zeCommandQueueExecuteCommandLists(
199199
// CKB8: ZE ---> zeCommandListClose(
200200
// CKB8: ZE ---> zeCommandQueueExecuteCommandLists(
201-
// CKALL: ---> piEventsWait(
201+
// CKALL: ---> piQueueFinish(
202202
// CKB3: ZE ---> zeCommandListClose(
203203
// CKB3: ZE ---> zeCommandQueueExecuteCommandLists(
204204
// CKB5: ZE ---> zeCommandListClose(

sycl/unittests/queue/CMakeLists.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,3 +1,4 @@
11
add_sycl_unittest(QueueTests OBJECT
22
EventClear.cpp
3+
Wait.cpp
34
)

sycl/unittests/queue/EventClear.cpp

Lines changed: 15 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,19 @@ std::unique_ptr<TestCtx> TestContext;
2525

2626
const int ExpectedEventThreshold = 128;
2727

28+
pi_result redefinedQueueCreate(pi_context context, pi_device device,
29+
pi_queue_properties properties,
30+
pi_queue *queue) {
31+
// Use in-order queues to force storing events for calling wait on them,
32+
// rather than calling piQueueFinish.
33+
if (properties & PI_QUEUE_OUT_OF_ORDER_EXEC_MODE_ENABLE) {
34+
return PI_INVALID_QUEUE_PROPERTIES;
35+
}
36+
return PI_SUCCESS;
37+
}
38+
39+
pi_result redefinedQueueRelease(pi_queue Queue) { return PI_SUCCESS; }
40+
2841
pi_result redefinedUSMEnqueueMemset(pi_queue queue, void *ptr, pi_int32 value,
2942
size_t count,
3043
pi_uint32 num_events_in_waitlist,
@@ -83,6 +96,8 @@ bool preparePiMock(platform &Plt) {
8396
}
8497

8598
unittest::PiMock Mock{Plt};
99+
Mock.redefine<detail::PiApiKind::piQueueCreate>(redefinedQueueCreate);
100+
Mock.redefine<detail::PiApiKind::piQueueRelease>(redefinedQueueRelease);
86101
Mock.redefine<detail::PiApiKind::piextUSMEnqueueMemset>(
87102
redefinedUSMEnqueueMemset);
88103
Mock.redefine<detail::PiApiKind::piEventsWait>(redefinedEventsWait);

0 commit comments

Comments
 (0)