feature: handle in-order allocation overflow

Related-To: NEO-7966

Signed-off-by: Dunajski, Bartosz <bartosz.dunajski@intel.com>
This commit is contained in:
Dunajski, Bartosz 2023-07-04 10:43:51 +00:00 committed by Compute-Runtime-Automation
parent 61f701aba5
commit 5b9d45f3bc
13 changed files with 124 additions and 27 deletions

View File

@ -169,7 +169,7 @@ struct CommandListCoreFamily : CommandListImp {
ze_result_t appendSignalEvent(ze_event_handle_t hEvent) override;
ze_result_t appendWaitOnEvents(uint32_t numEvents, ze_event_handle_t *phEvent, bool relaxedOrderingAllowed, bool trackDependencies, bool signalInOrderCompletion) override;
void appendWaitOnInOrderDependency(NEO::GraphicsAllocation *dependencyCounterAllocation, uint64_t waitValue, bool relaxedOrderingAllowed);
void appendWaitOnInOrderDependency(NEO::GraphicsAllocation *dependencyCounterAllocation, uint64_t waitValue, uint32_t offset, bool relaxedOrderingAllowed);
void appendSignalInOrderDependencyCounter();
ze_result_t appendWriteGlobalTimestamp(uint64_t *dstptr, ze_event_handle_t hSignalEvent,
uint32_t numWaitEvents, ze_event_handle_t *phWaitEvents) override;

View File

@ -2132,7 +2132,7 @@ inline ze_result_t CommandListCoreFamily<gfxCoreFamily>::addEventsToCmdList(uint
}
if (hasInOrderDependencies) {
CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(this->inOrderDependencyCounterAllocation, this->inOrderDependencyCounter, relaxedOrderingAllowed);
CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(this->inOrderDependencyCounterAllocation, this->inOrderDependencyCounter, this->inOrderAllocationOffset, relaxedOrderingAllowed);
}
if (numWaitEvents > 0) {
@ -2187,14 +2187,14 @@ ze_result_t CommandListCoreFamily<gfxCoreFamily>::appendSignalEvent(ze_event_han
}
template <GFXCORE_FAMILY gfxCoreFamily>
void CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(NEO::GraphicsAllocation *dependencyCounterAllocation, uint64_t waitValue, bool relaxedOrderingAllowed) {
void CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(NEO::GraphicsAllocation *dependencyCounterAllocation, uint64_t waitValue, uint32_t offset, bool relaxedOrderingAllowed) {
using COMPARE_OPERATION = typename GfxFamily::MI_SEMAPHORE_WAIT::COMPARE_OPERATION;
UNRECOVERABLE_IF(waitValue >= std::numeric_limits<uint32_t>::max());
UNRECOVERABLE_IF(waitValue > std::numeric_limits<uint32_t>::max());
commandContainer.addToResidencyContainer(dependencyCounterAllocation);
uint64_t gpuAddress = dependencyCounterAllocation->getGpuAddress();
uint64_t gpuAddress = dependencyCounterAllocation->getGpuAddress() + offset;
for (uint32_t i = 0; i < this->partitionCount; i++) {
if (relaxedOrderingAllowed) {
@ -2262,7 +2262,7 @@ ze_result_t CommandListCoreFamily<gfxCoreFamily>::appendWaitOnEvents(uint32_t nu
(event->getInOrderExecSignalValue() == this->inOrderDependencyCounter);
if (!eventFromPreviousAppend) {
CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(event->getInOrderExecDataAllocation(), event->getInOrderExecSignalValue(), relaxedOrderingAllowed);
CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(event->getInOrderExecDataAllocation(), event->getInOrderExecSignalValue(), event->getInOrderAllocationOffset(), relaxedOrderingAllowed);
}
continue;
}
@ -2318,7 +2318,9 @@ void CommandListCoreFamily<gfxCoreFamily>::appendSignalInOrderDependencyCounter(
auto lowPart = static_cast<uint32_t>(signalValue & 0x0000FFFFFFFFULL);
auto highPart = static_cast<uint32_t>(signalValue >> 32);
NEO::EncodeStoreMemory<GfxFamily>::programStoreDataImm(*commandContainer.getCommandStream(), this->inOrderDependencyCounterAllocation->getGpuAddress(),
uint64_t gpuVa = this->inOrderDependencyCounterAllocation->getGpuAddress() + this->inOrderAllocationOffset;
NEO::EncodeStoreMemory<GfxFamily>::programStoreDataImm(*commandContainer.getCommandStream(), gpuVa,
lowPart, highPart, true, (this->partitionCount > 1));
}

View File

@ -42,6 +42,7 @@ struct CommandListCoreFamilyImmediate : public CommandListCoreFamily<gfxCoreFami
using BaseClass::BaseClass;
using BaseClass::copyThroughLockedPtrEnabled;
using BaseClass::executeCommandListImmediate;
using BaseClass::inOrderAllocationOffset;
using BaseClass::isCopyOnly;
using BaseClass::isInOrderExecutionEnabled;
@ -183,8 +184,9 @@ struct CommandListCoreFamilyImmediate : public CommandListCoreFamily<gfxCoreFami
void printKernelsPrintfOutput(bool hangDetected);
ze_result_t synchronizeInOrderExecution(uint64_t timeout) const;
bool hasStallingCmdsForRelaxedOrdering(uint32_t numWaitEvents, bool relaxedOrderingDispatch);
bool hasStallingCmdsForRelaxedOrdering(uint32_t numWaitEvents, bool relaxedOrderingDispatch) const;
void setupFlushMethod(const NEO::RootDeviceEnvironment &rootDeviceEnvironment) override;
void handleInOrderDependencyCounter();
MOCKABLE_VIRTUAL void checkAssert();
ComputeFlushMethodType computeFlushMethod = nullptr;

View File

@ -330,7 +330,7 @@ bool CommandListCoreFamilyImmediate<gfxCoreFamily>::waitForEventsFromHost() {
}
template <GFXCORE_FAMILY gfxCoreFamily>
bool CommandListCoreFamilyImmediate<gfxCoreFamily>::hasStallingCmdsForRelaxedOrdering(uint32_t numWaitEvents, bool relaxedOrderingDispatch) {
bool CommandListCoreFamilyImmediate<gfxCoreFamily>::hasStallingCmdsForRelaxedOrdering(uint32_t numWaitEvents, bool relaxedOrderingDispatch) const {
return (!relaxedOrderingDispatch && (numWaitEvents > 0 || this->inOrderDependencyCounter > 0));
}
@ -393,7 +393,7 @@ ze_result_t CommandListCoreFamilyImmediate<gfxCoreFamily>::appendBarrier(
if (signalEvent->isEventTimestampFlagSet()) {
earlyReturn = false;
} else {
signalEvent->enableInOrderExecMode(*this->inOrderDependencyCounterAllocation, this->inOrderDependencyCounter);
signalEvent->enableInOrderExecMode(*this->inOrderDependencyCounterAllocation, this->inOrderDependencyCounter, this->inOrderAllocationOffset);
}
}
@ -765,9 +765,7 @@ ze_result_t CommandListCoreFamilyImmediate<gfxCoreFamily>::flushImmediate(ze_res
if (inputRet == ZE_RESULT_SUCCESS) {
if (isInOrderExecutionEnabled()) {
inOrderDependencyCounter++;
this->commandContainer.addToResidencyContainer(this->inOrderDependencyCounterAllocation);
handleInOrderDependencyCounter();
}
if (this->isFlushTaskSubmissionEnabled) {
@ -784,13 +782,35 @@ ze_result_t CommandListCoreFamilyImmediate<gfxCoreFamily>::flushImmediate(ze_res
signalEvent->setCsr(this->csr);
if (isInOrderExecutionEnabled()) {
signalEvent->enableInOrderExecMode(*this->inOrderDependencyCounterAllocation, this->inOrderDependencyCounter);
signalEvent->enableInOrderExecMode(*this->inOrderDependencyCounterAllocation, this->inOrderDependencyCounter, this->inOrderAllocationOffset);
}
}
return inputRet;
}
template <GFXCORE_FAMILY gfxCoreFamily>
void CommandListCoreFamilyImmediate<gfxCoreFamily>::handleInOrderDependencyCounter() {
if ((inOrderDependencyCounter + 1) == std::numeric_limits<uint32_t>::max()) {
CommandListCoreFamily<gfxCoreFamily>::appendWaitOnInOrderDependency(inOrderDependencyCounterAllocation, inOrderDependencyCounter + 1, inOrderAllocationOffset, false);
inOrderDependencyCounter = 0;
// multitile immediate writes are uint64_t aligned
uint32_t offset = this->partitionCount * static_cast<uint32_t>(sizeof(uint64_t));
inOrderAllocationOffset += offset;
UNRECOVERABLE_IF(inOrderAllocationOffset + offset >= inOrderDependencyCounterAllocation->getUnderlyingBufferSize());
CommandListCoreFamily<gfxCoreFamily>::appendSignalInOrderDependencyCounter(); // write 1 on new offset
}
inOrderDependencyCounter++;
this->commandContainer.addToResidencyContainer(this->inOrderDependencyCounterAllocation);
}
template <GFXCORE_FAMILY gfxCoreFamily>
bool CommandListCoreFamilyImmediate<gfxCoreFamily>::preferCopyThroughLockedPtr(CpuMemCopyInfo &cpuMemCopyInfo, uint32_t numWaitEvents, ze_event_handle_t *phWaitEvents) {
if (NEO::DebugManager.flags.ExperimentalForceCopyThroughLock.get() == 1) {
@ -1136,7 +1156,7 @@ ze_result_t CommandListCoreFamilyImmediate<gfxCoreFamily>::synchronizeInOrderExe
bool signaled = true;
auto hostAddress = static_cast<uint64_t *>(this->inOrderDependencyCounterAllocation->getUnderlyingBuffer());
auto hostAddress = static_cast<uint64_t *>(ptrOffset(this->inOrderDependencyCounterAllocation->getUnderlyingBuffer(), this->inOrderAllocationOffset));
for (uint32_t i = 0; i < this->partitionCount; i++) {
if (!NEO::WaitUtils::waitFunctionWithPredicate<const uint64_t>(hostAddress, waitValue, std::greater_equal<uint64_t>())) {

View File

@ -279,7 +279,7 @@ ze_result_t CommandListCoreFamily<gfxCoreFamily>::appendLaunchKernelWithParams(K
bool inOrderExecSignalRequired = (this->inOrderExecutionEnabled && !launchParams.isKernelSplitOperation);
if (inOrderExecSignalRequired && !event) {
dispatchKernelArgs.eventAddress = this->inOrderDependencyCounterAllocation->getGpuAddress();
dispatchKernelArgs.eventAddress = this->inOrderDependencyCounterAllocation->getGpuAddress() + this->inOrderAllocationOffset;
dispatchKernelArgs.postSyncImmValue = this->inOrderDependencyCounter + 1;
}

View File

@ -232,7 +232,7 @@ void CommandListImp::enableInOrderExecution() {
auto device = this->device->getNEODevice();
NEO::AllocationProperties allocationProperties{device->getRootDeviceIndex(), sizeof(uint32_t), NEO::AllocationType::TIMESTAMP_PACKET_TAG_BUFFER, device->getDeviceBitfield()};
NEO::AllocationProperties allocationProperties{device->getRootDeviceIndex(), MemoryConstants::pageSize64k, NEO::AllocationType::TIMESTAMP_PACKET_TAG_BUFFER, device->getDeviceBitfield()};
inOrderDependencyCounterAllocation = device->getMemoryManager()->allocateGraphicsMemoryWithProperties(allocationProperties);

View File

@ -44,6 +44,7 @@ struct CommandListImp : CommandList {
std::unique_ptr<NEO::LogicalStateHelper> nonImmediateLogicalStateHelper;
NEO::GraphicsAllocation *inOrderDependencyCounterAllocation = nullptr;
uint64_t inOrderDependencyCounter = 0;
uint32_t inOrderAllocationOffset = 0;
bool inOrderExecutionEnabled = false;
~CommandListImp() override = default;

View File

@ -391,11 +391,12 @@ void Event::setIsCompleted() {
unsetCmdQueue(true);
}
void Event::enableInOrderExecMode(NEO::GraphicsAllocation &inOrderDependenciesAllocation, uint64_t signalValue) {
void Event::enableInOrderExecMode(NEO::GraphicsAllocation &inOrderDependenciesAllocation, uint64_t signalValue, uint32_t allocationOffset) {
inOrderExecEvent = true;
inOrderExecSignalValue = signalValue;
inOrderExecDataAllocation = &inOrderDependenciesAllocation;
inOrderAllocationOffset = allocationOffset;
}
void Event::setLatestUsedCmdQueue(CommandQueue *newCmdQ) {

View File

@ -211,10 +211,11 @@ struct Event : _ze_event_handle_t {
void setMetricStreamer(MetricStreamer *metricStreamer) {
this->metricStreamer = metricStreamer;
}
void enableInOrderExecMode(NEO::GraphicsAllocation &inOrderDependenciesAllocation, uint64_t signalValue);
void enableInOrderExecMode(NEO::GraphicsAllocation &inOrderDependenciesAllocation, uint64_t signalValue, uint32_t allocationOffset);
bool isInOrderExecEvent() const { return inOrderExecEvent; }
NEO::GraphicsAllocation *getInOrderExecDataAllocation() const { return inOrderExecDataAllocation; }
uint64_t getInOrderExecSignalValue() const { return inOrderExecSignalValue; }
uint32_t getInOrderAllocationOffset() const { return inOrderAllocationOffset; }
void setLatestUsedCmdQueue(CommandQueue *newCmdQ);
void setReferenceTs(NEO::TimeStampData &timestamp) {
referenceTs = timestamp;
@ -233,6 +234,7 @@ struct Event : _ze_event_handle_t {
NEO::TimeStampData referenceTs{};
uint64_t inOrderExecSignalValue = 0;
uint32_t inOrderAllocationOffset = 0;
std::chrono::microseconds gpuHangCheckPeriod{500'000};
std::bitset<EventPacketsCount::maxKernelSplit> l3FlushAppliedOnKernel;

View File

@ -134,7 +134,7 @@ void EventImp<TagSizeT>::assignKernelEventCompletionData(void *address) {
template <typename TagSizeT>
ze_result_t EventImp<TagSizeT>::queryInOrderEventStatus() {
auto hostAddress = static_cast<uint64_t *>(this->inOrderExecDataAllocation->getUnderlyingBuffer());
auto hostAddress = static_cast<uint64_t *>(ptrOffset(this->inOrderExecDataAllocation->getUnderlyingBuffer(), this->inOrderAllocationOffset));
bool signaled = true;
for (uint32_t i = 0; i < this->getPacketsInUse(); i++) {
@ -403,6 +403,7 @@ ze_result_t EventImp<TagSizeT>::reset() {
inOrderExecEvent = false;
inOrderExecDataAllocation = nullptr;
inOrderExecSignalValue = 0;
inOrderAllocationOffset = 0;
}
unsetCmdQueue(false);
this->resetCompletionStatus();

View File

@ -75,6 +75,7 @@ struct WhiteBox<::L0::CommandListCoreFamily<gfxCoreFamily>>
using BaseClass::immediateCmdListHeapSharing;
using BaseClass::indirectAllocationsAllowed;
using BaseClass::initialize;
using BaseClass::inOrderAllocationOffset;
using BaseClass::isFlushTaskSubmissionEnabled;
using BaseClass::isRelaxedOrderingDispatchAllowed;
using BaseClass::isSyncModeQueue;

View File

@ -676,6 +676,7 @@ struct InOrderCmdListTests : public CommandListAppendLaunchKernel {
using EventImp<uint32_t>::maxPacketCount;
using EventImp<uint32_t>::inOrderExecDataAllocation;
using EventImp<uint32_t>::inOrderExecSignalValue;
using EventImp<uint32_t>::inOrderAllocationOffset;
};
void SetUp() override {
@ -766,22 +767,30 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenResetEventCalledThenResetEven
immCmdList->appendLaunchKernel(kernel->toHandle(), &groupCount, events[0]->toHandle(), 0, nullptr, launchParams, false);
EXPECT_EQ(MemoryConstants::pageSize64k, immCmdList->inOrderDependencyCounterAllocation->getUnderlyingBufferSize());
EXPECT_TRUE(events[0]->inOrderExecEvent);
EXPECT_EQ(events[0]->inOrderExecSignalValue, immCmdList->inOrderDependencyCounter);
EXPECT_EQ(events[0]->inOrderExecDataAllocation, immCmdList->inOrderDependencyCounterAllocation);
EXPECT_EQ(events[0]->inOrderAllocationOffset, 0u);
events[0]->inOrderAllocationOffset = 123;
events[0]->reset();
EXPECT_FALSE(events[0]->inOrderExecEvent);
EXPECT_EQ(events[0]->inOrderExecSignalValue, 0u);
EXPECT_EQ(events[0]->inOrderExecDataAllocation, nullptr);
EXPECT_EQ(events[0]->inOrderAllocationOffset, 0u);
}
HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenSubmittingThenProgramSemaphoreForPreviousDispatch, IsAtLeastXeHpCore) {
using MI_SEMAPHORE_WAIT = typename FamilyType::MI_SEMAPHORE_WAIT;
uint32_t counterOffset = 64;
auto immCmdList = createImmCmdList<gfxCoreFamily>();
immCmdList->inOrderAllocationOffset = counterOffset;
auto cmdStream = immCmdList->getCmdContainer().getCommandStream();
@ -804,7 +813,7 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenSubmittingThenProgramSemaphor
auto semaphoreCmd = genCmdCast<MI_SEMAPHORE_WAIT *>(*itor);
EXPECT_EQ(1u, semaphoreCmd->getSemaphoreDataDword());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress(), semaphoreCmd->getSemaphoreGraphicsAddress());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress() + counterOffset, semaphoreCmd->getSemaphoreGraphicsAddress());
EXPECT_EQ(MI_SEMAPHORE_WAIT::COMPARE_OPERATION::COMPARE_OPERATION_SAD_GREATER_THAN_OR_EQUAL_SDD, semaphoreCmd->getCompareOperation());
}
@ -874,7 +883,10 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenWaitingForEventFromAfterReset
HWTEST2_F(InOrderCmdListTests, givenInOrderEventModeWhenSubmittingThenProgramSemaphoreForEvent, IsAtLeastXeHpCore) {
using MI_SEMAPHORE_WAIT = typename FamilyType::MI_SEMAPHORE_WAIT;
uint32_t counterOffset = 64;
auto immCmdList = createImmCmdList<gfxCoreFamily>();
immCmdList->inOrderAllocationOffset = counterOffset;
auto eventPool = createEvents<FamilyType>(1, false);
@ -907,7 +919,7 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderEventModeWhenSubmittingThenProgramSem
auto semaphoreCmd = genCmdCast<MI_SEMAPHORE_WAIT *>(*itor);
EXPECT_EQ(2u, semaphoreCmd->getSemaphoreDataDword());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress(), semaphoreCmd->getSemaphoreGraphicsAddress());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress() + counterOffset, semaphoreCmd->getSemaphoreGraphicsAddress());
EXPECT_EQ(MI_SEMAPHORE_WAIT::COMPARE_OPERATION::COMPARE_OPERATION_SAD_GREATER_THAN_OR_EQUAL_SDD, semaphoreCmd->getCompareOperation());
}
@ -1038,7 +1050,10 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenProgrammingWalkerThenSignalSy
using MI_SEMAPHORE_WAIT = typename FamilyType::MI_SEMAPHORE_WAIT;
using MI_STORE_DATA_IMM = typename FamilyType::MI_STORE_DATA_IMM;
uint32_t counterOffset = 64;
auto immCmdList = createImmCmdList<gfxCoreFamily>();
immCmdList->inOrderAllocationOffset = counterOffset;
auto cmdStream = immCmdList->getCmdContainer().getCommandStream();
@ -1059,7 +1074,7 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenProgrammingWalkerThenSignalSy
EXPECT_EQ(POSTSYNC_DATA::OPERATION_WRITE_IMMEDIATE_DATA, postSync.getOperation());
EXPECT_EQ(1u, postSync.getImmediateData());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress(), postSync.getDestinationAddress());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress() + counterOffset, postSync.getDestinationAddress());
}
auto offset = cmdStream->getUsed();
@ -1096,13 +1111,13 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenProgrammingWalkerThenSignalSy
auto sdiCmd = genCmdCast<MI_STORE_DATA_IMM *>(++semaphoreCmd);
ASSERT_NE(nullptr, sdiCmd);
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress(), sdiCmd->getAddress());
EXPECT_EQ(immCmdList->inOrderDependencyCounterAllocation->getGpuAddress() + counterOffset, sdiCmd->getAddress());
EXPECT_EQ(1u, sdiCmd->getStoreQword());
EXPECT_EQ(2u, sdiCmd->getDataDword0());
EXPECT_EQ(0u, sdiCmd->getDataDword1());
}
auto hostAddress = static_cast<uint64_t *>(immCmdList->inOrderDependencyCounterAllocation->getUnderlyingBuffer());
auto hostAddress = static_cast<uint64_t *>(ptrOffset(immCmdList->inOrderDependencyCounterAllocation->getUnderlyingBuffer(), counterOffset));
*hostAddress = 1;
EXPECT_EQ(ZE_RESULT_NOT_READY, events[0]->hostSynchronize(1));
@ -1354,6 +1369,55 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenProgrammingAppendWaitOnEvents
EXPECT_EQ(2u, sdiCmd->getDataDword0());
}
HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenProgrammingCounterWithOverflowThenHandleItCorrectly, IsAtLeastXeHpCore) {
using MI_STORE_DATA_IMM = typename FamilyType::MI_STORE_DATA_IMM;
using MI_SEMAPHORE_WAIT = typename FamilyType::MI_SEMAPHORE_WAIT;
auto immCmdList = createImmCmdList<gfxCoreFamily>();
immCmdList->inOrderDependencyCounter = std::numeric_limits<uint32_t>::max() - 1;
auto cmdStream = immCmdList->getCmdContainer().getCommandStream();
auto eventPool = createEvents<FamilyType>(1, false);
auto eventHandle = events[0]->toHandle();
immCmdList->appendLaunchKernel(kernel->toHandle(), &groupCount, eventHandle, 0, nullptr, launchParams, false);
GenCmdList cmdList;
ASSERT_TRUE(FamilyType::PARSE::parseCommandBuffer(cmdList, cmdStream->getCpuBase(), cmdStream->getUsed()));
auto sdiItor = find<MI_STORE_DATA_IMM *>(cmdList.begin(), cmdList.end());
ASSERT_NE(cmdList.end(), sdiItor);
auto sdiCmd = genCmdCast<MI_STORE_DATA_IMM *>(*sdiItor);
uint64_t baseGpuVa = immCmdList->inOrderDependencyCounterAllocation->getGpuAddress();
EXPECT_EQ(baseGpuVa, sdiCmd->getAddress());
EXPECT_EQ(std::numeric_limits<uint32_t>::max(), sdiCmd->getDataDword0());
auto semaphoreCmd = genCmdCast<MI_SEMAPHORE_WAIT *>(++sdiCmd);
ASSERT_NE(nullptr, semaphoreCmd);
EXPECT_EQ(std::numeric_limits<uint32_t>::max(), semaphoreCmd->getSemaphoreDataDword());
EXPECT_EQ(baseGpuVa, semaphoreCmd->getSemaphoreGraphicsAddress());
sdiCmd = genCmdCast<MI_STORE_DATA_IMM *>(++semaphoreCmd);
ASSERT_NE(nullptr, sdiCmd);
uint32_t offset = static_cast<uint32_t>(sizeof(uint64_t));
EXPECT_EQ(baseGpuVa + offset, sdiCmd->getAddress());
EXPECT_EQ(1u, sdiCmd->getDataDword0());
EXPECT_EQ(1u, immCmdList->inOrderDependencyCounter);
EXPECT_EQ(offset, immCmdList->inOrderAllocationOffset);
EXPECT_EQ(1u, events[0]->inOrderExecSignalValue);
EXPECT_EQ(offset, events[0]->inOrderAllocationOffset);
}
HWTEST2_F(InOrderCmdListTests, givenCopyOnlyInOrderModeWhenProgrammingBarrierThenSignalInOrderAllocation, IsAtLeastXeHpCore) {
using MI_STORE_DATA_IMM = typename FamilyType::MI_STORE_DATA_IMM;
@ -1489,13 +1553,16 @@ HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenProgrammingAppendBarrierWitho
}
HWTEST2_F(InOrderCmdListTests, givenInOrderModeWhenCallingSyncThenHandleCompletion, IsAtLeastXeHpCore) {
uint32_t counterOffset = 64;
auto immCmdList = createImmCmdList<gfxCoreFamily>();
immCmdList->inOrderAllocationOffset = counterOffset;
auto ultCsr = static_cast<UltCommandStreamReceiver<FamilyType> *>(device->getNEODevice()->getDefaultEngine().commandStreamReceiver);
immCmdList->appendLaunchKernel(kernel->toHandle(), &groupCount, nullptr, 0, nullptr, launchParams, false);
auto hostAddress = static_cast<uint64_t *>(immCmdList->inOrderDependencyCounterAllocation->getUnderlyingBuffer());
auto hostAddress = static_cast<uint64_t *>(ptrOffset(immCmdList->inOrderDependencyCounterAllocation->getUnderlyingBuffer(), counterOffset));
*hostAddress = 0;
const uint32_t failCounter = 3;

View File

@ -3086,7 +3086,7 @@ HWTEST_F(EventTests, givenInOrderEventWhenHostEventSyncThenExpectDownloadEventAl
NEO::MockGraphicsAllocation syncAllocation(&storage, sizeof(storage));
event->enableInOrderExecMode(syncAllocation, 1);
event->enableInOrderExecMode(syncAllocation, 1, 0);
constexpr uint64_t timeout = std::numeric_limits<std::uint64_t>::max();
auto result = event->hostSynchronize(timeout);