From ffe45cdfef11458007feff3bf3181b2fd203ee80 Mon Sep 17 00:00:00 2001 From: Filitov Mikhail Date: Mon, 16 Sep 2024 15:12:49 +0200 Subject: [PATCH] Stable 24 3 8 analytics (#9022) Co-authored-by: Aidar Samerkhanov Co-authored-by: Vitalii Gridnev Co-authored-by: Tony-Romanov <150126326+Tony-Romanov@users.noreply.github.com> Co-authored-by: Whompe Co-authored-by: Vladislav Lukachik Co-authored-by: Whompe --- .../kqp/compile_service/kqp_compile_actor.cpp | 1 + .../compile_service/kqp_compile_service.cpp | 3 + .../kqp_compute_actor_factory.cpp | 15 +- .../compute_actor/kqp_compute_actor_factory.h | 1 + .../compute_actor/kqp_compute_actor_impl.h | 4 +- .../compute_actor/kqp_pure_compute_actor.cpp | 5 +- .../compute_actor/kqp_scan_compute_actor.cpp | 3 +- ydb/core/kqp/executer_actor/kqp_planner.cpp | 11 +- .../kqp/node_service/kqp_node_service.cpp | 4 +- ydb/core/kqp/opt/logical/kqp_opt_log.cpp | 3 +- ydb/core/kqp/opt/physical/kqp_opt_phy.cpp | 3 +- ydb/core/kqp/provider/yql_kikimr_exec.cpp | 3 +- ydb/core/kqp/provider/yql_kikimr_settings.cpp | 31 +- ydb/core/kqp/provider/yql_kikimr_settings.h | 5 + .../kqp/proxy_service/kqp_proxy_service.cpp | 9 +- .../kqp/query_compiler/kqp_query_compiler.cpp | 3 +- ydb/core/kqp/rm_service/kqp_rm_service.cpp | 180 +++-- ydb/core/kqp/rm_service/kqp_rm_service.h | 42 +- ydb/core/kqp/rm_service/kqp_rm_ut.cpp | 2 +- ydb/core/kqp/ut/olap/aggregations_ut.cpp | 43 ++ ydb/core/kqp/ut/olap/clickbench_ut.cpp | 8 +- ydb/core/kqp/ut/olap/kqp_olap_ut.cpp | 1 + ydb/core/kqp/ut/query/kqp_limits_ut.cpp | 6 +- .../kqp/ut/spilling/kqp_scan_spilling_ut.cpp | 91 +++ ydb/core/protos/table_service_config.proto | 6 +- ydb/core/tx/datashard/datashard_kqp.cpp | 10 +- .../datashard/datashard_kqp_lookup_table.cpp | 6 +- .../tx/datashard/datashard_kqp_read_table.cpp | 6 +- ydb/core/tx/program/registry.cpp | 12 +- .../yql/core/common_opt/yql_co_flow1.cpp | 4 +- .../yql/core/yql_aggregate_expander.cpp | 12 +- ydb/library/yql/core/yql_aggregate_expander.h | 10 +- ydb/library/yql/core/yql_opt_utils.cpp | 8 +- .../actors/compute/dq_async_compute_actor.cpp | 5 +- .../dq/actors/compute/dq_compute_actor.cpp | 5 +- .../dq/actors/compute/dq_compute_actor_impl.h | 4 + .../actors/compute/dq_compute_actor_stats.cpp | 8 + .../compute/dq_sync_compute_actor_base.h | 8 +- .../compute/dq_task_runner_exec_ctx.cpp | 22 +- .../actors/compute/dq_task_runner_exec_ctx.h | 12 +- .../yql/dq/actors/protos/dq_stats.proto | 8 + .../dq/actors/spilling/channel_storage.cpp | 14 +- .../yql/dq/actors/spilling/channel_storage.h | 7 +- .../actors/spilling/channel_storage_actor.cpp | 108 ++- .../actors/spilling/channel_storage_actor.h | 6 +- .../dq/actors/spilling/compute_storage.cpp | 5 +- .../yql/dq/actors/spilling/compute_storage.h | 4 +- .../actors/spilling/compute_storage_actor.cpp | 107 +-- .../actors/spilling/compute_storage_actor.h | 5 +- .../yql/dq/actors/spilling/spiller_factory.h | 15 +- .../dq/actors/spilling/spilling_counters.h | 10 + .../yql/dq/actors/spilling/spilling_file.cpp | 84 ++- .../yql/dq/actors/spilling/spilling_file.h | 5 + .../dq/actors/spilling/spilling_file_ut.cpp | 19 +- .../task_runner/task_runner_actor_local.cpp | 12 +- ydb/library/yql/dq/common/dq_common.h | 31 + ydb/library/yql/dq/opt/dq_opt_log.cpp | 5 +- ydb/library/yql/dq/opt/dq_opt_log.h | 3 +- .../yql/dq/runtime/dq_channel_storage.h | 2 - .../yql/dq/runtime/dq_tasks_runner.cpp | 23 +- ydb/library/yql/dq/runtime/dq_tasks_runner.h | 22 +- ydb/library/yql/dq/tasks/dq_task_program.cpp | 38 +- ydb/library/yql/dq/tasks/dq_task_program.h | 4 +- ydb/library/yql/minikql/aligned_page_pool.cpp | 1 + ydb/library/yql/minikql/aligned_page_pool.h | 9 + .../yql/minikql/comp_nodes/mkql_factory.cpp | 4 +- .../minikql/comp_nodes/mkql_grace_join.cpp | 19 +- .../minikql/comp_nodes/mkql_scalar_apply.cpp | 8 +- .../yql/minikql/comp_nodes/mkql_udf.cpp | 211 ++++-- .../minikql/comp_nodes/mkql_wide_combine.cpp | 636 +++++++++--------- .../comp_nodes/ut/mkql_computation_node_ut.h | 5 +- .../comp_nodes/ut/mkql_wide_combine_ut.cpp | 149 ++-- .../computation/mkql_computation_node.cpp | 3 + .../computation/mkql_computation_node.h | 32 +- .../mkql_computation_node_codegen.cpp | 4 - .../mkql_computation_node_codegen.h.txt | 7 - .../computation/mkql_spiller_factory.h | 4 + .../computation/mock_spiller_factory_ut.h | 21 + .../yql/minikql/mkql_program_builder.cpp | 22 - .../yql/minikql/mkql_program_builder.h | 4 + .../common/mkql/yql_provider_mkql.cpp | 7 - .../yql/providers/dq/actors/worker_actor.cpp | 5 +- .../providers/dq/common/yql_dq_settings.cpp | 26 +- .../yql/providers/dq/common/yql_dq_settings.h | 18 +- .../dq/local_gateway/yql_dq_gateway_local.cpp | 5 +- .../yql/providers/dq/opt/logical_optimize.cpp | 3 +- .../dq/planner/execution_planner.cpp | 7 +- .../provider/exec/yql_dq_exectransformer.cpp | 3 +- .../providers/dq/provider/yql_dq_validate.cpp | 2 +- .../yt/lib/lambda_builder/lambda_builder.cpp | 2 +- .../sql/dq_file/part0/canondata/result.json | 12 +- .../sql/dq_file/part1/canondata/result.json | 36 +- .../sql/dq_file/part10/canondata/result.json | 42 +- .../sql/dq_file/part11/canondata/result.json | 30 +- .../sql/dq_file/part12/canondata/result.json | 24 +- .../sql/dq_file/part13/canondata/result.json | 42 +- .../sql/dq_file/part14/canondata/result.json | 36 +- .../sql/dq_file/part15/canondata/result.json | 42 +- .../sql/dq_file/part16/canondata/result.json | 6 +- .../sql/dq_file/part17/canondata/result.json | 24 +- .../sql/dq_file/part18/canondata/result.json | 34 +- .../sql/dq_file/part19/canondata/result.json | 36 +- .../sql/dq_file/part2/canondata/result.json | 18 +- .../sql/dq_file/part3/canondata/result.json | 24 +- .../sql/dq_file/part4/canondata/result.json | 24 +- .../sql/dq_file/part5/canondata/result.json | 12 +- .../sql/dq_file/part6/canondata/result.json | 30 +- .../sql/dq_file/part7/canondata/result.json | 30 +- .../sql/dq_file/part8/canondata/result.json | 18 +- .../sql/dq_file/part9/canondata/result.json | 42 +- .../tests/sql/sql2yql/canondata/result.json | 14 + .../disable_blocks_with_spilling.cfg | 5 + .../disable_blocks_with_spilling.sql | 2 + ...e_all_with_python_input_stream-dq_fail.cfg | 4 + .../reduce_all_with_python_input_stream._sql | 14 + .../reduce_all_with_python_input_stream.cfg | 3 + .../reduce_with_python_input_stream._sql | 14 + .../reduce_with_python_input_stream.cfg | 2 + .../queries-original-plan-row-22 | 117 +--- .../queries-original-plan-row-9 | 105 +-- 120 files changed, 2001 insertions(+), 1210 deletions(-) create mode 100644 ydb/library/yql/minikql/computation/mock_spiller_factory_ut.h create mode 100644 ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.cfg create mode 100644 ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.sql create mode 100644 ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream-dq_fail.cfg create mode 100644 ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream._sql create mode 100644 ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream.cfg create mode 100644 ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream._sql create mode 100644 ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream.cfg diff --git a/ydb/core/kqp/compile_service/kqp_compile_actor.cpp b/ydb/core/kqp/compile_service/kqp_compile_actor.cpp index d8d04cbb80fc..7f1a4f2e4bb7 100644 --- a/ydb/core/kqp/compile_service/kqp_compile_actor.cpp +++ b/ydb/core/kqp/compile_service/kqp_compile_actor.cpp @@ -609,6 +609,7 @@ void ApplyServiceConfig(TKikimrConfiguration& kqpConfig, const TTableServiceConf kqpConfig.EnableSpillingGenericQuery = serviceConfig.GetEnableQueryServiceSpilling(); kqpConfig.DefaultCostBasedOptimizationLevel = serviceConfig.GetDefaultCostBasedOptimizationLevel(); kqpConfig.EnableConstantFolding = serviceConfig.GetEnableConstantFolding(); + kqpConfig.SetDefaultEnabledSpillingNodes(serviceConfig.GetEnableSpillingNodes()); if (const auto limit = serviceConfig.GetResourceManager().GetMkqlHeavyProgramMemoryLimit()) { kqpConfig._KqpYqlCombinerMemoryLimit = std::max(1_GB, limit - (limit >> 2U)); diff --git a/ydb/core/kqp/compile_service/kqp_compile_service.cpp b/ydb/core/kqp/compile_service/kqp_compile_service.cpp index 83ed02e73864..add71b771dc7 100644 --- a/ydb/core/kqp/compile_service/kqp_compile_service.cpp +++ b/ydb/core/kqp/compile_service/kqp_compile_service.cpp @@ -536,6 +536,8 @@ class TKqpCompileService : public TActorBootstrapped { ui64 defaultCostBasedOptimizationLevel = TableServiceConfig.GetDefaultCostBasedOptimizationLevel(); bool enableConstantFolding = TableServiceConfig.GetEnableConstantFolding(); + TString enableSpillingNodes = TableServiceConfig.GetEnableSpillingNodes(); + TableServiceConfig.Swap(event.MutableConfig()->MutableTableServiceConfig()); LOG_INFO(*TlsActivationContext, NKikimrServices::KQP_COMPILE_SERVICE, "Updated config"); @@ -562,6 +564,7 @@ class TKqpCompileService : public TActorBootstrapped { TableServiceConfig.GetExtractPredicateRangesLimit() != rangesLimit || TableServiceConfig.GetResourceManager().GetMkqlHeavyProgramMemoryLimit() != mkqlHeavyLimit || TableServiceConfig.GetIdxLookupJoinPointsLimit() != idxLookupPointsLimit || + TableServiceConfig.GetEnableSpillingNodes() != enableSpillingNodes || TableServiceConfig.GetEnableQueryServiceSpilling() != enableQueryServiceSpilling || TableServiceConfig.GetEnableImplicitQueryParameterTypes() != enableImplicitQueryParameterTypes || TableServiceConfig.GetDefaultCostBasedOptimizationLevel() != defaultCostBasedOptimizationLevel || diff --git a/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.cpp b/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.cpp index 83297b69f206..ca44ab103223 100644 --- a/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.cpp +++ b/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.cpp @@ -14,15 +14,13 @@ struct TMemoryQuotaManager : public NYql::NDq::TGuaranteeQuotaManager { , std::shared_ptr state , TIntrusivePtr tx , TIntrusivePtr task - , ui64 limit - , ui64 reasonableSpillingTreshold) + , ui64 limit) : NYql::NDq::TGuaranteeQuotaManager(limit, limit) , ResourceManager(std::move(resourceManager)) , MemoryPool(memoryPool) , State(std::move(state)) , Tx(std::move(tx)) , Task(std::move(task)) - , ReasonableSpillingTreshold(reasonableSpillingTreshold) { } @@ -57,7 +55,7 @@ struct TMemoryQuotaManager : public NYql::NDq::TGuaranteeQuotaManager { } bool IsReasonableToUseSpilling() const override { - return Tx->GetExtraMemoryAllocatedSize() >= ReasonableSpillingTreshold; + return Task->IsReasonableToStartSpilling(); } TString MemoryConsumptionDetails() const override { @@ -88,7 +86,6 @@ class TKqpCaFactory : public IKqpNodeComputeActorFactory { std::atomic MkqlLightProgramMemoryLimit = 0; std::atomic MkqlHeavyProgramMemoryLimit = 0; std::atomic MinChannelBufferSize = 0; - std::atomic ReasonableSpillingTreshold = 0; std::atomic MinMemAllocSize = 8_MB; std::atomic MinMemFreeSize = 32_MB; @@ -109,7 +106,6 @@ class TKqpCaFactory : public IKqpNodeComputeActorFactory { MkqlLightProgramMemoryLimit.store(config.GetMkqlLightProgramMemoryLimit()); MkqlHeavyProgramMemoryLimit.store(config.GetMkqlHeavyProgramMemoryLimit()); MinChannelBufferSize.store(config.GetMinChannelBufferSize()); - ReasonableSpillingTreshold.store(config.GetReasonableSpillingTreshold()); MinMemAllocSize.store(config.GetMinMemAllocSize()); MinMemFreeSize.store(config.GetMinMemFreeSize()); } @@ -164,14 +160,17 @@ class TKqpCaFactory : public IKqpNodeComputeActorFactory { std::move(args.State), std::move(args.TxInfo), std::move(task), - limit, - ReasonableSpillingTreshold.load()); + limit); auto runtimeSettings = args.RuntimeSettings; runtimeSettings.ExtraMemoryAllocationPool = args.MemoryPool; runtimeSettings.UseSpilling = args.WithSpilling; runtimeSettings.StatsMode = args.StatsMode; + if (runtimeSettings.UseSpilling) { + args.Task->SetEnableSpilling(runtimeSettings.UseSpilling); + } + if (args.Deadline) { runtimeSettings.Timeout = args.Deadline - TAppData::TimeProvider->Now(); } diff --git a/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.h b/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.h index c2a3325853ce..3922b0fdaa76 100644 --- a/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.h +++ b/ydb/core/kqp/compute_actor/kqp_compute_actor_factory.h @@ -124,6 +124,7 @@ struct IKqpNodeComputeActorFactory { const TInstant& Deadline; const bool ShareMailbox; const TMaybe& RlPath; + TComputeStagesWithScan* ComputesByStages = nullptr; std::shared_ptr State = nullptr; TComputeActorSchedulingOptions SchedulingOptions = {}; diff --git a/ydb/core/kqp/compute_actor/kqp_compute_actor_impl.h b/ydb/core/kqp/compute_actor/kqp_compute_actor_impl.h index 2633037dcb17..bdfbb2eeb7db 100644 --- a/ydb/core/kqp/compute_actor/kqp_compute_actor_impl.h +++ b/ydb/core/kqp/compute_actor/kqp_compute_actor_impl.h @@ -14,8 +14,8 @@ using namespace NYql::NDq; class TKqpTaskRunnerExecutionContext : public TDqTaskRunnerExecutionContext { public: - TKqpTaskRunnerExecutionContext(ui64 txId, bool withSpilling, IDqChannelStorage::TWakeUpCallback&& wakeUp) - : TDqTaskRunnerExecutionContext(txId, std::move(wakeUp)) + TKqpTaskRunnerExecutionContext(ui64 txId, bool withSpilling, TWakeUpCallback&& wakeUpCallback, TErrorCallback&& errorCallback) + : TDqTaskRunnerExecutionContext(txId, std::move(wakeUpCallback), std::move(errorCallback)) , WithSpilling_(withSpilling) { } diff --git a/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp b/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp index 2dbe94b66b2e..ab43bc1e5ff1 100644 --- a/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp +++ b/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp @@ -72,9 +72,10 @@ void TKqpComputeActor::DoBootstrap() { auto taskRunner = MakeDqTaskRunner(TBase::GetAllocatorPtr(), execCtx, settings, logger); SetTaskRunner(taskRunner); - auto wakeup = [this]{ ContinueExecute(); }; + auto wakeupCallback = [this]{ ContinueExecute(); }; + auto errorCallback = [this](const TString& error){ SendError(error); }; try { - PrepareTaskRunner(TKqpTaskRunnerExecutionContext(std::get(TxId), RuntimeSettings.UseSpilling, std::move(wakeup))); + PrepareTaskRunner(TKqpTaskRunnerExecutionContext(std::get(TxId), RuntimeSettings.UseSpilling, std::move(wakeupCallback), std::move(errorCallback))); } catch (const NMiniKQL::TKqpEnsureFail& e) { InternalError((TIssuesIds::EIssueCode) e.GetCode(), e.GetMessage()); return; diff --git a/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp b/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp index 8947e2740030..72d0e2af10a7 100644 --- a/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp +++ b/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp @@ -243,7 +243,8 @@ void TKqpScanComputeActor::DoBootstrap() { TBase::SetTaskRunner(taskRunner); auto wakeup = [this] { ContinueExecute(); }; - TBase::PrepareTaskRunner(TKqpTaskRunnerExecutionContext(std::get(TxId), RuntimeSettings.UseSpilling, std::move(wakeup))); + auto errorCallback = [this](const TString& error){ SendError(error); }; + TBase::PrepareTaskRunner(TKqpTaskRunnerExecutionContext(std::get(TxId), RuntimeSettings.UseSpilling, std::move(wakeup), std::move(errorCallback))); ComputeCtx.AddTableScan(0, Meta, GetStatsMode()); ScanData = &ComputeCtx.GetTableScan(0); diff --git a/ydb/core/kqp/executer_actor/kqp_planner.cpp b/ydb/core/kqp/executer_actor/kqp_planner.cpp index ad0b0d68e066..3c50dbd3e762 100644 --- a/ydb/core/kqp/executer_actor/kqp_planner.cpp +++ b/ydb/core/kqp/executer_actor/kqp_planner.cpp @@ -204,6 +204,7 @@ std::unique_ptr TKqpPlanner::SerializeReque request.SetStartAllOrFail(true); if (UseDataQueryPool) { request.MutableRuntimeSettings()->SetExecType(NYql::NDqProto::TComputeRuntimeSettings::DATA); + request.MutableRuntimeSettings()->SetUseSpilling(WithSpilling); } else { request.MutableRuntimeSettings()->SetExecType(NYql::NDqProto::TComputeRuntimeSettings::SCAN); request.MutableRuntimeSettings()->SetUseSpilling(WithSpilling); @@ -432,8 +433,14 @@ TString TKqpPlanner::ExecuteDataComputeTask(ui64 taskId, ui32 computeTasksSize) NYql::NDqProto::TDqTask* taskDesc = ArenaSerializeTaskToProto(TasksGraph, task, true); NYql::NDq::TComputeRuntimeSettings settings; if (!TxInfo) { + double memoryPoolPercent = 100; + if (UserRequestContext->PoolConfig.has_value()) { + memoryPoolPercent = UserRequestContext->PoolConfig->QueryMemoryLimitPercentPerNode; + } + TxInfo = MakeIntrusive( - TxId, TInstant::Now(), ResourceManager_->GetCounters()); + TxId, TInstant::Now(), ResourceManager_->GetCounters(), + UserRequestContext->PoolId, memoryPoolPercent, Database); } auto startResult = CaFactory_->CreateKqpComputeActor({ @@ -454,7 +461,7 @@ TString TKqpPlanner::ExecuteDataComputeTask(ui64 taskId, ui32 computeTasksSize) .StatsMode = GetDqStatsMode(StatsMode), .Deadline = Deadline, .ShareMailbox = (computeTasksSize <= 1), - .RlPath = Nothing() + .RlPath = Nothing(), }); if (const auto* rmResult = std::get_if(&startResult)) { diff --git a/ydb/core/kqp/node_service/kqp_node_service.cpp b/ydb/core/kqp/node_service/kqp_node_service.cpp index b4f8931674e7..162806f1de15 100644 --- a/ydb/core/kqp/node_service/kqp_node_service.cpp +++ b/ydb/core/kqp/node_service/kqp_node_service.cpp @@ -200,7 +200,9 @@ class TKqpNodeService : public TActorBootstrapped { } TIntrusivePtr txInfo = MakeIntrusive( - txId, TInstant::Now(), ResourceManager_->GetCounters()); + txId, TInstant::Now(), ResourceManager_->GetCounters(), + msg.GetSchedulerGroup(), msg.GetMemoryPoolPercent(), + msg.GetDatabase()); const ui32 tasksCount = msg.GetTasks().size(); for (auto& dqTask: *msg.MutableTasks()) { diff --git a/ydb/core/kqp/opt/logical/kqp_opt_log.cpp b/ydb/core/kqp/opt/logical/kqp_opt_log.cpp index e6715b1b2f35..99ad9ce69e98 100644 --- a/ydb/core/kqp/opt/logical/kqp_opt_log.cpp +++ b/ydb/core/kqp/opt/logical/kqp_opt_log.cpp @@ -141,7 +141,8 @@ class TKqpLogicalOptTransformer : public TOptimizeTransformerBase { true, // defaultWatermarksMode true); // syncActor } else { - output = DqRewriteAggregate(node, ctx, TypesCtx, false, KqpCtx.Config->HasOptEnableOlapPushdown() || KqpCtx.Config->HasOptUseFinalizeByKey(), KqpCtx.Config->HasOptUseFinalizeByKey()); + NDq::TSpillingSettings spillingSettings(KqpCtx.Config->GetEnabledSpillingNodes()); + output = DqRewriteAggregate(node, ctx, TypesCtx, false, KqpCtx.Config->HasOptEnableOlapPushdown() || KqpCtx.Config->HasOptUseFinalizeByKey(), KqpCtx.Config->HasOptUseFinalizeByKey(), spillingSettings.IsAggregationSpillingEnabled()); } if (output) { DumpAppliedRule("RewriteAggregate", node.Ptr(), output.Cast().Ptr(), ctx); diff --git a/ydb/core/kqp/opt/physical/kqp_opt_phy.cpp b/ydb/core/kqp/opt/physical/kqp_opt_phy.cpp index a4cfee447263..3f7ab21bc3c9 100644 --- a/ydb/core/kqp/opt/physical/kqp_opt_phy.cpp +++ b/ydb/core/kqp/opt/physical/kqp_opt_phy.cpp @@ -252,7 +252,8 @@ class TKqpPhysicalOptTransformer : public TOptimizeTransformerBase { } TMaybeNode ExpandAggregatePhase(TExprBase node, TExprContext& ctx) { - auto output = ExpandAggregatePeepholeImpl(node.Ptr(), ctx, TypesCtx, KqpCtx.Config->HasOptUseFinalizeByKey(), false); + NDq::TSpillingSettings spillingSettings(KqpCtx.Config->GetEnabledSpillingNodes()); + auto output = ExpandAggregatePeepholeImpl(node.Ptr(), ctx, TypesCtx, KqpCtx.Config->HasOptUseFinalizeByKey(), false, spillingSettings.IsAggregationSpillingEnabled()); DumpAppliedRule("ExpandAggregatePhase", node.Ptr(), output, ctx); return TExprBase(output); } diff --git a/ydb/core/kqp/provider/yql_kikimr_exec.cpp b/ydb/core/kqp/provider/yql_kikimr_exec.cpp index 7dc87ea7a79a..2cbccd660446 100644 --- a/ydb/core/kqp/provider/yql_kikimr_exec.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_exec.cpp @@ -815,10 +815,11 @@ class TKiSourceCallableExecutionTransformer : public TAsyncCallbackTransformer fakeReads; auto paramsType = NDq::CollectParameters(programLambda, ctx); + NDq::TSpillingSettings spillingSettings{SessionCtx->Config().GetEnabledSpillingNodes()}; lambda = NDq::BuildProgram( programLambda, *paramsType, compiler, SessionCtx->Query().QueryData->GetAllocState()->TypeEnv, *SessionCtx->Query().QueryData->GetAllocState()->HolderFactory.GetFunctionRegistry(), - ctx, fakeReads); + ctx, fakeReads, spillingSettings); NKikimr::NMiniKQL::TProgramBuilder programBuilder(SessionCtx->Query().QueryData->GetAllocState()->TypeEnv, *SessionCtx->Query().QueryData->GetAllocState()->HolderFactory.GetFunctionRegistry()); diff --git a/ydb/core/kqp/provider/yql_kikimr_settings.cpp b/ydb/core/kqp/provider/yql_kikimr_settings.cpp index 509c41b985f0..a57414bb300c 100644 --- a/ydb/core/kqp/provider/yql_kikimr_settings.cpp +++ b/ydb/core/kqp/provider/yql_kikimr_settings.cpp @@ -3,6 +3,8 @@ #include #include #include +#include +#include namespace NYql { @@ -23,6 +25,22 @@ EOptionalFlag GetOptionalFlagValue(const TMaybe& flag) { return EOptionalFlag::Disabled; } + +ui64 ParseEnableSpillingNodes(const TString &v) { + ui64 res = 0; + TVector vec; + StringSplitter(v).SplitBySet(",;| ").AddTo(&vec); + for (auto& s: vec) { + if (s.empty()) { + throw yexception() << "Empty value item"; + } + + auto value = FromString(s); + res |= ui64(value); + } + return res; +} + static inline bool GetFlagValue(const TMaybe& flag) { return flag ? flag.GetRef() : false; } @@ -73,6 +91,8 @@ TKikimrConfiguration::TKikimrConfiguration() { REGISTER_SETTING(*this, OptUseFinalizeByKey); REGISTER_SETTING(*this, CostBasedOptimizationLevel); + REGISTER_SETTING(*this, EnableSpillingNodes) + .Parser([](const TString& v) { return ParseEnableSpillingNodes(v); }); REGISTER_SETTING(*this, MaxDPccpDPTableSize); @@ -126,10 +146,9 @@ bool TKikimrSettings::HasOptEnableOlapProvideComputeSharding() const { } bool TKikimrSettings::HasOptUseFinalizeByKey() const { - return GetOptionalFlagValue(OptUseFinalizeByKey.Get()) != EOptionalFlag::Disabled; + return GetFlagValue(OptUseFinalizeByKey.Get().GetOrElse(true)) != EOptionalFlag::Disabled; } - EOptionalFlag TKikimrSettings::GetOptPredicateExtract() const { return GetOptionalFlagValue(OptEnablePredicateExtract.Get()); } @@ -151,4 +170,12 @@ TKikimrSettings::TConstPtr TKikimrConfiguration::Snapshot() const { return std::make_shared(*this); } +void TKikimrConfiguration::SetDefaultEnabledSpillingNodes(const TString& node) { + DefaultEnableSpillingNodes = ParseEnableSpillingNodes(node); +} + +ui64 TKikimrConfiguration::GetEnabledSpillingNodes() const { + return EnableSpillingNodes.Get().GetOrElse(DefaultEnableSpillingNodes); +} + } diff --git a/ydb/core/kqp/provider/yql_kikimr_settings.h b/ydb/core/kqp/provider/yql_kikimr_settings.h index 9183b6233ceb..c3963f079fcc 100644 --- a/ydb/core/kqp/provider/yql_kikimr_settings.h +++ b/ydb/core/kqp/provider/yql_kikimr_settings.h @@ -58,6 +58,7 @@ struct TKikimrSettings { NCommon::TConfSetting OptCardinalityHints; NCommon::TConfSetting OptJoinAlgoHints; NCommon::TConfSetting OptJoinOrderHints; + NCommon::TConfSetting OverrideStatistics; /* Disable optimizer rules */ NCommon::TConfSetting OptDisableTopSort; @@ -175,6 +176,10 @@ struct TKikimrConfiguration : public TKikimrSettings, public NCommon::TSettingDi bool EnableSpillingGenericQuery = false; ui32 DefaultCostBasedOptimizationLevel = 4; bool EnableConstantFolding = true; + ui64 DefaultEnableSpillingNodes = 0; + + void SetDefaultEnabledSpillingNodes(const TString& node); + ui64 GetEnabledSpillingNodes() const; }; } diff --git a/ydb/core/kqp/proxy_service/kqp_proxy_service.cpp b/ydb/core/kqp/proxy_service/kqp_proxy_service.cpp index 3c40f3f0b855..14db8e511e4b 100644 --- a/ydb/core/kqp/proxy_service/kqp_proxy_service.cpp +++ b/ydb/core/kqp/proxy_service/kqp_proxy_service.cpp @@ -46,6 +46,7 @@ #include #include +#include namespace NKikimr::NKqp { @@ -236,9 +237,15 @@ class TKqpProxyService : public TActorBootstrapped { ResourcePoolsCache.UpdateFeatureFlags(FeatureFlags, ActorContext()); if (auto& cfg = TableServiceConfig.GetSpillingServiceConfig().GetLocalFileConfig(); cfg.GetEnable()) { + TString spillingRoot = cfg.GetRoot(); + if (spillingRoot.empty()) { + spillingRoot = NYql::NDq::GetTmpSpillingRootForCurrentUser(); + MakeDirIfNotExist(spillingRoot); + } + SpillingService = TlsActivationContext->ExecutorThread.RegisterActor(NYql::NDq::CreateDqLocalFileSpillingService( NYql::NDq::TFileSpillingServiceConfig{ - .Root = cfg.GetRoot(), + .Root = spillingRoot, .MaxTotalSize = cfg.GetMaxTotalSize(), .MaxFileSize = cfg.GetMaxFileSize(), .MaxFilePartSize = cfg.GetMaxFilePartSize(), diff --git a/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp b/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp index 78be12f810e6..15f65fa95762 100644 --- a/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp +++ b/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp @@ -777,8 +777,9 @@ class TKqpQueryCompiler : public IKqpQueryCompiler { stageProto.SetIsEffectsStage(hasEffects || hasTxTableSink); auto paramsType = CollectParameters(stage, ctx); + NDq::TSpillingSettings spillingSettings{Config->GetEnabledSpillingNodes()}; auto programBytecode = NDq::BuildProgram(stage.Program(), *paramsType, *KqlCompiler, TypeEnv, FuncRegistry, - ctx, {}); + ctx, {}, spillingSettings); auto& programProto = *stageProto.MutableProgram(); programProto.SetRuntimeVersion(NYql::NDqProto::ERuntimeVersion::RUNTIME_VERSION_YQL_1_0); diff --git a/ydb/core/kqp/rm_service/kqp_rm_service.cpp b/ydb/core/kqp/rm_service/kqp_rm_service.cpp index ea16a237c12d..6632865dc0c5 100644 --- a/ydb/core/kqp/rm_service/kqp_rm_service.cpp +++ b/ydb/core/kqp/rm_service/kqp_rm_service.cpp @@ -20,6 +20,8 @@ #include +#include + namespace NKikimr { namespace NKqp { namespace NRm { @@ -43,42 +45,83 @@ using namespace NResourceBroker; namespace { -template -class TLimitedResource { +static constexpr double MYEPS = 1e-9; + +ui64 OverPercentage(ui64 limit, double percent) { + return static_cast(limit) / 100 * (100 - percent) + MYEPS; +} + +ui64 Percentage(ui64 limit, double percent) { + return static_cast(limit) / 100 * percent + MYEPS; +} + +class TMemoryResource : public TAtomicRefCount { public: - explicit TLimitedResource(T limit) - : Limit(limit) - , Used(0) {} + explicit TMemoryResource(ui64 baseLimit, double memoryPoolPercent, double overPercent) + : BaseLimit(baseLimit) + , Used(0) + , MemoryPoolPercent(memoryPoolPercent) + , OverPercent(overPercent) + , SpillingCookie(MakeIntrusive()) + { + SetActualLimits(); + } - T Available() const { + ui64 Available() const { return Limit > Used ? Limit - Used : 0; } - bool Has(T amount) const { + bool Has(ui64 amount) const { return Available() >= amount; } - bool Acquire(T value) { + bool AcquireIfAvailable(ui64 value) { if (Available() >= value) { Used += value; + UpdateCookie(); return true; } return false; } - void Release(T value) { + TIntrusivePtr GetSpillingCookie() const { + return SpillingCookie; + } + + void UpdateCookie() { + SpillingCookie->SpillingPercentReached.store(Available() < OverLimit); + } + + ui64 GetUsed() const { + return Used; + } + + void Release(ui64 value) { if (Used > value) { Used -= value; } else { Used = 0; } + + UpdateCookie(); + } + + void SetNewLimit(ui64 baseLimit, double memoryPoolPercent, double overPercent) { + if (abs(memoryPoolPercent - MemoryPoolPercent) < MYEPS && baseLimit == BaseLimit) + return; + + BaseLimit = baseLimit; + MemoryPoolPercent = memoryPoolPercent; + OverPercent = overPercent; + SetActualLimits(); } - void SetNewLimit(T limit) { - Limit = limit; + void SetActualLimits() { + Limit = Percentage(BaseLimit, MemoryPoolPercent); + OverLimit = OverPercentage(Limit, OverPercent); } - T GetLimit() const { + ui64 GetLimit() const { return Limit; } @@ -87,8 +130,14 @@ class TLimitedResource { } private: - T Limit; - T Used; + ui64 BaseLimit; + ui64 OverLimit; + ui64 Limit; + ui64 Used; + double MemoryPoolPercent; + double OverPercent; + + TIntrusivePtr SpillingCookie; }; struct TEvPrivate { @@ -116,7 +165,8 @@ class TKqpResourceManager : public IKqpResourceManager { : Counters(counters) , ExecutionUnitsResource(config.GetComputeActorsCount()) , ExecutionUnitsLimit(config.GetComputeActorsCount()) - , ScanQueryMemoryResource(config.GetQueryMemoryLimit()) + , SpillingPercent(config.GetSpillingPercent()) + , TotalMemoryResource(MakeIntrusive(config.GetQueryMemoryLimit(), (double)100, config.GetSpillingPercent())) , PublishResourcesByExchanger(config.GetEnablePublishResourcesByExchanger()) { SetConfigValues(config); @@ -209,11 +259,7 @@ class TKqpResourceManager : public IKqpResourceManager { } bool hasScanQueryMemory = true; - ui64 queryMemoryLimit = 0; - // NOTE(gvit): the first memory request always satisfied. - // all other requests are not guaranteed to be satisfied. - // In the nearest future we need to implement several layers of memory requests. bool isFirstAllocationRequest = (resources.ExecutionUnits > 0 && resources.MemoryPool == EKqpMemoryPool::DataQuery); if (isFirstAllocationRequest) { TKqpResourcesRequest newRequest = resources; @@ -231,17 +277,33 @@ class TKqpResourceManager : public IKqpResourceManager { return result; } - hasScanQueryMemory = ScanQueryMemoryResource.Has(resources.Memory); - if (hasScanQueryMemory) { - ScanQueryMemoryResource.Acquire(resources.Memory); - queryMemoryLimit = QueryMemoryLimit.load(); + hasScanQueryMemory = TotalMemoryResource->AcquireIfAvailable(resources.Memory); + task->TotalMemoryCookie = TotalMemoryResource->GetSpillingCookie(); + + if (hasScanQueryMemory && !tx->PoolId.empty() && tx->MemoryPoolPercent > 0) { + auto [it, success] = MemoryNamedPools.emplace(tx->MakePoolId(), nullptr); + + if (success) { + it->second = MakeIntrusive(TotalMemoryResource->GetLimit(), tx->MemoryPoolPercent, SpillingPercent.load()); + } else { + it->second->SetNewLimit(TotalMemoryResource->GetLimit(), tx->MemoryPoolPercent, SpillingPercent.load()); + } + + auto& poolMemory = it->second; + if (!poolMemory->AcquireIfAvailable(resources.Memory)) { + hasScanQueryMemory = false; + TotalMemoryResource->Release(resources.Memory); + } + + task->PoolMemoryCookie = poolMemory->GetSpillingCookie(); } - } // with_lock (Lock) + } if (!hasScanQueryMemory) { Counters->RmNotEnoughMemory->Inc(); TStringBuilder reason; - reason << "TxId: " << txId << ", taskId: " << taskId << ". Not enough memory for query, requested: " << resources.Memory; + reason << "TxId: " << txId << ", taskId: " << taskId << ". Not enough memory for query, requested: " << resources.Memory + << ". " << tx->ToString(); result.SetError(NKikimrKqp::TEvStartKqpTasksResponse::NOT_ENOUGH_MEMORY, reason); return result; } @@ -253,28 +315,29 @@ class TKqpResourceManager : public IKqpResourceManager { if (!result) { Counters->RmNotEnoughMemory->Inc(); with_lock (Lock) { - ScanQueryMemoryResource.Release(resources.Memory); - } // with_lock (Lock) + TotalMemoryResource->Release(resources.Memory); + if (!tx->PoolId.empty()) { + auto it = MemoryNamedPools.find(tx->MakePoolId()); + if (it != MemoryNamedPools.end()) { + it->second->Release(resources.Memory); + } + + if (it->second->GetUsed() == 0) { + MemoryNamedPools.erase(it); + } + } + } } }; - ui64 txTotalRequestedMemory = tx->GetExtraMemoryAllocatedSize() + resources.Memory; - if (txTotalRequestedMemory > queryMemoryLimit) { - TStringBuilder reason; - reason << "TxId: " << txId << ", taskId: " << taskId << ". Query memory limit exceeded: " - << "requested " << txTotalRequestedMemory; - result.SetError(NKikimrKqp::TEvStartKqpTasksResponse::QUERY_MEMORY_LIMIT_EXCEEDED, reason); - return result; - } - bool allocated = ResourceBroker->SubmitTaskInstant( TEvResourceBroker::TEvSubmitTask(rbTaskId, rbTaskName, {0, resources.Memory}, "kqp_query", 0, {}), SelfId); if (!allocated) { TStringBuilder reason; - reason << "TxId: " << txId << ", taskId: " << taskId << ". Not enough ScanQueryMemory: " - << "requested " << resources.Memory; + reason << "TxId: " << txId << ", taskId: " << taskId << ". Not enough memory for query, requested: " << resources.Memory + << ". " << tx->ToString(); LOG_AS_N(reason); result.SetError(NKikimrKqp::TEvStartKqpTasksResponse::NOT_ENOUGH_MEMORY, reason); return result; @@ -323,14 +386,25 @@ class TKqpResourceManager : public IKqpResourceManager { if (resources.Memory > 0) { with_lock (Lock) { - ScanQueryMemoryResource.Release(resources.Memory); - } // with_lock (Lock) + TotalMemoryResource->Release(resources.Memory); + if (!tx->PoolId.empty()) { + auto it = MemoryNamedPools.find(tx->MakePoolId()); + if (it != MemoryNamedPools.end()) { + it->second->Release(resources.Memory); + + if (it->second->GetUsed() == 0) { + MemoryNamedPools.erase(it); + } + } + } + } } - LOG_AS_D("TxId: " << tx->TxId << ", taskId: " << task->TaskId << ". Released resources, " - << "ScanQueryMemory: " << resources.Memory << ", " - << "ExternalDataQueryMemory " << resources.ExternalMemory << ", " - << "ExecutionUnits " << resources.ExecutionUnits << "."); + LOG_AS_D("TxId: " << tx->TxId << ", taskId: " << task->TaskId + << ". Released resources, " + << "Memory: " << resources.Memory << ", " + << "Free Tier: " << resources.ExternalMemory << ", " + << "ExecutionUnits: " << resources.ExecutionUnits << "."); FireResourcesPublishing(); } @@ -378,7 +452,7 @@ class TKqpResourceManager : public IKqpResourceManager { with_lock (Lock) { result.ExecutionUnits = ExecutionUnitsResource.load(); - result.Memory[EKqpMemoryPool::ScanQuery] = ScanQueryMemoryResource.Available(); + result.Memory[EKqpMemoryPool::ScanQuery] = TotalMemoryResource->Available(); } return result; @@ -427,6 +501,7 @@ class TKqpResourceManager : public IKqpResourceManager { MaxNonParallelTasksExecutionLimit.store(config.GetMaxNonParallelTasksExecutionLimit()); PreferLocalDatacenterExecution.store(config.GetPreferLocalDatacenterExecution()); MaxNonParallelDataQueryTasksLimit.store(config.GetMaxNonParallelDataQueryTasksLimit()); + SpillingPercent.store(config.GetSpillingPercent()); } ui32 GetNodeId() override { @@ -471,7 +546,8 @@ class TKqpResourceManager : public IKqpResourceManager { // limits (guarded by Lock) std::atomic ExecutionUnitsResource; std::atomic ExecutionUnitsLimit; - TLimitedResource ScanQueryMemoryResource; + std::atomic SpillingPercent; + TIntrusivePtr TotalMemoryResource; std::atomic ExternalDataQueryMemory = 0; std::atomic MaxNonParallelTopStageExecutionLimit = 1; std::atomic MaxNonParallelTasksExecutionLimit = 8; @@ -489,6 +565,8 @@ class TKqpResourceManager : public IKqpResourceManager { std::shared_ptr ResourceSnapshotState; bool PublishResourcesByExchanger; TActorId ResourceInfoExchanger = TActorId(); + + absl::flat_hash_map, TIntrusivePtr, THash>> MemoryNamedPools; }; struct TResourceManagers { @@ -651,7 +729,7 @@ class TKqpResourceManagerActor : public TActorBootstrapped 0) { with_lock (ResourceManager->Lock) { - ResourceManager->ScanQueryMemoryResource.SetNewLimit(queueConfig.GetLimit().GetMemory()); + ResourceManager->TotalMemoryResource->SetNewLimit(queueConfig.GetLimit().GetMemory(), (double)100, ResourceManager->SpillingPercent.load()); } LOG_I("Total node memory for scan queries: " << queueConfig.GetLimit().GetMemory() << " bytes"); } @@ -793,7 +871,7 @@ class TKqpResourceManagerActor : public TActorBootstrappedLock) { - str << "ScanQuery memory resource: " << ResourceManager->ScanQueryMemoryResource.ToString() << Endl; + str << "ScanQuery memory resource: " << ResourceManager->TotalMemoryResource->ToString() << Endl; str << "External DataQuery memory: " << ResourceManager->ExternalDataQueryMemory.load() << Endl; str << "ExecutionUnits resource: " << ResourceManager->ExecutionUnitsResource.load() << Endl; } @@ -891,13 +969,13 @@ class TKqpResourceManagerActor : public TActorBootstrappedLock) { payload.SetAvailableComputeActors(ResourceManager->ExecutionUnitsResource.load()); // legacy - payload.SetTotalMemory(ResourceManager->ScanQueryMemoryResource.GetLimit()); // legacy - payload.SetUsedMemory(ResourceManager->ScanQueryMemoryResource.GetLimit() - ResourceManager->ScanQueryMemoryResource.Available()); // legacy + payload.SetTotalMemory(ResourceManager->TotalMemoryResource->GetLimit()); // legacy + payload.SetUsedMemory(ResourceManager->TotalMemoryResource->GetLimit() - ResourceManager->TotalMemoryResource->Available()); // legacy payload.SetExecutionUnits(ResourceManager->ExecutionUnitsResource.load()); auto* pool = payload.MutableMemory()->Add(); pool->SetPool(EKqpMemoryPool::ScanQuery); - pool->SetAvailable(ResourceManager->ScanQueryMemoryResource.Available()); + pool->SetAvailable(ResourceManager->TotalMemoryResource->Available()); } if (PublishResourcesByExchanger) { diff --git a/ydb/core/kqp/rm_service/kqp_rm_service.h b/ydb/core/kqp/rm_service/kqp_rm_service.h index 213387b2a702..23138cf394c4 100644 --- a/ydb/core/kqp/rm_service/kqp_rm_service.h +++ b/ydb/core/kqp/rm_service/kqp_rm_service.h @@ -16,6 +16,7 @@ #include #include #include +#include namespace NKikimr { @@ -55,6 +56,11 @@ struct TKqpResourcesRequest { class TTxState; +class TMemoryResourceCookie : public TAtomicRefCount { +public: + std::atomic SpillingPercentReached{false}; +}; + class TTaskState : public TAtomicRefCount { friend TTxState; @@ -65,6 +71,8 @@ class TTaskState : public TAtomicRefCount { ui64 ExternalDataQueryMemory = 0; ui64 ResourceBrokerTaskId = 0; ui32 ExecutionUnits = 0; + TIntrusivePtr TotalMemoryCookie; + TIntrusivePtr PoolMemoryCookie; public: @@ -80,6 +88,11 @@ class TTaskState : public TAtomicRefCount { return resources; } + bool IsReasonableToStartSpilling() { + return (PoolMemoryCookie && PoolMemoryCookie->SpillingPercentReached.load()) + || (TotalMemoryCookie && TotalMemoryCookie->SpillingPercentReached.load()); + } + TKqpResourcesRequest FreeResourcesRequest() const { return TKqpResourcesRequest{ .ExecutionUnits=ExecutionUnits, @@ -101,26 +114,47 @@ class TTxState : public TAtomicRefCount { const ui64 TxId; const TInstant CreatedAt; TIntrusivePtr Counters; + const TString PoolId; + const double MemoryPoolPercent; + const TString Database; + private: std::atomic TxScanQueryMemory = 0; std::atomic TxExternalDataQueryMemory = 0; std::atomic TxExecutionUnits = 0; public: - explicit TTxState(ui64 txId, TInstant now, TIntrusivePtr counters) + explicit TTxState(ui64 txId, TInstant now, TIntrusivePtr counters, const TString& poolId, const double memoryPoolPercent, + const TString& database) : TxId(txId) , CreatedAt(now) , Counters(std::move(counters)) + , PoolId(poolId) + , MemoryPoolPercent(memoryPoolPercent) + , Database(database) {} + std::pair MakePoolId() const { + return std::make_pair(Database, PoolId); + } + TString ToString() const { - return TStringBuilder() << "TxResourcesInfo { " + auto res = TStringBuilder() << "TxResourcesInfo{ " << "TxId: " << TxId - << ", memory initially granted resources: " << TxExternalDataQueryMemory.load() - << ", tx total allocations " << TxScanQueryMemory.load() + << "Database: " << Database; + + if (!PoolId.empty()) { + res << ", PoolId: " << PoolId + << ", MemoryPoolPercent: " << Sprintf("%.2f", MemoryPoolPercent); + } + + res << ", memory initially granted resources: " << TxExternalDataQueryMemory.load() + << ", extra allocations " << TxScanQueryMemory.load() << ", execution units: " << TxExecutionUnits.load() << ", started at: " << CreatedAt << " }"; + + return res; } ui64 GetExtraMemoryAllocatedSize() { diff --git a/ydb/core/kqp/rm_service/kqp_rm_ut.cpp b/ydb/core/kqp/rm_service/kqp_rm_ut.cpp index 09d7c0536254..579e90bad452 100644 --- a/ydb/core/kqp/rm_service/kqp_rm_ut.cpp +++ b/ydb/core/kqp/rm_service/kqp_rm_ut.cpp @@ -186,7 +186,7 @@ class KqpRm : public TTestBase { } TIntrusivePtr MakeTx(ui64 txId, std::shared_ptr rm) { - return MakeIntrusive(txId, TInstant::Now(), rm->GetCounters()); + return MakeIntrusive(txId, TInstant::Now(), rm->GetCounters(), "", (double)100, ""); } TIntrusivePtr MakeTask(ui64 taskId, TIntrusivePtr tx) { diff --git a/ydb/core/kqp/ut/olap/aggregations_ut.cpp b/ydb/core/kqp/ut/olap/aggregations_ut.cpp index 9a70ee376fed..71cb222e1de5 100644 --- a/ydb/core/kqp/ut/olap/aggregations_ut.cpp +++ b/ydb/core/kqp/ut/olap/aggregations_ut.cpp @@ -171,6 +171,49 @@ Y_UNIT_TEST_SUITE(KqpOlapAggregations) { } } + Y_UNIT_TEST_TWIN(DisableBlockEngineInAggregationWithSpilling, AllowSpilling) { + auto settings = TKikimrSettings() + .SetWithSampleTables(false); + settings.AppConfig.MutableTableServiceConfig()->SetBlockChannelsMode(NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_FORCE); + if (AllowSpilling) { + settings.AppConfig.MutableTableServiceConfig()->SetEnableSpillingNodes("Aggregation"); + } else { + settings.AppConfig.MutableTableServiceConfig()->SetEnableSpillingNodes("None"); + } + TKikimrRunner kikimr(settings); + + TLocalHelper(kikimr).CreateTestOlapTable(); + auto client = kikimr.GetQueryClient(); + + { + WriteTestData(kikimr, "/Root/olapStore/olapTable", 10000, 3000000, 1000); + WriteTestData(kikimr, "/Root/olapStore/olapTable", 11000, 3001000, 1000); + WriteTestData(kikimr, "/Root/olapStore/olapTable", 12000, 3002000, 1000); + WriteTestData(kikimr, "/Root/olapStore/olapTable", 13000, 3003000, 1000); + WriteTestData(kikimr, "/Root/olapStore/olapTable", 14000, 3004000, 1000); + WriteTestData(kikimr, "/Root/olapStore/olapTable", 20000, 2000000, 7000); + WriteTestData(kikimr, "/Root/olapStore/olapTable", 30000, 1000000, 11000); + } + + { + TString query = R"( + --!syntax_v1 + SELECT + COUNT(*) + FROM `/Root/olapStore/olapTable` + GROUP BY level + )"; + + auto res = StreamExplainQuery(query, client); + UNIT_ASSERT_C(res.IsSuccess(), res.GetIssues().ToString()); + + auto plan = CollectStreamResult(res); + + bool hasWideCombiner = plan.QueryStats->Getquery_ast().Contains("WideCombiner"); + UNIT_ASSERT_C(hasWideCombiner == AllowSpilling, plan.QueryStats->Getquery_ast()); + } + } + Y_UNIT_TEST_TWIN(CountAllPushdown, UseLlvm) { auto settings = TKikimrSettings() .SetWithSampleTables(false); diff --git a/ydb/core/kqp/ut/olap/clickbench_ut.cpp b/ydb/core/kqp/ut/olap/clickbench_ut.cpp index 4f05c3ab05b8..e9f09d450e0c 100644 --- a/ydb/core/kqp/ut/olap/clickbench_ut.cpp +++ b/ydb/core/kqp/ut/olap/clickbench_ut.cpp @@ -166,9 +166,9 @@ Y_UNIT_TEST_SUITE(KqpOlapClickbench) { GROUP BY RegionID ORDER BY c DESC LIMIT 10 - )"); + )") //.SetExpectedReply("[[[\"40999\"];[4];1u];[[\"40998\"];[3];1u];[[\"40997\"];[2];1u]]") - // .SetExpectedReadNodeType("TableFullScan"); + .SetExpectedReadNodeType("TableFullScan"); // .SetExpectedReadNodeType("Aggregate-TableFullScan"); q9.FillExpectedAggregationGroupByPlanOptions(); @@ -213,9 +213,7 @@ Y_UNIT_TEST_SUITE(KqpOlapClickbench) { GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10; - )") - .AddExpectedPlanOptions("KqpOlapFilter"); - // .SetExpectedReadNodeType("TableFullScan"); + )"); q22.FillExpectedAggregationGroupByPlanOptions(); TAggregationTestCase q39; diff --git a/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp b/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp index f08611afd340..ca25c070056a 100644 --- a/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp +++ b/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp @@ -2491,6 +2491,7 @@ Y_UNIT_TEST_SUITE(KqpOlap) { NKikimrConfig::TAppConfig appConfig; appConfig.MutableTableServiceConfig()->SetEnableOlapSink(true); appConfig.MutableTableServiceConfig()->SetBlockChannelsMode(blockChannelsMode); + appConfig.MutableTableServiceConfig()->SetEnableSpillingNodes("None"); auto settings = TKikimrSettings() .SetAppConfig(appConfig) .SetWithSampleTables(true); diff --git a/ydb/core/kqp/ut/query/kqp_limits_ut.cpp b/ydb/core/kqp/ut/query/kqp_limits_ut.cpp index 5da239d6306c..d8fd16d5d754 100644 --- a/ydb/core/kqp/ut/query/kqp_limits_ut.cpp +++ b/ydb/core/kqp/ut/query/kqp_limits_ut.cpp @@ -6,6 +6,7 @@ #include #include +#include #include namespace NKikimr { @@ -160,12 +161,14 @@ Y_UNIT_TEST_SUITE(KqpLimits) { result.GetIssues().PrintTo(Cerr); UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::BAD_REQUEST); } - + Y_UNIT_TEST(ComputeActorMemoryAllocationFailure) { auto app = NKikimrConfig::TAppConfig(); app.MutableTableServiceConfig()->MutableResourceManager()->SetMkqlLightProgramMemoryLimit(10); app.MutableTableServiceConfig()->MutableResourceManager()->SetQueryMemoryLimit(2000); + app.MutableResourceBrokerConfig()->CopyFrom(MakeResourceBrokerTestConfig()); + TKikimrRunner kikimr(app); CreateLargeTable(kikimr, 0, 0, 0); @@ -180,7 +183,6 @@ Y_UNIT_TEST_SUITE(KqpLimits) { result.GetIssues().PrintTo(Cerr); UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::OVERLOADED); - UNIT_ASSERT_C(result.GetIssues().ToString().Contains("Mkql memory limit exceeded"), result.GetIssues().ToString()); } Y_UNIT_TEST(ComputeActorMemoryAllocationFailureQueryService) { diff --git a/ydb/core/kqp/ut/spilling/kqp_scan_spilling_ut.cpp b/ydb/core/kqp/ut/spilling/kqp_scan_spilling_ut.cpp index 1cd510746c1c..2f66b9f37bf9 100644 --- a/ydb/core/kqp/ut/spilling/kqp_scan_spilling_ut.cpp +++ b/ydb/core/kqp/ut/spilling/kqp_scan_spilling_ut.cpp @@ -32,6 +32,44 @@ NKikimrConfig::TAppConfig AppCfg() { return appCfg; } +NKikimrConfig::TAppConfig AppCfgLowComputeLimits(double reasonableTreshold, bool enableSpilling=true) { + NKikimrConfig::TAppConfig appCfg; + + auto* rm = appCfg.MutableTableServiceConfig()->MutableResourceManager(); + rm->SetMkqlLightProgramMemoryLimit(100); + rm->SetMkqlHeavyProgramMemoryLimit(300); + rm->SetSpillingPercent(reasonableTreshold); + appCfg.MutableTableServiceConfig()->SetEnableQueryServiceSpilling(true); + + auto* spilling = appCfg.MutableTableServiceConfig()->MutableSpillingServiceConfig()->MutableLocalFileConfig(); + + spilling->SetEnable(enableSpilling); + spilling->SetRoot("./spilling/"); + + return appCfg; +} + +void FillTableWithData(NQuery::TQueryClient& db, ui64 numRows=300) { + for (ui32 i = 0; i < numRows; ++i) { + auto result = db.ExecuteQuery(Sprintf(R"( + --!syntax_v1 + REPLACE INTO `/Root/KeyValue` (Key, Value) VALUES (%d, "%s") + )", i, TString(200000 + i, 'a' + (i % 26)).c_str()), NYdb::NQuery::TTxControl::BeginTx().CommitTx()).GetValueSync(); + UNIT_ASSERT_C(result.IsSuccess(), result.GetIssues().ToString()); + } +} + +constexpr auto SimpleGraceJoinWithSpillingQuery = R"( + --!syntax_v1 + PRAGMA ydb.EnableSpillingNodes="GraceJoin"; + PRAGMA ydb.CostBasedOptimizationLevel='0'; + PRAGMA ydb.HashJoinMode='graceandself'; + select t1.Key, t1.Value, t2.Key, t2.Value + from `/Root/KeyValue` as t1 full join `/Root/KeyValue` as t2 on t1.Value = t2.Value + order by t1.Value + )"; + + } // anonymous namespace Y_UNIT_TEST_SUITE(KqpScanSpilling) { @@ -54,6 +92,59 @@ Y_UNIT_TEST(SpillingPragmaParseError) { UNIT_ASSERT_VALUES_EQUAL_C(planres.GetStatus(), EStatus::GENERIC_ERROR, planres.GetIssues().ToString()); } +Y_UNIT_TEST_TWIN(SpillingInRuntimeNodes, EnabledSpilling) { + // tmp fix while runtime version is lower 50 + if (EnabledSpilling) return; + double reasonableTreshold = EnabledSpilling ? 0.01 : 100; + Cerr << "cwd: " << NFs::CurrentWorkingDirectory() << Endl; + TKikimrRunner kikimr(AppCfgLowComputeLimits(reasonableTreshold)); + + auto db = kikimr.GetQueryClient(); + + FillTableWithData(db); + + auto explainMode = NYdb::NQuery::TExecuteQuerySettings().ExecMode(NYdb::NQuery::EExecMode::Explain); + auto planres = db.ExecuteQuery(SimpleGraceJoinWithSpillingQuery, NYdb::NQuery::TTxControl::NoTx(), explainMode).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(planres.GetStatus(), EStatus::SUCCESS, planres.GetIssues().ToString()); + + Cerr << planres.GetStats()->GetAst() << Endl; + + auto result = db.ExecuteQuery(SimpleGraceJoinWithSpillingQuery, NYdb::NQuery::TTxControl::BeginTx().CommitTx(), NYdb::NQuery::TExecuteQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + + TKqpCounters counters(kikimr.GetTestServer().GetRuntime()->GetAppData().Counters); + if (EnabledSpilling) { + UNIT_ASSERT(counters.SpillingWriteBlobs->Val() > 0); + UNIT_ASSERT(counters.SpillingReadBlobs->Val() > 0); + } else { + UNIT_ASSERT(counters.SpillingWriteBlobs->Val() == 0); + UNIT_ASSERT(counters.SpillingReadBlobs->Val() == 0); + } +} + +Y_UNIT_TEST(HandleErrorsCorrectly) { + Cerr << "cwd: " << NFs::CurrentWorkingDirectory() << Endl; + TKikimrRunner kikimr(AppCfgLowComputeLimits(0.01, false)); + + auto db = kikimr.GetQueryClient(); + + FillTableWithData(db); + + auto explainMode = NYdb::NQuery::TExecuteQuerySettings().ExecMode(NYdb::NQuery::EExecMode::Explain); + auto planres = db.ExecuteQuery(SimpleGraceJoinWithSpillingQuery, NYdb::NQuery::TTxControl::NoTx(), explainMode).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(planres.GetStatus(), EStatus::SUCCESS, planres.GetIssues().ToString()); + + Cerr << planres.GetStats()->GetAst() << Endl; + + auto result = db.ExecuteQuery(SimpleGraceJoinWithSpillingQuery, NYdb::NQuery::TTxControl::BeginTx().CommitTx(), NYdb::NQuery::TExecuteQuerySettings()).ExtractValueSync(); + const auto errorMsg = result.GetIssues().ToString(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::INTERNAL_ERROR, errorMsg); + + const auto spillingPrefix = "[Compute spilling]"; + const auto pos = errorMsg.find(spillingPrefix); + UNIT_ASSERT_VALUES_UNEQUAL_C(pos, std::string::npos, "Spilling prefix not found in error message"); +} + Y_UNIT_TEST(SelfJoinQueryService) { Cerr << "cwd: " << NFs::CurrentWorkingDirectory() << Endl; diff --git a/ydb/core/protos/table_service_config.proto b/ydb/core/protos/table_service_config.proto index cc3223513685..2906289160dc 100644 --- a/ydb/core/protos/table_service_config.proto +++ b/ydb/core/protos/table_service_config.proto @@ -42,7 +42,7 @@ message TTableServiceConfig { optional TInfoExchangerSettings InfoExchangerSettings = 19; optional uint64 KqpPatternCachePatternAccessTimesBeforeTryToCompile = 20 [default = 5]; optional uint64 KqpPatternCacheCompiledCapacityBytes = 21 [default = 104857600]; // 100 MiB - optional uint64 ReasonableSpillingTreshold = 22 [default = 104857600]; // 100 MiB + optional double SpillingPercent = 22 [default = 80]; // 100 MiB optional uint64 MinMemAllocSize = 23 [default = 8388608]; // 8 MiB optional uint64 MinMemFreeSize = 24 [default = 33554432]; // 32 MiB @@ -55,8 +55,8 @@ message TTableServiceConfig { message TSpillingServiceConfig { message TLocalFileConfig { - optional bool Enable = 1 [default = false]; - optional string Root = 2 [default = "/tmp/kikimr_spilling/"]; + optional bool Enable = 1 [default = true]; + optional string Root = 2 [default = ""]; optional uint64 MaxTotalSize = 3 [default = 21474836480]; // 20 GiB optional uint64 MaxFileSize = 4 [default = 5368709120]; // 5 GiB optional uint64 MaxFilePartSize = 5 [default = 104857600]; // 100 MB diff --git a/ydb/core/tx/datashard/datashard_kqp.cpp b/ydb/core/tx/datashard/datashard_kqp.cpp index af3eb3c31c15..47928c55e8cc 100644 --- a/ydb/core/tx/datashard/datashard_kqp.cpp +++ b/ydb/core/tx/datashard/datashard_kqp.cpp @@ -1012,7 +1012,15 @@ class TKqpTaskRunnerExecutionContext: public NDq::IDqTaskRunnerExecutionContext return {}; } - std::function GetWakeupCallback() const override { + NDq::TWakeUpCallback GetWakeupCallback() const override { + return {}; + } + + NDq::TErrorCallback GetErrorCallback() const override { + return {}; + } + + TIntrusivePtr GetSpillingTaskCounters() const override { return {}; } diff --git a/ydb/core/tx/datashard/datashard_kqp_lookup_table.cpp b/ydb/core/tx/datashard/datashard_kqp_lookup_table.cpp index 27fc69b4f132..51752b1d206a 100644 --- a/ydb/core/tx/datashard/datashard_kqp_lookup_table.cpp +++ b/ydb/core/tx/datashard/datashard_kqp_lookup_table.cpp @@ -121,7 +121,7 @@ class TKqpLookupRowsWrapper : public TStatelessFlowComputationNode keyCells(ParseResult.KeyIndices.size()); - FillKeyTupleValue(key, ParseResult.KeyIndices, ParseResult.KeyTypes, keyCells, *ctx.TypeEnv); + FillKeyTupleValue(key, ParseResult.KeyIndices, ParseResult.KeyTypes, keyCells, ctx.TypeEnv); NUdf::TUnboxedValue result; TKqpTableStats stats; @@ -203,10 +203,10 @@ class TKqpLookupTableWrapper : public TStatelessFlowComputationNode fromCells(tableInfo->KeyColumns.size()); - FillKeyTupleValue(key, ParseResult.KeyIndices, ParseResult.KeyTypes, fromCells, *ctx.TypeEnv); + FillKeyTupleValue(key, ParseResult.KeyIndices, ParseResult.KeyTypes, fromCells, ctx.TypeEnv); TVector toCells(ParseResult.KeyIndices.size()); - FillKeyTupleValue(key, ParseResult.KeyIndices, ParseResult.KeyTypes, toCells, *ctx.TypeEnv); + FillKeyTupleValue(key, ParseResult.KeyIndices, ParseResult.KeyTypes, toCells, ctx.TypeEnv); auto range = TTableRange(fromCells, true, toCells, true); diff --git a/ydb/core/tx/datashard/datashard_kqp_read_table.cpp b/ydb/core/tx/datashard/datashard_kqp_read_table.cpp index 131165ae2999..6a06af8814c4 100644 --- a/ydb/core/tx/datashard/datashard_kqp_read_table.cpp +++ b/ydb/core/tx/datashard/datashard_kqp_read_table.cpp @@ -269,10 +269,10 @@ class TKqpWideReadTableWrapper : public TKqpWideReadTableWrapperBase EFetchResult ReadValue(TComputationContext& ctx, NUdf::TUnboxedValue* const* output) const final { if (!this->Iterator) { TVector fromCells; - BuildKeyTupleCells(ParseResult.FromTuple->GetType(), FromNode->GetValue(ctx), fromCells, *ctx.TypeEnv); + BuildKeyTupleCells(ParseResult.FromTuple->GetType(), FromNode->GetValue(ctx), fromCells, ctx.TypeEnv); TVector toCells; - BuildKeyTupleCells(ParseResult.ToTuple->GetType(), ToNode->GetValue(ctx), toCells, *ctx.TypeEnv); + BuildKeyTupleCells(ParseResult.ToTuple->GetType(), ToNode->GetValue(ctx), toCells, ctx.TypeEnv); auto range = TTableRange(fromCells, ParseResult.FromInclusive, toCells, ParseResult.ToInclusive); @@ -328,7 +328,7 @@ class TKqpWideReadTableRangesWrapper : public TKqpWideReadTableWrapperBaseComputeCtx.GetLocalTableId(ParseResult.TableId); const auto* tableInfo = this->ComputeCtx.Database->GetScheme().GetTableInfo(localTid); - Ranges = CreateTableRanges(ParseResult, RangesNode, *ctx.TypeEnv, ctx, tableInfo->KeyColumns.size()); + Ranges = CreateTableRanges(ParseResult, RangesNode, ctx.TypeEnv, ctx, tableInfo->KeyColumns.size()); RangeId = 0; if (ItemsLimit) { diff --git a/ydb/core/tx/program/registry.cpp b/ydb/core/tx/program/registry.cpp index f8ba71e37d78..5b68a06787e4 100644 --- a/ydb/core/tx/program/registry.cpp +++ b/ydb/core/tx/program/registry.cpp @@ -7,18 +7,18 @@ namespace NKikimr::NOlap { -::NTls::TValue Registry; +::NTls::TValue> Registry; bool TKernelsRegistry::Parse(const TString& serialized) { Y_ABORT_UNLESS(!!serialized); if (!Registry.Get()) { - Registry = NMiniKQL::CreateBuiltinRegistry(); + auto registry = NMiniKQL::CreateFunctionRegistry(NMiniKQL::CreateBuiltinRegistry())->Clone(); + NMiniKQL::FillStaticModules(*registry.Get()); + Registry = std::move(registry); } - auto copy = Registry.Get(); - auto functionRegistry = NMiniKQL::CreateFunctionRegistry(std::move(copy))->Clone(); - NMiniKQL::FillStaticModules(*functionRegistry); + auto nodeFactory = NMiniKQL::GetBuiltinFactory(); - auto kernels = NYql::LoadKernels(serialized, *functionRegistry, nodeFactory); + auto kernels = NYql::LoadKernels(serialized, *Registry.Get(), nodeFactory); Kernels.swap(kernels); for (const auto& kernel : Kernels) { arrow::compute::Arity arity(kernel->signature->in_types().size(), kernel->signature->is_varargs()); diff --git a/ydb/library/yql/core/common_opt/yql_co_flow1.cpp b/ydb/library/yql/core/common_opt/yql_co_flow1.cpp index eeb53b7e5815..d8c5138a4cc6 100644 --- a/ydb/library/yql/core/common_opt/yql_co_flow1.cpp +++ b/ydb/library/yql/core/common_opt/yql_co_flow1.cpp @@ -1409,7 +1409,7 @@ TExprNode::TPtr OptimizeFlatMap(const TExprNode::TPtr& node, TExprContext& ctx, return FuseFlatMapOverByKey(*node, ctx); } - if (node->Head().IsCallable({"PartitionByKey", "PartitionsByKeys"})) { + if (node->Head().IsCallable({"PartitionByKey", "PartitionsByKeys", "ShuffleByKeys"})) { return FuseFlatMapOverByKey(*node, ctx); } } @@ -1477,7 +1477,7 @@ TExprNode::TPtr OptimizeFlatMap(const TExprNode::TPtr& node, TExprContext& ctx, { auto canPush = [&](const auto& child) { // we push FlatMap over Extend only if it can later be fused with child - return child->IsCallable({Ordered ? "OrderedFlatMap" : "FlatMap", "GroupByKey", "CombineByKey", "PartitionByKey", "PartitionsByKeys", + return child->IsCallable({Ordered ? "OrderedFlatMap" : "FlatMap", "GroupByKey", "CombineByKey", "PartitionByKey", "PartitionsByKeys", "ShuffleByKeys", "ListIf", "FlatListIf", "AsList", "ToList"}) && optCtx.IsSingleUsage(*child); }; if (AllOf(node->Head().ChildrenList(), canPush)) { diff --git a/ydb/library/yql/core/yql_aggregate_expander.cpp b/ydb/library/yql/core/yql_aggregate_expander.cpp index e8ef9ca7e4ff..5eec73b41666 100644 --- a/ydb/library/yql/core/yql_aggregate_expander.cpp +++ b/ydb/library/yql/core/yql_aggregate_expander.cpp @@ -36,7 +36,7 @@ TExprNode::TPtr TAggregateExpander::ExpandAggregateWithFullOutput() HaveDistinct = AnyOf(AggregatedColumns->ChildrenList(), [](const auto& child) { return child->ChildrenSize() == 3; }); - EffectiveCompact = (HaveDistinct && CompactForDistinct && !TypesCtx.IsBlockEngineEnabled()) || ForceCompact || HasSetting(*settings, "compact"); + EffectiveCompact = (HaveDistinct && CompactForDistinct && !UseBlocks) || ForceCompact || HasSetting(*settings, "compact"); for (const auto& trait : Traits) { auto mergeLambda = trait->Child(5); if (mergeLambda->Tail().IsCallable("Void")) { @@ -67,7 +67,7 @@ TExprNode::TPtr TAggregateExpander::ExpandAggregateWithFullOutput() return GeneratePhases(); } - if (TypesCtx.IsBlockEngineEnabled()) { + if (UseBlocks) { if (Suffix == "Combine") { auto ret = TryGenerateBlockCombine(); if (ret) { @@ -2785,7 +2785,7 @@ TExprNode::TPtr TAggregateExpander::GeneratePhases() { streams.push_back(SerializeIdxSet(indicies)); } - if (TypesCtx.IsBlockEngineEnabled()) { + if (UseBlocks) { for (ui32 i = 0; i < unionAllInputs.size(); ++i) { unionAllInputs[i] = Ctx.Builder(Node->Pos()) .Callable("Map") @@ -2806,7 +2806,7 @@ TExprNode::TPtr TAggregateExpander::GeneratePhases() { } auto settings = cleanOutputSettings; - if (TypesCtx.IsBlockEngineEnabled()) { + if (UseBlocks) { settings = AddSetting(*settings, Node->Pos(), "many_streams", Ctx.NewList(Node->Pos(), std::move(streams)), Ctx); } @@ -2839,7 +2839,7 @@ TExprNode::TPtr TAggregateExpander::TryGenerateBlockCombine() { } TExprNode::TPtr TAggregateExpander::TryGenerateBlockMergeFinalize() { - if (UsePartitionsByKeys || !TypesCtx.IsBlockEngineEnabled()) { + if (UsePartitionsByKeys || !UseBlocks) { return nullptr; } @@ -2934,7 +2934,7 @@ TExprNode::TPtr ExpandAggregatePeephole(const TExprNode::TPtr& node, TExprContex return ret; } } - return ExpandAggregatePeepholeImpl(node, ctx, typesCtx, false, typesCtx.IsBlockEngineEnabled()); + return ExpandAggregatePeepholeImpl(node, ctx, typesCtx, false, typesCtx.IsBlockEngineEnabled(), false); } } // namespace NYql diff --git a/ydb/library/yql/core/yql_aggregate_expander.h b/ydb/library/yql/core/yql_aggregate_expander.h index 5333db934229..8161ad81751d 100644 --- a/ydb/library/yql/core/yql_aggregate_expander.h +++ b/ydb/library/yql/core/yql_aggregate_expander.h @@ -9,7 +9,7 @@ namespace NYql { class TAggregateExpander { public: TAggregateExpander(bool usePartitionsByKeys, const bool useFinalizeByKeys, const TExprNode::TPtr& node, TExprContext& ctx, TTypeAnnotationContext& typesCtx, - bool forceCompact = false, bool compactForDistinct = false, bool usePhases = false) + bool forceCompact = false, bool compactForDistinct = false, bool usePhases = false, bool allowSpilling = false) : Node(node) , Ctx(ctx) , TypesCtx(typesCtx) @@ -25,6 +25,7 @@ class TAggregateExpander { , HaveSessionSetting(false) , OriginalRowType(nullptr) , RowType(nullptr) + , UseBlocks(typesCtx.IsBlockEngineEnabled() && !allowSpilling) { PreMap = Ctx.Builder(node->Pos()) .Lambda() @@ -115,6 +116,7 @@ class TAggregateExpander { const TStructExprType* RowType; TVector RowItems; TExprNode::TPtr PreMap; + bool UseBlocks; TExprNode::TListType InitialColumnNames; TExprNode::TListType FinalColumnNames; @@ -130,8 +132,10 @@ class TAggregateExpander { std::unordered_map UdfWasChanged; }; -inline TExprNode::TPtr ExpandAggregatePeepholeImpl(const TExprNode::TPtr& node, TExprContext& ctx, TTypeAnnotationContext& typesCtx, const bool useFinalizeByKey, const bool useBlocks) { - TAggregateExpander aggExpander(!useFinalizeByKey && !useBlocks, useFinalizeByKey, node, ctx, typesCtx, true); +inline TExprNode::TPtr ExpandAggregatePeepholeImpl(const TExprNode::TPtr& node, TExprContext& ctx, TTypeAnnotationContext& typesCtx, + const bool useFinalizeByKey, const bool useBlocks, const bool allowSpilling) { + TAggregateExpander aggExpander(!useFinalizeByKey && !useBlocks, useFinalizeByKey, node, ctx, typesCtx, + true, false, false, allowSpilling); return aggExpander.ExpandAggregate(); } diff --git a/ydb/library/yql/core/yql_opt_utils.cpp b/ydb/library/yql/core/yql_opt_utils.cpp index a57c4f983e9a..814b2baeaa62 100644 --- a/ydb/library/yql/core/yql_opt_utils.cpp +++ b/ydb/library/yql/core/yql_opt_utils.cpp @@ -1813,7 +1813,8 @@ TExprNode::TPtr FindNonYieldTransparentNodeImpl(const TExprNode::TPtr& root, con || TCoForwardList::Match(node.Get()) || TCoApply::Match(node.Get()) || TCoSwitch::Match(node.Get()) - || node->IsCallable("DqReplicate"); + || node->IsCallable("DqReplicate") + || TCoPartitionsByKeys::Match(node.Get()); } ); @@ -1851,6 +1852,11 @@ TExprNode::TPtr FindNonYieldTransparentNodeImpl(const TExprNode::TPtr& root, con return node; } } + } else if (TCoPartitionsByKeys::Match(candidate.Get())) { + const auto handlerChild = candidate->Child(TCoPartitionsByKeys::idx_ListHandlerLambda); + if (auto node = FindNonYieldTransparentNodeImpl(handlerChild->TailPtr(), udfSupportsYield, TNodeSet{&handlerChild->Head().Head()})) { + return node; + } } } return {}; diff --git a/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp b/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp index f6629fc5b5e1..dac377c8896a 100644 --- a/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp @@ -127,8 +127,9 @@ class TDqAsyncComputeActor : public TDqComputeActorBase); - auto wakeup = [this]{ ContinueExecute(EResumeSource::CABootstrapWakeup); }; - std::shared_ptr execCtx = std::make_shared(TxId, std::move(wakeup)); + auto wakeupCallback = [this]{ ContinueExecute(EResumeSource::CABootstrapWakeup); }; + auto errorCallback = [this](const TString& error){ SendError(error); }; + std::shared_ptr execCtx = std::make_shared(TxId, std::move(wakeupCallback), std::move(errorCallback)); Send(TaskRunnerActorId, new NTaskRunnerActor::TEvTaskRunnerCreate( diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor.cpp b/ydb/library/yql/dq/actors/compute/dq_compute_actor.cpp index 8851f301f7dc..2f6335e11d78 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor.cpp @@ -58,8 +58,9 @@ class TDqComputeActor : public TDqSyncComputeActorBase { auto taskRunner = TaskRunnerFactory(GetAllocatorPtr(), Task, RuntimeSettings.StatsMode, logger); SetTaskRunner(taskRunner); - auto wakeup = [this]{ ContinueExecute(EResumeSource::CABootstrapWakeup); }; - TDqTaskRunnerExecutionContext execCtx(TxId, std::move(wakeup)); + auto wakeupCallback = [this]{ ContinueExecute(EResumeSource::CABootstrapWakeup); }; + auto errorCallback = [this](const TString& error){ SendError(error); }; + TDqTaskRunnerExecutionContext execCtx(TxId, std::move(wakeupCallback), std::move(errorCallback)); PrepareTaskRunner(execCtx); ContinueExecute(EResumeSource::CABootstrap); diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h b/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h index fdbba8eacaae..87afb807eed1 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h @@ -638,6 +638,10 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped } } + void SendError(const TString& error) { + this->Send(this->SelfId(), TEvDq::TEvAbortExecution::InternalError(error)); + } + protected: //TDqComputeActorChannels::ICallbacks //i64 GetInputChannelFreeSpace(ui64 channelId) is pure and must be overridded in derived class diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp b/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp index eb59ece2953c..8a35905543cd 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp @@ -62,6 +62,14 @@ void FillTaskRunnerStats(ui64 taskId, ui32 stageId, const TTaskRunnerStatsBase& protoTask->SetWaitInputTimeUs(taskStats.WaitInputTime.MicroSeconds()); protoTask->SetWaitOutputTimeUs(taskStats.WaitOutputTime.MicroSeconds()); + protoTask->SetSpillingComputeWriteBytes(taskStats.SpillingComputeWriteBytes); + protoTask->SetSpillingChannelWriteBytes(taskStats.SpillingChannelWriteBytes); + + protoTask->SetSpillingComputeReadTimeUs(taskStats.SpillingComputeReadTime.MicroSeconds()); + protoTask->SetSpillingComputeWriteTimeUs(taskStats.SpillingComputeWriteTime.MicroSeconds()); + protoTask->SetSpillingChannelReadTimeUs(taskStats.SpillingChannelReadTime.MicroSeconds()); + protoTask->SetSpillingChannelWriteTimeUs(taskStats.SpillingChannelWriteTime.MicroSeconds()); + if (StatsLevelCollectProfile(level)) { if (taskStats.ComputeCpuTimeByRun) { auto snapshot = taskStats.ComputeCpuTimeByRun->Snapshot(); diff --git a/ydb/library/yql/dq/actors/compute/dq_sync_compute_actor_base.h b/ydb/library/yql/dq/actors/compute/dq_sync_compute_actor_base.h index 8a776d83fbf3..3d3a78293aa9 100644 --- a/ydb/library/yql/dq/actors/compute/dq_sync_compute_actor_base.h +++ b/ydb/library/yql/dq/actors/compute/dq_sync_compute_actor_base.h @@ -209,13 +209,13 @@ class TDqSyncComputeActorBase: public TDqComputeActorBasePrepare(this->Task, limits, execCtx); - + if (this->Task.GetEnableSpilling()) { - TaskRunner->SetSpillerFactory(std::make_shared(execCtx.GetTxId(), NActors::TActivationContext::ActorSystem(), execCtx.GetWakeupCallback())); + TaskRunner->SetSpillerFactory(std::make_shared(execCtx.GetTxId(), NActors::TActivationContext::ActorSystem(), execCtx.GetWakeupCallback(), execCtx.GetErrorCallback())); } + TaskRunner->Prepare(this->Task, limits, execCtx); + for (auto& [channelId, channel] : this->InputChannelsMap) { channel.Channel = TaskRunner->GetInputChannel(channelId); } diff --git a/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.cpp b/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.cpp index 2f50fcbd5af0..e0127c420f55 100644 --- a/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.cpp @@ -6,9 +6,11 @@ namespace NYql { namespace NDq { -TDqTaskRunnerExecutionContext::TDqTaskRunnerExecutionContext(TTxId txId, IDqChannelStorage::TWakeUpCallback&& wakeUp) +TDqTaskRunnerExecutionContext::TDqTaskRunnerExecutionContext(TTxId txId, TWakeUpCallback&& wakeUpCallback, TErrorCallback&& errorCallback) : TxId_(txId) - , WakeUp_(std::move(wakeUp)) + , WakeUpCallback_(std::move(wakeUpCallback)) + , ErrorCallback_(std::move(errorCallback)) + , SpillingTaskCounters_(MakeIntrusive()) { } @@ -16,16 +18,24 @@ IDqChannelStorage::TPtr TDqTaskRunnerExecutionContext::CreateChannelStorage(ui64 return CreateChannelStorage(channelId, withSpilling, NActors::TlsActivationContext->ActorSystem()); } -IDqChannelStorage::TPtr TDqTaskRunnerExecutionContext::CreateChannelStorage(ui64 channelId, bool withSpilling, NActors::TActorSystem* actorSystem) const { +IDqChannelStorage::TPtr TDqTaskRunnerExecutionContext::CreateChannelStorage(ui64 channelId, bool withSpilling, NActors::TActorSystem* actorSystem) const { if (withSpilling) { - return CreateDqChannelStorage(TxId_, channelId, WakeUp_, actorSystem); + return CreateDqChannelStorage(TxId_, channelId, WakeUpCallback_, ErrorCallback_, SpillingTaskCounters_, actorSystem); } else { return nullptr; } } -std::function TDqTaskRunnerExecutionContext::GetWakeupCallback() const { - return WakeUp_; +TWakeUpCallback TDqTaskRunnerExecutionContext::GetWakeupCallback() const { + return WakeUpCallback_; +} + +TErrorCallback TDqTaskRunnerExecutionContext::GetErrorCallback() const { + return ErrorCallback_; +} + +TIntrusivePtr TDqTaskRunnerExecutionContext::GetSpillingTaskCounters() const { + return SpillingTaskCounters_; } TTxId TDqTaskRunnerExecutionContext::GetTxId() const { diff --git a/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.h b/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.h index f7fc7a0dea12..4dd1915d280c 100644 --- a/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.h +++ b/ydb/library/yql/dq/actors/compute/dq_task_runner_exec_ctx.h @@ -9,17 +9,21 @@ namespace NDq { class TDqTaskRunnerExecutionContext : public TDqTaskRunnerExecutionContextBase { public: - TDqTaskRunnerExecutionContext(TTxId txId, IDqChannelStorage::TWakeUpCallback&& wakeUp); + TDqTaskRunnerExecutionContext(TTxId txId, TWakeUpCallback&& WakeUpCallback_, TErrorCallback&& ErrorCallback_); IDqChannelStorage::TPtr CreateChannelStorage(ui64 channelId, bool withSpilling) const override; IDqChannelStorage::TPtr CreateChannelStorage(ui64 channelId, bool withSpilling, NActors::TActorSystem* actorSystem) const override; - - std::function GetWakeupCallback() const override; + + TWakeUpCallback GetWakeupCallback() const override; + TErrorCallback GetErrorCallback() const override; + TIntrusivePtr GetSpillingTaskCounters() const override; TTxId GetTxId() const override; private: const TTxId TxId_; - const IDqChannelStorage::TWakeUpCallback WakeUp_; + const TWakeUpCallback WakeUpCallback_; + const TErrorCallback ErrorCallback_; + const TIntrusivePtr SpillingTaskCounters_; }; } // namespace NDq diff --git a/ydb/library/yql/dq/actors/protos/dq_stats.proto b/ydb/library/yql/dq/actors/protos/dq_stats.proto index d03a02c31c2d..074e337f2f02 100644 --- a/ydb/library/yql/dq/actors/protos/dq_stats.proto +++ b/ydb/library/yql/dq/actors/protos/dq_stats.proto @@ -195,6 +195,14 @@ message TDqTaskStats { repeated TDqOutputChannelStats OutputChannels = 153; repeated TDqAsyncInputBufferStats InputTransforms = 155; + uint64 SpillingComputeWriteBytes = 160; + uint64 SpillingChannelWriteBytes = 161; + + uint64 SpillingComputeReadTimeUs = 162; + uint64 SpillingComputeWriteTimeUs = 163; + uint64 SpillingChannelReadTimeUs = 164; + uint64 SpillingChannelWriteTimeUs = 165; + // profile stats repeated TDqTableStats Tables = 10; diff --git a/ydb/library/yql/dq/actors/spilling/channel_storage.cpp b/ydb/library/yql/dq/actors/spilling/channel_storage.cpp index 79ca0fdae147..3d989f2d2c90 100644 --- a/ydb/library/yql/dq/actors/spilling/channel_storage.cpp +++ b/ydb/library/yql/dq/actors/spilling/channel_storage.cpp @@ -30,10 +30,11 @@ class TDqChannelStorage : public IDqChannelStorage { NThreading::TFuture IsBlobWrittenFuture_; }; public: - TDqChannelStorage(TTxId txId, ui64 channelId, TWakeUpCallback&& wakeUp, TActorSystem* actorSystem) + TDqChannelStorage(TTxId txId, ui64 channelId, TWakeUpCallback&& wakeUpCallback, TErrorCallback&& errorCallback, + TIntrusivePtr spillingTaskCounters, TActorSystem* actorSystem) : ActorSystem_(actorSystem) { - ChannelStorageActor_ = CreateDqChannelStorageActor(txId, channelId, std::move(wakeUp), actorSystem); + ChannelStorageActor_ = CreateDqChannelStorageActor(txId, channelId, std::move(wakeUpCallback), std::move(errorCallback), spillingTaskCounters, actorSystem); ChannelStorageActorId_ = ActorSystem_->Register(ChannelStorageActor_->GetActor()); } @@ -119,9 +120,14 @@ class TDqChannelStorage : public IDqChannelStorage { } // anonymous namespace -IDqChannelStorage::TPtr CreateDqChannelStorage(TTxId txId, ui64 channelId, IDqChannelStorage::TWakeUpCallback wakeUp, TActorSystem* actorSystem) + +IDqChannelStorage::TPtr CreateDqChannelStorage(TTxId txId, ui64 channelId, + TWakeUpCallback wakeUpCallback, + TErrorCallback errorCallback, + TIntrusivePtr spillingTaskCounters, + TActorSystem* actorSystem) { - return new TDqChannelStorage(txId, channelId, std::move(wakeUp), actorSystem); + return new TDqChannelStorage(txId, channelId, std::move(wakeUpCallback), std::move(errorCallback), spillingTaskCounters, actorSystem); } } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/channel_storage.h b/ydb/library/yql/dq/actors/spilling/channel_storage.h index 03ce1b8f6f53..909c0dce6bd0 100644 --- a/ydb/library/yql/dq/actors/spilling/channel_storage.h +++ b/ydb/library/yql/dq/actors/spilling/channel_storage.h @@ -1,5 +1,7 @@ #pragma once +#include "spilling_counters.h" + #include #include #include @@ -11,6 +13,9 @@ namespace NActors { namespace NYql::NDq { IDqChannelStorage::TPtr CreateDqChannelStorage(TTxId txId, ui64 channelId, - IDqChannelStorage::TWakeUpCallback wakeUpCb, NActors::TActorSystem* actorSystem); + TWakeUpCallback wakeUpCallback, + TErrorCallback errorCallback, + TIntrusivePtr spillingTaskCounters, + NActors::TActorSystem* actorSystem); } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/channel_storage_actor.cpp b/ydb/library/yql/dq/actors/spilling/channel_storage_actor.cpp index 93f707b21544..d050d54ac50d 100644 --- a/ydb/library/yql/dq/actors/spilling/channel_storage_actor.cpp +++ b/ydb/library/yql/dq/actors/spilling/channel_storage_actor.cpp @@ -41,12 +41,26 @@ class TDqChannelStorageActor : public IDqChannelStorageActor, public NActors::TActorBootstrapped { using TBase = TActorBootstrapped; + + struct TWritingBlobInfo { + ui64 Size; + NThreading::TPromise SavePromise; + TInstant OpBegin; + }; + + struct TLoadingBlobInfo { + NThreading::TPromise BlobPromise; + TInstant OpBegin; + }; public: - TDqChannelStorageActor(TTxId txId, ui64 channelId, IDqChannelStorage::TWakeUpCallback&& wakeUp, TActorSystem* actorSystem) + TDqChannelStorageActor(TTxId txId, ui64 channelId, TWakeUpCallback&& wakeUpCallback, TErrorCallback&& errorCallback, + TIntrusivePtr spillingTaskCounters, TActorSystem* actorSystem) : TxId_(txId) , ChannelId_(channelId) - , WakeUp_(std::move(wakeUp)) + , WakeUpCallback_(std::move(wakeUpCallback)) + , ErrorCallback_(std::move(errorCallback)) + , SpillingTaskCounters_(spillingTaskCounters) , ActorSystem_(actorSystem) {} @@ -63,6 +77,21 @@ class TDqChannelStorageActor : public IDqChannelStorageActor, return this; } +protected: + void FailWithError(const TString& error) { + if (!ErrorCallback_) Y_ABORT("Error: %s", error.c_str()); + + LOG_E("Error: " << error); + ErrorCallback_(TStringBuilder() << "[Channel spilling]" << error); + SendInternal(SpillingActorId_, new TEvents::TEvPoison); + PassAway(); + } + + void SendInternal(const TActorId& recipient, IEventBase* ev, TEventFlags flags = IEventHandle::FlagTrackDelivery) { + bool isSent = Send(recipient, ev, flags); + Y_ABORT_UNLESS(isSent, "Event was not sent"); + } + private: STATEFN(WorkState) { @@ -80,22 +109,30 @@ class TDqChannelStorageActor : public IDqChannelStorageActor, } } + + void HandleWork(TEvDqChannelSpilling::TEvGet::TPtr& ev) { auto& msg = *ev->Get(); LOG_T("[TEvGet] blobId: " << msg.BlobId_); + + auto opBegin = TInstant::Now(); - LoadingBlobs_.emplace(msg.BlobId_, std::move(msg.Promise_)); + auto loadingBlobInfo = TLoadingBlobInfo{std::move(msg.Promise_), opBegin}; + LoadingBlobs_.emplace(msg.BlobId_, std::move(loadingBlobInfo)); - Send(SpillingActorId_, new TEvDqSpilling::TEvRead(msg.BlobId_)); + SendInternal(SpillingActorId_, new TEvDqSpilling::TEvRead(msg.BlobId_)); } void HandleWork(TEvDqChannelSpilling::TEvPut::TPtr& ev) { auto& msg = *ev->Get(); LOG_T("[TEvPut] blobId: " << msg.BlobId_); - WritingBlobs_.emplace(msg.BlobId_, std::move(msg.Promise_)); + auto opBegin = TInstant::Now(); + + auto writingBlobInfo = TWritingBlobInfo{msg.Blob_.size(), std::move(msg.Promise_), opBegin}; + WritingBlobs_.emplace(msg.BlobId_, std::move(writingBlobInfo)); - Send(SpillingActorId_, new TEvDqSpilling::TEvWrite(msg.BlobId_, std::move(msg.Blob_))); + SendInternal(SpillingActorId_, new TEvDqSpilling::TEvWrite(msg.BlobId_, std::move(msg.Blob_))); } void HandleWork(TEvDqSpilling::TEvWriteResult::TPtr& ev) { @@ -104,19 +141,22 @@ class TDqChannelStorageActor : public IDqChannelStorageActor, const auto it = WritingBlobs_.find(msg.BlobId); if (it == WritingBlobs_.end()) { - LOG_E("Got unexpected TEvWriteResult, blobId: " << msg.BlobId); - - Error_ = "Internal error"; - - Send(SpillingActorId_, new TEvents::TEvPoison); + FailWithError(TStringBuilder() << "[TEvWriteResult] Got unexpected TEvWriteResult, blobId: " << msg.BlobId); return; } + auto& blobInfo = it->second; + + if (SpillingTaskCounters_) { + SpillingTaskCounters_->ChannelWriteBytes += blobInfo.Size; + auto opDuration = TInstant::Now() - blobInfo.OpBegin; + SpillingTaskCounters_->ChannelWriteTime += opDuration.MilliSeconds(); + } // Complete the future - it->second.SetValue(); + blobInfo.SavePromise.SetValue(); WritingBlobs_.erase(it); - WakeUp_(); + WakeUpCallback_(); } void HandleWork(TEvDqSpilling::TEvReadResult::TPtr& ev) { @@ -125,29 +165,30 @@ class TDqChannelStorageActor : public IDqChannelStorageActor, const auto it = LoadingBlobs_.find(msg.BlobId); if (it == LoadingBlobs_.end()) { - LOG_E("Got unexpected TEvReadResult, blobId: " << msg.BlobId); + FailWithError(TStringBuilder() << "[TEvReadResult] Got unexpected TEvReadResult, blobId: " << msg.BlobId); + return; + } - Error_ = "Internal error"; + auto& blobInfo = it->second; - Send(SpillingActorId_, new TEvents::TEvPoison); - return; + if (SpillingTaskCounters_) { + auto opDuration = TInstant::Now() - blobInfo.OpBegin; + SpillingTaskCounters_->ChannelReadTime += opDuration.MilliSeconds(); } - it->second.SetValue(std::move(msg.Blob)); + blobInfo.BlobPromise.SetValue(std::move(msg.Blob)); LoadingBlobs_.erase(it); - WakeUp_(); + WakeUpCallback_(); } void HandleWork(TEvDqSpilling::TEvError::TPtr& ev) { auto& msg = *ev->Get(); - LOG_D("[TEvError] " << msg.Message); - - Error_.ConstructInPlace(msg.Message); + FailWithError(TStringBuilder() << "[TEvError] " << msg.Message); } void PassAway() override { - Send(SpillingActorId_, new TEvents::TEvPoison); + SendInternal(SpillingActorId_, new TEvents::TEvPoison); TBase::PassAway(); } @@ -155,23 +196,30 @@ class TDqChannelStorageActor : public IDqChannelStorageActor, private: const TTxId TxId_; const ui64 ChannelId_; - IDqChannelStorage::TWakeUpCallback WakeUp_; + + TWakeUpCallback WakeUpCallback_; + TErrorCallback ErrorCallback_; + TIntrusivePtr SpillingTaskCounters_; TActorId SpillingActorId_; - // BlobId -> promise that blob is saved - std::unordered_map> WritingBlobs_; + // BlobId -> blob size + promise that blob is saved + std::unordered_map WritingBlobs_; // BlobId -> promise with requested blob - std::unordered_map> LoadingBlobs_; - TMaybe Error_; + std::unordered_map LoadingBlobs_; TActorSystem* ActorSystem_; }; } // anonymous namespace -IDqChannelStorageActor* CreateDqChannelStorageActor(TTxId txId, ui64 channelId, IDqChannelStorage::TWakeUpCallback&& wakeUp, NActors::TActorSystem* actorSystem) { - return new TDqChannelStorageActor(txId, channelId, std::move(wakeUp), actorSystem); +IDqChannelStorageActor* CreateDqChannelStorageActor(TTxId txId, ui64 channelId, + TWakeUpCallback&& wakeUpCallback, + TErrorCallback&& errorCallback, + TIntrusivePtr spillingTaskCounters, + NActors::TActorSystem* actorSystem) +{ + return new TDqChannelStorageActor(txId, channelId, std::move(wakeUpCallback), std::move(errorCallback), spillingTaskCounters, actorSystem); } } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/channel_storage_actor.h b/ydb/library/yql/dq/actors/spilling/channel_storage_actor.h index 7b509af38a6f..3d7dfb6536d0 100644 --- a/ydb/library/yql/dq/actors/spilling/channel_storage_actor.h +++ b/ydb/library/yql/dq/actors/spilling/channel_storage_actor.h @@ -1,3 +1,5 @@ +#include "spilling_counters.h" + #include #include "ydb/library/yql/dq/common/dq_common.h" @@ -49,6 +51,8 @@ class IDqChannelStorageActor virtual NActors::IActor* GetActor() = 0; }; -IDqChannelStorageActor* CreateDqChannelStorageActor(TTxId txId, ui64 channelId, IDqChannelStorage::TWakeUpCallback&& wakeUp, NActors::TActorSystem* actorSystem); + +IDqChannelStorageActor* CreateDqChannelStorageActor(TTxId txId, ui64 channelId, TWakeUpCallback&& wakeUpCallback, TErrorCallback&& errorCallback, + TIntrusivePtr spillingTaskCounters, NActors::TActorSystem* actorSystem); } // namespace NYql::NDq \ No newline at end of file diff --git a/ydb/library/yql/dq/actors/spilling/compute_storage.cpp b/ydb/library/yql/dq/actors/spilling/compute_storage.cpp index cba96b115a69..05361bc01497 100644 --- a/ydb/library/yql/dq/actors/spilling/compute_storage.cpp +++ b/ydb/library/yql/dq/actors/spilling/compute_storage.cpp @@ -6,10 +6,11 @@ namespace NYql::NDq { using namespace NActors; -TDqComputeStorage::TDqComputeStorage(TTxId txId, std::function wakeUpCallback, TActorSystem* actorSystem) : ActorSystem_(actorSystem) { +TDqComputeStorage::TDqComputeStorage(TTxId txId, TWakeUpCallback wakeUpCallback, TErrorCallback errorCallback, + TIntrusivePtr spillingTaskCounters, TActorSystem* actorSystem) : ActorSystem_(actorSystem) { TStringStream spillerName; spillerName << "Spiller" << "_" << CreateGuidAsString(); - ComputeStorageActor_ = CreateDqComputeStorageActor(txId, spillerName.Str(), wakeUpCallback); + ComputeStorageActor_ = CreateDqComputeStorageActor(txId, spillerName.Str(), wakeUpCallback, errorCallback, spillingTaskCounters); ComputeStorageActorId_ = ActorSystem_->Register(ComputeStorageActor_->GetActor()); } diff --git a/ydb/library/yql/dq/actors/spilling/compute_storage.h b/ydb/library/yql/dq/actors/spilling/compute_storage.h index 695770bfef29..1a728a04177c 100644 --- a/ydb/library/yql/dq/actors/spilling/compute_storage.h +++ b/ydb/library/yql/dq/actors/spilling/compute_storage.h @@ -16,8 +16,8 @@ namespace NYql::NDq { class TDqComputeStorage : public NKikimr::NMiniKQL::ISpiller { public: - - TDqComputeStorage(TTxId txId, std::function wakeUpCallback, NActors::TActorSystem* actorSystem); + TDqComputeStorage(TTxId txId, TWakeUpCallback wakeUpCallback, TErrorCallback errorCallback, + TIntrusivePtr spillingTaskCounters, NActors::TActorSystem* actorSystem); ~TDqComputeStorage(); diff --git a/ydb/library/yql/dq/actors/spilling/compute_storage_actor.cpp b/ydb/library/yql/dq/actors/spilling/compute_storage_actor.cpp index 8989f9da34c6..089b16cc6f91 100644 --- a/ydb/library/yql/dq/actors/spilling/compute_storage_actor.cpp +++ b/ydb/library/yql/dq/actors/spilling/compute_storage_actor.cpp @@ -33,17 +33,27 @@ class TDqComputeStorageActor : public NActors::TActorBootstrapped; - // size + promise with key - using TWritingBlobInfo = std::pair>; - // remove after read + promise with blob - using TLoadingBlobInfo = std::pair>>; + struct TWritingBlobInfo { + ui64 Size; + NThreading::TPromise BlobIdPromise; + TInstant OpBegin; + }; + + struct TLoadingBlobInfo { + bool RemoveAfterRead; + NThreading::TPromise> BlobPromise; + TInstant OpBegin; + }; // void promise that completes when block is removed using TDeletingBlobInfo = NThreading::TPromise; public: - TDqComputeStorageActor(TTxId txId, const TString& spillerName, std::function wakeupCallback) + TDqComputeStorageActor(TTxId txId, const TString& spillerName, TWakeUpCallback wakeupCallback, TErrorCallback errorCallback, + TIntrusivePtr spillingTaskCounters) : TxId_(txId), SpillerName_(spillerName), - WakeupCallback_(wakeupCallback) + WakeupCallback_(wakeupCallback), + ErrorCallback_(errorCallback), + SpillingTaskCounters_(spillingTaskCounters) { } @@ -60,14 +70,19 @@ class TDqComputeStorageActor : public NActors::TActorBootstrappedGet(); ui64 size = msg.Blob_.size(); + auto opBegin = TInstant::Now(); - Send(SpillingActorId_, new TEvDqSpilling::TEvWrite(NextBlobId, std::move(msg.Blob_))); + SendInternal(SpillingActorId_, new TEvDqSpilling::TEvWrite(NextBlobId, std::move(msg.Blob_))); - WritingBlobs_.emplace(NextBlobId, std::make_pair(size, std::move(msg.Promise_))); + auto writingBlobInfo = TWritingBlobInfo{size, std::move(msg.Promise_), opBegin}; + WritingBlobs_.emplace(NextBlobId, writingBlobInfo); WritingBlobsSize_ += size; ++NextBlobId; @@ -106,6 +123,7 @@ class TDqComputeStorageActor : public NActors::TActorBootstrappedGet(); + auto opBegin = TInstant::Now(); if (!StoredBlobs_.contains(msg.Key_)) { msg.Promise_.SetValue(std::nullopt); @@ -114,10 +132,10 @@ class TDqComputeStorageActor : public NActors::TActorBootstrappedsecond; + auto& blobInfo = it->second; - WritingBlobsSize_ -= size; + WritingBlobsSize_ -= blobInfo.Size; StoredBlobsCount_++; - StoredBlobsSize_ += size; + StoredBlobsSize_ += blobInfo.Size; + if (SpillingTaskCounters_) { + SpillingTaskCounters_->ComputeWriteBytes += blobInfo.Size; + auto opDuration = TInstant::Now() - blobInfo.OpBegin; + SpillingTaskCounters_->ComputeWriteTime += opDuration.MilliSeconds(); + } // complete future and wake up waiting compute node + auto& promise = blobInfo.BlobIdPromise; promise.SetValue(msg.BlobId); StoredBlobs_.emplace(msg.BlobId); @@ -177,22 +197,24 @@ class TDqComputeStorageActor : public NActors::TActorBootstrappedsecond; - Send(SpillingActorId_, new TEvents::TEvPoison); - return; + if (SpillingTaskCounters_) { + auto opDuration = TInstant::Now() - blobInfo.OpBegin; + SpillingTaskCounters_->ComputeReadTime += opDuration.MilliSeconds(); } - bool removedAfterRead = it->second.first; - if (removedAfterRead) { + if (blobInfo.RemoveAfterRead) { UpdateStatsAfterBlobDeletion(msg.Blob.Size(), msg.BlobId); } TRope res(TString(reinterpret_cast(msg.Blob.Data()), msg.Blob.Size())); - auto& promise = it->second.second; + auto& promise = blobInfo.BlobPromise; promise.SetValue(std::move(res)); LoadingBlobs_.erase(it); @@ -202,9 +224,7 @@ class TDqComputeStorageActor : public NActors::TActorBootstrappedGet(); - LOG_D("[TEvError] " << msg.Message); - - Error_.ConstructInPlace(msg.Message); + FailWithError(TStringBuilder() << "[TEvError] " << msg.Message); } bool HandleDelete(TKey blobId, ui64 size) { @@ -227,7 +247,7 @@ class TDqComputeStorageActor : public NActors::TActorBootstrapped DeletingBlobs_; - TMaybe Error_; - TKey NextBlobId = 0; TString SpillerName_; bool IsInitialized_ = false; - - std::function WakeupCallback_; + TWakeUpCallback WakeupCallback_; + TErrorCallback ErrorCallback_; + TIntrusivePtr SpillingTaskCounters_; TSet StoredBlobs_; - }; } // anonymous namespace -IDqComputeStorageActor* CreateDqComputeStorageActor(TTxId txId, const TString& spillerName, std::function wakeupCallback) { - return new TDqComputeStorageActor(txId, spillerName, wakeupCallback); +IDqComputeStorageActor* CreateDqComputeStorageActor(TTxId txId, const TString& spillerName, TWakeUpCallback wakeupCallback, + TErrorCallback errorCallback, TIntrusivePtr spillingTaskCounters) { + return new TDqComputeStorageActor(txId, spillerName, wakeupCallback, errorCallback, spillingTaskCounters); } } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/compute_storage_actor.h b/ydb/library/yql/dq/actors/spilling/compute_storage_actor.h index 5c680a54bffe..25bc30c6f8eb 100644 --- a/ydb/library/yql/dq/actors/spilling/compute_storage_actor.h +++ b/ydb/library/yql/dq/actors/spilling/compute_storage_actor.h @@ -1,5 +1,7 @@ #pragma once +#include "spilling_counters.h" + #include "ydb/library/yql/dq/common/dq_common.h" #include @@ -61,6 +63,7 @@ struct TEvDelete : NActors::TEventLocal Promise_; }; -IDqComputeStorageActor* CreateDqComputeStorageActor(TTxId txId, const TString& spillerName, std::function wakeupCallback); +IDqComputeStorageActor* CreateDqComputeStorageActor(TTxId txId, const TString& spillerName, TWakeUpCallback wakeupCallback, + TErrorCallback errorCallback, TIntrusivePtr spillingTaskCounters); } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/spiller_factory.h b/ydb/library/yql/dq/actors/spilling/spiller_factory.h index b8c0bc009041..052c5f0146b3 100644 --- a/ydb/library/yql/dq/actors/spilling/spiller_factory.h +++ b/ydb/library/yql/dq/actors/spilling/spiller_factory.h @@ -11,21 +11,28 @@ using namespace NActors; class TDqSpillerFactory : public NKikimr::NMiniKQL::ISpillerFactory { public: - TDqSpillerFactory(TTxId txId, TActorSystem* actorSystem, std::function wakeUpCallback) + TDqSpillerFactory(TTxId txId, TActorSystem* actorSystem, TWakeUpCallback wakeUpCallback, TErrorCallback errorCallback) : ActorSystem_(actorSystem), TxId_(txId), - WakeUpCallback_(wakeUpCallback) + WakeUpCallback_(wakeUpCallback), + ErrorCallback_(errorCallback) { } + void SetTaskCounters(TIntrusivePtr spillingTaskCounters) override { + SpillingTaskCounters_ = spillingTaskCounters; + } + NKikimr::NMiniKQL::ISpiller::TPtr CreateSpiller() override { - return std::make_shared(TxId_, WakeUpCallback_, ActorSystem_); + return std::make_shared(TxId_, WakeUpCallback_, ErrorCallback_, SpillingTaskCounters_, ActorSystem_); } private: TActorSystem* ActorSystem_; TTxId TxId_; - std::function WakeUpCallback_; + TWakeUpCallback WakeUpCallback_; + TErrorCallback ErrorCallback_; + TIntrusivePtr SpillingTaskCounters_; }; } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/spilling_counters.h b/ydb/library/yql/dq/actors/spilling/spilling_counters.h index 4122c13e7eb1..1672ac654afb 100644 --- a/ydb/library/yql/dq/actors/spilling/spilling_counters.h +++ b/ydb/library/yql/dq/actors/spilling/spilling_counters.h @@ -19,4 +19,14 @@ struct TSpillingCounters : public TThrRefBase { ::NMonitoring::TDynamicCounters::TCounterPtr SpillingIoErrors; }; +struct TSpillingTaskCounters : public TThrRefBase { + std::atomic ComputeWriteBytes = 0; + std::atomic ChannelWriteBytes = 0; + + std::atomic ComputeReadTime = 0; + std::atomic ComputeWriteTime = 0; + std::atomic ChannelReadTime = 0; + std::atomic ChannelWriteTime = 0; +}; + } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/actors/spilling/spilling_file.cpp b/ydb/library/yql/dq/actors/spilling/spilling_file.cpp index 9975e7d7f21e..49b8fba0781f 100644 --- a/ydb/library/yql/dq/actors/spilling/spilling_file.cpp +++ b/ydb/library/yql/dq/actors/spilling/spilling_file.cpp @@ -1,6 +1,7 @@ #include "spilling.h" #include "spilling_file.h" +#include #include #include @@ -13,6 +14,11 @@ #include #include #include +#include +#include +#include +#include +#include namespace NYql::NDq { @@ -159,6 +165,7 @@ class TDqLocalFileSpillingService : public TActorBootstrapped Error; }; + + struct TEvRemoveOldTmp : public TEventLocal { + TFsPath TmpRoot; + ui32 NodeId; + TString SpillingSessionId; + + TEvRemoveOldTmp(TFsPath tmpRoot, ui32 nodeId, TString spillingSessionId) + : TmpRoot(std::move(tmpRoot)), NodeId(nodeId), SpillingSessionId(std::move(spillingSessionId)) {} + }; }; struct TFileDesc; @@ -206,8 +222,11 @@ class TDqLocalFileSpillingService : public TActorBootstrapped(rootToRemoveOldTmp, nodeId, sessionId)); Become(&TDqLocalFileSpillingService::WorkState); } @@ -271,6 +292,7 @@ class TDqLocalFileSpillingService : public TActorBootstrappedSender << ", blobId: " << msg.BlobId << ", bytes: " << msg.Blob.size()); - Send(ev->Sender, new TEvDqSpilling::TEvError("File size limit exceeded")); + const auto usedMb = (fd.TotalSize + msg.Blob.size()) / 1024 / 1024; + const auto limitMb = Config_.MaxFileSize / 1024 / 1024; + + Send(ev->Sender, new TEvDqSpilling::TEvError(std::format("File size limit exceeded: {}/{}Mb", usedMb, limitMb))); Counters_->SpillingTooBigFileErrors->Inc(); return; @@ -395,7 +420,9 @@ class TDqLocalFileSpillingService : public TActorBootstrappedSender << ", blobId: " << msg.BlobId << ", bytes: " << msg.Blob.size()); - Send(ev->Sender, new TEvDqSpilling::TEvError("Total size limit exceeded")); + const auto usedMb = (TotalSize_ + msg.Blob.size()) / 1024 / 1024; + const auto limitMb = Config_.MaxTotalSize / 1024 / 1024; + Send(ev->Sender, new TEvDqSpilling::TEvError(std::format("Total size limit exceeded: {}/{}Mb", usedMb, limitMb))); Counters_->SpillingNoSpaceErrors->Inc(); return; @@ -712,6 +739,50 @@ class TDqLocalFileSpillingService : public TActorBootstrappedSender, new NMon::TEvHttpInfoRes(s.Str())); } + void HandleWork(TEvPrivate::TEvRemoveOldTmp::TPtr& ev) { + const auto& msg = *ev->Get(); + const auto& root = msg.TmpRoot; + const auto nodeIdString = ToString(msg.NodeId); + const auto& sessionId = msg.SpillingSessionId; + const auto& nodePrefix = this->NodePrefix_; + + LOG_I("[RemoveOldTmp] removing at root: " << root); + + const auto isDirOldTmp = [&nodePrefix, &nodeIdString, &sessionId](const TString& dirName) -> bool { + // dirName: node__ + TVector parts; + StringSplitter(dirName).Split('_').Limit(3).Collect(&parts); + + if (parts.size() < 3) { + return false; + } + return parts[0] == nodePrefix && parts[1] == nodeIdString && parts[2] != sessionId; + }; + + try { + TDirIterator iter(root, TDirIterator::TOptions().SetMaxLevel(1)); + + TVector oldTmps; + for (const auto& dirEntry : iter) { + if (dirEntry.fts_info == FTS_DP) { + continue; + } + + const auto dirName = dirEntry.fts_name; + if (isDirOldTmp(dirName)) { + LOG_D("[RemoveOldTmp] found old temporary at " << (root / dirName)); + oldTmps.emplace_back(std::move(dirName)); + } + } + + for (const auto& dirName : oldTmps) { + (root / dirName).ForceDelete(); + } + } catch (const yexception& e) { + LOG_E("[RemoveOldTmp] removing failed due to: " << e.what()); + } + } + private: void RunOp(TStringBuf opName, THolder op, TFileDesc& fd) { if (fd.HasActiveOp) { @@ -941,6 +1012,7 @@ class TDqLocalFileSpillingService : public TActorBootstrapped Counters_; @@ -952,6 +1024,12 @@ class TDqLocalFileSpillingService : public TActorBootstrapped #include +#include +#include namespace NYql::NDq { struct TFileSpillingServiceConfig { TString Root; + TString SpillingSessionId = CreateGuidAsString(); ui64 MaxTotalSize = 0; ui64 MaxFileSize = 0; ui64 MaxFilePartSize = 0; @@ -26,6 +29,8 @@ inline NActors::TActorId MakeDqLocalFileSpillingServiceID(ui32 nodeId) { return NActors::TActorId(nodeId, TStringBuf(name, 12)); } +TFsPath GetTmpSpillingRootForCurrentUser(); + NActors::IActor* CreateDqLocalFileSpillingActor(TTxId txId, const TString& details, const NActors::TActorId& client, bool removeBlobsAfterRead); NActors::IActor* CreateDqLocalFileSpillingService(const TFileSpillingServiceConfig& config, TIntrusivePtr counters); diff --git a/ydb/library/yql/dq/actors/spilling/spilling_file_ut.cpp b/ydb/library/yql/dq/actors/spilling/spilling_file_ut.cpp index fc98fac66318..cdf7a31cd8f8 100644 --- a/ydb/library/yql/dq/actors/spilling/spilling_file_ut.cpp +++ b/ydb/library/yql/dq/actors/spilling/spilling_file_ut.cpp @@ -48,13 +48,19 @@ class TTestActorRuntime: public TTestActorRuntimeBase { return str; } + const TString& GetSpillingSessionId() const { + return SpillingSessionId_; + } + TActorId StartSpillingService(ui64 maxTotalSize = 1000, ui64 maxFileSize = 500, ui64 maxFilePartSize = 100, const TFsPath& root = TFsPath::Cwd() / GetSpillingPrefix()) { SpillingRoot_ = root; + SpillingSessionId_ = CreateGuidAsString(); auto config = TFileSpillingServiceConfig{ .Root = root.GetPath(), + .SpillingSessionId = SpillingSessionId_, .MaxTotalSize = maxTotalSize, .MaxFileSize = maxFileSize, .MaxFilePartSize = maxFilePartSize @@ -91,6 +97,7 @@ class TTestActorRuntime: public TTestActorRuntimeBase { private: TFsPath SpillingRoot_; + TString SpillingSessionId_; }; TBuffer CreateBlob(ui32 size, char symbol) { @@ -263,7 +270,7 @@ Y_UNIT_TEST_SUITE(DqSpillingFileTests) { runtime.Send(new IEventHandle(spillingActor, tester, ev)); auto resp = runtime.GrabEdgeEvent(tester); - UNIT_ASSERT_STRINGS_EQUAL("Total size limit exceeded", resp->Get()->Message); + UNIT_ASSERT_STRINGS_EQUAL("Total size limit exceeded: 0/0Mb", resp->Get()->Message); } } @@ -290,7 +297,7 @@ Y_UNIT_TEST_SUITE(DqSpillingFileTests) { runtime.Send(new IEventHandle(spillingActor, tester, ev)); auto resp = runtime.GrabEdgeEvent(tester); - UNIT_ASSERT_STRINGS_EQUAL("File size limit exceeded", resp->Get()->Message); + UNIT_ASSERT_STRINGS_EQUAL("File size limit exceeded: 0/0Mb", resp->Get()->Message); } } @@ -303,8 +310,7 @@ Y_UNIT_TEST_SUITE(DqSpillingFileTests) { auto spillingActor = runtime.StartSpillingActor(tester); runtime.WaitBootstrap(); - - const TString filePrefix = TStringBuilder() << runtime.GetSpillingRoot().GetPath() << "/node_" << runtime.GetNodeId() << "/1_test_"; + const TString filePrefix = TStringBuilder() << runtime.GetSpillingRoot().GetPath() << "/node_" << runtime.GetNodeId() << "_" << runtime.GetSpillingSessionId() << "/1_test_"; for (ui32 i = 0; i < 5; ++i) { // Cerr << "---- store blob #" << i << Endl; @@ -346,7 +352,7 @@ Y_UNIT_TEST_SUITE(DqSpillingFileTests) { runtime.WaitBootstrap(); - const TString filePrefix = TStringBuilder() << runtime.GetSpillingRoot().GetPath() << "/node_" << runtime.GetNodeId() << "/1_test_"; + const TString filePrefix = TStringBuilder() << runtime.GetSpillingRoot().GetPath() << "/node_" << runtime.GetNodeId() << "_" << runtime.GetSpillingSessionId() << "/1_test_"; for (ui32 i = 0; i < 5; ++i) { // Cerr << "---- store blob #" << i << Endl; @@ -393,8 +399,7 @@ Y_UNIT_TEST_SUITE(DqSpillingFileTests) { auto resp = runtime.GrabEdgeEvent(tester); UNIT_ASSERT_VALUES_EQUAL(0, resp->Get()->BlobId); } - - auto nodePath = TFsPath("node_" + std::to_string(spillingSvc.NodeId())); + auto nodePath = TFsPath("node_" + std::to_string(spillingSvc.NodeId()) + "_" + runtime.GetSpillingSessionId()); (runtime.GetSpillingRoot() / nodePath / "1_test_0").ForceDelete(); { diff --git a/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp b/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp index 33cb50221d17..1f708c543810 100644 --- a/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp +++ b/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp @@ -86,6 +86,7 @@ class TLocalTaskRunnerActor } private: + void OnStatisticsRequest(TEvStatistics::TPtr& ev) { THashMap sinks; @@ -430,6 +431,12 @@ class TLocalTaskRunnerActor } } + if (settings.GetEnableSpilling()) { + auto wakeUpCallback = ev->Get()->ExecCtx->GetWakeupCallback(); + auto errorCallback = ev->Get()->ExecCtx->GetErrorCallback(); + TaskRunner->SetSpillerFactory(std::make_shared(TxId, NActors::TActivationContext::ActorSystem(), wakeUpCallback, errorCallback)); + } + TaskRunner->Prepare(settings, ev->Get()->MemoryLimits, *ev->Get()->ExecCtx); THashMap> inputTransforms; @@ -439,11 +446,6 @@ class TLocalTaskRunnerActor } } - if (settings.GetEnableSpilling()) { - auto wakeUpCallback = ev->Get()->ExecCtx->GetWakeupCallback(); - TaskRunner->SetSpillerFactory(std::make_shared(TxId, NActors::TActivationContext::ActorSystem(), wakeUpCallback)); - } - auto event = MakeHolder( TaskRunner->GetSecureParams(), TaskRunner->GetTaskParams(), diff --git a/ydb/library/yql/dq/common/dq_common.h b/ydb/library/yql/dq/common/dq_common.h index 557b3abaa3c1..ae29f742336d 100644 --- a/ydb/library/yql/dq/common/dq_common.h +++ b/ydb/library/yql/dq/common/dq_common.h @@ -11,6 +11,9 @@ using TTxId = std::variant; using TLogFunc = std::function; +using TWakeUpCallback = std::function; +using TErrorCallback = std::function; + template struct TBaseDqResManEvents { enum { @@ -89,6 +92,34 @@ enum class EHashJoinMode { GraceAndSelf /* "graceandself" */, }; +enum class EEnabledSpillingNodes : ui64 { + None = 0ULL /* "None" */, + GraceJoin = 1ULL /* "GraceJoin" */, + Aggregation = 2ULL /* "Aggregation" */, + All = ~0ULL /* "All" */, +}; + +class TSpillingSettings { +public: + TSpillingSettings() = default; + explicit TSpillingSettings(ui64 mask) : Mask(mask) {}; + + operator bool() const { + return Mask; + } + + bool IsGraceJoinSpillingEnabled() const { + return Mask & ui64(EEnabledSpillingNodes::GraceJoin); + } + + bool IsAggregationSpillingEnabled() const { + return Mask & ui64(EEnabledSpillingNodes::Aggregation); + } + +private: + const ui64 Mask = 0; +}; + } // namespace NYql::NDq IOutputStream& operator<<(IOutputStream& stream, const NYql::NDq::TTxId& txId); diff --git a/ydb/library/yql/dq/opt/dq_opt_log.cpp b/ydb/library/yql/dq/opt/dq_opt_log.cpp index 467fec490c9a..a0ad02b81860 100644 --- a/ydb/library/yql/dq/opt/dq_opt_log.cpp +++ b/ydb/library/yql/dq/opt/dq_opt_log.cpp @@ -17,12 +17,13 @@ using namespace NYql::NNodes; namespace NYql::NDq { TExprBase DqRewriteAggregate(TExprBase node, TExprContext& ctx, TTypeAnnotationContext& typesCtx, bool compactForDistinct, - bool usePhases, const bool useFinalizeByKey) + bool usePhases, const bool useFinalizeByKey, const bool allowSpilling) { if (!node.Maybe()) { return node; } - TAggregateExpander aggExpander(!typesCtx.IsBlockEngineEnabled() && !useFinalizeByKey, useFinalizeByKey, node.Ptr(), ctx, typesCtx, false, compactForDistinct, usePhases); + TAggregateExpander aggExpander(!typesCtx.IsBlockEngineEnabled() && !useFinalizeByKey, + useFinalizeByKey, node.Ptr(), ctx, typesCtx, false, compactForDistinct, usePhases, allowSpilling); auto result = aggExpander.ExpandAggregate(); YQL_ENSURE(result); diff --git a/ydb/library/yql/dq/opt/dq_opt_log.h b/ydb/library/yql/dq/opt/dq_opt_log.h index 560b2be62058..99c735a2f132 100644 --- a/ydb/library/yql/dq/opt/dq_opt_log.h +++ b/ydb/library/yql/dq/opt/dq_opt_log.h @@ -19,7 +19,8 @@ namespace NYql { namespace NYql::NDq { -NNodes::TExprBase DqRewriteAggregate(NNodes::TExprBase node, TExprContext& ctx, TTypeAnnotationContext& typesCtx, bool compactForDistinct, bool usePhases, const bool useFinalizeByKey); +NNodes::TExprBase DqRewriteAggregate(NNodes::TExprBase node, TExprContext& ctx, TTypeAnnotationContext& typesCtx, + bool compactForDistinct, bool usePhases, const bool useFinalizeByKey, const bool allowSpilling); NNodes::TExprBase DqRewriteTakeSortToTopSort(NNodes::TExprBase node, TExprContext& ctx, const TParentsMap& parents); diff --git a/ydb/library/yql/dq/runtime/dq_channel_storage.h b/ydb/library/yql/dq/runtime/dq_channel_storage.h index eaafab318052..62dcdebdb80f 100644 --- a/ydb/library/yql/dq/runtime/dq_channel_storage.h +++ b/ydb/library/yql/dq/runtime/dq_channel_storage.h @@ -15,8 +15,6 @@ class IDqChannelStorage : public TSimpleRefCount { public: using TPtr = TIntrusivePtr; - using TWakeUpCallback = std::function; - public: virtual ~IDqChannelStorage() = default; diff --git a/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp b/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp index 2c04e33985b0..6f909793c121 100644 --- a/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp +++ b/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp @@ -1,5 +1,6 @@ #include "dq_tasks_runner.h" +#include #include #include @@ -269,7 +270,7 @@ class TDqTaskRunner : public IDqTaskRunner { } void SetSpillerFactory(std::shared_ptr spillerFactory) override { - AllocatedHolder->ProgramParsed.CompGraph->GetContext().SpillerFactory = std::move(spillerFactory); + SpillerFactory = spillerFactory; } bool UseSeparatePatternAlloc(const TDqTaskSettings& taskSettings) const { @@ -457,7 +458,7 @@ class TDqTaskRunner : public IDqTaskRunner { auto opts = CreatePatternOpts(task, Alloc(), TypeEnv()); AllocatedHolder->ProgramParsed.CompGraph = AllocatedHolder->ProgramParsed.GetPattern()->Clone( - opts.ToComputationOptions(*Context.RandomProvider, *Context.TimeProvider, &TypeEnv())); + opts.ToComputationOptions(*Context.RandomProvider, *Context.TimeProvider)); TBindTerminator term(AllocatedHolder->ProgramParsed.CompGraph->GetTerminator()); @@ -512,6 +513,12 @@ class TDqTaskRunner : public IDqTaskRunner { auto& typeEnv = TypeEnv(); + SpillingTaskCounters = execCtx.GetSpillingTaskCounters(); + if (SpillerFactory) { + SpillerFactory->SetTaskCounters(SpillingTaskCounters); + } + AllocatedHolder->ProgramParsed.CompGraph->GetContext().SpillerFactory = std::move(SpillerFactory); + for (ui32 i = 0; i < task.InputsSize(); ++i) { auto& inputDesc = task.GetInputs(i); auto& inputStats = BillingStats.AddInputs(); @@ -716,6 +723,15 @@ class TDqTaskRunner : public IDqTaskRunner { auto runStatus = FetchAndDispatch(); if (Y_UNLIKELY(CollectFull())) { + if (SpillingTaskCounters) { + Stats->SpillingComputeWriteBytes = SpillingTaskCounters->ComputeWriteBytes.load(); + Stats->SpillingChannelWriteBytes = SpillingTaskCounters->ChannelWriteBytes.load(); + Stats->SpillingComputeReadTime = TDuration::MilliSeconds(SpillingTaskCounters->ComputeReadTime.load()); + Stats->SpillingComputeWriteTime = TDuration::MilliSeconds(SpillingTaskCounters->ComputeWriteTime.load()); + Stats->SpillingChannelReadTime = TDuration::MilliSeconds(SpillingTaskCounters->ChannelReadTime.load()); + Stats->SpillingChannelWriteTime = TDuration::MilliSeconds(SpillingTaskCounters->ChannelWriteTime.load()); + } + Stats->ComputeCpuTimeByRun->Collect(RunComputeTime.MilliSeconds()); if (AllocatedHolder->ProgramParsed.StatsRegistry) { @@ -928,6 +944,9 @@ class TDqTaskRunner : public IDqTaskRunner { } private: + std::shared_ptr SpillerFactory; + TIntrusivePtr SpillingTaskCounters; + ui64 TaskId = 0; TDqTaskRunnerContext Context; TDqTaskRunnerSettings Settings; diff --git a/ydb/library/yql/dq/runtime/dq_tasks_runner.h b/ydb/library/yql/dq/runtime/dq_tasks_runner.h index 065aa97d1b76..d79ea6210748 100644 --- a/ydb/library/yql/dq/runtime/dq_tasks_runner.h +++ b/ydb/library/yql/dq/runtime/dq_tasks_runner.h @@ -49,6 +49,14 @@ struct TTaskRunnerStatsBase { TDuration WaitInputTime; TDuration WaitOutputTime; + ui64 SpillingComputeWriteBytes; + ui64 SpillingChannelWriteBytes; + + TDuration SpillingComputeReadTime; + TDuration SpillingComputeWriteTime; + TDuration SpillingChannelReadTime; + TDuration SpillingChannelWriteTime; + // profile stats NMonitoring::IHistogramCollectorPtr ComputeCpuTimeByRun; // in millis @@ -138,7 +146,9 @@ class IDqTaskRunnerExecutionContext { virtual IDqChannelStorage::TPtr CreateChannelStorage(ui64 channelId, bool withSpilling) const = 0; virtual IDqChannelStorage::TPtr CreateChannelStorage(ui64 channelId, bool withSpilling, NActors::TActorSystem* actorSystem) const = 0; - virtual std::function GetWakeupCallback() const = 0; + virtual TWakeUpCallback GetWakeupCallback() const = 0; + virtual TErrorCallback GetErrorCallback() const = 0; + virtual TIntrusivePtr GetSpillingTaskCounters() const = 0; virtual TTxId GetTxId() const = 0; }; @@ -161,7 +171,15 @@ class TDqTaskRunnerExecutionContextDefault : public TDqTaskRunnerExecutionContex return {}; }; - std::function GetWakeupCallback() const override { + TWakeUpCallback GetWakeupCallback() const override { + return {}; + } + + TErrorCallback GetErrorCallback() const override { + return {}; + } + + TIntrusivePtr GetSpillingTaskCounters() const override { return {}; } diff --git a/ydb/library/yql/dq/tasks/dq_task_program.cpp b/ydb/library/yql/dq/tasks/dq_task_program.cpp index d06a6023b3b3..1f44d561c9ea 100644 --- a/ydb/library/yql/dq/tasks/dq_task_program.cpp +++ b/ydb/library/yql/dq/tasks/dq_task_program.cpp @@ -2,6 +2,7 @@ #include #include +#include #include namespace NYql::NDq { @@ -9,6 +10,33 @@ namespace NYql::NDq { using namespace NKikimr::NMiniKQL; using namespace NYql::NNodes; + +class TSpillingTransformProvider { +public: + + TSpillingTransformProvider(const TSpillingSettings& spillingSettings): SpillingSettings(spillingSettings){}; + + TCallableVisitFunc operator()(TInternName name) { + if (RuntimeVersion >= 50U && SpillingSettings.IsGraceJoinSpillingEnabled() && (name == "GraceJoin" || name == "GraceSelfJoin")) { + return [name](NKikimr::NMiniKQL::TCallable& callable, const TTypeEnvironment& env) { + TCallableBuilder callableBuilder(env, + TStringBuilder() << callable.GetType()->GetName() << "WithSpilling", + callable.GetType()->GetReturnType(), false); + for (ui32 i = 0; i < callable.GetInputsCount(); ++i) { + callableBuilder.Add(callable.GetInput(i)); + } + return TRuntimeNode(callableBuilder.Build(), false); + }; + } + + return TCallableVisitFunc(); + } + +private: + + TSpillingSettings SpillingSettings; +}; + const TStructExprType* CollectParameters(NNodes::TCoLambda program, TExprContext& ctx) { TVector memberTypes; @@ -27,7 +55,7 @@ const TStructExprType* CollectParameters(NNodes::TCoLambda program, TExprContext TString BuildProgram(NNodes::TCoLambda program, const TStructExprType& paramsType, const NCommon::IMkqlCallableCompiler& compiler, const TTypeEnvironment& typeEnv, - const IFunctionRegistry& funcRegistry, TExprContext& exprCtx, const TVector& reads) + const IFunctionRegistry& funcRegistry, TExprContext& exprCtx, const TVector& reads, const TSpillingSettings& spillingSettings) { TProgramBuilder pgmBuilder(typeEnv, funcRegistry); @@ -49,6 +77,13 @@ TString BuildProgram(NNodes::TCoLambda program, const TStructExprType& paramsTyp TRuntimeNode rootNode = MkqlBuildExpr(program.Body().Ref(), ctx); + TExploringNodeVisitor explorer; + if (spillingSettings) { + explorer.Walk(rootNode.GetNode(), typeEnv); + bool wereChanges = false; + rootNode = SinglePassVisitCallables(rootNode, explorer, TSpillingTransformProvider(spillingSettings), typeEnv, true, wereChanges); + } + TStructLiteralBuilder structBuilder(typeEnv); structBuilder.Add("Program", rootNode); structBuilder.Add("Inputs", pgmBuilder.NewTuple(inputNodes)); @@ -64,7 +99,6 @@ TString BuildProgram(NNodes::TCoLambda program, const TStructExprType& paramsTyp auto programNode = structBuilder.Build(); - TExploringNodeVisitor explorer; explorer.Walk(programNode, typeEnv); ui32 uniqueId = 0; for (auto& node : explorer.GetNodes()) { diff --git a/ydb/library/yql/dq/tasks/dq_task_program.h b/ydb/library/yql/dq/tasks/dq_task_program.h index 9bdf0de31953..cd11a301d040 100644 --- a/ydb/library/yql/dq/tasks/dq_task_program.h +++ b/ydb/library/yql/dq/tasks/dq_task_program.h @@ -7,6 +7,8 @@ #include #include +#include + namespace NYql::NDq { const TStructExprType* CollectParameters(NNodes::TCoLambda program, TExprContext& ctx); @@ -14,6 +16,6 @@ const TStructExprType* CollectParameters(NNodes::TCoLambda program, TExprContext TString BuildProgram(NNodes::TCoLambda program, const TStructExprType& paramsType, const NCommon::IMkqlCallableCompiler& compiler, const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, const NKikimr::NMiniKQL::IFunctionRegistry& funcRegistry, TExprContext& exprCtx, - const TVector& reads); + const TVector& reads, const TSpillingSettings& spillingSettings); } // namespace NYql::NDq diff --git a/ydb/library/yql/minikql/aligned_page_pool.cpp b/ydb/library/yql/minikql/aligned_page_pool.cpp index f4ad5f3550ff..72e08113df6b 100644 --- a/ydb/library/yql/minikql/aligned_page_pool.cpp +++ b/ydb/library/yql/minikql/aligned_page_pool.cpp @@ -504,6 +504,7 @@ void TAlignedPagePoolImpl::Free(void* ptr, size_t size) noexcept { template void TAlignedPagePoolImpl::UpdateMemoryYellowZone() { if (Limit == 0) return; + if (IsMemoryYellowZoneForcefullyChanged) return; if (IncreaseMemoryLimitCallback && !IsMaximumLimitValueReached) return; ui8 usedMemoryPercent = 100 * GetUsed() / Limit; diff --git a/ydb/library/yql/minikql/aligned_page_pool.h b/ydb/library/yql/minikql/aligned_page_pool.h index 869faf22ce17..9af3799d4261 100644 --- a/ydb/library/yql/minikql/aligned_page_pool.h +++ b/ydb/library/yql/minikql/aligned_page_pool.h @@ -222,6 +222,11 @@ class TAlignedPagePoolImpl { return IsMemoryYellowZoneReached; } + void ForcefullySetMemoryYellowZone(bool isEnabled) noexcept { + IsMemoryYellowZoneReached = isEnabled; + IsMemoryYellowZoneForcefullyChanged = true; + } + protected: void* Alloc(size_t size); void Free(void* ptr, size_t size) noexcept; @@ -268,6 +273,10 @@ class TAlignedPagePoolImpl { // Indicates when memory limit is almost reached. bool IsMemoryYellowZoneReached = false; + // Indicates that memory yellow zone was enabled or disabled forcefully. + // If the value of this variable is true, then the limits specified below will not be applied and + // changing the value can only be done manually. + bool IsMemoryYellowZoneForcefullyChanged = false; // This theshold is used to determine is memory limit is almost reached. // If TIncreaseMemoryLimitCallback is set this thresholds should be ignored. // The yellow zone turns on when memory consumption reaches 80% and turns off when consumption drops below 50%. diff --git a/ydb/library/yql/minikql/comp_nodes/mkql_factory.cpp b/ydb/library/yql/minikql/comp_nodes/mkql_factory.cpp index 24d0ab217db3..bfacc68bd9d4 100644 --- a/ydb/library/yql/minikql/comp_nodes/mkql_factory.cpp +++ b/ydb/library/yql/minikql/comp_nodes/mkql_factory.cpp @@ -234,8 +234,8 @@ struct TCallableComputationNodeBuilderFuncMapFiller { {"JoinDict", &WrapJoinDict}, {"GraceJoin", &WrapGraceJoin}, {"GraceSelfJoin", &WrapGraceSelfJoin}, - {"GraceJoinWithSpilling", &WrapGraceJoinWithSpilling}, - {"GraceSelfJoinWithSpilling", &WrapGraceSelfJoinWithSpilling}, + {"GraceJoinWithSpilling", &WrapGraceJoin}, + {"GraceSelfJoinWithSpilling", &WrapGraceSelfJoin}, {"MapJoinCore", &WrapMapJoinCore}, {"CommonJoinCore", &WrapCommonJoinCore}, {"CombineCore", &WrapCombineCore}, diff --git a/ydb/library/yql/minikql/comp_nodes/mkql_grace_join.cpp b/ydb/library/yql/minikql/comp_nodes/mkql_grace_join.cpp index 4d6562b84eca..f8c071dd837b 100644 --- a/ydb/library/yql/minikql/comp_nodes/mkql_grace_join.cpp +++ b/ydb/library/yql/minikql/comp_nodes/mkql_grace_join.cpp @@ -1096,7 +1096,7 @@ class TGraceJoinWrapper : public TStatefulWideFlowCodegeneratorNode struct TKernelState : public arrow::compute::KernelState { TKernelState(const TVector& argsTypes, TType* returnType, const TComputationContext& originalContext) : Alloc(__LOCATION__) + , TypeEnv(Alloc) , MemInfo("ScalarApply") - , HolderFactory(Alloc.Ref(), MemInfo) + , FunctionRegistry(originalContext.HolderFactory.GetFunctionRegistry()->Clone()) + , HolderFactory(Alloc.Ref(), MemInfo, FunctionRegistry.Get()) , ValueBuilder(HolderFactory, NUdf::EValidatePolicy::Exception) , PgBuilder(NYql::CreatePgBuilder()) , Accessors(argsTypes, returnType, *PgBuilder) , RandomProvider(CreateDefaultRandomProvider()) , TimeProvider(CreateDefaultTimeProvider()) , Ctx(HolderFactory, &ValueBuilder, TComputationOptsFull( - nullptr, Alloc.Ref(), *RandomProvider, *TimeProvider, NUdf::EValidatePolicy::Exception, nullptr), + nullptr, Alloc.Ref(), TypeEnv, *RandomProvider, *TimeProvider, NUdf::EValidatePolicy::Exception, originalContext.SecureParamsProvider, originalContext.CountersProvider), originalContext.Mutables, *NYql::NUdf::GetYqlMemoryPool()) { Alloc.Release(); @@ -64,7 +66,9 @@ class TScalarApplyWrapper : public TMutableComputationNode } TScopedAlloc Alloc; + TTypeEnvironment TypeEnv; TMemoryUsageInfo MemInfo; + const IFunctionRegistry::TPtr FunctionRegistry; THolderFactory HolderFactory; TDefaultValueBuilder ValueBuilder; std::unique_ptr PgBuilder; diff --git a/ydb/library/yql/minikql/comp_nodes/mkql_udf.cpp b/ydb/library/yql/minikql/comp_nodes/mkql_udf.cpp index fdddf886ee48..0d8e8c1578ad 100644 --- a/ydb/library/yql/minikql/comp_nodes/mkql_udf.cpp +++ b/ydb/library/yql/minikql/comp_nodes/mkql_udf.cpp @@ -15,17 +15,73 @@ namespace NMiniKQL { namespace { -class TUdfRunCodegeneratorNode: public TUnboxedImmutableRunCodegeneratorNode { +template +class TSimpleUdfWrapper: public TMutableComputationNode> { +using TBaseComputation = TMutableComputationNode>; public: - TUdfRunCodegeneratorNode(TMemoryUsageInfo* memInfo, - NUdf::TUnboxedValue&& value, const TString& moduleIRUniqID, const TString& moduleIR, const TString& functionName) - : TUnboxedImmutableRunCodegeneratorNode(memInfo, std::move(value)) - , ModuleIRUniqID(moduleIRUniqID) - , ModuleIR(moduleIR) - , FunctionName(functionName) + TSimpleUdfWrapper( + TComputationMutables& mutables, + TString&& functionName, + TString&& typeConfig, + NUdf::TSourcePosition pos, + const TCallableType* callableType, + TType* userType) + : TBaseComputation(mutables, EValueRepresentation::Boxed) + , FunctionName(std::move(functionName)) + , TypeConfig(std::move(typeConfig)) + , Pos(pos) + , CallableType(callableType) + , UserType(userType) { + this->Stateless = false; + } + + NUdf::TUnboxedValuePod DoCalculate(TComputationContext& ctx) const { + ui32 flags = 0; + TFunctionTypeInfo funcInfo; + const auto status = ctx.HolderFactory.GetFunctionRegistry()->FindFunctionTypeInfo( + ctx.TypeEnv, ctx.TypeInfoHelper, ctx.CountersProvider, FunctionName, UserType->IsVoid() ? nullptr : UserType, + TypeConfig, flags, Pos, ctx.SecureParamsProvider, &funcInfo); + + MKQL_ENSURE(status.IsOk(), status.GetError()); + MKQL_ENSURE(funcInfo.Implementation, "UDF implementation is not set for function " << FunctionName); + NUdf::TUnboxedValue udf(NUdf::TUnboxedValuePod(funcInfo.Implementation.Release())); + TValidate::WrapCallable(CallableType, udf, TStringBuilder() << "FunctionWithConfig<" << FunctionName << ">"); + return udf.Release(); } +private: + void RegisterDependencies() const final {} + const TString FunctionName; + const TString TypeConfig; + const NUdf::TSourcePosition Pos; + const TCallableType *const CallableType; + TType *const UserType; +}; + +class TUdfRunCodegeneratorNode: public TSimpleUdfWrapper> +#ifndef MKQL_DISABLE_CODEGEN + , public ICodegeneratorRunNode +#endif +{ +public: + TUdfRunCodegeneratorNode( + TComputationMutables& mutables, + TString&& functionName, + TString&& typeConfig, + NUdf::TSourcePosition pos, + const TCallableType* callableType, + TType* userType, + TString&& moduleIRUniqID, + TString&& moduleIR, + TString&& fuctioNameIR, + NUdf::TUniquePtr&& impl) + : TSimpleUdfWrapper(mutables, std::move(functionName), std::move(typeConfig), pos, callableType, userType) + , ModuleIRUniqID(std::move(moduleIRUniqID)) + , ModuleIR(std::move(moduleIR)) + , IRFunctionName(std::move(fuctioNameIR)) + , Impl(std::move(impl)) + {} #ifndef MKQL_DISABLE_CODEGEN void CreateRun(const TCodegenContext& ctx, BasicBlock*& block, Value* result, Value* args) const final { ctx.Codegen.LoadBitCode(ModuleIR, ModuleIRUniqID); @@ -35,110 +91,148 @@ class TUdfRunCodegeneratorNode: public TUnboxedImmutableRunCodegeneratorNode { const auto type = Type::getInt128Ty(context); YQL_ENSURE(result->getType() == PointerType::getUnqual(type)); - const auto data = ConstantInt::get(Type::getInt64Ty(context), reinterpret_cast(UnboxedValue.AsBoxed().Get())); + const auto data = ConstantInt::get(Type::getInt64Ty(context), reinterpret_cast(Impl.Get())); const auto ptrStructType = PointerType::getUnqual(StructType::get(context)); const auto boxed = CastInst::Create(Instruction::IntToPtr, data, ptrStructType, "boxed", block); const auto builder = ctx.GetBuilder(); const auto funType = FunctionType::get(Type::getVoidTy(context), {boxed->getType(), result->getType(), builder->getType(), args->getType()}, false); - const auto runFunc = ctx.Codegen.GetModule().getOrInsertFunction(llvm::StringRef(FunctionName.data(), FunctionName.size()), funType); + const auto runFunc = ctx.Codegen.GetModule().getOrInsertFunction(llvm::StringRef(IRFunctionName.data(), IRFunctionName.size()), funType); CallInst::Create(runFunc, {boxed, result, builder, args}, "", block); } #endif private: const TString ModuleIRUniqID; const TString ModuleIR; - const TString FunctionName; + const TString IRFunctionName; + const NUdf::TUniquePtr Impl; }; - template class TUdfWrapper: public TMutableCodegeneratorPtrNode> { - typedef TMutableCodegeneratorPtrNode> TBaseComputation; +using TBaseComputation = TMutableCodegeneratorPtrNode>; public: TUdfWrapper( TComputationMutables& mutables, - IComputationNode* functionImpl, TString&& functionName, + TString&& typeConfig, + NUdf::TSourcePosition pos, IComputationNode* runConfigNode, - const TCallableType* callableType) + const TCallableType* callableType, + TType* userType) : TBaseComputation(mutables, EValueRepresentation::Boxed) - , FunctionImpl(functionImpl) , FunctionName(std::move(functionName)) + , TypeConfig(std::move(typeConfig)) + , Pos(pos) , RunConfigNode(runConfigNode) , CallableType(callableType) + , UserType(userType) + , UdfIndex(mutables.CurValueIndex++) { this->Stateless = false; } NUdf::TUnboxedValue DoCalculate(TComputationContext& ctx) const { + auto& udf = ctx.MutableValues[UdfIndex]; + if (!udf.HasValue()) { + MakeUdf(ctx, udf); + } const auto runConfig = RunConfigNode->GetValue(ctx); - auto callable = FunctionImpl->GetValue(ctx).Run(ctx.Builder, &runConfig); + auto callable = udf.Run(ctx.Builder, &runConfig); Wrap(callable); return callable; } - #ifndef MKQL_DISABLE_CODEGEN void DoGenerateGetValue(const TCodegenContext& ctx, Value* pointer, BasicBlock*& block) const { auto& context = ctx.Codegen.GetContext(); - GetNodeValue(pointer, RunConfigNode, ctx, block); - const auto conf = new LoadInst(Type::getInt128Ty(context), pointer, "conf", block); + const auto valueType = Type::getInt128Ty(context); + + const auto udfPtr = GetElementPtrInst::CreateInBounds(valueType, ctx.GetMutables(), {ConstantInt::get(Type::getInt32Ty(context), UdfIndex)}, "udf_ptr", block); + + const auto make = BasicBlock::Create(context, "make", ctx.Func); + const auto main = BasicBlock::Create(context, "main", ctx.Func); + + const auto ptrType = PointerType::getUnqual(StructType::get(context)); + const auto self = CastInst::Create(Instruction::IntToPtr, ConstantInt::get(Type::getInt64Ty(context), uintptr_t(this)), ptrType, "self", block); - const auto func = GetNodeValue(FunctionImpl, ctx, block); + BranchInst::Create(main, make, HasValue(udfPtr, block), block); - CallBoxedValueVirtualMethod(pointer, func, ctx.Codegen, block, ctx.GetBuilder(), pointer); + block = make; + + const auto makeFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TUdfWrapper::MakeUdf)); + const auto makeType = FunctionType::get(Type::getVoidTy(context), {self->getType(), ctx.Ctx->getType(), udfPtr->getType()}, false); + const auto makeFuncPtr = CastInst::Create(Instruction::IntToPtr, makeFunc, PointerType::getUnqual(makeType), "function", block); + CallInst::Create(makeType, makeFuncPtr, {self, ctx.Ctx, udfPtr}, "", block); + BranchInst::Create(main, block); + + block = main; + + GetNodeValue(pointer, RunConfigNode, ctx, block); + const auto conf = new LoadInst(valueType, pointer, "conf", block); + const auto udf = new LoadInst(valueType, udfPtr, "udf", block); + + CallBoxedValueVirtualMethod(pointer, udf, ctx.Codegen, block, ctx.GetBuilder(), pointer); ValueUnRef(RunConfigNode->GetRepresentation(), conf, ctx, block); - const auto ptrType = PointerType::getUnqual(StructType::get(context)); const auto wrap = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TUdfWrapper::Wrap)); - const auto self = CastInst::Create(Instruction::IntToPtr, ConstantInt::get(Type::getInt64Ty(context), uintptr_t(this)), ptrType, "self", block); const auto funType = FunctionType::get(Type::getVoidTy(context), {self->getType(), pointer->getType()}, false); const auto doFuncPtr = CastInst::Create(Instruction::IntToPtr, wrap, PointerType::getUnqual(funType), "function", block); CallInst::Create(funType, doFuncPtr, {self, pointer}, "", block); } #endif private: + void MakeUdf(TComputationContext& ctx, NUdf::TUnboxedValue& udf) const { + ui32 flags = 0; + TFunctionTypeInfo funcInfo; + const auto status = ctx.HolderFactory.GetFunctionRegistry()->FindFunctionTypeInfo( + ctx.TypeEnv, ctx.TypeInfoHelper, ctx.CountersProvider, FunctionName, UserType->IsVoid() ? nullptr : UserType, + TypeConfig, flags, Pos, ctx.SecureParamsProvider, &funcInfo); + + MKQL_ENSURE(status.IsOk(), status.GetError()); + MKQL_ENSURE(funcInfo.Implementation, "UDF implementation is not set for function " << FunctionName); + udf = NUdf::TUnboxedValuePod(funcInfo.Implementation.Release()); + } + void Wrap(NUdf::TUnboxedValue& callable) const { MKQL_ENSURE(bool(callable), "Returned empty value in function: " << FunctionName); TValidate::WrapCallable(CallableType, callable, TStringBuilder() << "FunctionWithConfig<" << FunctionName << ">"); } void RegisterDependencies() const final { - this->DependsOn(FunctionImpl); this->DependsOn(RunConfigNode); } - IComputationNode* const FunctionImpl; const TString FunctionName; + const TString TypeConfig; + const NUdf::TSourcePosition Pos; IComputationNode* const RunConfigNode; - const TCallableType* const CallableType; + const TCallableType* CallableType; + TType* const UserType; + const ui32 UdfIndex; }; -inline IComputationNode* CreateUdfWrapper( - const TComputationNodeFactoryContext& ctx, - NUdf::TUnboxedValue&& functionImpl, - TString&& functionName, - IComputationNode* runConfigNode, - const TCallableType* callableType) +template +using TWrapper = std::conditional_t, TUdfWrapper>; + +template +inline IComputationNode* CreateUdfWrapper(const TComputationNodeFactoryContext& ctx, TArgs&&...args) { - const auto node = ctx.NodeFactory.CreateImmutableNode(std::move(functionImpl)); - ctx.NodePushBack(node); switch (ctx.ValidateMode) { case NUdf::EValidateMode::None: - return new TUdfWrapper>(ctx.Mutables, std::move(node), std::move(functionName), runConfigNode, callableType); + return new TWrapper>(ctx.Mutables, std::forward(args)...); case NUdf::EValidateMode::Lazy: if (ctx.ValidatePolicy == NUdf::EValidatePolicy::Fail) { - return new TUdfWrapper>(ctx.Mutables, std::move(node), std::move(functionName), runConfigNode, callableType); + return new TWrapper>(ctx.Mutables, std::forward(args)...); } else { - return new TUdfWrapper>(ctx.Mutables, std::move(node), std::move(functionName), runConfigNode, callableType); + return new TWrapper>(ctx.Mutables, std::forward(args)...); } case NUdf::EValidateMode::Greedy: if (ctx.ValidatePolicy == NUdf::EValidatePolicy::Fail) { - return new TUdfWrapper>(ctx.Mutables, std::move(node), std::move(functionName), runConfigNode, callableType); + return new TWrapper>(ctx.Mutables, std::forward(args)...); } else { - return new TUdfWrapper>(ctx.Mutables, std::move(node), std::move(functionName), runConfigNode, callableType); + return new TWrapper>(ctx.Mutables, std::forward(args)...); } default: Y_ABORT("Unexpected validate mode: %u", static_cast(ctx.ValidateMode)); @@ -159,7 +253,7 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont MKQL_ENSURE(userTypeNode.GetStaticType()->IsType(), "Expected type"); TString funcName(AS_VALUE(TDataLiteral, funcNameNode)->AsValue().AsStringRef()); - TStringBuf typeConfig(AS_VALUE(TDataLiteral, typeCfgNode)->AsValue().AsStringRef()); + TString typeConfig(AS_VALUE(TDataLiteral, typeCfgNode)->AsValue().AsStringRef()); NUdf::TSourcePosition pos; if (callable.GetInputsCount() == 7) { @@ -171,6 +265,7 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont ui32 flags = 0; TFunctionTypeInfo funcInfo; const auto userType = static_cast(userTypeNode.GetNode()); + const auto status = ctx.FunctionRegistry.FindFunctionTypeInfo( ctx.Env, ctx.TypeInfoHelper, ctx.CountersProvider, funcName, userType->IsVoid() ? nullptr : userType, typeConfig, flags, pos, ctx.SecureParamsProvider, &funcInfo); @@ -186,35 +281,18 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont MKQL_ENSURE(typesMatch, "RunConfig '" << funcName << "' type mismatch, expected: " << PrintNode(runCfgNode.GetStaticType(), true) << ", actual: " << PrintNode(runConfigType, true)); - NUdf::TUnboxedValue impl(NUdf::TUnboxedValuePod(funcInfo.Implementation.Release())); if (runConfigType->IsVoid()) { - // use function implementation as is - if (ctx.ValidateMode == NUdf::EValidateMode::None && funcInfo.ModuleIR && funcInfo.IRFunctionName) { - return new TUdfRunCodegeneratorNode(&ctx.HolderFactory.GetMemInfo(), std::move(impl), funcInfo.ModuleIRUniqID, funcInfo.ModuleIR, funcInfo.IRFunctionName); + return new TUdfRunCodegeneratorNode( + ctx.Mutables, std::move(funcName), std::move(typeConfig), pos, funcInfo.FunctionType, userType, + std::move(funcInfo.ModuleIRUniqID), std::move(funcInfo.ModuleIR), std::move(funcInfo.IRFunctionName), std::move(funcInfo.Implementation) + ); } - - if (ctx.ValidateMode != NUdf::EValidateMode::None) { - if (ctx.ValidateMode == NUdf::EValidateMode::Lazy) { - if (ctx.ValidatePolicy == NUdf::EValidatePolicy::Fail) { - TValidate>::WrapCallable(funcInfo.FunctionType, impl, TStringBuilder() << "FunctionWrapper<" << funcName << ">"); - } else { - TValidate>::WrapCallable(funcInfo.FunctionType, impl, TStringBuilder() << "FunctionWrapper<" << funcName << ">"); - } - } else { - if (ctx.ValidatePolicy == NUdf::EValidatePolicy::Fail) { - TValidate>::WrapCallable(funcInfo.FunctionType, impl, TStringBuilder() << "FunctionWrapper<" << funcName << ">"); - } else { - TValidate>::WrapCallable(funcInfo.FunctionType, impl, TStringBuilder() << "FunctionWrapper<" << funcName << ">"); - } - } - } - return ctx.NodeFactory.CreateImmutableNode(std::move(impl)); + return CreateUdfWrapper(ctx, std::move(funcName), std::move(typeConfig), pos, funcInfo.FunctionType, userType); } - // use function factory to get implementation by runconfig in runtime const auto runCfgCompNode = LocateNode(ctx.NodeLocator, *runCfgNode.GetNode()); - return CreateUdfWrapper(ctx, std::move(impl), std::move(funcName), runCfgCompNode, funcInfo.FunctionType); + return CreateUdfWrapper(ctx, std::move(funcName), std::move(typeConfig), pos, runCfgCompNode, funcInfo.FunctionType, userType); } IComputationNode* WrapScriptUdf(TCallable& callable, const TComputationNodeFactoryContext& ctx) { @@ -227,7 +305,7 @@ IComputationNode* WrapScriptUdf(TCallable& callable, const TComputationNodeFacto MKQL_ENSURE(userTypeNode.IsImmediate() && userTypeNode.GetStaticType()->IsType(), "Expected immediate type"); TString funcName(AS_VALUE(TDataLiteral, funcNameNode)->AsValue().AsStringRef()); - TStringBuf typeConfig(AS_VALUE(TDataLiteral, typeConfigNode)->AsValue().AsStringRef()); + TString typeConfig(AS_VALUE(TDataLiteral, typeConfigNode)->AsValue().AsStringRef()); NUdf::TSourcePosition pos; if (callable.GetInputsCount() == 7) { @@ -236,10 +314,11 @@ IComputationNode* WrapScriptUdf(TCallable& callable, const TComputationNodeFacto pos.Column_ = AS_VALUE(TDataLiteral, callable.GetInput(6))->AsValue().Get(); } + const auto userType = static_cast(userTypeNode.GetNode()); ui32 flags = 0; TFunctionTypeInfo funcInfo; const auto status = ctx.FunctionRegistry.FindFunctionTypeInfo( - ctx.Env, ctx.TypeInfoHelper, ctx.CountersProvider, funcName, static_cast(userTypeNode.GetNode()), + ctx.Env, ctx.TypeInfoHelper, ctx.CountersProvider, funcName, userType, typeConfig, flags, pos, ctx.SecureParamsProvider, &funcInfo); MKQL_ENSURE(status.IsOk(), status.GetError()); MKQL_ENSURE(funcInfo.Implementation, "UDF implementation is not set"); @@ -252,7 +331,7 @@ IComputationNode* WrapScriptUdf(TCallable& callable, const TComputationNodeFacto const auto funcTypeInfo = static_cast(callableResultType); const auto programCompNode = LocateNode(ctx.NodeLocator, *programNode.GetNode()); - return CreateUdfWrapper(ctx, NUdf::TUnboxedValuePod(funcInfo.Implementation.Release()), std::move(funcName), programCompNode, funcTypeInfo); + return CreateUdfWrapper(ctx, std::move(funcName), std::move(typeConfig), pos, programCompNode, funcTypeInfo, userType); } } diff --git a/ydb/library/yql/minikql/comp_nodes/mkql_wide_combine.cpp b/ydb/library/yql/minikql/comp_nodes/mkql_wide_combine.cpp index c28e8dee937c..d8b69ad188af 100644 --- a/ydb/library/yql/minikql/comp_nodes/mkql_wide_combine.cpp +++ b/ydb/library/yql/minikql/comp_nodes/mkql_wide_combine.cpp @@ -132,6 +132,27 @@ struct TCombinerNodes { } } + void ConsumeRawData(TComputationContext&, NUdf::TUnboxedValue* keys, NUdf::TUnboxedValue** from, NUdf::TUnboxedValue* to) const { + std::fill_n(keys, KeyResultNodes.size(), NUdf::TUnboxedValuePod()); + for (ui32 i = 0U; i < ItemNodes.size(); ++i) { + if (from[i] && IsInputItemNodeUsed(i)) { + to[i] = std::move(*(from[i])); + } + } + } + + void ExtractRawData(TComputationContext& ctx, NUdf::TUnboxedValue* from, NUdf::TUnboxedValue* keys) const { + for (ui32 i = 0U; i != ItemNodes.size(); ++i) { + if (IsInputItemNodeUsed(i)) { + ItemNodes[i]->SetValue(ctx, std::move(from[i])); + } + } + for (ui32 i = 0U; i < KeyNodes.size(); ++i) { + auto& key = KeyNodes[i]->RefValue(ctx); + *keys++ = key = KeyResultNodes[i]->GetValue(ctx); + } + } + void ProcessItem(TComputationContext& ctx, NUdf::TUnboxedValue* keys, NUdf::TUnboxedValue* state) const { if (keys) { std::fill_n(keys, KeyResultNodes.size(), NUdf::TUnboxedValuePod()); @@ -346,62 +367,61 @@ class TSpillingSupportState : public TComputationValue { enum class ETasteResult: i8 { Init = -1, Update, - Skip + ConsumeRawData + }; + + enum class EUpdateResult: i8 { + Yield = -1, + ExtractRawData, + ReadInput, + Extract, + Finish }; TSpillingSupportState( - TMemoryUsageInfo* memInfo, size_t wideFieldsIndex, + TMemoryUsageInfo* memInfo, const TMultiType* usedInputItemType, const TMultiType* keyAndStateType, ui32 keyWidth, size_t itemNodesSize, const THashFunc& hash, const TEqualsFunc& equal, bool allowSpilling, TComputationContext& ctx ) : TBase(memInfo) , InMemoryProcessingState(memInfo, keyWidth, keyAndStateType->GetElementsCount() - keyWidth, hash, equal) - , WideFieldsIndex(wideFieldsIndex) , UsedInputItemType(usedInputItemType) , KeyAndStateType(keyAndStateType) , KeyWidth(keyWidth) , ItemNodesSize(itemNodesSize) , Hasher(hash) , Mode(EOperatingMode::InMemory) + , ViewForKeyAndState(keyAndStateType->GetElementsCount()) , MemInfo(memInfo) , Equal(equal) , AllowSpilling(allowSpilling) , Ctx(ctx) { BufferForUsedInputItems.reserve(usedInputItemType->GetElementsCount()); - BufferForKeyAndState.reserve(keyAndStateType->GetElementsCount()); Tongue = InMemoryProcessingState.Tongue; Throat = InMemoryProcessingState.Throat; } - bool HasAnyData() const { - return !SpilledBuckets.empty(); - } + EUpdateResult Update() { + if (IsEverythingExtracted) return EUpdateResult::Finish; - bool IsProcessingRequired() const { - if (InputStatus != EFetchResult::Finish) return true; - - return HasDataForProcessing; - } - - bool UpdateAndWait() { switch (GetMode()) { case EOperatingMode::InMemory: { + Tongue = InMemoryProcessingState.Tongue; if (CheckMemoryAndSwitchToSpilling()) { - return UpdateAndWait(); + return Update(); } - return false; + if (InputStatus == EFetchResult::Finish) return EUpdateResult::Extract; + + return EUpdateResult::ReadInput; } - - case EOperatingMode::ProcessSpilled: - return ProcessSpilledDataAndWait(); case EOperatingMode::Spilling: { UpdateSpillingBuckets(); - if (!HasMemoryForProcessing() && InputStatus != EFetchResult::Finish && TryToReduceMemoryAndWait()) return true; + if (!HasMemoryForProcessing() && InputStatus != EFetchResult::Finish && TryToReduceMemoryAndWait()) return EUpdateResult::Yield; if (BufferForUsedInputItems.size()) { auto& bucket = SpilledBuckets[BufferForUsedInputItemsBucketId]; - if (bucket.AsyncWriteOperation.has_value()) return true; + if (bucket.AsyncWriteOperation.has_value()) return EUpdateResult::Yield; bucket.AsyncWriteOperation = bucket.SpilledData->WriteWideItem(BufferForUsedInputItems); BufferForUsedInputItems.resize(0); //for freeing allocated key value asap @@ -409,10 +429,10 @@ class TSpillingSupportState : public TComputationValue { if (InputStatus == EFetchResult::Finish) return FlushSpillingBuffersAndWait(); - // Prepare buffer for reading new key - BufferForKeyAndState.resize(KeyWidth); - return false; + return EUpdateResult::ReadInput; } + case EOperatingMode::ProcessSpilled: + return ProcessSpilledData(); } } @@ -420,78 +440,59 @@ class TSpillingSupportState : public TComputationValue { if (GetMode() == EOperatingMode::InMemory) { bool isNew = InMemoryProcessingState.TasteIt(); Throat = InMemoryProcessingState.Throat; - Tongue = InMemoryProcessingState.Tongue; return isNew ? ETasteResult::Init : ETasteResult::Update; } if (GetMode() == EOperatingMode::ProcessSpilled) { // while restoration we process buckets one by one starting from the first in a queue bool isNew = SpilledBuckets.front().InMemoryProcessingState->TasteIt(); Throat = SpilledBuckets.front().InMemoryProcessingState->Throat; - Tongue = SpilledBuckets.front().InMemoryProcessingState->Tongue; + BufferForUsedInputItems.resize(0); return isNew ? ETasteResult::Init : ETasteResult::Update; } - MKQL_ENSURE(!BufferForKeyAndState.empty(), "Internal logic error"); - - auto hash = Hasher(BufferForKeyAndState.data()); + auto hash = Hasher(ViewForKeyAndState.data()); auto bucketId = hash % SpilledBucketCount; auto& bucket = SpilledBuckets[bucketId]; if (bucket.BucketState == TSpilledBucket::EBucketState::InMemory) { - MoveKeyToBucket(bucket); + std::copy_n(ViewForKeyAndState.data(), KeyWidth, static_cast(bucket.InMemoryProcessingState->Tongue)); + + bool isNew = bucket.InMemoryProcessingState->TasteIt(); Throat = bucket.InMemoryProcessingState->Throat; - return bucket.InMemoryProcessingState->TasteIt()? ETasteResult::Init : ETasteResult::Update; + return isNew ? ETasteResult::Init : ETasteResult::Update; } - // Corresponding bucket is spilled, we don't need a key anymore, full input will be spilled - BufferForKeyAndState.resize(0); - TryToSpillRawData(bucket, bucketId); - - return ETasteResult::Skip; + // Prepare space for raw data + MKQL_ENSURE(BufferForUsedInputItems.size() == 0, "Internal logic error"); + BufferForUsedInputItems.resize(ItemNodesSize); + BufferForUsedInputItemsBucketId = bucketId; + + Throat = BufferForUsedInputItems.data(); + + return ETasteResult::ConsumeRawData; } NUdf::TUnboxedValuePod* Extract() { - if (GetMode() == EOperatingMode::InMemory) return static_cast(InMemoryProcessingState.Extract()); + NUdf::TUnboxedValue* value = nullptr; + if (GetMode() == EOperatingMode::InMemory) { + value = static_cast(InMemoryProcessingState.Extract()); + if (!value) IsEverythingExtracted = true; + return value; + } MKQL_ENSURE(SpilledBuckets.front().BucketState == TSpilledBucket::EBucketState::InMemory, "Internal logic error"); MKQL_ENSURE(SpilledBuckets.size() > 0, "Internal logic error"); - auto value = static_cast(SpilledBuckets.front().InMemoryProcessingState->Extract()); + value = static_cast(SpilledBuckets.front().InMemoryProcessingState->Extract()); if (!value) { SpilledBuckets.pop_front(); + if (SpilledBuckets.empty()) IsEverythingExtracted = true; } return value; } private: - void MoveKeyToBucket(TSpilledBucket& bucket) { - for (size_t i = 0; i < KeyWidth; ++i) { - //jumping into unsafe world, refusing ownership - static_cast(bucket.InMemoryProcessingState->Tongue[i]) = std::move(BufferForKeyAndState[i]); - } - BufferForKeyAndState.resize(0); - } - - // Copies data from WideFields to local and tries to spill it using suitable bucket. - // if the bucket is already busy, then the buffer will wait for the next iteration. - void TryToSpillRawData(TSpilledBucket& bucket, size_t bucketId) { - auto **fields = Ctx.WideFields.data() + WideFieldsIndex; - MKQL_ENSURE(BufferForUsedInputItems.empty(), "Internal logic error"); - - for (size_t i = 0; i < ItemNodesSize; ++i) { - if (fields[i]) { - BufferForUsedInputItems.push_back(*fields[i]); - } - } - if (bucket.AsyncWriteOperation.has_value()) { - BufferForUsedInputItemsBucketId = bucketId; - return; - } - bucket.AsyncWriteOperation = bucket.SpilledData->WriteWideItem(BufferForUsedInputItems); - BufferForUsedInputItems.resize(0); - } - - bool FlushSpillingBuffersAndWait() { + EUpdateResult FlushSpillingBuffersAndWait() { UpdateSpillingBuckets(); ui64 finishedCount = 0; @@ -507,12 +508,11 @@ class TSpillingSupportState : public TComputationValue { } } - if (finishedCount != SpilledBuckets.size()) return true; + if (finishedCount != SpilledBuckets.size()) return EUpdateResult::Yield; - YQL_LOG(INFO) << "switching to ProcessSpilled"; SwitchMode(EOperatingMode::ProcessSpilled); - return ProcessSpilledDataAndWait(); + return ProcessSpilledData(); } void SplitStateIntoBuckets() { @@ -539,11 +539,7 @@ class TSpillingSupportState : public TComputationValue { bool CheckMemoryAndSwitchToSpilling() { if (AllowSpilling && Ctx.SpillerFactory && IsSwitchToSpillingModeCondition()) { - const auto used = TlsAllocState->GetUsed(); - const auto limit = TlsAllocState->GetLimit(); - - YQL_LOG(INFO) << "yellow zone reached " << (used*100/limit) << "%=" << used << "/" << limit; - YQL_LOG(INFO) << "switching Memory mode to Spilling"; + LogMemoryUsage(); SwitchMode(EOperatingMode::Spilling); return true; @@ -552,6 +548,19 @@ class TSpillingSupportState : public TComputationValue { return false; } + void LogMemoryUsage() const { + const auto used = TlsAllocState->GetUsed(); + const auto limit = TlsAllocState->GetLimit(); + TStringBuilder logmsg; + logmsg << "Memory usage: "; + if (limit) { + logmsg << (used*100/limit) << "%="; + } + logmsg << (used/1_MB) << "MB/" << (limit/1_MB) << "MB"; + + YQL_LOG(INFO) << logmsg; + } + void SpillMoreStateFromBucket(TSpilledBucket& bucket) { MKQL_ENSURE(!bucket.AsyncWriteOperation.has_value(), "Internal logic error"); @@ -608,11 +617,9 @@ class TSpillingSupportState : public TComputationValue { return false; } - bool ProcessSpilledDataAndWait() { - if (SpilledBuckets.empty()) return false; - + EUpdateResult ProcessSpilledData() { if (AsyncReadOperation) { - if (!AsyncReadOperation->HasValue()) return true; + if (!AsyncReadOperation->HasValue()) return EUpdateResult::Yield; if (RecoverState) { SpilledBuckets[0].SpilledState->AsyncReadCompleted(AsyncReadOperation->ExtractValue().value(), Ctx.HolderFactory); } else { @@ -620,28 +627,28 @@ class TSpillingSupportState : public TComputationValue { } AsyncReadOperation = std::nullopt; } + auto& bucket = SpilledBuckets.front(); - if (bucket.BucketState == TSpilledBucket::EBucketState::InMemory) return false; + if (bucket.BucketState == TSpilledBucket::EBucketState::InMemory) return EUpdateResult::Extract; + //recover spilled state while(!bucket.SpilledState->Empty()) { RecoverState = true; - BufferForKeyAndState.resize(KeyAndStateType->GetElementsCount()); - AsyncReadOperation = bucket.SpilledState->ExtractWideItem(BufferForKeyAndState); + TTemporaryUnboxedValueVector bufferForKeyAndState(KeyAndStateType->GetElementsCount()); + AsyncReadOperation = bucket.SpilledState->ExtractWideItem(bufferForKeyAndState); if (AsyncReadOperation) { - BufferForKeyAndState.resize(0); - return true; + return EUpdateResult::Yield; } for (size_t i = 0; i< KeyWidth; ++i) { //jumping into unsafe world, refusing ownership - static_cast(bucket.InMemoryProcessingState->Tongue[i]) = std::move(BufferForKeyAndState[i]); + static_cast(bucket.InMemoryProcessingState->Tongue[i]) = std::move(bufferForKeyAndState[i]); } auto isNew = bucket.InMemoryProcessingState->TasteIt(); MKQL_ENSURE(isNew, "Internal logic error"); for (size_t i = KeyWidth; i < KeyAndStateType->GetElementsCount(); ++i) { //jumping into unsafe world, refusing ownership - static_cast(bucket.InMemoryProcessingState->Throat[i - KeyWidth]) = std::move(BufferForKeyAndState[i]); + static_cast(bucket.InMemoryProcessingState->Throat[i - KeyWidth]) = std::move(bufferForKeyAndState[i]); } - BufferForKeyAndState.resize(0); } //process spilled data if (!bucket.SpilledData->Empty()) { @@ -649,24 +656,16 @@ class TSpillingSupportState : public TComputationValue { BufferForUsedInputItems.resize(UsedInputItemType->GetElementsCount()); AsyncReadOperation = bucket.SpilledData->ExtractWideItem(BufferForUsedInputItems); if (AsyncReadOperation) { - return true; - } - auto **fields = Ctx.WideFields.data() + WideFieldsIndex; - for (size_t i = 0, j = 0; i < ItemNodesSize; ++i) { - if (fields[i]) { - fields[i] = &(BufferForUsedInputItems[j++]); - } + return EUpdateResult::Yield; } + Throat = BufferForUsedInputItems.data(); Tongue = bucket.InMemoryProcessingState->Tongue; - Throat = bucket.InMemoryProcessingState->Throat; - HasDataForProcessing = true; - return false; + return EUpdateResult::ExtractRawData; } bucket.BucketState = TSpilledBucket::EBucketState::InMemory; - HasDataForProcessing = false; - return false; + return EUpdateResult::Extract; } EOperatingMode GetMode() const { @@ -676,10 +675,12 @@ class TSpillingSupportState : public TComputationValue { void SwitchMode(EOperatingMode mode) { switch(mode) { case EOperatingMode::InMemory: { + YQL_LOG(INFO) << "switching Memory mode to InMemory"; MKQL_ENSURE(false, "Internal logic error"); break; } case EOperatingMode::Spilling: { + YQL_LOG(INFO) << "switching Memory mode to Spilling"; MKQL_ENSURE(EOperatingMode::InMemory == Mode, "Internal logic error"); SpilledBuckets.resize(SpilledBucketCount); auto spiller = Ctx.SpillerFactory->CreateSpiller(); @@ -690,14 +691,13 @@ class TSpillingSupportState : public TComputationValue { } SplitStateIntoBuckets(); - BufferForKeyAndState.resize(KeyAndStateType->GetElementsCount()); - Tongue = BufferForKeyAndState.data(); + Tongue = ViewForKeyAndState.data(); break; } case EOperatingMode::ProcessSpilled: { + YQL_LOG(INFO) << "switching Memory mode to ProcessSpilled"; MKQL_ENSURE(EOperatingMode::Spilling == Mode, "Internal logic error"); MKQL_ENSURE(SpilledBuckets.size() == SpilledBucketCount, "Internal logic error"); - BufferForKeyAndState.resize(0); break; } @@ -710,9 +710,7 @@ class TSpillingSupportState : public TComputationValue { } bool IsSwitchToSpillingModeCondition() const { - return false; - // TODO: YQL-18033 - // return !HasMemoryForProcessing(); + return !HasMemoryForProcessing(); } public: @@ -723,10 +721,9 @@ class TSpillingSupportState : public TComputationValue { private: ui64 NextBucketToSpill = 0; - bool HasDataForProcessing = false; + bool IsEverythingExtracted = false; TState InMemoryProcessingState; - const size_t WideFieldsIndex; const TMultiType* const UsedInputItemType; const TMultiType* const KeyAndStateType; const size_t KeyWidth; @@ -740,7 +737,7 @@ class TSpillingSupportState : public TComputationValue { std::deque SpilledBuckets; ui64 BufferForUsedInputItemsBucketId; TUnboxedValueVector BufferForUsedInputItems; - TUnboxedValueVector BufferForKeyAndState; + std::vector> ViewForKeyAndState; TMemoryUsageInfo* MemInfo = nullptr; TEqualsFunc const Equal; @@ -1246,48 +1243,48 @@ using TBaseComputation = TStatefulWideFlowCodegeneratorNodeUpdateAndWait()) { - return EFetchResult::Yield; - } - - if (ptr->InputStatus != EFetchResult::Finish) { - switch (ptr->InputStatus = Flow->FetchValues(ctx, fields)) { - case EFetchResult::One: - break; - case EFetchResult::Finish: - continue; - case EFetchResult::Yield: - return EFetchResult::Yield; - } - } - - if (ptr->IsProcessingRequired()) { - Nodes.ExtractKey(ctx, fields, static_cast(ptr->Tongue)); - - switch(ptr->TasteIt()) { - case TSpillingSupportState::ETasteResult::Init: - Nodes.ProcessItem(ctx, nullptr, static_cast(ptr->Throat)); - break; - case TSpillingSupportState::ETasteResult::Update: - Nodes.ProcessItem(ctx, static_cast(ptr->Tongue), static_cast(ptr->Throat)); - break; - case TSpillingSupportState::ETasteResult::Skip: - break; + switch(ptr->Update()) { + case TSpillingSupportState::EUpdateResult::ReadInput: { + for (auto i = 0U; i < Nodes.ItemNodes.size(); ++i) + fields[i] = Nodes.GetUsedInputItemNodePtrOrNull(ctx, i); + switch (ptr->InputStatus = Flow->FetchValues(ctx, fields)) { + case EFetchResult::One: + break; + case EFetchResult::Finish: + continue; + case EFetchResult::Yield: + return EFetchResult::Yield; + } + Nodes.ExtractKey(ctx, fields, static_cast(ptr->Tongue)); + break; } - continue; + case TSpillingSupportState::EUpdateResult::Yield: + return EFetchResult::Yield; + case TSpillingSupportState::EUpdateResult::ExtractRawData: + Nodes.ExtractRawData(ctx, static_cast(ptr->Throat), static_cast(ptr->Tongue)); + break; + case TSpillingSupportState::EUpdateResult::Extract: + if (const auto values = static_cast(ptr->Extract())) { + Nodes.FinishItem(ctx, values, output); + return EFetchResult::One; + } + continue; + case TSpillingSupportState::EUpdateResult::Finish: + return EFetchResult::Finish; } - if (const auto values = static_cast(ptr->Extract())) { - Nodes.FinishItem(ctx, values, output); - return EFetchResult::One; + switch(ptr->TasteIt()) { + case TSpillingSupportState::ETasteResult::Init: + Nodes.ProcessItem(ctx, nullptr, static_cast(ptr->Throat)); + break; + case TSpillingSupportState::ETasteResult::Update: + Nodes.ProcessItem(ctx, static_cast(ptr->Tongue), static_cast(ptr->Throat)); + break; + case TSpillingSupportState::ETasteResult::ConsumeRawData: + Nodes.ConsumeRawData(ctx, static_cast(ptr->Tongue), fields, static_cast(ptr->Throat)); + break; } - if (!ptr->HasAnyData()) { - return EFetchResult::Finish; - } } } Y_UNREACHABLE(); @@ -1326,220 +1323,247 @@ using TBaseComputation = TStatefulWideFlowCodegeneratorNodegetType()}, false); BranchInst::Create(more, block); - const auto full = BasicBlock::Create(context, "full", ctx.Func); + const auto pull = BasicBlock::Create(context, "pull", ctx.Func); + const auto rest = BasicBlock::Create(context, "rest", ctx.Func); + const auto test = BasicBlock::Create(context, "test", ctx.Func); + const auto good = BasicBlock::Create(context, "good", ctx.Func); + const auto load = BasicBlock::Create(context, "load", ctx.Func); + const auto fill = BasicBlock::Create(context, "fill", ctx.Func); + const auto data = BasicBlock::Create(context, "data", ctx.Func); + const auto done = BasicBlock::Create(context, "done", ctx.Func); const auto over = BasicBlock::Create(context, "over", ctx.Func); + const auto stub = BasicBlock::Create(context, "stub", ctx.Func); + + new UnreachableInst(context, stub); + const auto result = PHINode::Create(statusType, 4U, "result", over); - { - const auto test = BasicBlock::Create(context, "test", ctx.Func); - const auto pull = BasicBlock::Create(context, "pull", ctx.Func); - const auto rest = BasicBlock::Create(context, "rest", ctx.Func); - const auto proc = BasicBlock::Create(context, "proc", ctx.Func); - const auto good = BasicBlock::Create(context, "good", ctx.Func); + std::vector phis(Nodes.ItemNodes.size(), nullptr); + auto j = 0U; + std::generate(phis.begin(), phis.end(), [&]() { + return Nodes.IsInputItemNodeUsed(j++) ? + PHINode::Create(valueType, 2U, (TString("item_") += ToString(j)).c_str(), test) : nullptr; + }); - block = more; + block = more; - const auto waitMoreFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::UpdateAndWait)); - const auto waitMoreFuncPtr = CastInst::Create(Instruction::IntToPtr, waitMoreFunc, PointerType::getUnqual(boolFuncType), "wait_more_func", block); - const auto waitMore = CallInst::Create(boolFuncType, waitMoreFuncPtr, { stateArg }, "wait_more", block); + const auto updateFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::Update)); + const auto updateType = FunctionType::get(wayType, {stateArg->getType()}, false); + const auto updateFuncPtr = CastInst::Create(Instruction::IntToPtr, updateFunc, PointerType::getUnqual(updateType), "update_func", block); + const auto update = CallInst::Create(updateType, updateFuncPtr, { stateArg }, "update", block); - result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::Yield)), block); + result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::Yield)), block); - BranchInst::Create(over, test, waitMore, block); + const auto updateWay = SwitchInst::Create(update, stub, 5U, block); + updateWay->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::EUpdateResult::Yield)), over); + updateWay->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::EUpdateResult::Extract)), fill); + updateWay->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::EUpdateResult::Finish)), done); + updateWay->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::EUpdateResult::ReadInput)), pull); + updateWay->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::EUpdateResult::ExtractRawData)), load); - block = test; + block = load; - const auto statusPtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetStatus() }, "last", block); - const auto last = new LoadInst(statusType, statusPtr, "last", block); - const auto finish = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_EQ, last, ConstantInt::get(last->getType(), static_cast(EFetchResult::Finish)), "finish", block); + const auto extractorPtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetThroat() }, "extractor_ptr", block); + const auto extractor = new LoadInst(ptrValueType, extractorPtr, "extractor", block); - BranchInst::Create(good, pull, finish, block); + std::vector items(phis.size(), nullptr); + for (ui32 i = 0U; i < items.size(); ++i) { + const auto ptr = GetElementPtrInst::CreateInBounds(valueType, extractor, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("load_ptr_") += ToString(i)).c_str(), block); + if (phis[i]) + items[i] = new LoadInst(valueType, ptr, (TString("load_") += ToString(i)).c_str(), block); + if (i < Nodes.ItemNodes.size() && Nodes.ItemNodes[i]->GetDependencesCount() > 0U) + EnsureDynamicCast(Nodes.ItemNodes[i])->CreateSetValue(ctx, block, items[i]); + } - block = pull; + for (ui32 i = 0U; i < phis.size(); ++i) { + if (const auto phi = phis[i]) { + phi->addIncoming(items[i], block); + } + } - const auto getres = GetNodeValues(Flow, ctx, block); + BranchInst::Create(test, block); - result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::Yield)), block); + block = pull; - const auto choise = SwitchInst::Create(getres.first, good, 2U, block); - choise->addCase(ConstantInt::get(statusType, static_cast(EFetchResult::Yield)), over); - choise->addCase(ConstantInt::get(statusType, static_cast(EFetchResult::Finish)), rest); + const auto getres = GetNodeValues(Flow, ctx, block); - block = rest; - new StoreInst(ConstantInt::get(statusType, static_cast(EFetchResult::Finish)), statusPtr, block); - BranchInst::Create(more, block); + result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::Yield)), block); - block = good; + const auto choise = SwitchInst::Create(getres.first, good, 2U, block); + choise->addCase(ConstantInt::get(statusType, static_cast(EFetchResult::Yield)), over); + choise->addCase(ConstantInt::get(statusType, static_cast(EFetchResult::Finish)), rest); - const auto processingFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::IsProcessingRequired)); - const auto processingFuncPtr = CastInst::Create(Instruction::IntToPtr, processingFunc, PointerType::getUnqual(boolFuncType), "processing_func", block); - const auto processing = CallInst::Create(boolFuncType, processingFuncPtr, { stateArg }, "processing", block); + block = rest; + const auto statusPtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetStatus() }, "last", block); + new StoreInst(ConstantInt::get(statusType, static_cast(EFetchResult::Finish)), statusPtr, block); + BranchInst::Create(more, block); - BranchInst::Create(proc, full, processing, block); + block = good; - block = proc; + for (ui32 i = 0U; i < items.size(); ++i) { + if (phis[i]) + items[i] = getres.second[i](ctx, block); + if (Nodes.ItemNodes[i]->GetDependencesCount() > 0U) + EnsureDynamicCast(Nodes.ItemNodes[i])->CreateSetValue(ctx, block, items[i]); + } - std::vector items(Nodes.ItemNodes.size(), nullptr); - for (ui32 i = 0U; i < items.size(); ++i) { - if (Nodes.ItemNodes[i]->GetDependencesCount() > 0U) - EnsureDynamicCast(Nodes.ItemNodes[i])->CreateSetValue(ctx, block, items[i] = getres.second[i](ctx, block)); - else if (Nodes.PasstroughtItems[i]) - items[i] = getres.second[i](ctx, block); + for (ui32 i = 0U; i < phis.size(); ++i) { + if (const auto phi = phis[i]) { + phi->addIncoming(items[i], block); } + } - const auto tonguePtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetTongue() }, "tongue_ptr", block); - const auto tongue = new LoadInst(ptrValueType, tonguePtr, "tongue", block); + BranchInst::Create(test, block); - std::vector keyPointers(Nodes.KeyResultNodes.size(), nullptr), keys(Nodes.KeyResultNodes.size(), nullptr); - for (ui32 i = 0U; i < Nodes.KeyResultNodes.size(); ++i) { - auto& key = keys[i]; - const auto keyPtr = keyPointers[i] = GetElementPtrInst::CreateInBounds(valueType, tongue, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("key_") += ToString(i)).c_str(), block); - if (const auto map = Nodes.KeysOnItems[i]) { - auto& it = items[*map]; - if (!it) - it = getres.second[*map](ctx, block); - key = it; - } else { - key = GetNodeValue(Nodes.KeyResultNodes[i], ctx, block); - } + block = test; - if (Nodes.KeyNodes[i]->GetDependencesCount() > 0U) - EnsureDynamicCast(Nodes.KeyNodes[i])->CreateSetValue(ctx, block, key); + const auto tonguePtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetTongue() }, "tongue_ptr", block); + const auto tongue = new LoadInst(ptrValueType, tonguePtr, "tongue", block); - new StoreInst(key, keyPtr, block); + std::vector keyPointers(Nodes.KeyResultNodes.size(), nullptr), keys(Nodes.KeyResultNodes.size(), nullptr); + for (ui32 i = 0U; i < Nodes.KeyResultNodes.size(); ++i) { + auto& key = keys[i]; + const auto keyPtr = keyPointers[i] = GetElementPtrInst::CreateInBounds(valueType, tongue, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("key_") += ToString(i)).c_str(), block); + if (const auto map = Nodes.KeysOnItems[i]) { + key = phis[*map]; + } else { + key = GetNodeValue(Nodes.KeyResultNodes[i], ctx, block); } - const auto atFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::TasteIt)); - const auto atType = FunctionType::get(wayType, {stateArg->getType()}, false); - const auto atPtr = CastInst::Create(Instruction::IntToPtr, atFunc, PointerType::getUnqual(atType), "function", block); - const auto taste= CallInst::Create(atType, atPtr, {stateArg}, "taste", block); + if (Nodes.KeyNodes[i]->GetDependencesCount() > 0U) + EnsureDynamicCast(Nodes.KeyNodes[i])->CreateSetValue(ctx, block, key); - const auto init = BasicBlock::Create(context, "init", ctx.Func); - const auto next = BasicBlock::Create(context, "next", ctx.Func); + new StoreInst(key, keyPtr, block); + } - const auto throatPtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetThroat() }, "throat_ptr", block); - const auto throat = new LoadInst(ptrValueType, throatPtr, "throat", block); + const auto atFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::TasteIt)); + const auto atType = FunctionType::get(wayType, {stateArg->getType()}, false); + const auto atPtr = CastInst::Create(Instruction::IntToPtr, atFunc, PointerType::getUnqual(atType), "function", block); + const auto taste= CallInst::Create(atType, atPtr, {stateArg}, "taste", block); - std::vector pointers; - pointers.reserve(Nodes.StateNodes.size()); - for (ui32 i = 0U; i < Nodes.StateNodes.size(); ++i) { - pointers.emplace_back(GetElementPtrInst::CreateInBounds(valueType, throat, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("state_") += ToString(i)).c_str(), block)); - } + const auto init = BasicBlock::Create(context, "init", ctx.Func); + const auto next = BasicBlock::Create(context, "next", ctx.Func); + const auto save = BasicBlock::Create(context, "save", ctx.Func); - const auto way = SwitchInst::Create(taste, more, 2U, block); - way->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::ETasteResult::Init)), init); - way->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::ETasteResult::Update)), next); + const auto throatPtr = GetElementPtrInst::CreateInBounds(stateType, stateArg, { stateFields.This(), stateFields.GetThroat() }, "throat_ptr", block); + const auto throat = new LoadInst(ptrValueType, throatPtr, "throat", block); - block = init; + std::vector pointers; + const auto width = std::max(Nodes.StateNodes.size(), phis.size()); + pointers.reserve(width); + for (ui32 i = 0U; i < width; ++i) { + pointers.emplace_back(GetElementPtrInst::CreateInBounds(valueType, throat, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("state_") += ToString(i)).c_str(), block)); + } - for (ui32 i = 0U; i < Nodes.KeyResultNodes.size(); ++i) { - ValueAddRef(Nodes.KeyResultNodes[i]->GetRepresentation(), keyPointers[i], ctx, block); - } + const auto way = SwitchInst::Create(taste, stub, 3U, block); + way->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::ETasteResult::Init)), init); + way->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::ETasteResult::Update)), next); + way->addCase(ConstantInt::get(wayType, static_cast(TSpillingSupportState::ETasteResult::ConsumeRawData)), save); - for (ui32 i = 0U; i < Nodes.InitResultNodes.size(); ++i) { - if (const auto map = Nodes.InitOnItems[i]) { - auto& it = items[*map]; - if (!it) - it = getres.second[*map](ctx, block); - new StoreInst(it, pointers[i], block); - ValueAddRef(Nodes.InitResultNodes[i]->GetRepresentation(), it, ctx, block); - } else if (const auto map = Nodes.InitOnKeys[i]) { - const auto key = keys[*map]; - new StoreInst(key, pointers[i], block); - ValueAddRef(Nodes.InitResultNodes[i]->GetRepresentation(), key, ctx, block); - } else { - GetNodeValue(pointers[i], Nodes.InitResultNodes[i], ctx, block); - } + block = init; + + for (ui32 i = 0U; i < Nodes.KeyResultNodes.size(); ++i) { + ValueAddRef(Nodes.KeyResultNodes[i]->GetRepresentation(), keyPointers[i], ctx, block); + } + + for (ui32 i = 0U; i < Nodes.InitResultNodes.size(); ++i) { + if (const auto map = Nodes.InitOnItems[i]) { + const auto item = phis[*map]; + new StoreInst(item, pointers[i], block); + ValueAddRef(Nodes.InitResultNodes[i]->GetRepresentation(), item, ctx, block); + } else if (const auto map = Nodes.InitOnKeys[i]) { + const auto key = keys[*map]; + new StoreInst(key, pointers[i], block); + ValueAddRef(Nodes.InitResultNodes[i]->GetRepresentation(), key, ctx, block); + } else { + GetNodeValue(pointers[i], Nodes.InitResultNodes[i], ctx, block); } + } - BranchInst::Create(more, block); + BranchInst::Create(more, block); - block = next; + block = next; - std::vector stored(Nodes.StateNodes.size(), nullptr); - for (ui32 i = 0U; i < stored.size(); ++i) { - const bool hasDependency = Nodes.StateNodes[i]->GetDependencesCount() > 0U; - if (const auto map = Nodes.StateOnUpdate[i]) { - if (hasDependency || i != *map) { - stored[i] = new LoadInst(valueType, pointers[i], (TString("state_") += ToString(i)).c_str(), block); - if (hasDependency) - EnsureDynamicCast(Nodes.StateNodes[i])->CreateSetValue(ctx, block, stored[i]); - } - } else if (hasDependency) { - EnsureDynamicCast(Nodes.StateNodes[i])->CreateSetValue(ctx, block, pointers[i]); - } else { - ValueUnRef(Nodes.StateNodes[i]->GetRepresentation(), pointers[i], ctx, block); + std::vector stored(Nodes.StateNodes.size(), nullptr); + for (ui32 i = 0U; i < stored.size(); ++i) { + const bool hasDependency = Nodes.StateNodes[i]->GetDependencesCount() > 0U; + if (const auto map = Nodes.StateOnUpdate[i]) { + if (hasDependency || i != *map) { + stored[i] = new LoadInst(valueType, pointers[i], (TString("state_") += ToString(i)).c_str(), block); + if (hasDependency) + EnsureDynamicCast(Nodes.StateNodes[i])->CreateSetValue(ctx, block, stored[i]); } + } else if (hasDependency) { + EnsureDynamicCast(Nodes.StateNodes[i])->CreateSetValue(ctx, block, pointers[i]); + } else { + ValueUnRef(Nodes.StateNodes[i]->GetRepresentation(), pointers[i], ctx, block); } + } - for (ui32 i = 0U; i < Nodes.UpdateResultNodes.size(); ++i) { - if (const auto map = Nodes.UpdateOnState[i]) { - if (const auto j = *map; i != j) { - auto& it = stored[j]; - if (!it) - it = new LoadInst(valueType, pointers[j], (TString("state_") += ToString(j)).c_str(), block); - new StoreInst(it, pointers[i], block); - if (i != *Nodes.StateOnUpdate[j]) - ValueAddRef(Nodes.UpdateResultNodes[i]->GetRepresentation(), it, ctx, block); - } - } else if (const auto map = Nodes.UpdateOnItems[i]) { - auto& it = items[*map]; - if (!it) - it = getres.second[*map](ctx, block); + for (ui32 i = 0U; i < Nodes.UpdateResultNodes.size(); ++i) { + if (const auto map = Nodes.UpdateOnState[i]) { + if (const auto j = *map; i != j) { + const auto it = stored[j]; new StoreInst(it, pointers[i], block); - ValueAddRef(Nodes.UpdateResultNodes[i]->GetRepresentation(), it, ctx, block); - } else if (const auto map = Nodes.UpdateOnKeys[i]) { - const auto key = keys[*map]; - new StoreInst(key, pointers[i], block); - ValueAddRef(Nodes.UpdateResultNodes[i]->GetRepresentation(), key, ctx, block); - } else { - GetNodeValue(pointers[i], Nodes.UpdateResultNodes[i], ctx, block); + if (i != *Nodes.StateOnUpdate[j]) + ValueAddRef(Nodes.UpdateResultNodes[i]->GetRepresentation(), it, ctx, block); } + } else if (const auto map = Nodes.UpdateOnItems[i]) { + const auto item = phis[*map]; + new StoreInst(item, pointers[i], block); + ValueAddRef(Nodes.UpdateResultNodes[i]->GetRepresentation(), item, ctx, block); + } else if (const auto map = Nodes.UpdateOnKeys[i]) { + const auto key = keys[*map]; + new StoreInst(key, pointers[i], block); + ValueAddRef(Nodes.UpdateResultNodes[i]->GetRepresentation(), key, ctx, block); + } else { + GetNodeValue(pointers[i], Nodes.UpdateResultNodes[i], ctx, block); } - - BranchInst::Create(more, block); } - { - block = full; + BranchInst::Create(more, block); - const auto good = BasicBlock::Create(context, "good", ctx.Func); - const auto last = BasicBlock::Create(context, "last", ctx.Func); + block = save; - const auto extractFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::Extract)); - const auto extractType = FunctionType::get(ptrValueType, {stateArg->getType()}, false); - const auto extractPtr = CastInst::Create(Instruction::IntToPtr, extractFunc, PointerType::getUnqual(extractType), "extract", block); - const auto out = CallInst::Create(extractType, extractPtr, {stateArg}, "out", block); - const auto has = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_NE, out, ConstantPointerNull::get(ptrValueType), "has", block); + for (ui32 i = 0U; i < phis.size(); ++i) { + if (const auto item = phis[i]) { + new StoreInst(item, pointers[i], block); + ValueAddRef(Nodes.ItemNodes[i]->GetRepresentation(), item, ctx, block); + } + } - BranchInst::Create(good, last, has, block); + BranchInst::Create(more, block); - block = good; + block = fill; - for (ui32 i = 0U; i < Nodes.FinishNodes.size(); ++i) { - const auto ptr = GetElementPtrInst::CreateInBounds(valueType, out, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("out_key_") += ToString(i)).c_str(), block); - if (Nodes.FinishNodes[i]->GetDependencesCount() > 0 || Nodes.ItemsOnResult[i]) - EnsureDynamicCast(Nodes.FinishNodes[i])->CreateSetValue(ctx, block, ptr); - else - ValueUnRef(Nodes.FinishNodes[i]->GetRepresentation(), ptr, ctx, block); - } + const auto extractFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::Extract)); + const auto extractType = FunctionType::get(ptrValueType, {stateArg->getType()}, false); + const auto extractPtr = CastInst::Create(Instruction::IntToPtr, extractFunc, PointerType::getUnqual(extractType), "extract", block); + const auto out = CallInst::Create(extractType, extractPtr, {stateArg}, "out", block); + const auto has = CmpInst::Create(Instruction::ICmp, ICmpInst::ICMP_NE, out, ConstantPointerNull::get(ptrValueType), "has", block); - result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::One)), block); - BranchInst::Create(over, block); + BranchInst::Create(data, more, has, block); - block = last; + block = data; - const auto hasDataFunc = ConstantInt::get(Type::getInt64Ty(context), GetMethodPtr(&TSpillingSupportState::HasAnyData)); - const auto hasDataFuncPtr = CastInst::Create(Instruction::IntToPtr, hasDataFunc, PointerType::getUnqual(boolFuncType), "has_data_func", block); - const auto hasData = CallInst::Create(boolFuncType, hasDataFuncPtr, { stateArg }, "has_data", block); + for (ui32 i = 0U; i < Nodes.FinishNodes.size(); ++i) { + const auto ptr = GetElementPtrInst::CreateInBounds(valueType, out, {ConstantInt::get(Type::getInt32Ty(context), i)}, (TString("out_key_") += ToString(i)).c_str(), block); + if (Nodes.FinishNodes[i]->GetDependencesCount() > 0 || Nodes.ItemsOnResult[i]) + EnsureDynamicCast(Nodes.FinishNodes[i])->CreateSetValue(ctx, block, ptr); + else + ValueUnRef(Nodes.FinishNodes[i]->GetRepresentation(), ptr, ctx, block); + } - result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::Finish)), block); + result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::One)), block); - BranchInst::Create(more, over, hasData, block); - } + BranchInst::Create(over, block); + + block = done; + + result->addIncoming(ConstantInt::get(statusType, static_cast(EFetchResult::Finish)), block); + BranchInst::Create(over, block); block = over; @@ -1553,8 +1577,7 @@ using TBaseComputation = TStatefulWideFlowCodegeneratorNode(WideFieldsIndex, - UsedInputItemType, KeyAndStateType, + state = ctx.HolderFactory.Create(UsedInputItemType, KeyAndStateType, Nodes.KeyNodes.size(), Nodes.ItemNodes.size(), #ifdef MKQL_DISABLE_CODEGEN @@ -1690,15 +1713,8 @@ IComputationNode* WrapWideCombinerT(TCallable& callable, const TComputationNodeF if (const auto wide = dynamic_cast(flow)) { if constexpr (Last) { const auto inputItemTypes = GetWideComponents(inputType); - std::vector usedInputItemTypes; - usedInputItemTypes.reserve(inputItemTypes.size()); - for (size_t i = 0; i != inputItemTypes.size(); ++i) { - if (nodes.IsInputItemNodeUsed(i)) { - usedInputItemTypes.push_back(inputItemTypes[i]); - } - } return new TWideLastCombinerWrapper(ctx.Mutables, wide, std::move(nodes), - TMultiType::Create(usedInputItemTypes.size(), usedInputItemTypes.data(), ctx.Env), + TMultiType::Create(inputItemTypes.size(), inputItemTypes.data(), ctx.Env), std::move(keyTypes), TMultiType::Create(keyAndStateItemTypes.size(),keyAndStateItemTypes.data(), ctx.Env), allowSpilling diff --git a/ydb/library/yql/minikql/comp_nodes/ut/mkql_computation_node_ut.h b/ydb/library/yql/minikql/comp_nodes/ut/mkql_computation_node_ut.h index 7fa1164bbb67..761382dd6970 100644 --- a/ydb/library/yql/minikql/comp_nodes/ut/mkql_computation_node_ut.h +++ b/ydb/library/yql/minikql/comp_nodes/ut/mkql_computation_node_ut.h @@ -48,6 +48,7 @@ void N(NUnitTest::TTestContext&) #define Y_UNIT_TEST_LLVM(N) Y_UNIT_TEST_TWIN(N, LLVM) +#define Y_UNIT_TEST_LLVM_SPILLING(N) Y_UNIT_TEST_QUAD(N, LLVM, SPILLING) #define Y_UNIT_TEST_QUAD(N, OPT1, OPT2) \ template void N(NUnitTest::TTestContext&); \ @@ -79,7 +80,7 @@ struct TUdfModuleInfo { NUdf::TUniquePtr Module; }; -template +template struct TSetup { explicit TSetup(TComputationNodeFactory nodeFactory = GetTestFactory(), TVector&& modules = {}) : Alloc(__LOCATION__) @@ -96,6 +97,8 @@ struct TSetup { FunctionRegistry = mutableRegistry; } + Alloc.Ref().ForcefullySetMemoryYellowZone(EnableSpilling); + RandomProvider = CreateDeterministicRandomProvider(1); TimeProvider = CreateDeterministicTimeProvider(10000000); diff --git a/ydb/library/yql/minikql/comp_nodes/ut/mkql_wide_combine_ut.cpp b/ydb/library/yql/minikql/comp_nodes/ut/mkql_wide_combine_ut.cpp index 1984d7bde28e..f89d68ffb342 100644 --- a/ydb/library/yql/minikql/comp_nodes/ut/mkql_wide_combine_ut.cpp +++ b/ydb/library/yql/minikql/comp_nodes/ut/mkql_wide_combine_ut.cpp @@ -5,9 +5,9 @@ #include #include +#include #include -#include #include namespace NKikimr { @@ -30,6 +30,7 @@ using TBaseComputation = TMutableComputationNode; {} private: NUdf::EFetchStatus Fetch(NUdf::TUnboxedValue& result) override { + constexpr auto size = Y_ARRAY_SIZE(g_TestYieldStreamData); if (Index == size) { return NUdf::EFetchStatus::Finish; @@ -47,6 +48,7 @@ using TBaseComputation = TMutableComputationNode; items[1] = NUdf::TUnboxedValuePod(MakeString(ToString(val))); ++Index; + return NUdf::EFetchStatus::Ok; } @@ -117,6 +119,13 @@ TRuntimeNode Combine(TProgramBuilder& pb, TRuntimeNode stream, std::function +TRuntimeNode WideLastCombiner(TProgramBuilder& pb, TRuntimeNode flow, const TProgramBuilder::TWideLambda& extractor, const TProgramBuilder::TBinaryWideLambda& init, const TProgramBuilder::TTernaryWideLambda& update, const TProgramBuilder::TBinaryWideLambda& finish) { + return SPILLING ? + pb.WideLastCombinerWithSpilling(flow, extractor, init, update, finish): + pb.WideLastCombiner(flow, extractor, init, update, finish); +} + } // unnamed #if !defined(MKQL_RUNTIME_VERSION) || MKQL_RUNTIME_VERSION >= 18u @@ -996,8 +1005,11 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideCombinerPerfTest) { #endif #if !defined(MKQL_RUNTIME_VERSION) || MKQL_RUNTIME_VERSION >= 29u Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { - Y_UNIT_TEST_LLVM(TestLongStringsRefCounting) { - TSetup setup; + Y_UNIT_TEST_LLVM_SPILLING(TestLongStringsRefCounting) { + // callable WideLastCombinerWithSpilling was introduced in 49 version of runtime + if (MKQL_RUNTIME_VERSION < 49U && SPILLING) return; + + TSetup setup; TProgramBuilder& pb = *setup.PgmBuilder; const auto dataType = pb.NewDataType(NUdf::TDataType::Id); @@ -1035,7 +1047,7 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { const auto list = pb.NewList(tupleType, {data1, data2, data3, data4, data5, data6, data7, data8, data9}); - const auto pgmReturn = pb.Collect(pb.NarrowMap(pb.WideLastCombiner(pb.ExpandMap(pb.ToFlow(list), + const auto pgmReturn = pb.Collect(pb.NarrowMap(WideLastCombiner(pb, pb.ExpandMap(pb.ToFlow(list), [&](TRuntimeNode item) -> TRuntimeNode::TList { return {pb.Nth(item, 0U), pb.Nth(item, 1U)}; }), [&](TRuntimeNode::TList items) -> TRuntimeNode::TList { return {items.front()}; }, [&](TRuntimeNode::TList keys, TRuntimeNode::TList items) -> TRuntimeNode::TList { @@ -1059,22 +1071,35 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { )); const auto graph = setup.BuildGraph(pgmReturn); + if (SPILLING) { + graph->GetContext().SpillerFactory = std::make_shared(); + } const auto iterator = graph->GetValue().GetListIterator(); + + std::unordered_set expected { + "key one", + "very long value 2 / key two", + "very long key one", + "very long value 8 / very long value 7 / very long value 6" + }; + NUdf::TUnboxedValue item; - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "key one"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long value 2 / key two"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long key one"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long value 8 / very long value 7 / very long value 6"); + while (!expected.empty()) { + UNIT_ASSERT(iterator.Next(item)); + const auto actual = TString(item.AsStringRef()); + + auto it = expected.find(actual); + UNIT_ASSERT(it != expected.end()); + expected.erase(it); + } UNIT_ASSERT(!iterator.Next(item)); UNIT_ASSERT(!iterator.Next(item)); } - Y_UNIT_TEST_LLVM(TestLongStringsPasstroughtRefCounting) { - TSetup setup; + Y_UNIT_TEST_LLVM_SPILLING(TestLongStringsPasstroughtRefCounting) { + // callable WideLastCombinerWithSpilling was introduced in 49 version of runtime + if (MKQL_RUNTIME_VERSION < 49U && SPILLING) return; + TSetup setup; TProgramBuilder& pb = *setup.PgmBuilder; const auto dataType = pb.NewDataType(NUdf::TDataType::Id); @@ -1111,7 +1136,7 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { const auto list = pb.NewList(tupleType, {data1, data2, data3, data4, data5, data6, data7, data8, data9}); - const auto pgmReturn = pb.Collect(pb.NarrowMap(pb.WideLastCombiner(pb.ExpandMap(pb.ToFlow(list), + const auto pgmReturn = pb.Collect(pb.NarrowMap(WideLastCombiner(pb, pb.ExpandMap(pb.ToFlow(list), [&](TRuntimeNode item) -> TRuntimeNode::TList { return {pb.Nth(item, 0U), pb.Nth(item, 1U)}; }), [&](TRuntimeNode::TList items) -> TRuntimeNode::TList { return {items.front()}; }, [&](TRuntimeNode::TList keys, TRuntimeNode::TList items) -> TRuntimeNode::TList { @@ -1134,22 +1159,38 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { )); const auto graph = setup.BuildGraph(pgmReturn); + if (SPILLING) { + graph->GetContext().SpillerFactory = std::make_shared(); + } const auto iterator = graph->GetValue().GetListIterator(); + + std::unordered_set expected { + "very long value 1 / key one / very long value 1 / key one", + "very long value 3 / key two / very long value 2 / key two", + "very long value 4 / very long key one / very long value 4 / very long key one", + "very long value 9 / very long key two / very long value 5 / very long key two" + }; + NUdf::TUnboxedValue item; - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long value 1 / key one / very long value 1 / key one"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long value 3 / key two / very long value 2 / key two"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long value 4 / very long key one / very long value 4 / very long key one"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "very long value 9 / very long key two / very long value 5 / very long key two"); + while (!expected.empty()) { + UNIT_ASSERT(iterator.Next(item)); + const auto actual = TString(item.AsStringRef()); + + auto it = expected.find(actual); + UNIT_ASSERT(it != expected.end()); + expected.erase(it); + } UNIT_ASSERT(!iterator.Next(item)); UNIT_ASSERT(!iterator.Next(item)); } - Y_UNIT_TEST_LLVM(TestDoNotCalculateUnusedInput) { - TSetup setup; + Y_UNIT_TEST_LLVM_SPILLING(TestDoNotCalculateUnusedInput) { + // Test is broken. Remove this if after YQL-18808. + if (SPILLING) return; + + // callable WideLastCombinerWithSpilling was introduced in 49 version of runtime + if (MKQL_RUNTIME_VERSION < 49U && SPILLING) return; + TSetup setup; TProgramBuilder& pb = *setup.PgmBuilder; const auto dataType = pb.NewDataType(NUdf::TDataType::Id); @@ -1183,7 +1224,7 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { const auto landmine = pb.NewDataLiteral("ACHTUNG MINEN!"); - const auto pgmReturn = pb.Collect(pb.NarrowMap(pb.WideLastCombiner(pb.ExpandMap(pb.ToFlow(list), + const auto pgmReturn = pb.Collect(pb.NarrowMap(WideLastCombiner(pb, pb.ExpandMap(pb.ToFlow(list), [&](TRuntimeNode item) -> TRuntimeNode::TList { return {pb.Nth(item, 0U), pb.Unwrap(pb.Nth(item, 1U), landmine, __FILE__, __LINE__, 0), pb.Nth(item, 2U)}; }), [&](TRuntimeNode::TList items) -> TRuntimeNode::TList { return {items.front()}; }, [&](TRuntimeNode::TList keys, TRuntimeNode::TList items) -> TRuntimeNode::TList { @@ -1207,18 +1248,32 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { )); const auto graph = setup.BuildGraph(pgmReturn); + if (SPILLING) { + graph->GetContext().SpillerFactory = std::make_shared(); + } + std::unordered_set expected { + "key one / value 2 / value 1 / value 5 / value 4", + "key two / value 4 / value 3 / value 3 / value 2" + }; + const auto iterator = graph->GetValue().GetListIterator(); NUdf::TUnboxedValue item; - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "key one / value 2 / value 1 / value 5 / value 4"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "key two / value 4 / value 3 / value 3 / value 2"); + while (!expected.empty()) { + UNIT_ASSERT(iterator.Next(item)); + const auto actual = TString(item.AsStringRef()); + + auto it = expected.find(actual); + UNIT_ASSERT(it != expected.end()); + expected.erase(it); + } UNIT_ASSERT(!iterator.Next(item)); UNIT_ASSERT(!iterator.Next(item)); } - Y_UNIT_TEST_LLVM(TestDoNotCalculateUnusedOutput) { - TSetup setup; + Y_UNIT_TEST_LLVM_SPILLING(TestDoNotCalculateUnusedOutput) { + // callable WideLastCombinerWithSpilling was introduced in 49 version of runtime + if (MKQL_RUNTIME_VERSION < 49U && SPILLING) return; + TSetup setup; TProgramBuilder& pb = *setup.PgmBuilder; const auto dataType = pb.NewDataType(NUdf::TDataType::Id); @@ -1252,7 +1307,7 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { const auto landmine = pb.NewDataLiteral("ACHTUNG MINEN!"); - const auto pgmReturn = pb.Collect(pb.NarrowMap(pb.WideLastCombiner(pb.ExpandMap(pb.ToFlow(list), + const auto pgmReturn = pb.Collect(pb.NarrowMap(WideLastCombiner(pb, pb.ExpandMap(pb.ToFlow(list), [&](TRuntimeNode item) -> TRuntimeNode::TList { return {pb.Nth(item, 0U), pb.Nth(item, 1U), pb.Nth(item, 2U)}; }), [&](TRuntimeNode::TList items) -> TRuntimeNode::TList { return {items.front()}; }, [&](TRuntimeNode::TList, TRuntimeNode::TList items) -> TRuntimeNode::TList { @@ -1268,18 +1323,32 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { )); const auto graph = setup.BuildGraph(pgmReturn); + if (SPILLING) { + graph->GetContext().SpillerFactory = std::make_shared(); + } + std::unordered_set expected { + "key one: value 1, value 4, value 5, value 1, value 2", + "key two: value 2, value 3, value 3, value 4" + }; + const auto iterator = graph->GetValue().GetListIterator(); NUdf::TUnboxedValue item; - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "key one: value 1, value 4, value 5, value 1, value 2"); - UNIT_ASSERT(iterator.Next(item)); - UNBOXED_VALUE_STR_EQUAL(item, "key two: value 2, value 3, value 3, value 4"); + while (!expected.empty()) { + UNIT_ASSERT(iterator.Next(item)); + const auto actual = TString(item.AsStringRef()); + + auto it = expected.find(actual); + UNIT_ASSERT(it != expected.end()); + expected.erase(it); + } UNIT_ASSERT(!iterator.Next(item)); UNIT_ASSERT(!iterator.Next(item)); } - Y_UNIT_TEST_LLVM(TestThinAllLambdas) { - TSetup setup; + Y_UNIT_TEST_LLVM_SPILLING(TestThinAllLambdas) { + // callable WideLastCombinerWithSpilling was introduced in 49 version of runtime + if (MKQL_RUNTIME_VERSION < 49U && SPILLING) return; + TSetup setup; TProgramBuilder& pb = *setup.PgmBuilder; const auto tupleType = pb.NewTupleType({}); @@ -1287,7 +1356,7 @@ Y_UNIT_TEST_SUITE(TMiniKQLWideLastCombinerTest) { const auto list = pb.NewList(tupleType, {data, data, data, data}); - const auto pgmReturn = pb.Collect(pb.NarrowMap(pb.WideLastCombiner(pb.ExpandMap(pb.ToFlow(list), + const auto pgmReturn = pb.Collect(pb.NarrowMap(WideLastCombiner(pb, pb.ExpandMap(pb.ToFlow(list), [](TRuntimeNode) -> TRuntimeNode::TList { return {}; }), [](TRuntimeNode::TList items) { return items; }, [](TRuntimeNode::TList, TRuntimeNode::TList items) { return items; }, diff --git a/ydb/library/yql/minikql/computation/mkql_computation_node.cpp b/ydb/library/yql/minikql/computation/mkql_computation_node.cpp index 6e5020a6b98b..ebcdd790cd2e 100644 --- a/ydb/library/yql/minikql/computation/mkql_computation_node.cpp +++ b/ydb/library/yql/minikql/computation/mkql_computation_node.cpp @@ -55,6 +55,9 @@ TComputationContext::TComputationContext(const THolderFactory& holderFactory, , WideFields(mutables.CurWideFieldsIndex, nullptr) , TypeEnv(opts.TypeEnv) , Mutables(mutables) + , TypeInfoHelper(new TTypeInfoHelper) + , CountersProvider(opts.CountersProvider) + , SecureParamsProvider(opts.SecureParamsProvider) { std::fill_n(MutableValues.get(), mutables.CurValueIndex, NUdf::TUnboxedValue(NUdf::TUnboxedValuePod::Invalid())); diff --git a/ydb/library/yql/minikql/computation/mkql_computation_node.h b/ydb/library/yql/minikql/computation/mkql_computation_node.h index 79070c1dfe6b..cad7d9d3a7c4 100644 --- a/ydb/library/yql/minikql/computation/mkql_computation_node.h +++ b/ydb/library/yql/minikql/computation/mkql_computation_node.h @@ -45,33 +45,25 @@ struct TComputationOpts { }; struct TComputationOptsFull: public TComputationOpts { - TComputationOptsFull(IStatsRegistry* stats, TAllocState& allocState, IRandomProvider& randomProvider, - ITimeProvider& timeProvider, NUdf::EValidatePolicy validatePolicy, const NUdf::ISecureParamsProvider* secureParamsProvider) + TComputationOptsFull(IStatsRegistry* stats, TAllocState& allocState, const TTypeEnvironment& typeEnv, IRandomProvider& randomProvider, + ITimeProvider& timeProvider, NUdf::EValidatePolicy validatePolicy, const NUdf::ISecureParamsProvider* secureParamsProvider, NUdf::ICountersProvider* countersProvider) : TComputationOpts(stats) , AllocState(allocState) - , RandomProvider(randomProvider) - , TimeProvider(timeProvider) - , ValidatePolicy(validatePolicy) - , SecureParamsProvider(secureParamsProvider) - {} - - TComputationOptsFull(IStatsRegistry* stats, TTypeEnvironment* typeEnv, IRandomProvider& randomProvider, - ITimeProvider& timeProvider, NUdf::EValidatePolicy validatePolicy, const NUdf::ISecureParamsProvider* secureParamsProvider) - : TComputationOpts(stats) - , AllocState(typeEnv->GetAllocator().Ref()) , TypeEnv(typeEnv) , RandomProvider(randomProvider) , TimeProvider(timeProvider) , ValidatePolicy(validatePolicy) , SecureParamsProvider(secureParamsProvider) + , CountersProvider(countersProvider) {} TAllocState& AllocState; - TTypeEnvironment* TypeEnv = nullptr; + const TTypeEnvironment& TypeEnv; IRandomProvider& RandomProvider; ITimeProvider& TimeProvider; NUdf::EValidatePolicy ValidatePolicy; - const NUdf::ISecureParamsProvider* SecureParamsProvider; + const NUdf::ISecureParamsProvider *const SecureParamsProvider; + NUdf::ICountersProvider *const CountersProvider; }; struct TWideFieldsInitInfo { @@ -119,9 +111,12 @@ struct TComputationContext : public TComputationContextLLVM { bool ExecuteLLVM = false; arrow::MemoryPool& ArrowMemoryPool; std::vector WideFields; - TTypeEnvironment* TypeEnv = nullptr; + const TTypeEnvironment& TypeEnv; const TComputationMutables Mutables; std::shared_ptr SpillerFactory; + const NUdf::ITypeInfoHelper::TPtr TypeInfoHelper; + NUdf::ICountersProvider *const CountersProvider; + const NUdf::ISecureParamsProvider *const SecureParamsProvider; TComputationContext(const THolderFactory& holderFactory, const NUdf::IValueBuilder* builder, @@ -391,13 +386,8 @@ struct TComputationPatternOpts { NUdf::ICountersProvider* CountersProvider = nullptr; const NUdf::ISecureParamsProvider* SecureParamsProvider = nullptr; - /// \todo split and exclude TComputationOptsFull ToComputationOptions(IRandomProvider& randomProvider, ITimeProvider& timeProvider, TAllocState* allocStatePtr = nullptr) const { - return TComputationOptsFull(Stats, allocStatePtr ? *allocStatePtr : AllocState, randomProvider, timeProvider, ValidatePolicy, SecureParamsProvider); - } - - TComputationOptsFull ToComputationOptions(IRandomProvider& randomProvider, ITimeProvider& timeProvider, TTypeEnvironment* typeEnv) const { - return TComputationOptsFull(Stats, typeEnv, randomProvider, timeProvider, ValidatePolicy, SecureParamsProvider); + return TComputationOptsFull(Stats, allocStatePtr ? *allocStatePtr : AllocState, Env, randomProvider, timeProvider, ValidatePolicy, SecureParamsProvider, CountersProvider); } }; diff --git a/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.cpp b/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.cpp index 8d9f9bedcf6c..21504a860564 100644 --- a/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.cpp +++ b/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.cpp @@ -991,10 +991,6 @@ Value* TUnboxedImmutableCodegeneratorNode::CreateGetValue(const TCodegenContext& return ConstantInt::get(Type::getInt128Ty(ctx.Codegen.GetContext()), APInt(128, 2, reinterpret_cast(&UnboxedValue))); } -TUnboxedImmutableRunCodegeneratorNode::TUnboxedImmutableRunCodegeneratorNode(TMemoryUsageInfo* memInfo, NUdf::TUnboxedValue&& value) - : TUnboxedImmutableComputationNode(memInfo, std::move(value)) -{} - TExternalCodegeneratorNode::TExternalCodegeneratorNode(TComputationMutables& mutables, EValueRepresentation kind) : TExternalComputationNode(mutables, kind) {} diff --git a/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.h.txt b/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.h.txt index 149beee42ea6..4d3c38b59cbf 100644 --- a/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.h.txt +++ b/ydb/library/yql/minikql/computation/mkql_computation_node_codegen.h.txt @@ -11,7 +11,6 @@ using NYql::GetMethodPtrIndex; using NYql::GetMethodIndex; using TUnboxedImmutableCodegeneratorNode = TUnboxedImmutableComputationNode; -using TUnboxedImmutableRunCodegeneratorNode = TUnboxedImmutableComputationNode; using TExternalCodegeneratorNode = TExternalComputationNode; using TWideFlowProxyCodegeneratorNode = TWideFlowProxyComputationNode; @@ -944,12 +943,6 @@ private: Value* CreateGetValue(const TCodegenContext& ctx, BasicBlock*&) const final; }; -class TUnboxedImmutableRunCodegeneratorNode: public TUnboxedImmutableComputationNode, public ICodegeneratorRunNode -{ -public: - TUnboxedImmutableRunCodegeneratorNode(TMemoryUsageInfo* memInfo, NUdf::TUnboxedValue&& value); -}; - class TExternalCodegeneratorNode: public TExternalComputationNode, public ICodegeneratorExternalNode { public: diff --git a/ydb/library/yql/minikql/computation/mkql_spiller_factory.h b/ydb/library/yql/minikql/computation/mkql_spiller_factory.h index 473686a235c5..4eed968079bb 100644 --- a/ydb/library/yql/minikql/computation/mkql_spiller_factory.h +++ b/ydb/library/yql/minikql/computation/mkql_spiller_factory.h @@ -2,6 +2,8 @@ #include "mkql_spiller.h" +#include + namespace NKikimr::NMiniKQL { class ISpillerFactory : private TNonCopyable @@ -9,6 +11,8 @@ class ISpillerFactory : private TNonCopyable public: virtual ISpiller::TPtr CreateSpiller() = 0; + virtual void SetTaskCounters(TIntrusivePtr spillingTaskCounters) = 0; + virtual ~ISpillerFactory(){} }; diff --git a/ydb/library/yql/minikql/computation/mock_spiller_factory_ut.h b/ydb/library/yql/minikql/computation/mock_spiller_factory_ut.h new file mode 100644 index 000000000000..21e6328c2a15 --- /dev/null +++ b/ydb/library/yql/minikql/computation/mock_spiller_factory_ut.h @@ -0,0 +1,21 @@ +#pragma once + +#include +#include + +namespace NKikimr::NMiniKQL { + +using namespace NActors; + +class TMockSpillerFactory : public ISpillerFactory +{ +public: + void SetTaskCounters(TIntrusivePtr /*spillingTaskCounters*/) override { + } + + ISpiller::TPtr CreateSpiller() override { + return CreateMockSpiller(); + } +}; + +} // namespace NKikimr::NMiniKQL \ No newline at end of file diff --git a/ydb/library/yql/minikql/mkql_program_builder.cpp b/ydb/library/yql/minikql/mkql_program_builder.cpp index 2272ae640b3e..3bb7463b776e 100644 --- a/ydb/library/yql/minikql/mkql_program_builder.cpp +++ b/ydb/library/yql/minikql/mkql_program_builder.cpp @@ -2160,28 +2160,6 @@ TRuntimeNode TProgramBuilder::GraceSelfJoin(TRuntimeNode flowLeft, EJoinKind jo return GraceJoinCommon(__func__, flowLeft, {}, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings); } -TRuntimeNode TProgramBuilder::GraceJoinWithSpilling(TRuntimeNode flowLeft, TRuntimeNode flowRight, EJoinKind joinKind, - const TArrayRef& leftKeyColumns, const TArrayRef& rightKeyColumns, - const TArrayRef& leftRenames, const TArrayRef& rightRenames, TType* returnType, EAnyJoinSettings anyJoinSettings ) { - - if constexpr (RuntimeVersion < 50U) { - THROW yexception() << "Runtime version (" << RuntimeVersion << ") too old for " << __func__; - } - - return GraceJoinCommon(__func__, flowLeft, flowRight, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings); -} - -TRuntimeNode TProgramBuilder::GraceSelfJoinWithSpilling(TRuntimeNode flowLeft, EJoinKind joinKind, - const TArrayRef& leftKeyColumns, const TArrayRef& rightKeyColumns, - const TArrayRef& leftRenames, const TArrayRef& rightRenames, TType* returnType, EAnyJoinSettings anyJoinSettings ) { - - if constexpr (RuntimeVersion < 50U) { - THROW yexception() << "Runtime version (" << RuntimeVersion << ") too old for " << __func__; - } - - return GraceJoinCommon(__func__, flowLeft, {}, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings); -} - TRuntimeNode TProgramBuilder::ToSortedDict(TRuntimeNode list, bool all, const TUnaryLambda& keySelector, const TUnaryLambda& payloadSelector, bool isCompact, ui64 itemsCountHint) { return ToDict(list, all, keySelector, payloadSelector, __func__, isCompact, itemsCountHint); diff --git a/ydb/library/yql/minikql/mkql_program_builder.h b/ydb/library/yql/minikql/mkql_program_builder.h index 97daf163c586..f8a6d5cd8827 100644 --- a/ydb/library/yql/minikql/mkql_program_builder.h +++ b/ydb/library/yql/minikql/mkql_program_builder.h @@ -86,6 +86,10 @@ inline void AddAnyJoinSide(EAnyJoinSettings& combined, EAnyJoinSettings value) { combined = (EAnyJoinSettings)combinedVal; } +inline bool HasSpillingFlag(const TCallable& callable) { + return TStringBuf(callable.GetType()->GetName()).EndsWith("WithSpilling"_sb); +} + #define MKQL_SCRIPT_TYPES(xx) \ xx(Unknown, 0, unknown, false) \ xx(Python, 1, python, false) \ diff --git a/ydb/library/yql/providers/common/mkql/yql_provider_mkql.cpp b/ydb/library/yql/providers/common/mkql/yql_provider_mkql.cpp index f72f6f32e1bb..d1aa870f24eb 100644 --- a/ydb/library/yql/providers/common/mkql/yql_provider_mkql.cpp +++ b/ydb/library/yql/providers/common/mkql/yql_provider_mkql.cpp @@ -1704,13 +1704,6 @@ TMkqlCommonCallableCompiler::TShared::TShared() { const auto returnType = BuildType(node, *node.GetTypeAnn(), ctx.ProgramBuilder); - // TODO: use PRAGMA - bool IsSpillingAllowed = false; - if (RuntimeVersion >= 50U && IsSpillingAllowed) { - return selfJoin - ? ctx.ProgramBuilder.GraceSelfJoinWithSpilling(flowLeft, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings) - : ctx.ProgramBuilder.GraceJoinWithSpilling(flowLeft, flowRight, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings); - } return selfJoin ? ctx.ProgramBuilder.GraceSelfJoin(flowLeft, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings) : ctx.ProgramBuilder.GraceJoin(flowLeft, flowRight, joinKind, leftKeyColumns, rightKeyColumns, leftRenames, rightRenames, returnType, anyJoinSettings); diff --git a/ydb/library/yql/providers/dq/actors/worker_actor.cpp b/ydb/library/yql/providers/dq/actors/worker_actor.cpp index f7a33da883ec..c34ba8e1e5b3 100644 --- a/ydb/library/yql/providers/dq/actors/worker_actor.cpp +++ b/ydb/library/yql/providers/dq/actors/worker_actor.cpp @@ -280,8 +280,9 @@ class TDqWorker: public TRichActor limits.ChannelBufferSize = 20_MB; limits.OutputChunkMaxSize = 2_MB; - auto wakeup = [this]{ ResumeExecution(EResumeSource::Default); }; - std::shared_ptr execCtx = std::make_shared(TraceId, std::move(wakeup)); + auto wakeupCallback = [this]{ ResumeExecution(EResumeSource::Default); }; + auto errorCallback = [this](const TString& error){ this->Send(this->SelfId(), new TEvDqFailure(StatusIds::INTERNAL_ERROR, error)); }; + std::shared_ptr execCtx = std::make_shared(TraceId, std::move(wakeupCallback), std::move(errorCallback)); Send(TaskRunnerActor, new TEvTaskRunnerCreate(std::move(ev->Get()->Record.GetTask()), limits, NDqProto::DQ_STATS_MODE_BASIC, execCtx)); } diff --git a/ydb/library/yql/providers/dq/common/yql_dq_settings.cpp b/ydb/library/yql/providers/dq/common/yql_dq_settings.cpp index 457474767d03..68e8fdf84269 100644 --- a/ydb/library/yql/providers/dq/common/yql_dq_settings.cpp +++ b/ydb/library/yql/providers/dq/common/yql_dq_settings.cpp @@ -1,4 +1,5 @@ #include "yql_dq_settings.h" +#include namespace NYql { @@ -79,10 +80,12 @@ TDqConfiguration::TDqConfiguration() { REGISTER_SETTING(*this, SpillingEngine) .Parser([](const TString& v) { return FromString(v); - }) - .ValueSetter([this](const TString&, TDqSettings::ESpillingEngine value) { - SpillingEngine = value; - if (value != TDqSettings::ESpillingEngine::Disable) { + }); + + REGISTER_SETTING(*this, EnableSpillingInChannels) + .ValueSetter([this](const TString&, bool value) { + EnableSpillingInChannels = value; + if (value) { SplitStageOnDqReplicate = false; EnableDqReplicate = true; } @@ -98,6 +101,21 @@ TDqConfiguration::TDqConfiguration() { REGISTER_SETTING(*this, _MaxAttachmentsSize); REGISTER_SETTING(*this, DisableCheckpoints); + REGISTER_SETTING(*this, EnableSpillingNodes) + .Parser([](const TString& v) { + ui64 res = 0; + TVector vec; + StringSplitter(v).SplitBySet(",;| ").AddTo(&vec); + for (auto& s: vec) { + if (s.empty()) { + throw yexception() << "Empty value item"; + } + + auto value = FromString(s); + res |= ui64(value); + } + return res; + }); } } // namespace NYql diff --git a/ydb/library/yql/providers/dq/common/yql_dq_settings.h b/ydb/library/yql/providers/dq/common/yql_dq_settings.h index 436b54f1acb4..169129a70442 100644 --- a/ydb/library/yql/providers/dq/common/yql_dq_settings.h +++ b/ydb/library/yql/providers/dq/common/yql_dq_settings.h @@ -60,6 +60,8 @@ struct TDqSettings { static constexpr ui32 MaxDPccpDPTableSize = 40000U; static constexpr ui64 MaxAttachmentsSize = 2_GB; static constexpr bool SplitStageOnDqReplicate = true; + static constexpr ui64 EnableSpillingNodes = 0; + static constexpr bool EnableSpillingInChannels = false; }; using TPtr = std::shared_ptr; @@ -131,6 +133,9 @@ struct TDqSettings { NCommon::TConfSetting DisableLLVMForBlockStages; NCommon::TConfSetting SplitStageOnDqReplicate; + NCommon::TConfSetting EnableSpillingNodes; + NCommon::TConfSetting EnableSpillingInChannels; + NCommon::TConfSetting _MaxAttachmentsSize; NCommon::TConfSetting DisableCheckpoints; @@ -185,6 +190,7 @@ struct TDqSettings { SAVE_SETTING(ExportStats); SAVE_SETTING(TaskRunnerStats); SAVE_SETTING(SpillingEngine); + SAVE_SETTING(EnableSpillingInChannels); SAVE_SETTING(DisableCheckpoints); #undef SAVE_SETTING } @@ -211,10 +217,20 @@ struct TDqSettings { } } - bool IsSpillingEnabled() const { + bool IsSpillingEngineEnabled() const { return SpillingEngine.Get().GetOrElse(TDqSettings::TDefault::SpillingEngine) != ESpillingEngine::Disable; } + bool IsSpillingInChannelsEnabled() const { + if (!IsSpillingEngineEnabled()) return false; + return EnableSpillingInChannels.Get().GetOrElse(TDqSettings::TDefault::EnableSpillingInChannels) != false; + } + + ui64 GetEnabledSpillingNodes() const { + if (!IsSpillingEngineEnabled()) return 0; + return EnableSpillingNodes.Get().GetOrElse(TDqSettings::TDefault::EnableSpillingNodes); + } + bool IsDqReplicateEnabled(const TTypeAnnotationContext& typesCtx) const { return EnableDqReplicate.Get().GetOrElse( typesCtx.BlockEngineMode != EBlockEngineMode::Disable || TDqSettings::TDefault::EnableDqReplicate); diff --git a/ydb/library/yql/providers/dq/local_gateway/yql_dq_gateway_local.cpp b/ydb/library/yql/providers/dq/local_gateway/yql_dq_gateway_local.cpp index 8b87f8aebcd0..7c4f592454c6 100644 --- a/ydb/library/yql/providers/dq/local_gateway/yql_dq_gateway_local.cpp +++ b/ydb/library/yql/providers/dq/local_gateway/yql_dq_gateway_local.cpp @@ -80,9 +80,8 @@ class TLocalServiceHolder { TActorSetupCmd(resman, TMailboxType::Simple, 0)); if (withSpilling) { - char tempDir[MAX_PATH]; - if (MakeTempDir(tempDir, nullptr) != 0) - ythrow yexception() << "LocalServiceHolder: Can't create temporary directory " << tempDir; + auto tempDir = NDq::GetTmpSpillingRootForCurrentUser(); + MakeDirIfNotExist(tempDir); auto spillingActor = NDq::CreateDqLocalFileSpillingService(NDq::TFileSpillingServiceConfig{.Root = tempDir, .CleanupOnShutdown = true}, MakeIntrusive(lwmGroup)); diff --git a/ydb/library/yql/providers/dq/opt/logical_optimize.cpp b/ydb/library/yql/providers/dq/opt/logical_optimize.cpp index 6cd3c38240df..a02369aadb4f 100644 --- a/ydb/library/yql/providers/dq/opt/logical_optimize.cpp +++ b/ydb/library/yql/providers/dq/opt/logical_optimize.cpp @@ -137,7 +137,8 @@ class TDqsLogicalOptProposalTransformer : public TOptimizeTransformerBase { bool syncActor = Config->ComputeActorType.Get() != "async"; return NHopping::RewriteAsHoppingWindow(node, ctx, input.Cast(), analyticsHopping, lateArrivalDelay, defaultWatermarksMode, syncActor); } else { - return DqRewriteAggregate(node, ctx, TypesCtx, true, Config->UseAggPhases.Get().GetOrElse(false), Config->UseFinalizeByKey.Get().GetOrElse(false)); + NDq::TSpillingSettings spillingSettings(Config->GetEnabledSpillingNodes()); + return DqRewriteAggregate(node, ctx, TypesCtx, true, Config->UseAggPhases.Get().GetOrElse(false), Config->UseFinalizeByKey.Get().GetOrElse(false), spillingSettings.IsAggregationSpillingEnabled()); } } return node; diff --git a/ydb/library/yql/providers/dq/planner/execution_planner.cpp b/ydb/library/yql/providers/dq/planner/execution_planner.cpp index aa3aca5b4468..793ab143295a 100644 --- a/ydb/library/yql/providers/dq/planner/execution_planner.cpp +++ b/ydb/library/yql/providers/dq/planner/execution_planner.cpp @@ -442,7 +442,7 @@ namespace NYql::NDqs { bool enableSpilling = false; if (task.Outputs.size() > 1) { - enableSpilling = Settings->IsSpillingEnabled(); + enableSpilling = Settings->IsSpillingInChannelsEnabled(); } for (auto& output : task.Outputs) { FillOutputDesc(*taskDesc.AddOutputs(), output, enableSpilling); @@ -457,7 +457,7 @@ namespace NYql::NDqs { taskMeta.SetStageId(publicId); taskDesc.MutableMeta()->PackFrom(taskMeta); taskDesc.SetStageId(stageId); - taskDesc.SetEnableSpilling(Settings->IsSpillingEnabled()); + taskDesc.SetEnableSpilling(Settings->GetEnabledSpillingNodes()); if (Settings->DisableLLVMForBlockStages.Get().GetOrElse(true)) { auto& stage = TasksGraph.GetStageInfo(task.StageId).Meta.Stage; @@ -685,10 +685,11 @@ namespace NYql::NDqs { Y_ABORT_UNLESS(false); } */ + TSpillingSettings spillingSettings{Settings->GetEnabledSpillingNodes()}; StagePrograms[stageInfo.first] = std::make_tuple( NDq::BuildProgram( stage.Program(), *paramsType, compiler, typeEnv, *FunctionRegistry, - ExprContext, fakeReads), + ExprContext, fakeReads, spillingSettings), stageId, publicId); } } diff --git a/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp b/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp index 7f0683a01254..1e4f1e69ff45 100644 --- a/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp +++ b/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp @@ -760,9 +760,10 @@ class TDqExecTransformer: public TExecTransformerBase, TCounters TVector fakeReads; auto paramsType = NDq::CollectParameters(programLambda, ctx); + NDq::TSpillingSettings spillingSettings{State->Settings->GetEnabledSpillingNodes()}; *lambda = NDq::BuildProgram( programLambda, *paramsType, compiler, typeEnv, *State->FunctionRegistry, - ctx, fakeReads); + ctx, fakeReads, spillingSettings); } auto block = MeasureBlock("RuntimeNodeVisitor"); diff --git a/ydb/library/yql/providers/dq/provider/yql_dq_validate.cpp b/ydb/library/yql/providers/dq/provider/yql_dq_validate.cpp index c5c014568982..b01c855618d7 100644 --- a/ydb/library/yql/providers/dq/provider/yql_dq_validate.cpp +++ b/ydb/library/yql/providers/dq/provider/yql_dq_validate.cpp @@ -147,7 +147,7 @@ class TDqExecutionValidator { , State_(state) , CheckSelfJoin_(!TypeCtx_.ForceDq && !State_->Settings->SplitStageOnDqReplicate.Get().GetOrElse(TDqSettings::TDefault::SplitStageOnDqReplicate) - && !State_->Settings->IsSpillingEnabled()) + && !State_->Settings->IsSpillingInChannelsEnabled()) {} bool ValidateDqExecution(const TExprNode& node) { diff --git a/ydb/library/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp b/ydb/library/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp index bbf8732e69b3..f74d35b82d54 100644 --- a/ydb/library/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp +++ b/ydb/library/yql/providers/yt/lib/lambda_builder/lambda_builder.cpp @@ -198,7 +198,7 @@ THolder TLambdaBuilder::BuildGraph( auto pattern = preparePatternFunc(); YQL_ENSURE(pattern); - const TComputationOptsFull computeOpts(JobStats, Alloc.Ref(), *randomProvider, *timeProvider, validatePolicy, SecureParamsProvider); + const TComputationOptsFull computeOpts(JobStats, Alloc.Ref(), GetTypeEnvironment(), *randomProvider, *timeProvider, validatePolicy, SecureParamsProvider, Counters); auto graph = pattern->Clone(computeOpts); return MakeHolder(std::move(pattern), std::move(graph)); } diff --git a/ydb/library/yql/tests/sql/dq_file/part0/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part0/canondata/result.json index 9c9ff8126547..129986fc39db 100644 --- a/ydb/library/yql/tests/sql/dq_file/part0/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part0/canondata/result.json @@ -376,9 +376,9 @@ ], "test.test[aggregate-group_by_ru_join_qualified-default.txt-Debug]": [ { - "checksum": "50084dafc2ab2c5cce34d81fb4f874dc", - "size": 6271, - "uri": "https://{canondata_backend}/1899731/b7c6d9f20471cbb09c67e8664b660b2739bcb261/resource.tar.gz#test.test_aggregate-group_by_ru_join_qualified-default.txt-Debug_/opt.yql_patched" + "checksum": "ce88db3b4db16f8b52732b13ccb9c654", + "size": 6326, + "uri": "https://{canondata_backend}/1924537/435b3ac583db04c7fee418a89b1f7da98bd4b6d3/resource.tar.gz#test.test_aggregate-group_by_ru_join_qualified-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join_qualified-default.txt-Plan]": [ @@ -2622,9 +2622,9 @@ ], "test.test[pg-tpch-q05-default.txt-Debug]": [ { - "checksum": "c693e64e390003eeabe6b63953474b35", - "size": 14983, - "uri": "https://{canondata_backend}/1899731/b7c6d9f20471cbb09c67e8664b660b2739bcb261/resource.tar.gz#test.test_pg-tpch-q05-default.txt-Debug_/opt.yql_patched" + "checksum": "1b1037dff3bea5e1d8505bc5a1f592f9", + "size": 15018, + "uri": "https://{canondata_backend}/1903885/6c9c0a088205fc4d8727c537e045c6fb7c1b497c/resource.tar.gz#test.test_pg-tpch-q05-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q05-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part1/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part1/canondata/result.json index f9e203a97ebd..d8396ac8365b 100644 --- a/ydb/library/yql/tests/sql/dq_file/part1/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part1/canondata/result.json @@ -452,9 +452,9 @@ ], "test.test[aggregate-group_by_gs_duo--Debug]": [ { - "checksum": "25b53d13ad1aaa9594a636741e68d3ea", - "size": 4031, - "uri": "https://{canondata_backend}/1937027/591a1ceca790d81eaf524a7a3e730722b0d7bdb7/resource.tar.gz#test.test_aggregate-group_by_gs_duo--Debug_/opt.yql_patched" + "checksum": "59257d703f2930e08d3b7f69094f53c9", + "size": 4083, + "uri": "https://{canondata_backend}/1130705/223d79eda7e49588c54267c8b7c488154ed801c9/resource.tar.gz#test.test_aggregate-group_by_gs_duo--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_duo--Plan]": [ @@ -474,9 +474,9 @@ ], "test.test[aggregate-group_by_gs_flatten-default.txt-Debug]": [ { - "checksum": "49cd9fa336a5fec980795d60d46618cc", - "size": 3857, - "uri": "https://{canondata_backend}/1871102/093ef1237a5eb90e2e1f6670f45824dd7aa652e1/resource.tar.gz#test.test_aggregate-group_by_gs_flatten-default.txt-Debug_/opt.yql_patched" + "checksum": "78c0e51be35dd604e8e44b570af2881b", + "size": 3909, + "uri": "https://{canondata_backend}/1937424/0a7fa81182305af7b414a8e11e361266a61bc724/resource.tar.gz#test.test_aggregate-group_by_gs_flatten-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_flatten-default.txt-Plan]": [ @@ -496,9 +496,9 @@ ], "test.test[aggregate-group_by_ru_join_simple--Debug]": [ { - "checksum": "a62a866c286f0e79edc3e7d637dd8286", - "size": 5861, - "uri": "https://{canondata_backend}/1937492/31087c071b969ea75eb59570bea0f09cbff39e8c/resource.tar.gz#test.test_aggregate-group_by_ru_join_simple--Debug_/opt.yql_patched" + "checksum": "8091f4227f492e86b7a87449851aae3b", + "size": 5916, + "uri": "https://{canondata_backend}/1817427/c9d5452335f1bec2244311087380ff1cb48cc606/resource.tar.gz#test.test_aggregate-group_by_ru_join_simple--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join_simple--Plan]": [ @@ -1139,9 +1139,9 @@ ], "test.test[hor_join-out_sampling--Debug]": [ { - "checksum": "a7a731afc7145a0cee7006d262255061", - "size": 5549, - "uri": "https://{canondata_backend}/1917492/1ed6d08398686e90568735860251083949d84e4e/resource.tar.gz#test.test_hor_join-out_sampling--Debug_/opt.yql_patched" + "checksum": "8606ab607900ab1c285f8ba104e78a82", + "size": 5612, + "uri": "https://{canondata_backend}/1880306/b2c00ff823e390f0263acf2dbb68c876e0b31abd/resource.tar.gz#test.test_hor_join-out_sampling--Debug_/opt.yql_patched" } ], "test.test[hor_join-out_sampling--Plan]": [ @@ -2447,9 +2447,9 @@ ], "test.test[pg-tpch-q22-default.txt-Debug]": [ { - "checksum": "bf86e2ecfe907ef46642b1c2b774198d", - "size": 12643, - "uri": "https://{canondata_backend}/1937492/31087c071b969ea75eb59570bea0f09cbff39e8c/resource.tar.gz#test.test_pg-tpch-q22-default.txt-Debug_/opt.yql_patched" + "checksum": "887c733dbb7fff78b7a31c2119e08130", + "size": 12647, + "uri": "https://{canondata_backend}/1937150/ada84f9d693b3fd476e8c136d4ac0c0ff177279b/resource.tar.gz#test.test_pg-tpch-q22-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q22-default.txt-Plan]": [ @@ -2914,9 +2914,9 @@ ], "test.test[table_range-range_with_view--Debug]": [ { - "checksum": "8c3ab9a7d76fc314c4d5be392e075ad7", - "size": 3947, - "uri": "https://{canondata_backend}/1917492/1ed6d08398686e90568735860251083949d84e4e/resource.tar.gz#test.test_table_range-range_with_view--Debug_/opt.yql_patched" + "checksum": "5b6927fd57b28a292fc0a17afefdf28a", + "size": 3982, + "uri": "https://{canondata_backend}/1871102/a14717a3c8c558cf8fefe6d46cd5b04ed47ccc80/resource.tar.gz#test.test_table_range-range_with_view--Debug_/opt.yql_patched" } ], "test.test[table_range-range_with_view--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part10/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part10/canondata/result.json index 581daf054064..752b0176f516 100644 --- a/ydb/library/yql/tests/sql/dq_file/part10/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part10/canondata/result.json @@ -303,9 +303,9 @@ ], "test.test[aggregate-group_by_mul_gs_ru--Debug]": [ { - "checksum": "0638146b59bc226b8388bd56d7b4313f", - "size": 6072, - "uri": "https://{canondata_backend}/1923547/320f607d9e9c19a93a835d3183938f1fba6dd52c/resource.tar.gz#test.test_aggregate-group_by_mul_gs_ru--Debug_/opt.yql_patched" + "checksum": "ea41d7d534054a215601887475a08f36", + "size": 6129, + "uri": "https://{canondata_backend}/1773845/c255bb2163f40b1ca08f81b23e10624ae1969605/resource.tar.gz#test.test_aggregate-group_by_mul_gs_ru--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_mul_gs_ru--Plan]": [ @@ -340,23 +340,23 @@ "test.test[aggregate-percentiles_ungrouped--Results]": [], "test.test[aggregate-subquery_aggregation--Analyze]": [ { - "checksum": "9dec3798de4c88e1f79c01259fea4c30", - "size": 6627, - "uri": "https://{canondata_backend}/1600758/32cfdeb8c6377a2e7e62c6c4adbb95f25af7669b/resource.tar.gz#test.test_aggregate-subquery_aggregation--Analyze_/plan.txt" + "checksum": "49fb64ebed43ca080f30a0b8387033c3", + "size": 6149, + "uri": "https://{canondata_backend}/1773845/27425423327af8b10415a6bbb80a5aec7c55b13f/resource.tar.gz#test.test_aggregate-subquery_aggregation--Analyze_/plan.txt" } ], "test.test[aggregate-subquery_aggregation--Debug]": [ { - "checksum": "be5464a62cb392e7ba87b2cce10d0a26", - "size": 3147, - "uri": "https://{canondata_backend}/937458/3ce9d3f90d17a09aa182a1ae8e08f2f065219fab/resource.tar.gz#test.test_aggregate-subquery_aggregation--Debug_/opt.yql_patched" + "checksum": "9ebd4251fdeffd3409adc7fd7de958e7", + "size": 3531, + "uri": "https://{canondata_backend}/1923547/3423d2190488ae10821f2c300e70cd1b3ed9fc6d/resource.tar.gz#test.test_aggregate-subquery_aggregation--Debug_/opt.yql_patched" } ], "test.test[aggregate-subquery_aggregation--Plan]": [ { - "checksum": "9dec3798de4c88e1f79c01259fea4c30", - "size": 6627, - "uri": "https://{canondata_backend}/1600758/32cfdeb8c6377a2e7e62c6c4adbb95f25af7669b/resource.tar.gz#test.test_aggregate-subquery_aggregation--Plan_/plan.txt" + "checksum": "49fb64ebed43ca080f30a0b8387033c3", + "size": 6149, + "uri": "https://{canondata_backend}/1781765/e399fc9631f4d96fcb5c717860c01dc3564d9b6c/resource.tar.gz#test.test_aggregate-subquery_aggregation--Plan_/plan.txt" } ], "test.test[aggregate-subquery_aggregation--Results]": [], @@ -1097,9 +1097,9 @@ ], "test.test[in-in_sorted_by_tuple--Debug]": [ { - "checksum": "0658166c4e33cfd60f9a064b587f6535", - "size": 6168, - "uri": "https://{canondata_backend}/1597364/79684f84de863c4a9442337abe85ae217290f982/resource.tar.gz#test.test_in-in_sorted_by_tuple--Debug_/opt.yql_patched" + "checksum": "25eac6930d6256ef93f7c557aa32f2ce", + "size": 6188, + "uri": "https://{canondata_backend}/1923547/9b592402cd0b362fd0d6fa97573b2d5e8102c60a/resource.tar.gz#test.test_in-in_sorted_by_tuple--Debug_/opt.yql_patched" } ], "test.test[in-in_sorted_by_tuple--Plan]": [ @@ -1864,9 +1864,9 @@ ], "test.test[pg-aggregate_combine--Debug]": [ { - "checksum": "9fa29a162f8152ed7a7959ce4ec57f6e", - "size": 3715, - "uri": "https://{canondata_backend}/995452/59cb21feb51bcd4aaf002804abbfbb4a05ffe65f/resource.tar.gz#test.test_pg-aggregate_combine--Debug_/opt.yql_patched" + "checksum": "e50861c99c37ca8ce4bdd20148fa0753", + "size": 3750, + "uri": "https://{canondata_backend}/1903280/f275d6f071715007b59c0fcf1ce9a3d4eafb9599/resource.tar.gz#test.test_pg-aggregate_combine--Debug_/opt.yql_patched" } ], "test.test[pg-aggregate_combine--Plan]": [ @@ -2760,9 +2760,9 @@ ], "test.test[window-win_func_over_group_by_list_names--Debug]": [ { - "checksum": "93272f345d0939b83721190ea5597f30", - "size": 8213, - "uri": "https://{canondata_backend}/1923547/320f607d9e9c19a93a835d3183938f1fba6dd52c/resource.tar.gz#test.test_window-win_func_over_group_by_list_names--Debug_/opt.yql_patched" + "checksum": "a18ba714975de5a61f001dc9c971bd68", + "size": 8248, + "uri": "https://{canondata_backend}/1871182/0dad491c929525221cf344b9f6f54cd14f2ce4f0/resource.tar.gz#test.test_window-win_func_over_group_by_list_names--Debug_/opt.yql_patched" } ], "test.test[window-win_func_over_group_by_list_names--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part11/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part11/canondata/result.json index 930136ad66a7..df4c04a53094 100644 --- a/ydb/library/yql/tests/sql/dq_file/part11/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part11/canondata/result.json @@ -300,9 +300,9 @@ ], "test.test[aggregate-group_by_expr_only_join--Debug]": [ { - "checksum": "140831743035ca28afaa66e2ee8c34aa", - "size": 3398, - "uri": "https://{canondata_backend}/1817427/e0163eda749656672c865f5d2efac3ac10a5fe44/resource.tar.gz#test.test_aggregate-group_by_expr_only_join--Debug_/opt.yql_patched" + "checksum": "624474ec7bbb1e5dc661400cacc95efe", + "size": 3433, + "uri": "https://{canondata_backend}/1130705/868ff3a15d881fd30a076f428e6c4dbcfce68902/resource.tar.gz#test.test_aggregate-group_by_expr_only_join--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_expr_only_join--Plan]": [ @@ -322,9 +322,9 @@ ], "test.test[aggregate-group_by_gs_few_empty--Debug]": [ { - "checksum": "0ac1e3214471087b86a97fa21e69c3a2", - "size": 6528, - "uri": "https://{canondata_backend}/1936947/a99026e839b7e22714c2a9a81971a3b5e3ed1eb4/resource.tar.gz#test.test_aggregate-group_by_gs_few_empty--Debug_/opt.yql_patched" + "checksum": "3bbb4eb639e235c657a648557ce521ce", + "size": 6580, + "uri": "https://{canondata_backend}/1937367/e5d3b4a217429148a8315cf4e228d45b21f861fe/resource.tar.gz#test.test_aggregate-group_by_gs_few_empty--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_few_empty--Plan]": [ @@ -344,9 +344,9 @@ ], "test.test[aggregate-group_by_gs_simp--Debug]": [ { - "checksum": "f97f89c53ae93b6a6c7b9912e3950550", - "size": 5421, - "uri": "https://{canondata_backend}/1936947/a99026e839b7e22714c2a9a81971a3b5e3ed1eb4/resource.tar.gz#test.test_aggregate-group_by_gs_simp--Debug_/opt.yql_patched" + "checksum": "873e07e43b5b691bd8457dc9da486ff2", + "size": 5474, + "uri": "https://{canondata_backend}/1937367/e2a772964cf46b8a14a828d48a136378216522b9/resource.tar.gz#test.test_aggregate-group_by_gs_simp--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_simp--Plan]": [ @@ -366,9 +366,9 @@ ], "test.test[aggregate-group_by_rollup_grouping--Debug]": [ { - "checksum": "1bc4644b215619613368c778f07fa578", - "size": 7107, - "uri": "https://{canondata_backend}/1903280/419b5c18140d44a17c33d80899398c8647846b33/resource.tar.gz#test.test_aggregate-group_by_rollup_grouping--Debug_/opt.yql_patched" + "checksum": "c24d41f35785c78d23a30c634eacb1f8", + "size": 7167, + "uri": "https://{canondata_backend}/1773845/e1901cc86dc30911ad22db641ffb6fe66c04423b/resource.tar.gz#test.test_aggregate-group_by_rollup_grouping--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_grouping--Plan]": [ @@ -388,9 +388,9 @@ ], "test.test[aggregate-having_distinct_expr--Debug]": [ { - "checksum": "6a45904e076ad4dbbe5a94c52b69c43f", - "size": 3276, - "uri": "https://{canondata_backend}/1942173/5dda369a5c566435d55e882d65f0212fa3dfb906/resource.tar.gz#test.test_aggregate-having_distinct_expr--Debug_/opt.yql_patched" + "checksum": "4bdc0046de46875ae83dcdda879b5ba7", + "size": 3311, + "uri": "https://{canondata_backend}/1847551/155e040f6efb509114e481612c26cc3259caa89d/resource.tar.gz#test.test_aggregate-having_distinct_expr--Debug_/opt.yql_patched" } ], "test.test[aggregate-having_distinct_expr--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part12/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part12/canondata/result.json index 011170cad82e..89f710a03a5a 100644 --- a/ydb/library/yql/tests/sql/dq_file/part12/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part12/canondata/result.json @@ -272,9 +272,9 @@ ], "test.test[aggregate-group_by_rollup_grouping_hum--Debug]": [ { - "checksum": "6f56ba8703599d31b777058eed994bf0", - "size": 7947, - "uri": "https://{canondata_backend}/1777230/915011f8f5c826c23aaf0fd2e67aa8d2fb1f93cd/resource.tar.gz#test.test_aggregate-group_by_rollup_grouping_hum--Debug_/opt.yql_patched" + "checksum": "0fa3ad8c20368a624649240306680c0e", + "size": 8006, + "uri": "https://{canondata_backend}/1847551/cea98224a0242fa122932bfd335599c5107ce35b/resource.tar.gz#test.test_aggregate-group_by_rollup_grouping_hum--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_grouping_hum--Plan]": [ @@ -2838,9 +2838,9 @@ ], "test.test[pg-tpch-q12-default.txt-Debug]": [ { - "checksum": "e15d575beb51b6a932c2e163858f834e", - "size": 8284, - "uri": "https://{canondata_backend}/1942671/d6f4266e568a96dce8044dc5678d1f94b3d26355/resource.tar.gz#test.test_pg-tpch-q12-default.txt-Debug_/opt.yql_patched" + "checksum": "0cba1d5cb55b1e38ee956ec4335ad562", + "size": 8319, + "uri": "https://{canondata_backend}/1937150/0e405e71b6db1ea5ac683d7aa86fab7fc3bc2f91/resource.tar.gz#test.test_pg-tpch-q12-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q12-default.txt-Plan]": [ @@ -2860,9 +2860,9 @@ ], "test.test[pg-tpch-q16-default.txt-Debug]": [ { - "checksum": "2b525e16ed037906ebaca105d20f8572", - "size": 11528, - "uri": "https://{canondata_backend}/1942671/d6f4266e568a96dce8044dc5678d1f94b3d26355/resource.tar.gz#test.test_pg-tpch-q16-default.txt-Debug_/opt.yql_patched" + "checksum": "c0a6af4c997d706a81fe06e5b3359d6a", + "size": 11563, + "uri": "https://{canondata_backend}/1942278/c55aab0e2fa55c1a618538101f229f9f5347d7e6/resource.tar.gz#test.test_pg-tpch-q16-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q16-default.txt-Plan]": [ @@ -3349,9 +3349,9 @@ ], "test.test[select-trivial_having-default.txt-Debug]": [ { - "checksum": "1cfeecb75ebfcbc6dd730c75edbb7b8f", - "size": 2380, - "uri": "https://{canondata_backend}/1942173/50b4ae48e906d86b27ee0b68ed5a08b5ad6bf50e/resource.tar.gz#test.test_select-trivial_having-default.txt-Debug_/opt.yql_patched" + "checksum": "36e41a7925a3f6f19fda9f5efced4017", + "size": 2415, + "uri": "https://{canondata_backend}/1784117/b9574b141cedb261a8563a97d9a372d9277da2ce/resource.tar.gz#test.test_select-trivial_having-default.txt-Debug_/opt.yql_patched" } ], "test.test[select-trivial_having-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part13/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part13/canondata/result.json index 7ff935a83e11..c21387780b14 100644 --- a/ydb/library/yql/tests/sql/dq_file/part13/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part13/canondata/result.json @@ -297,9 +297,9 @@ ], "test.test[aggregate-group_by_gs_flatten_columns-default.txt-Debug]": [ { - "checksum": "b6dc4cebc2a2d3c9213e4fd6ca2237c0", - "size": 4044, - "uri": "https://{canondata_backend}/1946324/e871328b5487b9b2c440f1dd14b427a10459f3e7/resource.tar.gz#test.test_aggregate-group_by_gs_flatten_columns-default.txt-Debug_/opt.yql_patched" + "checksum": "5c2ce1d7d4e6df05a04c23fcfb7ae23e", + "size": 4096, + "uri": "https://{canondata_backend}/1923547/45484b99c033020b648870c9707d8e325a2db399/resource.tar.gz#test.test_aggregate-group_by_gs_flatten_columns-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_flatten_columns-default.txt-Plan]": [ @@ -347,9 +347,9 @@ ], "test.test[aggregate-group_by_rollup_duo--Debug]": [ { - "checksum": "76e92c8e5426e3591ceef5287ec16f90", - "size": 5431, - "uri": "https://{canondata_backend}/1936997/93899b3de50fae3f9677baacc98094a7a629590a/resource.tar.gz#test.test_aggregate-group_by_rollup_duo--Debug_/opt.yql_patched" + "checksum": "992877925fc779de4a52be671e977582", + "size": 5485, + "uri": "https://{canondata_backend}/1871102/3fcf32ea5c486527b20a5dea1db1e9ccf2e36a61/resource.tar.gz#test.test_aggregate-group_by_rollup_duo--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_duo--Plan]": [ @@ -369,9 +369,9 @@ ], "test.test[aggregate-group_by_ru_with_select_distinct--Debug]": [ { - "checksum": "4808ad562e2399291f99c9803027468f", - "size": 3337, - "uri": "https://{canondata_backend}/1924537/36fe336db7de347902767b13c1e0d63cf42757cd/resource.tar.gz#test.test_aggregate-group_by_ru_with_select_distinct--Debug_/opt.yql_patched" + "checksum": "960048221203aedc00af6f3a504bcbdc", + "size": 3425, + "uri": "https://{canondata_backend}/1775059/34e3c4f18af78c5b08d1779bfd2babe42d60869d/resource.tar.gz#test.test_aggregate-group_by_ru_with_select_distinct--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_with_select_distinct--Plan]": [ @@ -640,9 +640,9 @@ ], "test.test[dq-dq_replicate_ok-default.txt-Debug]": [ { - "checksum": "ff1a4f4a469760d70199f021a8762373", - "size": 3228, - "uri": "https://{canondata_backend}/1936842/0049c952a1bcb0ee8c00f8d262e8ccbc9a964444/resource.tar.gz#test.test_dq-dq_replicate_ok-default.txt-Debug_/opt.yql_patched" + "checksum": "35116c36ee83610ad04b16310fe91c46", + "size": 3263, + "uri": "https://{canondata_backend}/1871002/87af0e803663459b2fc0b931b22ed73d40f91575/resource.tar.gz#test.test_dq-dq_replicate_ok-default.txt-Debug_/opt.yql_patched" } ], "test.test[dq-dq_replicate_ok-default.txt-Plan]": [ @@ -1787,9 +1787,9 @@ ], "test.test[pg-aggregate_minus_zero--Debug]": [ { - "checksum": "fcb73f959fec7a9ed07627915c534d5e", - "size": 3129, - "uri": "https://{canondata_backend}/1903280/2010996c42ed76fd6d1e7bedccdf6026ec5a5fdb/resource.tar.gz#test.test_pg-aggregate_minus_zero--Debug_/opt.yql_patched" + "checksum": "dcab7563006ab8666b45c33ecb72bad2", + "size": 3164, + "uri": "https://{canondata_backend}/1923547/b752d090106a138d94de874b3e8b115ee6aa3c61/resource.tar.gz#test.test_pg-aggregate_minus_zero--Debug_/opt.yql_patched" } ], "test.test[pg-aggregate_minus_zero--Plan]": [ @@ -2276,9 +2276,9 @@ ], "test.test[pg-tpch-q13-default.txt-Debug]": [ { - "checksum": "562386df8d04ef440659016224b62731", - "size": 8912, - "uri": "https://{canondata_backend}/1937429/d4acef4fb2bfbf6a6f11d023507fabf22523d224/resource.tar.gz#test.test_pg-tpch-q13-default.txt-Debug_/opt.yql_patched" + "checksum": "f4cd33c81869dc46d10e192831eb7a61", + "size": 8947, + "uri": "https://{canondata_backend}/1923547/3f2874a2af3dc92454f155e96d4a4c91f2fbb16d/resource.tar.gz#test.test_pg-tpch-q13-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q13-default.txt-Plan]": [ @@ -2978,9 +2978,9 @@ ], "test.test[window-win_func_over_group_by_list_names_order_prefix--Debug]": [ { - "checksum": "d08e7e7b15a07c3b90bf5c42b0398cae", - "size": 8116, - "uri": "https://{canondata_backend}/1946324/e871328b5487b9b2c440f1dd14b427a10459f3e7/resource.tar.gz#test.test_window-win_func_over_group_by_list_names_order_prefix--Debug_/opt.yql_patched" + "checksum": "f27e3bd65da12d156812818f8f3007ec", + "size": 8151, + "uri": "https://{canondata_backend}/1784826/19728bd1f1bdea5d0605d9a498ec2970c5f3e92a/resource.tar.gz#test.test_window-win_func_over_group_by_list_names_order_prefix--Debug_/opt.yql_patched" } ], "test.test[window-win_func_over_group_by_list_names_order_prefix--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part14/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part14/canondata/result.json index c2fe3d7f12a3..a62ccd85e215 100644 --- a/ydb/library/yql/tests/sql/dq_file/part14/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part14/canondata/result.json @@ -284,9 +284,9 @@ ], "test.test[aggregate-group_by_cube_expr_trio--Debug]": [ { - "checksum": "286329169f0fff5035fb80c4c27045cd", - "size": 6977, - "uri": "https://{canondata_backend}/1777230/b8c638a79c26a4c14c582731ad5b06fe98478bb4/resource.tar.gz#test.test_aggregate-group_by_cube_expr_trio--Debug_/opt.yql_patched" + "checksum": "17691e421b8446c9f5e8d1e1e3367214", + "size": 7033, + "uri": "https://{canondata_backend}/1847551/0c915a458dc980cf235b6b87ce642ec9efee9e93/resource.tar.gz#test.test_aggregate-group_by_cube_expr_trio--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_cube_expr_trio--Plan]": [ @@ -350,9 +350,9 @@ ], "test.test[aggregate-group_by_ru_join_simple_fs_multiusage--Debug]": [ { - "checksum": "cbd01ac665ac34c8bd05b467b3eba7d4", - "size": 5960, - "uri": "https://{canondata_backend}/1942173/ff54d8318d63b24b6b066b083d340bc05cdcdf57/resource.tar.gz#test.test_aggregate-group_by_ru_join_simple_fs_multiusage--Debug_/opt.yql_patched" + "checksum": "40c120a2482c758ea695dcf009e7cab4", + "size": 6014, + "uri": "https://{canondata_backend}/1937150/28c16a1eb7a8543dba701afbdce5ac2e805d7dae/resource.tar.gz#test.test_aggregate-group_by_ru_join_simple_fs_multiusage--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join_simple_fs_multiusage--Plan]": [ @@ -1196,9 +1196,9 @@ ], "test.test[hor_join-empty_out_hor_join-default.txt-Debug]": [ { - "checksum": "56ffbe8e5c55caa1f8f55738fddc784c", - "size": 3086, - "uri": "https://{canondata_backend}/1777230/b8c638a79c26a4c14c582731ad5b06fe98478bb4/resource.tar.gz#test.test_hor_join-empty_out_hor_join-default.txt-Debug_/opt.yql_patched" + "checksum": "e475bbc392cba96d700981f3d64093d0", + "size": 3139, + "uri": "https://{canondata_backend}/1871182/e02f7cf2d403eeff46ab74696026e36be4ded9a9/resource.tar.gz#test.test_hor_join-empty_out_hor_join-default.txt-Debug_/opt.yql_patched" } ], "test.test[hor_join-empty_out_hor_join-default.txt-Plan]": [ @@ -2664,9 +2664,9 @@ ], "test.test[pg-tpch-q18-default.txt-Debug]": [ { - "checksum": "f58a286d7a1afb4bb16271f66b235848", - "size": 13230, - "uri": "https://{canondata_backend}/1942173/ff54d8318d63b24b6b066b083d340bc05cdcdf57/resource.tar.gz#test.test_pg-tpch-q18-default.txt-Debug_/opt.yql_patched" + "checksum": "736db3f7371060e631eb540d754522ea", + "size": 13265, + "uri": "https://{canondata_backend}/1689644/56cb6d928f012a044a6a92f477a5d1230cb126f7/resource.tar.gz#test.test_pg-tpch-q18-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q18-default.txt-Plan]": [ @@ -2976,9 +2976,9 @@ ], "test.test[select-discard-default.txt-Debug]": [ { - "checksum": "2a0c2447a72b0b87c4520f00afaff980", - "size": 8088, - "uri": "https://{canondata_backend}/1942173/ff54d8318d63b24b6b066b083d340bc05cdcdf57/resource.tar.gz#test.test_select-discard-default.txt-Debug_/opt.yql_patched" + "checksum": "41e050db4594c943d6ea2e087509688a", + "size": 8143, + "uri": "https://{canondata_backend}/1931696/84d8870f69a0ed5ab587dcc90a72bf470f59cbb8/resource.tar.gz#test.test_select-discard-default.txt-Debug_/opt.yql_patched" } ], "test.test[select-discard-default.txt-Plan]": [ @@ -3152,9 +3152,9 @@ ], "test.test[tpch-q11-default.txt-Debug]": [ { - "checksum": "a9a36525a02243bf5cb5956c9ba4c39e", - "size": 8641, - "uri": "https://{canondata_backend}/1942173/ff54d8318d63b24b6b066b083d340bc05cdcdf57/resource.tar.gz#test.test_tpch-q11-default.txt-Debug_/opt.yql_patched" + "checksum": "5818205132db06e5a70dd75361ccb6ef", + "size": 8676, + "uri": "https://{canondata_backend}/1600758/dd5b840770076286dd527fb7b5b51c5add954a15/resource.tar.gz#test.test_tpch-q11-default.txt-Debug_/opt.yql_patched" } ], "test.test[tpch-q11-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part15/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part15/canondata/result.json index 77986a6a716b..66f009640acc 100644 --- a/ydb/library/yql/tests/sql/dq_file/part15/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part15/canondata/result.json @@ -416,9 +416,9 @@ ], "test.test[aggregate-group_by_cube_grouping_and_expr-default.txt-Debug]": [ { - "checksum": "e8bf5cb7c6827564cb98e82877d2d931", - "size": 5268, - "uri": "https://{canondata_backend}/1777230/8d6bc20c3c548691ed47463aed0d508dcd185ce3/resource.tar.gz#test.test_aggregate-group_by_cube_grouping_and_expr-default.txt-Debug_/opt.yql_patched" + "checksum": "b29a1121295e35678097e04a6be69d1e", + "size": 5331, + "uri": "https://{canondata_backend}/1847551/5836720bd6edb7a20e88f4ea2ae09a4e4b561093/resource.tar.gz#test.test_aggregate-group_by_cube_grouping_and_expr-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_cube_grouping_and_expr-default.txt-Plan]": [ @@ -438,9 +438,9 @@ ], "test.test[aggregate-group_by_cube_join_count--Debug]": [ { - "checksum": "0433f4a612a605f11b127a4b76667aa7", - "size": 6828, - "uri": "https://{canondata_backend}/1942173/424cb3883e2778dea5912fc54f803111231f15cf/resource.tar.gz#test.test_aggregate-group_by_cube_join_count--Debug_/opt.yql_patched" + "checksum": "2b660a85546417936f69fb47b3f70a8c", + "size": 6889, + "uri": "https://{canondata_backend}/1599023/6b1d1e5bfb6806200cf7b045d1d0a5c785f8ce64/resource.tar.gz#test.test_aggregate-group_by_cube_join_count--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_cube_join_count--Plan]": [ @@ -482,9 +482,9 @@ ], "test.test[aggregate-group_by_ru_join_star-default.txt-Debug]": [ { - "checksum": "11c453cda6c770c216aac8b7a4d65a19", - "size": 3799, - "uri": "https://{canondata_backend}/1942173/424cb3883e2778dea5912fc54f803111231f15cf/resource.tar.gz#test.test_aggregate-group_by_ru_join_star-default.txt-Debug_/opt.yql_patched" + "checksum": "bfc59d33705bb7a2a54873e86a41baaa", + "size": 3852, + "uri": "https://{canondata_backend}/1889210/bb0c0a0adac57f0ecc091e7093fb3f5bc2de0c19/resource.tar.gz#test.test_aggregate-group_by_ru_join_star-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join_star-default.txt-Plan]": [ @@ -504,9 +504,9 @@ ], "test.test[aggregate-group_by_ru_with_window_func--Debug]": [ { - "checksum": "8b9927290f5ebf82c428cbd684bd128b", - "size": 3737, - "uri": "https://{canondata_backend}/1916746/8af1fb7747dc5b2dccf47bca5be44479c7ae6621/resource.tar.gz#test.test_aggregate-group_by_ru_with_window_func--Debug_/opt.yql_patched" + "checksum": "86f3ff358eb385a0af0c6c099e82b162", + "size": 3825, + "uri": "https://{canondata_backend}/1600758/6536c12aedebd96c5fc71915b4cfc72252cfa630/resource.tar.gz#test.test_aggregate-group_by_ru_with_window_func--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_with_window_func--Plan]": [ @@ -2302,23 +2302,23 @@ "test.test[pg-tpch-q06-default.txt-Results]": [], "test.test[pg-tpch-q11-default.txt-Analyze]": [ { - "checksum": "b7dac2c41dd84112e0bebb14c5527bdd", - "size": 15506, - "uri": "https://{canondata_backend}/212715/17fecc70dc3d82af132727a3aa20d5edbdd4d7dd/resource.tar.gz#test.test_pg-tpch-q11-default.txt-Analyze_/plan.txt" + "checksum": "d925fa20a9243269d72fd62951392b7e", + "size": 14875, + "uri": "https://{canondata_backend}/1814674/5f3c7350d8e72ada6a702e29e5f3bfddaa73df08/resource.tar.gz#test.test_pg-tpch-q11-default.txt-Analyze_/plan.txt" } ], "test.test[pg-tpch-q11-default.txt-Debug]": [ { - "checksum": "3b78e96d9b704580d2b9f8758da6cb07", - "size": 10970, - "uri": "https://{canondata_backend}/1942173/424cb3883e2778dea5912fc54f803111231f15cf/resource.tar.gz#test.test_pg-tpch-q11-default.txt-Debug_/opt.yql_patched" + "checksum": "2bed4f0772b70e6e544dd5602659ba43", + "size": 12018, + "uri": "https://{canondata_backend}/1946324/c19334a8bb1758dfb02761e1ce101dc42cf675fa/resource.tar.gz#test.test_pg-tpch-q11-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q11-default.txt-Plan]": [ { - "checksum": "b7dac2c41dd84112e0bebb14c5527bdd", - "size": 15506, - "uri": "https://{canondata_backend}/1777230/8d6bc20c3c548691ed47463aed0d508dcd185ce3/resource.tar.gz#test.test_pg-tpch-q11-default.txt-Plan_/plan.txt" + "checksum": "d925fa20a9243269d72fd62951392b7e", + "size": 14875, + "uri": "https://{canondata_backend}/1130705/acc206ee45aa5b7f7e78d232059c9535f249dda0/resource.tar.gz#test.test_pg-tpch-q11-default.txt-Plan_/plan.txt" } ], "test.test[pg-tpch-q11-default.txt-Results]": [], diff --git a/ydb/library/yql/tests/sql/dq_file/part16/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part16/canondata/result.json index 101f85779650..360e900145f0 100644 --- a/ydb/library/yql/tests/sql/dq_file/part16/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part16/canondata/result.json @@ -312,9 +312,9 @@ ], "test.test[aggregate-group_by_mul_gs_expr_and_column--Debug]": [ { - "checksum": "c8865311fd357cef6557d884d5dd042c", - "size": 4576, - "uri": "https://{canondata_backend}/1937492/8cdefcd6d0d86a9eaa2af7a5c2ce6fe8014c7ac2/resource.tar.gz#test.test_aggregate-group_by_mul_gs_expr_and_column--Debug_/opt.yql_patched" + "checksum": "83bd986a36702e60beb6441068da6ef3", + "size": 4628, + "uri": "https://{canondata_backend}/1130705/984266b47af8d517834a10674242c9e900f41724/resource.tar.gz#test.test_aggregate-group_by_mul_gs_expr_and_column--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_mul_gs_expr_and_column--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part17/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part17/canondata/result.json index 43d98e28181f..74a9b4702330 100644 --- a/ydb/library/yql/tests/sql/dq_file/part17/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part17/canondata/result.json @@ -420,9 +420,9 @@ ], "test.test[aggregate-group_by_ru_join_agg--Debug]": [ { - "checksum": "0c0aa4b6f7a61844347fb8d8d572782c", - "size": 6120, - "uri": "https://{canondata_backend}/1942173/b272062375a42ae80db813301c8c3dff3241b41a/resource.tar.gz#test.test_aggregate-group_by_ru_join_agg--Debug_/opt.yql_patched" + "checksum": "310156d4e22408fd37fc19e6160ad3fc", + "size": 6175, + "uri": "https://{canondata_backend}/1925842/cdbda15640e6186d6a55e022cf27805bb41951ca/resource.tar.gz#test.test_aggregate-group_by_ru_join_agg--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join_agg--Plan]": [ @@ -718,9 +718,9 @@ ], "test.test[csee-yql-7237--Debug]": [ { - "checksum": "6cdc40bb095fa8825818537693155684", - "size": 5275, - "uri": "https://{canondata_backend}/1942173/b272062375a42ae80db813301c8c3dff3241b41a/resource.tar.gz#test.test_csee-yql-7237--Debug_/opt.yql_patched" + "checksum": "d259e9d9f542189642d65c36c51ee74e", + "size": 5310, + "uri": "https://{canondata_backend}/1925842/bf44abaff14fdc233f3e96010fd1617ed0eb9d56/resource.tar.gz#test.test_csee-yql-7237--Debug_/opt.yql_patched" } ], "test.test[csee-yql-7237--Plan]": [ @@ -1512,9 +1512,9 @@ ], "test.test[join-two_aggrs-default.txt-Debug]": [ { - "checksum": "6a69e0238ee504fa3002f1895b072af8", - "size": 4087, - "uri": "https://{canondata_backend}/1942173/b272062375a42ae80db813301c8c3dff3241b41a/resource.tar.gz#test.test_join-two_aggrs-default.txt-Debug_/opt.yql_patched" + "checksum": "3edfc0e61aada69a4da33f6f72f3f00f", + "size": 4105, + "uri": "https://{canondata_backend}/1937429/2f571d847455277d1442fcb7f010661e820848b9/resource.tar.gz#test.test_join-two_aggrs-default.txt-Debug_/opt.yql_patched" } ], "test.test[join-two_aggrs-default.txt-Plan]": [ @@ -2617,9 +2617,9 @@ ], "test.test[table_range-concat_with_view--Debug]": [ { - "checksum": "8c3ab9a7d76fc314c4d5be392e075ad7", - "size": 3947, - "uri": "https://{canondata_backend}/1942173/e32f1de19c4f2770a6f215d1dc22bc97e318bf22/resource.tar.gz#test.test_table_range-concat_with_view--Debug_/opt.yql_patched" + "checksum": "5b6927fd57b28a292fc0a17afefdf28a", + "size": 3982, + "uri": "https://{canondata_backend}/1925842/5ebb3b5676761f341f736480110bd8ab8a78d858/resource.tar.gz#test.test_table_range-concat_with_view--Debug_/opt.yql_patched" } ], "test.test[table_range-concat_with_view--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part18/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part18/canondata/result.json index 54f4b9bea41b..a804fe7db9d4 100644 --- a/ydb/library/yql/tests/sql/dq_file/part18/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part18/canondata/result.json @@ -244,6 +244,34 @@ } ], "test.test[aggregate-aggregate_with_deep_aggregated_column--Results]": [], + "test.test[aggregate-disable_blocks_with_spilling--Analyze]": [ + { + "checksum": "7887cfe87307d36449cd6afe65636dd1", + "size": 4615, + "uri": "https://{canondata_backend}/1775059/60aa9c77d2376aa1beb6e616fcbdc82d0b2724be/resource.tar.gz#test.test_aggregate-disable_blocks_with_spilling--Analyze_/plan.txt" + } + ], + "test.test[aggregate-disable_blocks_with_spilling--Debug]": [ + { + "checksum": "c3bb2f21048ee6f5a0e7846fd28f481d", + "size": 2423, + "uri": "https://{canondata_backend}/1775059/60aa9c77d2376aa1beb6e616fcbdc82d0b2724be/resource.tar.gz#test.test_aggregate-disable_blocks_with_spilling--Debug_/opt.yql_patched" + } + ], + "test.test[aggregate-disable_blocks_with_spilling--Plan]": [ + { + "checksum": "7887cfe87307d36449cd6afe65636dd1", + "size": 4615, + "uri": "https://{canondata_backend}/1775059/60aa9c77d2376aa1beb6e616fcbdc82d0b2724be/resource.tar.gz#test.test_aggregate-disable_blocks_with_spilling--Plan_/plan.txt" + } + ], + "test.test[aggregate-disable_blocks_with_spilling--Results]": [ + { + "checksum": "42f51df2ad014764141d891357b0b6b6", + "size": 915, + "uri": "https://{canondata_backend}/1775059/60aa9c77d2376aa1beb6e616fcbdc82d0b2724be/resource.tar.gz#test.test_aggregate-disable_blocks_with_spilling--Results_/results.txt" + } + ], "test.test[aggregate-group_by_column_alias_reuse-default.txt-Analyze]": [ { "checksum": "bf546487bcd475b8555f2a7883d1f6a0", @@ -1849,9 +1877,9 @@ ], "test.test[optimizers-sort_constraint_in_left--Debug]": [ { - "checksum": "b1709776e175b9e244b7f9552358e74a", - "size": 4067, - "uri": "https://{canondata_backend}/1777230/00c02c2221ad7773f9cfecb5ec1bd067dbaacbc5/resource.tar.gz#test.test_optimizers-sort_constraint_in_left--Debug_/opt.yql_patched" + "checksum": "ddafcba99ec8f9ffc2ca1656003dacd5", + "size": 4102, + "uri": "https://{canondata_backend}/1916746/2673649875bad3e7ba633862e83892bc57334832/resource.tar.gz#test.test_optimizers-sort_constraint_in_left--Debug_/opt.yql_patched" } ], "test.test[optimizers-sort_constraint_in_left--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part19/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part19/canondata/result.json index 26919f550374..ff877bbc31f1 100644 --- a/ydb/library/yql/tests/sql/dq_file/part19/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part19/canondata/result.json @@ -400,9 +400,9 @@ ], "test.test[aggregate-group_by_gs_join_aliases-default.txt-Debug]": [ { - "checksum": "5bdb64f88d9f039ee97f5b8b9cf8ae3f", - "size": 5484, - "uri": "https://{canondata_backend}/1784826/ac5e05ca91b417814012c384cae2340a173d491c/resource.tar.gz#test.test_aggregate-group_by_gs_join_aliases-default.txt-Debug_/opt.yql_patched" + "checksum": "05a809d287b7cfdc8af2ef37a6f96417", + "size": 5541, + "uri": "https://{canondata_backend}/1937429/28dc671beb94ae1c8b1b265c49c4a0f32f7f68f1/resource.tar.gz#test.test_aggregate-group_by_gs_join_aliases-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_join_aliases-default.txt-Plan]": [ @@ -422,9 +422,9 @@ ], "test.test[aggregate-group_by_rollup_column_reuse--Debug]": [ { - "checksum": "64b83869160e04718b08ddf7a81b1c6f", - "size": 5666, - "uri": "https://{canondata_backend}/1689644/57f5e520abfb96651cc218a0d82eb6ee0fe38907/resource.tar.gz#test.test_aggregate-group_by_rollup_column_reuse--Debug_/opt.yql_patched" + "checksum": "46f4e870b0e81b54be5d46294a169761", + "size": 5720, + "uri": "https://{canondata_backend}/1937429/d03442e328dca2de744539eee34693d8645faba4/resource.tar.gz#test.test_aggregate-group_by_rollup_column_reuse--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_column_reuse--Plan]": [ @@ -488,9 +488,9 @@ ], "test.test[aggregate-rollup_with_dict--Debug]": [ { - "checksum": "6d691609766138d06960a172da915c65", - "size": 4416, - "uri": "https://{canondata_backend}/1689644/57f5e520abfb96651cc218a0d82eb6ee0fe38907/resource.tar.gz#test.test_aggregate-rollup_with_dict--Debug_/opt.yql_patched" + "checksum": "a81bca6bbe1adbd873aff7caffaa9ddd", + "size": 4451, + "uri": "https://{canondata_backend}/1937429/8d4678be89a5cdafec6099b5b174e1923a6b1b84/resource.tar.gz#test.test_aggregate-rollup_with_dict--Debug_/opt.yql_patched" } ], "test.test[aggregate-rollup_with_dict--Plan]": [ @@ -1815,9 +1815,9 @@ ], "test.test[optimizers-unused_columns_group_one_of_multi--Debug]": [ { - "checksum": "8586e1601cc4af8fb7c6f307354f025e", - "size": 4250, - "uri": "https://{canondata_backend}/1889210/2fbf7f68942208b15ab6eb23b14b78640f078541/resource.tar.gz#test.test_optimizers-unused_columns_group_one_of_multi--Debug_/opt.yql_patched" + "checksum": "cf9ded228af2a2d59c007f6cb9534552", + "size": 4285, + "uri": "https://{canondata_backend}/1903280/45bea04670ac04ec37c2deab5ff4a786ae244430/resource.tar.gz#test.test_optimizers-unused_columns_group_one_of_multi--Debug_/opt.yql_patched" } ], "test.test[optimizers-unused_columns_group_one_of_multi--Plan]": [ @@ -2823,9 +2823,9 @@ ], "test.test[tpch-q18-default.txt-Debug]": [ { - "checksum": "c144d57016fc7dd0d6416396016a0173", - "size": 9887, - "uri": "https://{canondata_backend}/1784826/ac5e05ca91b417814012c384cae2340a173d491c/resource.tar.gz#test.test_tpch-q18-default.txt-Debug_/opt.yql_patched" + "checksum": "cb07b68236013d6f29c7d815b3d9913a", + "size": 9922, + "uri": "https://{canondata_backend}/1931696/175b453808baee5138a78eebd922e34fb0603ae4/resource.tar.gz#test.test_tpch-q18-default.txt-Debug_/opt.yql_patched" } ], "test.test[tpch-q18-default.txt-Plan]": [ @@ -3180,9 +3180,9 @@ ], "test.test[window-win_func_over_group_by_compl--Debug]": [ { - "checksum": "aa0b1b5fa400fefc235a52fc4ae77cb4", - "size": 7515, - "uri": "https://{canondata_backend}/1689644/57f5e520abfb96651cc218a0d82eb6ee0fe38907/resource.tar.gz#test.test_window-win_func_over_group_by_compl--Debug_/opt.yql_patched" + "checksum": "c539ed09917e7c678800243c531d5926", + "size": 7550, + "uri": "https://{canondata_backend}/1784117/1ee1d854fe43f6981a4bf1da95e36e8be387e233/resource.tar.gz#test.test_window-win_func_over_group_by_compl--Debug_/opt.yql_patched" } ], "test.test[window-win_func_over_group_by_compl--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part2/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part2/canondata/result.json index 474b7542e21d..9cc8f40422cf 100644 --- a/ydb/library/yql/tests/sql/dq_file/part2/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part2/canondata/result.json @@ -231,9 +231,9 @@ ], "test.test[aggregate-group_by_gs_flatten_expr-default.txt-Debug]": [ { - "checksum": "de14e0c1e81ca5678d67ba6ea2aea701", - "size": 4253, - "uri": "https://{canondata_backend}/1936947/79f6f05a619e566dcfd3200df680cadf79a1ceda/resource.tar.gz#test.test_aggregate-group_by_gs_flatten_expr-default.txt-Debug_/opt.yql_patched" + "checksum": "c2833f137482cbef45f302716a027fad", + "size": 4305, + "uri": "https://{canondata_backend}/1920236/b75e9728ed12152b2d9ddc60dd94c08dfc4796e2/resource.tar.gz#test.test_aggregate-group_by_gs_flatten_expr-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_flatten_expr-default.txt-Plan]": [ @@ -253,9 +253,9 @@ ], "test.test[aggregate-group_by_rollup_column_ref--Debug]": [ { - "checksum": "d5b5ee1843f9d09cd8cc6180e3bf9c10", - "size": 5562, - "uri": "https://{canondata_backend}/1936947/79f6f05a619e566dcfd3200df680cadf79a1ceda/resource.tar.gz#test.test_aggregate-group_by_rollup_column_ref--Debug_/opt.yql_patched" + "checksum": "cb168c91086a659cb5fdff91b51d4619", + "size": 5616, + "uri": "https://{canondata_backend}/1917492/53254226c3bcc22f2f64563cade92bbf7d103511/resource.tar.gz#test.test_aggregate-group_by_rollup_column_ref--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_column_ref--Plan]": [ @@ -275,9 +275,9 @@ ], "test.test[aggregate-group_by_ru_partition_by_grouping-default.txt-Debug]": [ { - "checksum": "ed88dd60c2213f296962c78fc79f8176", - "size": 7933, - "uri": "https://{canondata_backend}/1937367/6af906d8e8515951055311e09244912c4095ac7f/resource.tar.gz#test.test_aggregate-group_by_ru_partition_by_grouping-default.txt-Debug_/opt.yql_patched" + "checksum": "11d1fc2cfa24b26e10c896218e286158", + "size": 7968, + "uri": "https://{canondata_backend}/1903280/7cce39a9f54f953ff747437b2afbf4cd7340892a/resource.tar.gz#test.test_aggregate-group_by_ru_partition_by_grouping-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_partition_by_grouping-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part3/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part3/canondata/result.json index 46cf24c00663..2e0ba4df30d6 100644 --- a/ydb/library/yql/tests/sql/dq_file/part3/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part3/canondata/result.json @@ -266,9 +266,9 @@ ], "test.test[aggregate-group_by_gs_subselect-default.txt-Debug]": [ { - "checksum": "5c59c4ff97dae61eb25049435c055898", - "size": 4179, - "uri": "https://{canondata_backend}/1937027/7e92a59557f254d8b58c96118ce2e626b197c0b1/resource.tar.gz#test.test_aggregate-group_by_gs_subselect-default.txt-Debug_/opt.yql_patched" + "checksum": "39d2d47db756d2b49474b5c199c4c38b", + "size": 4229, + "uri": "https://{canondata_backend}/1130705/85069899508bcd3b8be2b6d75961f8852e8ff128/resource.tar.gz#test.test_aggregate-group_by_gs_subselect-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_subselect-default.txt-Plan]": [ @@ -288,9 +288,9 @@ ], "test.test[aggregate-group_by_mul_gs_gs--Debug]": [ { - "checksum": "ac786ff228c9aabff4056702026f8aab", - "size": 7013, - "uri": "https://{canondata_backend}/1923547/331b1de1b2a9544651bd249eccea1d8975558c09/resource.tar.gz#test.test_aggregate-group_by_mul_gs_gs--Debug_/opt.yql_patched" + "checksum": "b2f65819e77ed498d1bb503eeda1d0a9", + "size": 7078, + "uri": "https://{canondata_backend}/1599023/b8ad35209f29e3ae308d0f48b10aeffda01d29c8/resource.tar.gz#test.test_aggregate-group_by_mul_gs_gs--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_mul_gs_gs--Plan]": [ @@ -310,9 +310,9 @@ ], "test.test[aggregate-group_by_rollup_duo_opt--Debug]": [ { - "checksum": "ba75c73258dd7e79ff2c1ec574b0a1f4", - "size": 5169, - "uri": "https://{canondata_backend}/1923547/331b1de1b2a9544651bd249eccea1d8975558c09/resource.tar.gz#test.test_aggregate-group_by_rollup_duo_opt--Debug_/opt.yql_patched" + "checksum": "51b1c16826e339f1620bd946cf0cec42", + "size": 5224, + "uri": "https://{canondata_backend}/1599023/2257627623f9ecc02660dc51c2da964bf24e60cf/resource.tar.gz#test.test_aggregate-group_by_rollup_duo_opt--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_duo_opt--Plan]": [ @@ -2704,9 +2704,9 @@ ], "test.test[window-win_func_over_group_by--Debug]": [ { - "checksum": "7b029a85f005715959accf5ec1599fdb", - "size": 5122, - "uri": "https://{canondata_backend}/1809005/bace128d842e0e2cef93390c0800c74269352290/resource.tar.gz#test.test_window-win_func_over_group_by--Debug_/opt.yql_patched" + "checksum": "bbb96c3242a6663b3235bb4f06748647", + "size": 5157, + "uri": "https://{canondata_backend}/1031349/110747d194be1b3ec565c8629bddeb11bdda85c5/resource.tar.gz#test.test_window-win_func_over_group_by--Debug_/opt.yql_patched" } ], "test.test[window-win_func_over_group_by--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part4/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part4/canondata/result.json index 20b2db0e021c..3898f73b0bc4 100644 --- a/ydb/library/yql/tests/sql/dq_file/part4/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part4/canondata/result.json @@ -321,9 +321,9 @@ ], "test.test[aggregate-group_by_gs_with_rollup--Debug]": [ { - "checksum": "5b153496b7f2d8c0a862d538785a0169", - "size": 6668, - "uri": "https://{canondata_backend}/1936947/c075b3a6b857003250f6fcdaddd6e5508fb9d58f/resource.tar.gz#test.test_aggregate-group_by_gs_with_rollup--Debug_/opt.yql_patched" + "checksum": "ea486f54e35e838bed83cb661fa50c97", + "size": 6719, + "uri": "https://{canondata_backend}/1871102/61453f3b91b2a15fb0e2d2c6c61875e360070851/resource.tar.gz#test.test_aggregate-group_by_gs_with_rollup--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_with_rollup--Plan]": [ @@ -399,9 +399,9 @@ ], "test.test[aggregate-group_by_rollup_aggr_expr--Debug]": [ { - "checksum": "66366d52a0c559024f7ca2bf1b1924f0", - "size": 7262, - "uri": "https://{canondata_backend}/1871102/a3f4b73c2abee8d4bf99b0344946712ef43a9193/resource.tar.gz#test.test_aggregate-group_by_rollup_aggr_expr--Debug_/opt.yql_patched" + "checksum": "1a3dd2ce4733e65654f8996550c61af9", + "size": 7318, + "uri": "https://{canondata_backend}/1937027/4608fb8cff903881d29660feb5fbd40491ccea1b/resource.tar.gz#test.test_aggregate-group_by_rollup_aggr_expr--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_aggr_expr--Plan]": [ @@ -1871,9 +1871,9 @@ ], "test.test[optimizers-yql-8953_logical_fuse_with_table_props--Debug]": [ { - "checksum": "df81654bc258b3a04795f7928a2f02b7", - "size": 3896, - "uri": "https://{canondata_backend}/1889210/97759537b96117d0b783b60421791e318b61fc40/resource.tar.gz#test.test_optimizers-yql-8953_logical_fuse_with_table_props--Debug_/opt.yql_patched" + "checksum": "1887847b798f6ee89f31bedc55d03c90", + "size": 3931, + "uri": "https://{canondata_backend}/1925842/834d93a5fe5fd54aa84f6328cbfc56994ae169c9/resource.tar.gz#test.test_optimizers-yql-8953_logical_fuse_with_table_props--Debug_/opt.yql_patched" } ], "test.test[optimizers-yql-8953_logical_fuse_with_table_props--Plan]": [ @@ -2435,9 +2435,9 @@ ], "test.test[pg-tpch-q09-default.txt-Debug]": [ { - "checksum": "f4dde9ad230c88872c2f7886c5e9a435", - "size": 15675, - "uri": "https://{canondata_backend}/1903280/86dcb437348050335359deea8ac19bec0e50955c/resource.tar.gz#test.test_pg-tpch-q09-default.txt-Debug_/opt.yql_patched" + "checksum": "2cf1f345c46819f8a52b8f5048db9270", + "size": 15710, + "uri": "https://{canondata_backend}/1931696/ad557a9af2f482ab6e8f91faf53872856e3eaac1/resource.tar.gz#test.test_pg-tpch-q09-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q09-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part5/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part5/canondata/result.json index b124c29b73c5..b178697f9b98 100644 --- a/ydb/library/yql/tests/sql/dq_file/part5/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part5/canondata/result.json @@ -339,9 +339,9 @@ ], "test.test[aggregate-group_by_mul_ru_ru--Debug]": [ { - "checksum": "9cba3bd36f11cf360a6ccd255109a281", - "size": 13190, - "uri": "https://{canondata_backend}/1777230/4b5479e2ebed213e8e8d9a64aa0b5a72bb3ea4dd/resource.tar.gz#test.test_aggregate-group_by_mul_ru_ru--Debug_/opt.yql_patched" + "checksum": "95444261d8a55a1c6c91b36ebf9b93fb", + "size": 13268, + "uri": "https://{canondata_backend}/1899731/0bc935d3f61810d330a6462fb133ddcc4ac126c6/resource.tar.gz#test.test_aggregate-group_by_mul_ru_ru--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_mul_ru_ru--Plan]": [ @@ -2561,9 +2561,9 @@ ], "test.test[pg-tpch-q04-default.txt-Debug]": [ { - "checksum": "a4c37d592840463a5d678ae8f6bbcd87", - "size": 9290, - "uri": "https://{canondata_backend}/1784826/e5ca4fb88654f6c9311174bbb05a921b55612738/resource.tar.gz#test.test_pg-tpch-q04-default.txt-Debug_/opt.yql_patched" + "checksum": "4666a7269563392f9265afabac7f77c2", + "size": 9294, + "uri": "https://{canondata_backend}/1917492/efbefb09221dc3e19769efa112dda52dbfbdaf53/resource.tar.gz#test.test_pg-tpch-q04-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q04-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part6/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part6/canondata/result.json index f47b443b52eb..d71c43219083 100644 --- a/ydb/library/yql/tests/sql/dq_file/part6/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part6/canondata/result.json @@ -259,9 +259,9 @@ ], "test.test[aggregate-group_by_cube_duo--Debug]": [ { - "checksum": "3cf8eaf5ce4f0c8e668f517cbb2dca3e", - "size": 6624, - "uri": "https://{canondata_backend}/1936947/2fc43e3b7bf2ac6312b395248938656a7fa50fcc/resource.tar.gz#test.test_aggregate-group_by_cube_duo--Debug_/opt.yql_patched" + "checksum": "9236a886d3fa78e7288be9bd8912d9d2", + "size": 6680, + "uri": "https://{canondata_backend}/1775059/0211445827e77a089557f709a929c720409a58d4/resource.tar.gz#test.test_aggregate-group_by_cube_duo--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_cube_duo--Plan]": [ @@ -281,9 +281,9 @@ ], "test.test[aggregate-group_by_cube_grouping--Debug]": [ { - "checksum": "2af97603ecbeb4c0e4aa6c85de640b60", - "size": 12278, - "uri": "https://{canondata_backend}/1777230/af30d016cca75b9d11b6ed54e7d270e255deb404/resource.tar.gz#test.test_aggregate-group_by_cube_grouping--Debug_/opt.yql_patched" + "checksum": "3add42ae160fd7a1653ae657a859a559", + "size": 12353, + "uri": "https://{canondata_backend}/1775059/e0d64b1cb8ae8bb7052270bb8eeef5ba21a3c131/resource.tar.gz#test.test_aggregate-group_by_cube_grouping--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_cube_grouping--Plan]": [ @@ -303,9 +303,9 @@ ], "test.test[aggregate-group_by_rollup_udf--Debug]": [ { - "checksum": "2a155992061f0284cf6d7ee6d793b893", - "size": 7037, - "uri": "https://{canondata_backend}/1936947/2fc43e3b7bf2ac6312b395248938656a7fa50fcc/resource.tar.gz#test.test_aggregate-group_by_rollup_udf--Debug_/opt.yql_patched" + "checksum": "a70eed5b18e68b90e50476bcc64470bd", + "size": 7095, + "uri": "https://{canondata_backend}/1784117/357d3ccdef7d0372b6d86bbe259ca7f35b60e595/resource.tar.gz#test.test_aggregate-group_by_rollup_udf--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_udf--Plan]": [ @@ -796,9 +796,9 @@ ], "test.test[epochs-use_sorted_by_complex_type--Debug]": [ { - "checksum": "b8ad92842200b0e27e2b58dd4534b53c", - "size": 5261, - "uri": "https://{canondata_backend}/1903885/dc53b4edac607ebf3b277ca9598c7c26218fd737/resource.tar.gz#test.test_epochs-use_sorted_by_complex_type--Debug_/opt.yql_patched" + "checksum": "189229cc9f067077257f0ba8cbf3b3b0", + "size": 5296, + "uri": "https://{canondata_backend}/1923547/9c9bffa7195403e2170e840b06e42ba0867654f8/resource.tar.gz#test.test_epochs-use_sorted_by_complex_type--Debug_/opt.yql_patched" } ], "test.test[epochs-use_sorted_by_complex_type--Plan]": [ @@ -2417,9 +2417,9 @@ ], "test.test[pg-tpch-q07-default.txt-Debug]": [ { - "checksum": "7b32eabedab349cd6443fd3f1d9e878c", - "size": 15272, - "uri": "https://{canondata_backend}/1814674/c6d381127a358df0aadb7c021f8fd353a231b060/resource.tar.gz#test.test_pg-tpch-q07-default.txt-Debug_/opt.yql_patched" + "checksum": "2259fabdd491e9cd8a3c7327bfb18027", + "size": 15307, + "uri": "https://{canondata_backend}/1775319/4822fc307d7eab1238a19a49bb254227c5051fce/resource.tar.gz#test.test_pg-tpch-q07-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q07-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part7/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part7/canondata/result.json index e5b4d9e49c2a..f4784e9a1059 100644 --- a/ydb/library/yql/tests/sql/dq_file/part7/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part7/canondata/result.json @@ -250,9 +250,9 @@ ], "test.test[aggregate-compare_by_tuple--Debug]": [ { - "checksum": "e91261faf8733b739c279fa03e5eb806", - "size": 3545, - "uri": "https://{canondata_backend}/1923547/4a11bf336fd7fb8da5f5162c16271b830cef13e4/resource.tar.gz#test.test_aggregate-compare_by_tuple--Debug_/opt.yql_patched" + "checksum": "baef0bf57c57837e4624860775faaaa4", + "size": 3580, + "uri": "https://{canondata_backend}/1937492/caaa9a42499278fe8d2abe06fe2b17bceeb09e18/resource.tar.gz#test.test_aggregate-compare_by_tuple--Debug_/opt.yql_patched" } ], "test.test[aggregate-compare_by_tuple--Plan]": [ @@ -272,9 +272,9 @@ ], "test.test[aggregate-group_by_gs_grouping--Debug]": [ { - "checksum": "53143a2969289e7363d37c8ab8bbb950", - "size": 4190, - "uri": "https://{canondata_backend}/1871102/680e072d487740a733846c6fb8acae02496a7035/resource.tar.gz#test.test_aggregate-group_by_gs_grouping--Debug_/opt.yql_patched" + "checksum": "8724627da064c332ff9d3557432a46fc", + "size": 4242, + "uri": "https://{canondata_backend}/937458/dab17048102e83ee373249c500ad7a1a9718d95c/resource.tar.gz#test.test_aggregate-group_by_gs_grouping--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_grouping--Plan]": [ @@ -294,9 +294,9 @@ ], "test.test[aggregate-group_by_rollup_column_ref_same_names--Debug]": [ { - "checksum": "64b83869160e04718b08ddf7a81b1c6f", - "size": 5666, - "uri": "https://{canondata_backend}/1871102/680e072d487740a733846c6fb8acae02496a7035/resource.tar.gz#test.test_aggregate-group_by_rollup_column_ref_same_names--Debug_/opt.yql_patched" + "checksum": "46f4e870b0e81b54be5d46294a169761", + "size": 5720, + "uri": "https://{canondata_backend}/1925842/1c73675b2ef22d3db833d7bb81e6d092b9398bca/resource.tar.gz#test.test_aggregate-group_by_rollup_column_ref_same_names--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_column_ref_same_names--Plan]": [ @@ -316,9 +316,9 @@ ], "test.test[aggregate-group_by_rollup_key_check--Debug]": [ { - "checksum": "2f9650ad69bc0d377c5476e54a7d408b", - "size": 5335, - "uri": "https://{canondata_backend}/1871102/680e072d487740a733846c6fb8acae02496a7035/resource.tar.gz#test.test_aggregate-group_by_rollup_key_check--Debug_/opt.yql_patched" + "checksum": "f783d973cc24ef82d7b55505b7a41612", + "size": 5389, + "uri": "https://{canondata_backend}/1937492/63930c59f78bd833253a0a3dd62479c8ad6cb321/resource.tar.gz#test.test_aggregate-group_by_rollup_key_check--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_key_check--Plan]": [ @@ -338,9 +338,9 @@ ], "test.test[aggregate-group_by_ru_join--Debug]": [ { - "checksum": "446a2c5070f635026a365400bbb6f18f", - "size": 3719, - "uri": "https://{canondata_backend}/1900335/37d3d334f87aa8919dccdac3d307eef9f104cf54/resource.tar.gz#test.test_aggregate-group_by_ru_join--Debug_/opt.yql_patched" + "checksum": "8772e8bd3cac771890514d055f7dce5f", + "size": 3772, + "uri": "https://{canondata_backend}/1775319/3b8377e8fd4479a1afeb2968193880fa658d75a9/resource.tar.gz#test.test_aggregate-group_by_ru_join--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join--Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part8/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part8/canondata/result.json index fac0b45f076b..9792e3314956 100644 --- a/ydb/library/yql/tests/sql/dq_file/part8/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part8/canondata/result.json @@ -377,9 +377,9 @@ ], "test.test[aggregate-group_by_rollup_with_filter--Debug]": [ { - "checksum": "ed571b15ab8e9fd8d94f910d0edea0a8", - "size": 5333, - "uri": "https://{canondata_backend}/1923547/995d9d96bbba94053a60009ae7ba99979f31a5bf/resource.tar.gz#test.test_aggregate-group_by_rollup_with_filter--Debug_/opt.yql_patched" + "checksum": "a21a86dc117dd7c1063ba939cfaf9f2b", + "size": 5388, + "uri": "https://{canondata_backend}/1925842/5d65124ba39bfe8e6fec32f860c67c97e48531b0/resource.tar.gz#test.test_aggregate-group_by_rollup_with_filter--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_with_filter--Plan]": [ @@ -399,9 +399,9 @@ ], "test.test[aggregate-group_by_ru_join_grouping-default.txt-Debug]": [ { - "checksum": "cb490b2a8c3ee307111e4d7091686b4c", - "size": 6576, - "uri": "https://{canondata_backend}/1899731/d0ecf1a0e81a1395a457b2649cb0a7edf4d6f0ed/resource.tar.gz#test.test_aggregate-group_by_ru_join_grouping-default.txt-Debug_/opt.yql_patched" + "checksum": "a5235bf71851f3855e21f74a5b8e1f34", + "size": 6598, + "uri": "https://{canondata_backend}/1925821/5d0d0417f7a22d79e29fcb1ab2c6feb128861ec5/resource.tar.gz#test.test_aggregate-group_by_ru_join_grouping-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_ru_join_grouping-default.txt-Plan]": [ @@ -2651,9 +2651,9 @@ ], "test.test[pg-tpch-q01-default.txt-Debug]": [ { - "checksum": "fccd9597f2b0ede9cd3e500e92f340ba", - "size": 11372, - "uri": "https://{canondata_backend}/212715/d4ecb16ed782ab92fb83434c6219e42fc504abe9/resource.tar.gz#test.test_pg-tpch-q01-default.txt-Debug_/opt.yql_patched" + "checksum": "168e1cea646413f323310cc015341f97", + "size": 11407, + "uri": "https://{canondata_backend}/1775319/171cd905ac5402be308349251477ae77481201b4/resource.tar.gz#test.test_pg-tpch-q01-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q01-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/dq_file/part9/canondata/result.json b/ydb/library/yql/tests/sql/dq_file/part9/canondata/result.json index 259b1c128115..8b8edb5db8b2 100644 --- a/ydb/library/yql/tests/sql/dq_file/part9/canondata/result.json +++ b/ydb/library/yql/tests/sql/dq_file/part9/canondata/result.json @@ -278,9 +278,9 @@ ], "test.test[aggregate-group_by_gs_and_having-default.txt-Debug]": [ { - "checksum": "4a1478e67226b3f286ef4acba5e7bab1", - "size": 4805, - "uri": "https://{canondata_backend}/1931696/76a7bb8a2aaec831535cf7ca5b0ffb62ccdb717e/resource.tar.gz#test.test_aggregate-group_by_gs_and_having-default.txt-Debug_/opt.yql_patched" + "checksum": "3a69321fb54a5b53c2e98e4bbfffbc90", + "size": 4857, + "uri": "https://{canondata_backend}/1775319/ca8674d999cbbb16d2b8ee3ea7569f01aeebffef/resource.tar.gz#test.test_aggregate-group_by_gs_and_having-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_and_having-default.txt-Plan]": [ @@ -300,9 +300,9 @@ ], "test.test[aggregate-group_by_gs_subselect_asterisk-default.txt-Debug]": [ { - "checksum": "989c3a034a4d95b30c28a5c70de56990", - "size": 4177, - "uri": "https://{canondata_backend}/995452/7ec622aec5d016fd39433889930514dc0a6b650d/resource.tar.gz#test.test_aggregate-group_by_gs_subselect_asterisk-default.txt-Debug_/opt.yql_patched" + "checksum": "39d2d47db756d2b49474b5c199c4c38b", + "size": 4229, + "uri": "https://{canondata_backend}/1775059/7880e12b17712e34b83d0a19290f284c039a3892/resource.tar.gz#test.test_aggregate-group_by_gs_subselect_asterisk-default.txt-Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_gs_subselect_asterisk-default.txt-Plan]": [ @@ -322,9 +322,9 @@ ], "test.test[aggregate-group_by_mul_gb_ru--Debug]": [ { - "checksum": "e22296177388156b996f9c7d4cbeba13", - "size": 7205, - "uri": "https://{canondata_backend}/995452/7ec622aec5d016fd39433889930514dc0a6b650d/resource.tar.gz#test.test_aggregate-group_by_mul_gb_ru--Debug_/opt.yql_patched" + "checksum": "4eef89c55c02a52ff6a5363e05a677ab", + "size": 7267, + "uri": "https://{canondata_backend}/1775059/af971073e577c583556209f50933dfff4201794c/resource.tar.gz#test.test_aggregate-group_by_mul_gb_ru--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_mul_gb_ru--Plan]": [ @@ -344,9 +344,9 @@ ], "test.test[aggregate-group_by_rollup_grouping_hum_bind--Debug]": [ { - "checksum": "6f56ba8703599d31b777058eed994bf0", - "size": 7947, - "uri": "https://{canondata_backend}/995452/7ec622aec5d016fd39433889930514dc0a6b650d/resource.tar.gz#test.test_aggregate-group_by_rollup_grouping_hum_bind--Debug_/opt.yql_patched" + "checksum": "0fa3ad8c20368a624649240306680c0e", + "size": 8006, + "uri": "https://{canondata_backend}/1809005/eaed2cb21c41d953b2deee98fb8e34f4399b8dbc/resource.tar.gz#test.test_aggregate-group_by_rollup_grouping_hum_bind--Debug_/opt.yql_patched" } ], "test.test[aggregate-group_by_rollup_grouping_hum_bind--Plan]": [ @@ -366,9 +366,9 @@ ], "test.test[aggregate-list_with_fold_map--Debug]": [ { - "checksum": "1e6c9744360e47f9075b1f1cc549d8ea", - "size": 2572, - "uri": "https://{canondata_backend}/1937367/1710911e4cee83432c347ca77fc35e2630f78589/resource.tar.gz#test.test_aggregate-list_with_fold_map--Debug_/opt.yql_patched" + "checksum": "f4a374cb0de77c173c4391757339cc24", + "size": 2607, + "uri": "https://{canondata_backend}/1920236/5181d12d98fbd8d9942aee7bad394d888116e5c2/resource.tar.gz#test.test_aggregate-list_with_fold_map--Debug_/opt.yql_patched" } ], "test.test[aggregate-list_with_fold_map--Plan]": [ @@ -498,9 +498,9 @@ ], "test.test[blocks-group_by_complex_key--Debug]": [ { - "checksum": "1241daa2f0acc468897ebf5b942e1960", - "size": 3256, - "uri": "https://{canondata_backend}/1889210/02c3d838178ec7378a674d9517d94cf16e6f7cb4/resource.tar.gz#test.test_blocks-group_by_complex_key--Debug_/opt.yql_patched" + "checksum": "46963505228339eaf80c52ce41c1afb1", + "size": 3291, + "uri": "https://{canondata_backend}/1773845/c2740b835a2bd143c8d121693abd92ce773d1c6b/resource.tar.gz#test.test_blocks-group_by_complex_key--Debug_/opt.yql_patched" } ], "test.test[blocks-group_by_complex_key--Plan]": [ @@ -2074,9 +2074,9 @@ ], "test.test[pg-tpch-q08-default.txt-Debug]": [ { - "checksum": "60b5035ba66dbca9a182bd52be8208a7", - "size": 20602, - "uri": "https://{canondata_backend}/1899731/913a9cc37b2bac41869d9049064ff8974738e594/resource.tar.gz#test.test_pg-tpch-q08-default.txt-Debug_/opt.yql_patched" + "checksum": "35ffe4f3b62268a11b89ef00043ccf5b", + "size": 20637, + "uri": "https://{canondata_backend}/1936273/cb7c5fe42d55b43de4d71f64638d96e5d2301a14/resource.tar.gz#test.test_pg-tpch-q08-default.txt-Debug_/opt.yql_patched" } ], "test.test[pg-tpch-q08-default.txt-Plan]": [ diff --git a/ydb/library/yql/tests/sql/sql2yql/canondata/result.json b/ydb/library/yql/tests/sql/sql2yql/canondata/result.json index 41f50062d845..7b3b56f5aa0c 100644 --- a/ydb/library/yql/tests/sql/sql2yql/canondata/result.json +++ b/ydb/library/yql/tests/sql/sql2yql/canondata/result.json @@ -1868,6 +1868,13 @@ "uri": "https://{canondata_backend}/1942415/e6af6d354a98ef890e03fc9f0ff5926afc11a26b/resource.tar.gz#test_sql2yql.test_aggregate-dedup_state_keys_/sql.yql" } ], + "test_sql2yql.test[aggregate-disable_blocks_with_spilling]": [ + { + "checksum": "e1c9df055ae7de78e0d0364ec949dec4", + "size": 1398, + "uri": "https://{canondata_backend}/1936947/cdbc6e86b3a08f513dc20af9f537f10f6b930f5d/resource.tar.gz#test_sql2yql.test_aggregate-disable_blocks_with_spilling_/sql.yql" + } + ], "test_sql2yql.test[aggregate-ensure_count]": [ { "checksum": "680e664bf810c0f13951de38d3cf94f7", @@ -21132,6 +21139,13 @@ "uri": "https://{canondata_backend}/1880306/64654158d6bfb1289c66c626a8162239289559d0/resource.tar.gz#test_sql_format.test_aggregate-dedup_state_keys_/formatted.sql" } ], + "test_sql_format.test[aggregate-disable_blocks_with_spilling]": [ + { + "checksum": "ed1c0334420d2ec08b8ccc4020e4fb6b", + "size": 88, + "uri": "https://{canondata_backend}/1920236/3d99d8b2ede4d290229a75d3c17d5a932a859473/resource.tar.gz#test_sql_format.test_aggregate-disable_blocks_with_spilling_/formatted.sql" + } + ], "test_sql_format.test[aggregate-ensure_count]": [ { "checksum": "7a2ea2eeaf67cc395330f6718ce49635", diff --git a/ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.cfg b/ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.cfg new file mode 100644 index 000000000000..bb375970b492 --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.cfg @@ -0,0 +1,5 @@ +in Input input.txt + +providers dq +pragma dq.SpillingEngine="file"; +pragma dq.EnableSpillingNodes="Aggregation"; diff --git a/ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.sql b/ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.sql new file mode 100644 index 000000000000..0d8671bf0b21 --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/aggregate/disable_blocks_with_spilling.sql @@ -0,0 +1,2 @@ +pragma BlockEngine='force'; +select count(key) from plato.Input group by key; diff --git a/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream-dq_fail.cfg b/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream-dq_fail.cfg new file mode 100644 index 000000000000..c426164726cd --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream-dq_fail.cfg @@ -0,0 +1,4 @@ +xfail +in Input1 input1.txt +udf python3_udf +providers dq diff --git a/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream._sql b/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream._sql new file mode 100644 index 000000000000..82b3767cc507 --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream._sql @@ -0,0 +1,14 @@ +/* postgres can not */ +USE plato; + +$udfScript = @@ +import functools +def Len(stream): + sums = [functools.reduce(lambda x,y: x + y, pair[1], 0) for pair in stream] + return {"sumByAllVal":functools.reduce(lambda x,y: x + y, sums, 0)} +@@; + +$udf = Python3::Len(Callable<(Stream>>)->Struct>, $udfScript); + +--INSERT INTO Output +REDUCE Input1 ON key USING ALL $udf(cast(value as uint32) ?? 0); diff --git a/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream.cfg b/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream.cfg new file mode 100644 index 000000000000..13bb8734c430 --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/produce/reduce_all_with_python_input_stream.cfg @@ -0,0 +1,3 @@ +in Input1 input1.txt +udf python3_udf +providers yt diff --git a/ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream._sql b/ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream._sql new file mode 100644 index 000000000000..f244cd5c0d8c --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream._sql @@ -0,0 +1,14 @@ +/* postgres can not */ +USE plato; + +$udfScript = @@ +import functools +def Len(key, input): + return {"value":functools.reduce(lambda x,y: x + 1, input, 0)} +@@; + +$udf = Python::Len(Callable<(String, Stream)->Struct>, $udfScript); + +$res = (REDUCE Input1 ON key USING $udf(value)); + +select * from $res order by value; diff --git a/ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream.cfg b/ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream.cfg new file mode 100644 index 000000000000..b16f832837fc --- /dev/null +++ b/ydb/library/yql/tests/sql/suites/produce/reduce_with_python_input_stream.cfg @@ -0,0 +1,2 @@ +in Input1 input1.txt +udf python3_udf diff --git a/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-22 b/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-22 index deb1342c6620..93a614eb97ae 100644 --- a/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-22 +++ b/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-22 @@ -11,7 +11,7 @@ "Plans": [ { "Node Type": "ResultSet", - "PlanNodeId": 12, + "PlanNodeId": 11, "PlanNodeType": "ResultSet", "Plans": [ { @@ -20,18 +20,18 @@ { "Inputs": [ { - "ExternalPlanNodeId": 10 + "ExternalPlanNodeId": 9 } ], "Limit": "10", "Name": "Limit" } ], - "PlanNodeId": 11, + "PlanNodeId": 10, "Plans": [ { "Node Type": "Merge", - "PlanNodeId": 10, + "PlanNodeId": 9, "PlanNodeType": "Connection", "Plans": [ { @@ -40,7 +40,7 @@ { "Inputs": [ { - "ExternalPlanNodeId": 8 + "ExternalPlanNodeId": 7 } ], "Limit": "10", @@ -48,14 +48,14 @@ "TopSortBy": "argument.Count0" } ], - "PlanNodeId": 9, + "PlanNodeId": 8, "Plans": [ { "KeyColumns": [ "SearchPhrase" ], "Node Type": "HashShuffle", - "PlanNodeId": 8, + "PlanNodeId": 7, "PlanNodeType": "Connection", "Plans": [ { @@ -64,41 +64,31 @@ { "Inputs": [ { - "ExternalPlanNodeId": 6 + "ExternalPlanNodeId": 5 }, { - "ExternalPlanNodeId": 4 + "ExternalPlanNodeId": 3 } ], "Name": "Union" } ], - "PlanNodeId": 7, + "PlanNodeId": 6, "Plans": [ { "Node Type": "UnionAll", - "PlanNodeId": 6, + "PlanNodeId": 5, "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Aggregate-Filter-TableFullScan", + "Node Type": "Filter-TableFullScan", "Operators": [ { - "Aggregation": "{_yql_agg_0: SUM(state._yql_agg_0,1),_yql_agg_2: MIN(item.URL,state._yql_agg_2),_yql_agg_3: MIN(item.Title,state._yql_agg_3)}", - "GroupBy": "item.SearchPhrase", "Inputs": [ { "InternalOperatorId": 1 } ], - "Name": "Aggregate" - }, - { - "Inputs": [ - { - "InternalOperatorId": 2 - } - ], "Name": "Filter", "Predicate": "item.Title StringContains \"Google\" And Not item.URL StringContains \".google.\" And item.SearchPhrase != \"\"" }, @@ -122,7 +112,9 @@ "Table": "clickbench/plans/row/hits" } ], - "PlanNodeId": 5, + "Parent Relationship": "InitPlan", + "PlanNodeId": 4, + "Subplan Name": "CTE Filter-TableFullScan_9", "Tables": [ "clickbench/plans/row/hits" ] @@ -131,7 +123,7 @@ }, { "Node Type": "UnionAll", - "PlanNodeId": 4, + "PlanNodeId": 3, "PlanNodeType": "Connection", "Plans": [ { @@ -142,74 +134,23 @@ "GroupBy": "item.SearchPhrase", "Inputs": [ { - "ExternalPlanNodeId": 2 + "ExternalPlanNodeId": 1 } ], "Name": "Aggregate" } ], - "PlanNodeId": 3, + "PlanNodeId": 2, "Plans": [ { + "CTE Name": "Filter-TableFullScan_9", "KeyColumns": [ "SearchPhrase", "UserID" ], "Node Type": "HashShuffle", - "PlanNodeId": 2, - "PlanNodeType": "Connection", - "Plans": [ - { - "Node Type": "Aggregate-Filter-TableFullScan", - "Operators": [ - { - "Aggregation": "state", - "GroupBy": "", - "Inputs": [ - { - "InternalOperatorId": 1 - }, - { - "InternalOperatorId": 1 - } - ], - "Name": "Aggregate" - }, - { - "Inputs": [ - { - "InternalOperatorId": 2 - } - ], - "Name": "Filter", - "Predicate": "item.Title StringContains \"Google\" And Not item.URL StringContains \".google.\" And item.SearchPhrase != \"\"" - }, - { - "Inputs": [], - "Name": "TableFullScan", - "ReadColumns": [ - "SearchPhrase", - "Title", - "URL", - "UserID" - ], - "ReadRanges": [ - "CounterID (-\u221e, +\u221e)", - "EventDate (-\u221e, +\u221e)", - "UserID (-\u221e, +\u221e)", - "EventTime (-\u221e, +\u221e)", - "WatchID (-\u221e, +\u221e)" - ], - "Scan": "Parallel", - "Table": "clickbench/plans/row/hits" - } - ], - "PlanNodeId": 1, - "Tables": [ - "clickbench/plans/row/hits" - ] - } - ] + "PlanNodeId": 1, + "PlanNodeType": "Connection" } ] } @@ -236,22 +177,6 @@ { "name": "/local/clickbench/plans/row/hits", "reads": [ - { - "columns": [ - "SearchPhrase", - "Title", - "URL", - "UserID" - ], - "scan_by": [ - "CounterID (-\u221e, +\u221e)", - "EventDate (-\u221e, +\u221e)", - "UserID (-\u221e, +\u221e)", - "EventTime (-\u221e, +\u221e)", - "WatchID (-\u221e, +\u221e)" - ], - "type": "FullScan" - }, { "columns": [ "SearchPhrase", diff --git a/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-9 b/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-9 index 94f862d68bf5..f198c6aafd4e 100644 --- a/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-9 +++ b/ydb/tests/functional/clickbench/canondata/test.test_plans_row_/queries-original-plan-row-9 @@ -11,7 +11,7 @@ "Plans": [ { "Node Type": "ResultSet", - "PlanNodeId": 12, + "PlanNodeId": 11, "PlanNodeType": "ResultSet", "Plans": [ { @@ -20,18 +20,18 @@ { "Inputs": [ { - "ExternalPlanNodeId": 10 + "ExternalPlanNodeId": 9 } ], "Limit": "10", "Name": "Limit" } ], - "PlanNodeId": 11, + "PlanNodeId": 10, "Plans": [ { "Node Type": "Merge", - "PlanNodeId": 10, + "PlanNodeId": 9, "PlanNodeType": "Connection", "Plans": [ { @@ -40,7 +40,7 @@ { "Inputs": [ { - "ExternalPlanNodeId": 8 + "ExternalPlanNodeId": 7 } ], "Limit": "10", @@ -48,14 +48,14 @@ "TopSortBy": "argument.Count0" } ], - "PlanNodeId": 9, + "PlanNodeId": 8, "Plans": [ { "KeyColumns": [ "RegionID" ], "Node Type": "HashShuffle", - "PlanNodeId": 8, + "PlanNodeId": 7, "PlanNodeType": "Connection", "Plans": [ { @@ -64,35 +64,25 @@ { "Inputs": [ { - "ExternalPlanNodeId": 6 + "ExternalPlanNodeId": 5 }, { - "ExternalPlanNodeId": 4 + "ExternalPlanNodeId": 3 } ], "Name": "Union" } ], - "PlanNodeId": 7, + "PlanNodeId": 6, "Plans": [ { "Node Type": "UnionAll", - "PlanNodeId": 6, + "PlanNodeId": 5, "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Aggregate-TableFullScan", + "Node Type": "TableFullScan", "Operators": [ - { - "Aggregation": "{_yql_agg_1: SUM(state._yql_agg_1,1),_yql_agg_3: SUM(item.AdvEngineID,state._yql_agg_3)}", - "GroupBy": "item.RegionID", - "Inputs": [ - { - "InternalOperatorId": 1 - } - ], - "Name": "Aggregate" - }, { "Inputs": [], "Name": "TableFullScan", @@ -113,7 +103,9 @@ "Table": "clickbench/plans/row/hits" } ], - "PlanNodeId": 5, + "Parent Relationship": "InitPlan", + "PlanNodeId": 4, + "Subplan Name": "CTE TableFullScan_9", "Tables": [ "clickbench/plans/row/hits" ] @@ -122,7 +114,7 @@ }, { "Node Type": "UnionAll", - "PlanNodeId": 4, + "PlanNodeId": 3, "PlanNodeType": "Connection", "Plans": [ { @@ -133,62 +125,23 @@ "GroupBy": "item.RegionID", "Inputs": [ { - "ExternalPlanNodeId": 2 + "ExternalPlanNodeId": 1 } ], "Name": "Aggregate" } ], - "PlanNodeId": 3, + "PlanNodeId": 2, "Plans": [ { + "CTE Name": "TableFullScan_9", "KeyColumns": [ "RegionID", "UserID" ], "Node Type": "HashShuffle", - "PlanNodeId": 2, - "PlanNodeType": "Connection", - "Plans": [ - { - "Node Type": "Aggregate-TableFullScan", - "Operators": [ - { - "Aggregation": "state", - "GroupBy": "", - "Inputs": [ - { - "InternalOperatorId": 1 - } - ], - "Name": "Aggregate" - }, - { - "Inputs": [], - "Name": "TableFullScan", - "ReadColumns": [ - "AdvEngineID", - "RegionID", - "ResolutionWidth", - "UserID" - ], - "ReadRanges": [ - "CounterID (-\u221e, +\u221e)", - "EventDate (-\u221e, +\u221e)", - "UserID (-\u221e, +\u221e)", - "EventTime (-\u221e, +\u221e)", - "WatchID (-\u221e, +\u221e)" - ], - "Scan": "Parallel", - "Table": "clickbench/plans/row/hits" - } - ], - "PlanNodeId": 1, - "Tables": [ - "clickbench/plans/row/hits" - ] - } - ] + "PlanNodeId": 1, + "PlanNodeType": "Connection" } ] } @@ -215,22 +168,6 @@ { "name": "/local/clickbench/plans/row/hits", "reads": [ - { - "columns": [ - "AdvEngineID", - "RegionID", - "ResolutionWidth", - "UserID" - ], - "scan_by": [ - "CounterID (-\u221e, +\u221e)", - "EventDate (-\u221e, +\u221e)", - "UserID (-\u221e, +\u221e)", - "EventTime (-\u221e, +\u221e)", - "WatchID (-\u221e, +\u221e)" - ], - "type": "FullScan" - }, { "columns": [ "AdvEngineID",