Skip to content

Commit 94a20df

Browse files
authored
Merge ebecfd4 into 79ef89b
2 parents 79ef89b + ebecfd4 commit 94a20df

File tree

93 files changed

+2562
-82
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

93 files changed

+2562
-82
lines changed

ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,7 @@
11
#include "kqp_pure_compute_actor.h"
2+
#include <ydb/core/base/appdata.h>
23
#include <ydb/core/base/feature_flags.h>
34

4-
55
namespace NKikimr {
66
namespace NKqp {
77

@@ -15,7 +15,7 @@ TKqpComputeActor::TKqpComputeActor(const TActorId& executerId, ui64 txId, NDqPro
1515
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
1616
NWilson::TTraceId traceId, TIntrusivePtr<NActors::TProtoArenaHolder> arena,
1717
const std::optional<TKqpFederatedQuerySetup>& federatedQuerySetup, const TGUCSettings::TPtr& GUCSettings)
18-
: TBase(executerId, txId, task, std::move(asyncIoFactory), settings, memoryLimits, /* ownMemoryQuota = */ true, /* passExceptions = */ true, /*taskCounters = */ nullptr, std::move(traceId), std::move(arena), GUCSettings)
18+
: TBase(executerId, txId, task, std::move(asyncIoFactory), AppData()->FunctionRegistry, settings, memoryLimits, /* ownMemoryQuota = */ true, /* passExceptions = */ true, /*taskCounters = */ nullptr, std::move(traceId), std::move(arena), GUCSettings)
1919
, ComputeCtx(settings.StatsMode)
2020
, FederatedQuerySetup(federatedQuerySetup)
2121
{
@@ -42,7 +42,7 @@ void TKqpComputeActor::DoBootstrap() {
4242

4343
TDqTaskRunnerContext execCtx;
4444

45-
execCtx.FuncRegistry = AppData()->FunctionRegistry;
45+
execCtx.FuncRegistry = TBase::FunctionRegistry;
4646
execCtx.RandomProvider = TAppData::RandomProvider.Get();
4747
execCtx.TimeProvider = TAppData::TimeProvider.Get();
4848
execCtx.ComputeCtx = &ComputeCtx;

ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ TKqpScanComputeActor::TKqpScanComputeActor(const TActorId& executerId, ui64 txId
2727
IDqAsyncIoFactory::TPtr asyncIoFactory,
2828
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits, NWilson::TTraceId traceId,
2929
TIntrusivePtr<NActors::TProtoArenaHolder> arena)
30-
: TBase(executerId, txId, task, std::move(asyncIoFactory), settings,
30+
: TBase(executerId, txId, task, std::move(asyncIoFactory), AppData()->FunctionRegistry, settings,
3131
memoryLimits, /* ownMemoryQuota = */ true, /* passExceptions = */ true, /*taskCounters = */ nullptr, std::move(traceId), std::move(arena))
3232
, ComputeCtx(settings.StatsMode)
3333
{
@@ -180,7 +180,7 @@ void TKqpScanComputeActor::PollSources(ui64 prevFreeSpace) {
180180
void TKqpScanComputeActor::DoBootstrap() {
181181
CA_LOG_D("EVLOGKQP START");
182182
NDq::TDqTaskRunnerContext execCtx;
183-
execCtx.FuncRegistry = AppData()->FunctionRegistry;
183+
execCtx.FuncRegistry = TBase::FunctionRegistry;
184184
execCtx.ComputeCtx = &ComputeCtx;
185185
execCtx.ComputationFactory = NMiniKQL::GetKqpActorComputeFactory(&ComputeCtx, std::nullopt);
186186
execCtx.RandomProvider = TAppData::RandomProvider.Get();

ydb/library/yql/cfg/tests/gateways.conf

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -216,3 +216,10 @@ Fs {
216216
TargetUrl: "arc:/$1"
217217
}
218218
}
219+
220+
Solomon {
221+
DefaultSettings {
222+
Name: "_EnableReading"
223+
Value: "1"
224+
}
225+
}

ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -75,13 +75,13 @@ class TDqAsyncComputeActor : public TDqComputeActorBase<TDqAsyncComputeActor, TC
7575
static constexpr bool HasAsyncTaskRunner = true;
7676

7777
TDqAsyncComputeActor(const TActorId& executerId, const TTxId& txId, NDqProto::TDqTask* task,
78-
IDqAsyncIoFactory::TPtr asyncIoFactory,
78+
IDqAsyncIoFactory::TPtr asyncIoFactory, const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
7979
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
8080
const NTaskRunnerActor::ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory,
8181
const ::NMonitoring::TDynamicCounterPtr& taskCounters,
8282
const TActorId& quoterServiceActorId,
8383
bool ownCounters)
84-
: TBase(executerId, txId, task, std::move(asyncIoFactory), settings, memoryLimits, /* ownMemoryQuota = */ false, false, taskCounters)
84+
: TBase(executerId, txId, task, std::move(asyncIoFactory), functionRegistry, settings, memoryLimits, /* ownMemoryQuota = */ false, false, taskCounters)
8585
, TaskRunnerActorFactory(taskRunnerActorFactory)
8686
, ReadyToCheckpointFlag(false)
8787
, SentStatsRequest(false)
@@ -1164,15 +1164,15 @@ class TDqAsyncComputeActor : public TDqComputeActorBase<TDqAsyncComputeActor, TC
11641164

11651165

11661166
IActor* CreateDqAsyncComputeActor(const TActorId& executerId, const TTxId& txId, NYql::NDqProto::TDqTask* task,
1167-
IDqAsyncIoFactory::TPtr asyncIoFactory,
1167+
IDqAsyncIoFactory::TPtr asyncIoFactory, const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
11681168
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
11691169
const NTaskRunnerActor::ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory,
11701170
::NMonitoring::TDynamicCounterPtr taskCounters,
11711171
const TActorId& quoterServiceActorId,
11721172
bool ownCounters)
11731173
{
11741174
return new TDqAsyncComputeActor(executerId, txId, task, std::move(asyncIoFactory),
1175-
settings, memoryLimits, taskRunnerActorFactory, taskCounters, quoterServiceActorId, ownCounters);
1175+
functionRegistry, settings, memoryLimits, taskRunnerActorFactory, taskCounters, quoterServiceActorId, ownCounters);
11761176
}
11771177

11781178
} // namespace NDq

ydb/library/yql/dq/actors/compute/dq_async_compute_actor.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@ namespace NYql {
1818
namespace NDq {
1919

2020
NActors::IActor* CreateDqAsyncComputeActor(const NActors::TActorId& executerId, const TTxId& txId, NDqProto::TDqTask* task,
21-
IDqAsyncIoFactory::TPtr asyncIoFactory,
21+
IDqAsyncIoFactory::TPtr asyncIoFactory, const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
2222
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
2323
const NTaskRunnerActor::ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory,
2424
::NMonitoring::TDynamicCounterPtr taskCounters = nullptr,

ydb/library/yql/dq/actors/compute/dq_compute_actor.cpp

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,10 +35,11 @@ class TDqComputeActor : public TDqSyncComputeActorBase<TDqComputeActor> {
3535

3636
TDqComputeActor(const TActorId& executerId, const TTxId& txId, NDqProto::TDqTask* task,
3737
IDqAsyncIoFactory::TPtr asyncIoFactory,
38+
const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
3839
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
3940
const TTaskRunnerFactory& taskRunnerFactory,
4041
::NMonitoring::TDynamicCounterPtr taskCounters)
41-
: TBase(executerId, txId, task, std::move(asyncIoFactory), settings, memoryLimits, true, false, taskCounters)
42+
: TBase(executerId, txId, task, std::move(asyncIoFactory), functionRegistry, settings, memoryLimits, true, false, taskCounters)
4243
, TaskRunnerFactory(taskRunnerFactory)
4344
{
4445
InitializeTask();
@@ -74,12 +75,13 @@ class TDqComputeActor : public TDqSyncComputeActorBase<TDqComputeActor> {
7475

7576
IActor* CreateDqComputeActor(const TActorId& executerId, const TTxId& txId, NYql::NDqProto::TDqTask* task,
7677
IDqAsyncIoFactory::TPtr asyncIoFactory,
78+
const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
7779
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
7880
const TTaskRunnerFactory& taskRunnerFactory,
7981
::NMonitoring::TDynamicCounterPtr taskCounters)
8082
{
8183
return new TDqComputeActor(executerId, txId, task, std::move(asyncIoFactory),
82-
settings, memoryLimits, taskRunnerFactory, taskCounters);
84+
functionRegistry, settings, memoryLimits, taskRunnerFactory, taskCounters);
8385
}
8486

8587
} // namespace NDq

ydb/library/yql/dq/actors/compute/dq_compute_actor.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -382,7 +382,7 @@ void FillTaskRunnerStats(ui64 taskId, ui32 stageId, const TTaskRunnerStatsBase&
382382
NDqProto::TDqTaskStats* protoTask, TCollectStatsLevel level);
383383

384384
NActors::IActor* CreateDqComputeActor(const NActors::TActorId& executerId, const TTxId& txId, NDqProto::TDqTask* task,
385-
IDqAsyncIoFactory::TPtr asyncIoFactory,
385+
IDqAsyncIoFactory::TPtr asyncIoFactory, const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
386386
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
387387
const TTaskRunnerFactory& taskRunnerFactory,
388388
::NMonitoring::TDynamicCounterPtr taskCounters = nullptr);

ydb/library/yql/dq/actors/compute/dq_compute_actor_async_input_helper.h

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,6 +4,8 @@
44
#include "dq_compute_actor_metrics.h"
55
#include "dq_compute_actor_watermarks.h"
66

7+
#include <ydb/library/yql/minikql/mkql_program_builder.h>
8+
79
//must be included the last
810
#include "dq_compute_actor_log.h"
911

@@ -22,6 +24,7 @@ struct TComputeActorAsyncInputHelper {
2224
const NDqProto::EWatermarksMode WatermarksMode = NDqProto::EWatermarksMode::WATERMARKS_MODE_DISABLED;
2325
const NKikimr::NMiniKQL::TType* ValueType = nullptr;
2426
TMaybe<TInstant> PendingWatermark = Nothing();
27+
TMaybe<NKikimr::NMiniKQL::TProgramBuilder> ProgramBuilder;
2528
public:
2629
TComputeActorAsyncInputHelper(
2730
const TString& logPrefix,
@@ -122,4 +125,3 @@ struct TComputeActorAsyncInputHelperSync: public TComputeActorAsyncInputHelper
122125
};
123126

124127
} //namespace NYql::NDq
125-

ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -256,6 +256,7 @@ struct IDqAsyncIoFactory : public TThrRefBase {
256256
const NActors::TActorId& ComputeActorId;
257257
const NKikimr::NMiniKQL::TTypeEnvironment& TypeEnv;
258258
const NKikimr::NMiniKQL::THolderFactory& HolderFactory;
259+
NKikimr::NMiniKQL::TProgramBuilder& ProgramBuilder;
259260
::NMonitoring::TDynamicCounterPtr TaskCounters;
260261
std::shared_ptr<NKikimr::NMiniKQL::TScopedAlloc> Alloc;
261262
IMemoryQuotaManager::TPtr MemoryQuotaManager;

ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -161,6 +161,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped<TDerived>
161161
protected:
162162
TDqComputeActorBase(const NActors::TActorId& executerId, const TTxId& txId, NDqProto::TDqTask* task,
163163
IDqAsyncIoFactory::TPtr asyncIoFactory,
164+
const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
164165
const TComputeRuntimeSettings& settings, const TComputeMemoryLimits& memoryLimits,
165166
bool ownMemoryQuota = true, bool passExceptions = false,
166167
const ::NMonitoring::TDynamicCounterPtr& taskCounters = nullptr,
@@ -174,6 +175,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped<TDerived>
174175
, MemoryLimits(memoryLimits)
175176
, CanAllocateExtraMemory(RuntimeSettings.ExtraMemoryAllocationPool != 0)
176177
, AsyncIoFactory(std::move(asyncIoFactory))
178+
, FunctionRegistry(functionRegistry)
177179
, CheckpointingMode(GetTaskCheckpointingMode(Task))
178180
, State(Task.GetCreateSuspended() ? NDqProto::COMPUTE_STATE_UNKNOWN : NDqProto::COMPUTE_STATE_EXECUTING)
179181
, WatermarksTracker(this->SelfId(), TxId, Task.GetId())
@@ -1256,6 +1258,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped<TDerived>
12561258
const auto& inputDesc = Task.GetInputs(inputIndex);
12571259
Y_ABORT_UNLESS(inputDesc.HasSource());
12581260
source.Type = inputDesc.GetSource().GetType();
1261+
source.ProgramBuilder.ConstructInPlace(typeEnv, *FunctionRegistry);
12591262
const auto& settings = Task.GetSourceSettings();
12601263
Y_ABORT_UNLESS(settings.empty() || inputIndex < settings.size());
12611264
CA_LOG_D("Create source for input " << inputIndex << " " << inputDesc);
@@ -1273,6 +1276,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped<TDerived>
12731276
.ComputeActorId = this->SelfId(),
12741277
.TypeEnv = typeEnv,
12751278
.HolderFactory = holderFactory,
1279+
.ProgramBuilder = *source.ProgramBuilder,
12761280
.TaskCounters = TaskCounters,
12771281
.Alloc = Alloc,
12781282
.MemoryQuotaManager = MemoryLimits.MemoryQuotaManager,
@@ -1856,6 +1860,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped<TDerived>
18561860
TComputeMemoryLimits MemoryLimits;
18571861
const bool CanAllocateExtraMemory = false;
18581862
const IDqAsyncIoFactory::TPtr AsyncIoFactory;
1863+
const NKikimr::NMiniKQL::IFunctionRegistry* FunctionRegistry = nullptr;
18591864
const NDqProto::ECheckpointingMode CheckpointingMode;
18601865
TDqComputeActorChannels* Channels = nullptr;
18611866
TDqComputeActorCheckpoints* Checkpoints = nullptr;

ydb/library/yql/dq/actors/compute/dq_sync_compute_actor_base.h

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -369,4 +369,3 @@ class TDqSyncComputeActorBase: public TDqComputeActorBase<TDerived, TComputeActo
369369
};
370370

371371
} //namespace NYql::NDq
372-

ydb/library/yql/dq/actors/compute/ya.make

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ PEERDIR(
2626
ydb/library/yql/dq/runtime
2727
ydb/library/yql/dq/tasks
2828
ydb/library/yql/dq/actors/spilling
29+
ydb/library/yql/minikql
2930
ydb/library/yql/minikql/comp_nodes
3031
ydb/library/yql/public/issue
3132
ydb/core/quoter/public

ydb/library/yql/dq/actors/task_runner/events.h

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,12 +28,12 @@ struct TTaskRunnerEvents {
2828

2929
EvOutputChannelDataRequest,
3030
EvOutputChannelData,
31-
31+
3232
EvInputChannelData,
3333
EvInputChannelDataAck,
34-
34+
3535
// EvContinueRun -> TaskRunner->Run() -> TEvTaskRunFinished
36-
EvContinueRun,
36+
EvContinueRun,
3737
EvRunFinished,
3838

3939
EvSourceDataAck,

ydb/library/yql/providers/dq/actors/compute_actor.cpp

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -67,6 +67,7 @@ IActor* CreateComputeActor(
6767
operationId,
6868
task,
6969
options.AsyncIoFactory,
70+
options.FunctionRegistry,
7071
computeRuntimeSettings,
7172
memoryLimits,
7273
taskRunnerFactory,
@@ -77,6 +78,7 @@ IActor* CreateComputeActor(
7778
operationId,
7879
task,
7980
options.AsyncIoFactory,
81+
options.FunctionRegistry,
8082
computeRuntimeSettings,
8183
memoryLimits,
8284
taskRunnerActorFactory,

ydb/library/yql/providers/dq/actors/worker_actor.cpp

Lines changed: 10 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -11,6 +11,7 @@
1111
#include <ydb/library/yql/utils/log/log.h>
1212

1313
#include <ydb/library/yql/minikql/mkql_string_util.h>
14+
#include <ydb/library/yql/minikql/mkql_program_builder.h>
1415

1516
#include <ydb/library/actors/core/event_pb.h>
1617
#include <ydb/library/actors/core/hfunc.h>
@@ -56,6 +57,7 @@ struct TSourceInfo {
5657
bool PushStarted = false;
5758
bool Finished = false;
5859
NKikimr::NMiniKQL::TTypeEnvironment* TypeEnv = nullptr;
60+
std::optional<NKikimr::NMiniKQL::TProgramBuilder> ProgramBuilder;
5961
};
6062

6163
struct TSinkInfo {
@@ -93,10 +95,12 @@ class TDqWorker: public TRichActor<TDqWorker>
9395
explicit TDqWorker(
9496
const ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory,
9597
const IDqAsyncIoFactory::TPtr& asyncIoFactory,
98+
const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry,
9699
TWorkerRuntimeData* runtimeData,
97100
const TString& traceId)
98101
: TRichActor<TDqWorker>(&TDqWorker::Handler)
99102
, AsyncIoFactory(asyncIoFactory)
103+
, FunctionRegistry(functionRegistry)
100104
, TaskRunnerActorFactory(taskRunnerActorFactory)
101105
, RuntimeData(runtimeData)
102106
, TraceId(traceId)
@@ -294,6 +298,7 @@ class TDqWorker: public TRichActor<TDqWorker>
294298
if (input.HasSource()) {
295299
auto& source = SourcesMap[inputId];
296300
source.TypeEnv = const_cast<NKikimr::NMiniKQL::TTypeEnvironment*>(&typeEnv);
301+
source.ProgramBuilder.emplace(*source.TypeEnv, *FunctionRegistry);
297302
std::tie(source.Source, source.Actor) =
298303
AsyncIoFactory->CreateDqSource(
299304
IDqAsyncIoFactory::TSourceArguments {
@@ -307,6 +312,7 @@ class TDqWorker: public TRichActor<TDqWorker>
307312
.ComputeActorId = SelfId(),
308313
.TypeEnv = typeEnv,
309314
.HolderFactory = holderFactory,
315+
.ProgramBuilder = *source.ProgramBuilder,
310316
.MemoryQuotaManager = MemoryQuotaManager
311317
});
312318
RegisterLocalChild(source.Actor);
@@ -769,6 +775,7 @@ class TDqWorker: public TRichActor<TDqWorker>
769775
/*_________________________________________________________*/
770776

771777
IDqAsyncIoFactory::TPtr AsyncIoFactory;
778+
const NKikimr::NMiniKQL::IFunctionRegistry* FunctionRegistry;
772779
ITaskRunnerActorFactory::TPtr TaskRunnerActorFactory;
773780
NTaskRunnerActor::ITaskRunnerActor* Actor = nullptr;
774781
TActorId TaskRunnerActor;
@@ -808,13 +815,15 @@ NActors::IActor* CreateWorkerActor(
808815
TWorkerRuntimeData* runtimeData,
809816
const TString& traceId,
810817
const ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory,
811-
const IDqAsyncIoFactory::TPtr& asyncIoFactory)
818+
const IDqAsyncIoFactory::TPtr& asyncIoFactory,
819+
const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry)
812820
{
813821
Y_ABORT_UNLESS(taskRunnerActorFactory);
814822
return new TLogWrapReceive(
815823
new TDqWorker(
816824
taskRunnerActorFactory,
817825
asyncIoFactory,
826+
functionRegistry,
818827
runtimeData,
819828
traceId), traceId);
820829
}

ydb/library/yql/providers/dq/actors/worker_actor.h

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@ namespace NYql::NDqs {
2525
TWorkerRuntimeData* runtimeData,
2626
const TString& traceId,
2727
const NDq::NTaskRunnerActor::ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory,
28-
const NDq::IDqAsyncIoFactory::TPtr& asyncIoFactory);
28+
const NDq::IDqAsyncIoFactory::TPtr& asyncIoFactory,
29+
const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry);
2930

3031
} // namespace NYql::NDqs

ydb/library/yql/providers/dq/worker_manager/local_worker_manager.cpp

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -392,7 +392,8 @@ class TLocalWorkerManager: public TWorkerManagerCommon<TLocalWorkerManager> {
392392
Options.RuntimeData,
393393
traceId,
394394
Options.TaskRunnerActorFactory,
395-
Options.AsyncIoFactory));
395+
Options.AsyncIoFactory,
396+
Options.FunctionRegistry));
396397
}
397398
allocationInfo.WorkerActors.ActorIds.emplace_back(RegisterChild(
398399
actor.Release(), createComputeActor ? NYql::NDq::TEvDq::TEvAbortExecution::Unavailable("Aborted by LWM").Release() : nullptr

0 commit comments

Comments
 (0)