diff --git a/.gitignore b/.gitignore index f97992275a5d..bde4ce084232 100644 --- a/.gitignore +++ b/.gitignore @@ -26,6 +26,9 @@ __pycache__/ *.pb.h *.pb.cc +# Other generated +*.fbs.h + # MacOS specific .DS_Store diff --git a/ydb/core/kqp/compile_service/kqp_compile_actor.cpp b/ydb/core/kqp/compile_service/kqp_compile_actor.cpp index 2f629eee29e5..0ca07f3f8c2e 100644 --- a/ydb/core/kqp/compile_service/kqp_compile_actor.cpp +++ b/ydb/core/kqp/compile_service/kqp_compile_actor.cpp @@ -452,9 +452,9 @@ class TKqpCompileActor : public TActorBootstrapped { } void FillCompileResult(std::unique_ptr preparingQuery, NKikimrKqp::EQueryType queryType, - bool allowCache) { + bool allowCache, bool success) { auto preparedQueryHolder = std::make_shared( - preparingQuery.release(), AppData()->FunctionRegistry); + preparingQuery.release(), AppData()->FunctionRegistry, !success); preparedQueryHolder->MutableLlvmSettings().Fill(Config, queryType); KqpCompileResult->PreparedQuery = preparedQueryHolder; KqpCompileResult->AllowCache = CanCacheQuery(KqpCompileResult->PreparedQuery->GetPhysicalQuery()) && allowCache; @@ -503,7 +503,7 @@ class TKqpCompileActor : public TActorBootstrapped { if (status == Ydb::StatusIds::SUCCESS) { YQL_ENSURE(kqpResult.PreparingQuery); - FillCompileResult(std::move(kqpResult.PreparingQuery), queryType, kqpResult.AllowCache); + FillCompileResult(std::move(kqpResult.PreparingQuery), queryType, kqpResult.AllowCache, true); auto now = TInstant::Now(); auto duration = now - StartTime; @@ -514,7 +514,7 @@ class TKqpCompileActor : public TActorBootstrapped { << ", duration: " << duration); } else { if (kqpResult.PreparingQuery) { - FillCompileResult(std::move(kqpResult.PreparingQuery), queryType, kqpResult.AllowCache); + FillCompileResult(std::move(kqpResult.PreparingQuery), queryType, kqpResult.AllowCache, false); } LOG_ERROR_S(ctx, NKikimrServices::KQP_COMPILE_ACTOR, "Compilation failed" diff --git a/ydb/core/kqp/executer_actor/kqp_data_executer.cpp b/ydb/core/kqp/executer_actor/kqp_data_executer.cpp index 87dd3ec0142e..50488467d667 100644 --- a/ydb/core/kqp/executer_actor/kqp_data_executer.cpp +++ b/ydb/core/kqp/executer_actor/kqp_data_executer.cpp @@ -206,41 +206,9 @@ class TKqpDataExecuter : public TKqpExecuterBaseCollectStatsByLongTasks && HasOlapTable; - } - - void FillResponseStats(Ydb::StatusIds::StatusCode status) { - auto& response = *ResponseEv->Record.MutableResponse(); - - response.SetStatus(status); - - if (Stats) { - ReportEventElapsedTime(); - - Stats->FinishTs = TInstant::Now(); - Stats->Finish(); - - if (LogStatsByLongTasks() || CollectFullStats(Request.StatsMode)) { - for (ui32 txId = 0; txId < Request.Transactions.size(); ++txId) { - const auto& tx = Request.Transactions[txId].Body; - auto planWithStats = AddExecStatsToTxPlan(tx->GetPlan(), response.GetResult().GetStats()); - response.MutableResult()->MutableStats()->AddTxPlansWithStats(planWithStats); - } - } - - if (LogStatsByLongTasks()) { - const auto& txPlansWithStats = response.GetResult().GetStats().GetTxPlansWithStats(); - if (!txPlansWithStats.empty()) { - LOG_N("Full stats: " << txPlansWithStats); - } - } - - Stats.reset(); - } - } - void Finalize() { + YQL_ENSURE(!AlreadyReplied); + if (LocksBroken) { TString message = "Transaction locks invalidated."; @@ -248,9 +216,7 @@ class TKqpDataExecuter : public TKqpExecuterBaseRecord.MutableResponse(); - - FillResponseStats(Ydb::StatusIds::SUCCESS); + ResponseEv->Record.MutableResponse()->SetStatus(Ydb::StatusIds::SUCCESS); Counters->TxProxyMon->ReportStatusOK->Inc(); auto addLocks = [this](const auto& data) { @@ -287,7 +253,7 @@ class TKqpDataExecuter : public TKqpExecuterBaseLockHandle = std::move(LockHandle); } - BuildLocks(*response.MutableResult()->MutableLocks(), Locks); + BuildLocks(*ResponseEv->Record.MutableResponse()->MutableResult()->MutableLocks(), Locks); } auto resultSize = ResponseEv->GetByteSize(); @@ -313,9 +279,7 @@ class TKqpDataExecuter : public TKqpExecuterBaseResultsSize()); - Send(Target, ResponseEv.release()); + AlreadyReplied = true; PassAway(); } @@ -355,6 +319,8 @@ class TKqpDataExecuter : public TKqpExecuterBaseGet()->Record.GetState() == NDqProto::COMPUTE_STATE_FAILURE) { CancelProposal(0); } - HandleComputeStats(ev); + HandleComputeState(ev); } void HandlePrepare(TEvPipeCache::TEvDeliveryProblem::TPtr& ev) { @@ -1041,7 +1007,7 @@ class TKqpDataExecuter : public TKqpExecuterBaseGetPendingComputeTasks().empty() && Planner->GetPendingComputeActors().empty()) { + LOG_I("Shutdown immediately - nothing to wait"); + PassAway(); + } else { + this->Become(&TThis::WaitShutdownState); + LOG_I("Waiting for shutdown of " << Planner->GetPendingComputeTasks().size() << " tasks and " + << Planner->GetPendingComputeActors().size() << " compute actors"); + // TODO(ilezhankin): the CA awaiting timeout should be configurable. + TActivationContext::Schedule(TDuration::Seconds(10), new IEventHandle(SelfId(), SelfId(), new TEvents::TEvPoison)); + } + } else { + PassAway(); + } + } + void PassAway() override { auto totalTime = TInstant::Now() - StartTime; Counters->Counters->DataTxTotalTimeHistogram->Collect(totalTime.MilliSeconds()); @@ -2648,6 +2631,54 @@ class TKqpDataExecuter : public TKqpExecuterBaseGetTypeRewrite()) { + hFunc(TEvDqCompute::TEvState, HandleShutdown); + hFunc(TEvInterconnect::TEvNodeDisconnected, HandleShutdown); + hFunc(TEvents::TEvPoison, HandleShutdown); + default: + LOG_E("Unexpected event: " << ev->GetTypeName()); // ignore all other events + } + } + + void HandleShutdown(TEvDqCompute::TEvState::TPtr& ev) { + HandleComputeStats(ev); + + if (Planner->GetPendingComputeTasks().empty() && Planner->GetPendingComputeActors().empty()) { + PassAway(); + } + } + + void HandleShutdown(TEvInterconnect::TEvNodeDisconnected::TPtr& ev) { + const auto nodeId = ev->Get()->NodeId; + LOG_N("Node has disconnected while shutdown: " << nodeId); + + YQL_ENSURE(Planner); + + for (const auto& task : TasksGraph.GetTasks()) { + if (task.Meta.NodeId == nodeId && !task.Meta.Completed) { + if (task.ComputeActorId) { + Planner->CompletedCA(task.Id, task.ComputeActorId); + } else { + Planner->TaskNotStarted(task.Id); + } + } + } + + if (Planner->GetPendingComputeTasks().empty() && Planner->GetPendingComputeActors().empty()) { + PassAway(); + } + } + + void HandleShutdown(TEvents::TEvPoison::TPtr& ev) { + // Self-poison means timeout - don't wait anymore. + LOG_I("Timed out on waiting for Compute Actors to finish - forcing shutdown"); + + if (ev->Sender == SelfId()) { + PassAway(); + } + } + private: void ReplyTxStateUnknown(ui64 shardId) { auto message = TStringBuilder() << "Tx state unknown for shard " << shardId << ", txid " << TxId; diff --git a/ydb/core/kqp/executer_actor/kqp_executer_impl.h b/ydb/core/kqp/executer_actor/kqp_executer_impl.h index 25b897d3dd41..9e99b72f31ff 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_impl.h +++ b/ydb/core/kqp/executer_actor/kqp_executer_impl.h @@ -177,10 +177,10 @@ class TKqpExecuterBase : public TActorBootstrapped { } void ReportEventElapsedTime() { - if (Stats) { - ui64 elapsedMicros = TlsActivationContext->GetCurrentEventTicksAsSeconds() * 1'000'000; - Stats->ExecuterCpuTime += TDuration::MicroSeconds(elapsedMicros); - } + YQL_ENSURE(Stats); + + ui64 elapsedMicros = TlsActivationContext->GetCurrentEventTicksAsSeconds() * 1'000'000; + Stats->ExecuterCpuTime += TDuration::MicroSeconds(elapsedMicros); } protected: @@ -330,11 +330,10 @@ class TKqpExecuterBase : public TActorBootstrapped { } YQL_ENSURE(channel.DstTask == 0); + YQL_ENSURE(Stats); - if (Stats) { - Stats->ResultBytes += batch.Size(); - Stats->ResultRows += batch.RowCount(); - } + Stats->ResultBytes += batch.Size(); + Stats->ResultRows += batch.RowCount(); LOG_T("Got result, channelId: " << channel.Id << ", shardId: " << task.Meta.ShardId << ", inputIndex: " << channel.DstInputIndex << ", from: " << ev->Sender @@ -380,7 +379,7 @@ class TKqpExecuterBase : public TActorBootstrapped { this->Send(channelComputeActorId, ackEv.Release(), /* TODO: undelivery */ 0, /* cookie */ channelId); } - void HandleComputeStats(NYql::NDq::TEvDqCompute::TEvState::TPtr& ev) { + bool HandleComputeStats(NYql::NDq::TEvDqCompute::TEvState::TPtr& ev) { TActorId computeActor = ev->Sender; auto& state = ev->Get()->Record; ui64 taskId = state.GetTaskId(); @@ -391,7 +390,9 @@ class TKqpExecuterBase : public TActorBootstrapped { << ", state: " << NYql::NDqProto::EComputeState_Name((NYql::NDqProto::EComputeState) state.GetState()) << ", stats: " << state.GetStats()); - if (Stats && state.HasStats() && Request.ProgressStatsPeriod) { + YQL_ENSURE(Stats); + + if (state.HasStats() && Request.ProgressStatsPeriod) { Stats->UpdateTaskStats(taskId, state.GetStats()); auto now = TInstant::Now(); if (LastProgressStats + Request.ProgressStatsPeriod <= now) { @@ -408,84 +409,62 @@ class TKqpExecuterBase : public TActorBootstrapped { } } + YQL_ENSURE(Planner); + bool ack = Planner->AcknowledgeCA(taskId, computeActor, &state); + + switch (state.GetState()) { + case NYql::NDqProto::COMPUTE_STATE_FAILURE: + case NYql::NDqProto::COMPUTE_STATE_FINISHED: + // Don't finalize stats twice. + if (Planner->CompletedCA(taskId, computeActor)) { + ExtraData[computeActor].Swap(state.MutableExtraData()); + + Stats->AddComputeActorStats( + computeActor.NodeId(), + std::move(*state.MutableStats()), + TDuration::MilliSeconds(AggregationSettings.GetCollectLongTasksStatsTimeoutMs()) + ); + + LastTaskId = taskId; + LastComputeActorId = computeActor.ToString(); + } + default: + ; // ignore all other states. + } + + return ack; + } + + void HandleComputeState(NYql::NDq::TEvDqCompute::TEvState::TPtr& ev) { + TActorId computeActor = ev->Sender; + auto& state = ev->Get()->Record; + ui64 taskId = state.GetTaskId(); + + bool populateChannels = HandleComputeStats(ev); + switch (state.GetState()) { case NYql::NDqProto::COMPUTE_STATE_UNKNOWN: { YQL_ENSURE(false, "unexpected state from " << computeActor << ", task: " << taskId); return; } - case NYql::NDqProto::COMPUTE_STATE_FAILURE: { - ReplyErrorAndDie(NYql::NDq::DqStatusToYdbStatus(state.GetStatusCode()), state.MutableIssues()); - return; - } - case NYql::NDqProto::COMPUTE_STATE_EXECUTING: { - // initial TEvState event from Compute Actor - // there can be race with RM answer - if (Planner) { - if (Planner->GetPendingComputeTasks().erase(taskId)) { - auto it = Planner->GetPendingComputeActors().emplace(computeActor, TProgressStat()); - YQL_ENSURE(it.second); - - if (state.HasStats()) { - it.first->second.Set(state.GetStats()); - } - - auto& task = TasksGraph.GetTask(taskId); - task.ComputeActorId = computeActor; - - THashMap> updates; - CollectTaskChannelsUpdates(task, updates); - PropagateChannelsUpdates(updates); - } else { - auto it = Planner->GetPendingComputeActors().find(computeActor); - if (it != Planner->GetPendingComputeActors().end()) { - if (state.HasStats()) { - it->second.Set(state.GetStats()); - } - } - } + if (populateChannels) { + auto& task = TasksGraph.GetTask(taskId); + THashMap> updates; + CollectTaskChannelsUpdates(task, updates); + PropagateChannelsUpdates(updates); } break; } - case NYql::NDqProto::COMPUTE_STATE_FINISHED: { - if (Stats) { - Stats->AddComputeActorStats( - computeActor.NodeId(), - std::move(*state.MutableStats()), - TDuration::MilliSeconds(AggregationSettings.GetCollectLongTasksStatsTimeoutMs()) - ); - } - ExtraData[computeActor].Swap(state.MutableExtraData()); - - LastTaskId = taskId; - LastComputeActorId = computeActor.ToString(); - - if (Planner) { - auto it = Planner->GetPendingComputeActors().find(computeActor); - if (it == Planner->GetPendingComputeActors().end()) { - LOG_W("Got execution state for compute actor: " << computeActor - << ", task: " << taskId - << ", state: " << NYql::NDqProto::EComputeState_Name((NYql::NDqProto::EComputeState) state.GetState()) - << ", too early (waiting reply from RM)"); - - if (Planner && Planner->GetPendingComputeTasks().erase(taskId)) { - LOG_E("Got execution state for compute actor: " << computeActor - << ", for unknown task: " << state.GetTaskId() - << ", state: " << NYql::NDqProto::EComputeState_Name((NYql::NDqProto::EComputeState) state.GetState())); - return; - } - } else { - if (state.HasStats()) { - it->second.Set(state.GetStats()); - } - LastStats.emplace_back(std::move(it->second)); - Planner->GetPendingComputeActors().erase(it); - YQL_ENSURE(Planner->GetPendingComputeTasks().find(taskId) == Planner->GetPendingComputeTasks().end()); - } - } - } + default: + ; // ignore all other states. + } + + if (state.GetState() == NYql::NDqProto::COMPUTE_STATE_FAILURE) { + ReplyErrorAndDie(NYql::NDq::DqStatusToYdbStatus(state.GetStatusCode()), state.MutableIssues()); + return; } static_cast(this)->CheckExecutionComplete(); @@ -532,9 +511,9 @@ class TKqpExecuterBase : public TActorBootstrapped { auto now = TAppData::TimeProvider->Now(); StartResolveTime = now; - if (Stats) { - Stats->StartTs = now; - } + YQL_ENSURE(Stats); + + Stats->StartTs = now; } TMaybe FindReadRangesSource(const NKqpProto::TKqpPhyStage& stage) { @@ -683,20 +662,14 @@ class TKqpExecuterBase : public TActorBootstrapped { auto taskId = startedTask.GetTaskId(); auto& task = TasksGraph.GetTask(taskId); - task.ComputeActorId = ActorIdFromProto(startedTask.GetActorId()); - - LOG_D("Executing task: " << taskId << " on compute actor: " << task.ComputeActorId); - - if (Planner) { - if (Planner->GetPendingComputeTasks().erase(taskId) == 0) { - LOG_D("Executing task: " << taskId << ", compute actor: " << task.ComputeActorId << ", already finished"); - } else { - auto result = Planner->GetPendingComputeActors().emplace(std::make_pair(task.ComputeActorId, TProgressStat())); - YQL_ENSURE(result.second); - - CollectTaskChannelsUpdates(task, channelsUpdates); - } + TActorId computeActorId = ActorIdFromProto(startedTask.GetActorId()); + LOG_D("Executing task: " << taskId << " on compute actor: " << computeActorId); + YQL_ENSURE(Planner); + bool channelUpdates = Planner->AcknowledgeCA(taskId, computeActorId, nullptr); + if (channelUpdates) { + CollectTaskChannelsUpdates(task, channelsUpdates); } + } PropagateChannelsUpdates(channelsUpdates); @@ -711,7 +684,7 @@ class TKqpExecuterBase : public TActorBootstrapped { if (statusCode == Ydb::StatusIds::INTERNAL_ERROR) { InternalError(issues); } else if (statusCode == Ydb::StatusIds::TIMEOUT) { - AbortExecutionAndDie(ev->Sender, NYql::NDqProto::StatusIds::TIMEOUT, "Request timeout exceeded"); + TimeoutError(ev->Sender); } else { RuntimeError(NYql::NDq::DqStatusToYdbStatus(msg.GetStatusCode()), issues); } @@ -789,16 +762,9 @@ class TKqpExecuterBase : public TActorBootstrapped { LastResourceUsageUpdate = now; TProgressStat::TEntry consumption; - if (Planner) { - for (const auto& p : Planner->GetPendingComputeActors()) { - const auto& t = p.second.GetLastUsage(); - consumption += t; - } - } - for (const auto& p : LastStats) { - const auto& t = p.GetLastUsage(); - consumption += t; + if (Planner) { + consumption += Planner->CalculateConsumptionUpdate(); } auto ru = NRuCalc::CalcRequestUnit(consumption); @@ -811,13 +777,7 @@ class TKqpExecuterBase : public TActorBootstrapped { return; if (Planner) { - for (auto& p : Planner->GetPendingComputeActors()) { - p.second.Update(); - } - } - - for (auto& p : LastStats) { - p.Update(); + Planner->ShiftConsumption(); } if (Request.RlPath) { @@ -1187,8 +1147,9 @@ class TKqpExecuterBase : public TActorBootstrapped { : Nothing(); YQL_ENSURE(!shardsResolved || nodeId); + YQL_ENSURE(Stats); - if (shardId && Stats) { + if (shardId) { Stats->AffectedShards.insert(*shardId); } @@ -1256,11 +1217,13 @@ class TKqpExecuterBase : public TActorBootstrapped { if (partitions.size() > 0 && source.GetSequentialInFlightShards() > 0 && partitions.size() > source.GetSequentialInFlightShards()) { auto [startShard, shardInfo] = MakeVirtualTablePartition(source, stageInfo, HolderFactory(), TypeEnv()); - if (Stats) { - for (auto& [shardId, _] : partitions) { - Stats->AffectedShards.insert(shardId); - } + + YQL_ENSURE(Stats); + + for (auto& [shardId, _] : partitions) { + Stats->AffectedShards.insert(shardId); } + if (shardInfo.KeyReadRanges) { addPartiton(startShard, {}, shardInfo, source.GetSequentialInFlightShards()); fillRangesForTasks(); @@ -1523,6 +1486,8 @@ class TKqpExecuterBase : public TActorBootstrapped { THashMap assignedShardsCount; auto& stage = stageInfo.Meta.GetStage(stageInfo.Id); + YQL_ENSURE(Stats); + const auto& tableInfo = stageInfo.Meta.TableConstInfo; const auto& keyTypes = tableInfo->KeyColumnTypes; ui32 metaId = 0; @@ -1551,7 +1516,7 @@ class TKqpExecuterBase : public TActorBootstrapped { nodeShards[nodeId].emplace_back(TShardInfoWithId(i.first, std::move(i.second))); } - if (Stats && CollectProfileStats(Request.StatsMode)) { + if (CollectProfileStats(Request.StatsMode)) { for (auto&& i : nodeShards) { Stats->AddNodeShardsCount(stageInfo.Id.StageId, i.first, i.second.size()); } @@ -1658,14 +1623,14 @@ class TKqpExecuterBase : public TActorBootstrapped { protected: void TerminateComputeActors(Ydb::StatusIds::StatusCode code, const NYql::TIssues& issues) { for (const auto& task : this->TasksGraph.GetTasks()) { - if (task.ComputeActorId) { + if (task.ComputeActorId && !task.Meta.Completed) { LOG_I("aborting compute actor execution, message: " << issues.ToOneLineString() << ", compute actor: " << task.ComputeActorId << ", task: " << task.Id); auto ev = MakeHolder(NYql::NDq::YdbStatusToDqStatus(code), issues); this->Send(task.ComputeActorId, ev.Release()); } else { - LOG_I("task: " << task.Id << ", does not have Compute ActorId yet"); + LOG_I("task: " << task.Id << ", does not have the CA id yet or is already complete"); } } } @@ -1683,7 +1648,6 @@ class TKqpExecuterBase : public TActorBootstrapped { void InternalError(const NYql::TIssues& issues) { LOG_E(issues.ToOneLineString()); - TerminateComputeActors(Ydb::StatusIds::INTERNAL_ERROR, issues); auto issue = NYql::YqlIssue({}, NYql::TIssuesIds::UNEXPECTED, "Internal error while executing transaction."); for (const NYql::TIssue& i : issues) { issue.AddSubIssue(MakeIntrusive(i)); @@ -1697,7 +1661,6 @@ class TKqpExecuterBase : public TActorBootstrapped { void ReplyUnavailable(const TString& message) { LOG_E("UNAVAILABLE: " << message); - TerminateComputeActors(Ydb::StatusIds::UNAVAILABLE, message); auto issue = NYql::YqlIssue({}, NYql::TIssuesIds::KIKIMR_TEMPORARILY_UNAVAILABLE); issue.AddSubIssue(new NYql::TIssue(message)); ReplyErrorAndDie(Ydb::StatusIds::UNAVAILABLE, issue); @@ -1705,7 +1668,6 @@ class TKqpExecuterBase : public TActorBootstrapped { void RuntimeError(Ydb::StatusIds::StatusCode code, const NYql::TIssues& issues) { LOG_E(Ydb::StatusIds_StatusCode_Name(code) << ": " << issues.ToOneLineString()); - TerminateComputeActors(code, issues); ReplyErrorAndDie(code, issues); } @@ -1721,48 +1683,46 @@ class TKqpExecuterBase : public TActorBootstrapped { ReplyErrorAndDie(status, &issues); } - void AbortExecutionAndDie(TActorId abortSender, NYql::NDqProto::StatusIds::StatusCode status, const TString& message) { + void TimeoutError(TActorId abortSender) { if (AlreadyReplied) { + LOG_E("Timeout when we already replied - not good" << Endl << TBackTrace().PrintToString() << Endl); return; } + const auto status = NYql::NDqProto::StatusIds::TIMEOUT; + const TString message = "Request timeout exceeded"; + + TerminateComputeActors(Ydb::StatusIds::TIMEOUT, message); + + AlreadyReplied = true; + LOG_E("Abort execution: " << NYql::NDqProto::StatusIds_StatusCode_Name(status) << "," << message); if (ExecuterSpan) { ExecuterSpan.EndError(TStringBuilder() << NYql::NDqProto::StatusIds_StatusCode_Name(status)); } - static_cast(this)->FillResponseStats(Ydb::StatusIds::TIMEOUT); + ResponseEv->Record.MutableResponse()->SetStatus(Ydb::StatusIds::TIMEOUT); // TEvAbortExecution can come from either ComputeActor or SessionActor (== Target). if (abortSender != Target) { - auto abortEv = MakeHolder(status, "Request timeout exceeded"); + auto abortEv = MakeHolder(status, message); this->Send(Target, abortEv.Release()); } - AlreadyReplied = true; LOG_E("Sending timeout response to: " << Target); - this->Send(Target, ResponseEv.release()); - Request.Transactions.crop(0); - TerminateComputeActors(Ydb::StatusIds::TIMEOUT, message); - this->PassAway(); + this->Shutdown(); } virtual void ReplyErrorAndDie(Ydb::StatusIds::StatusCode status, google::protobuf::RepeatedPtrField* issues) { if (AlreadyReplied) { + LOG_E("Error when we already replied - not good" << Endl << TBackTrace().PrintToString() << Endl); return; } - if (Planner) { - for (auto computeActor : Planner->GetPendingComputeActors()) { - LOG_D("terminate compute actor " << computeActor.first); - - auto ev = MakeHolder(NYql::NDq::YdbStatusToDqStatus(status), "Terminate execution"); - this->Send(computeActor.first, ev.Release()); - } - } + TerminateComputeActors(status, "Terminate execution"); AlreadyReplied = true; auto& response = *ResponseEv->Record.MutableResponse(); @@ -1786,9 +1746,7 @@ class TKqpExecuterBase : public TActorBootstrapped { ExecuterSpan.EndError(response.DebugString()); ExecuterStateSpan.EndError(response.DebugString()); - Request.Transactions.crop(0); - this->Send(Target, ResponseEv.release()); - this->PassAway(); + this->Shutdown(); } protected: @@ -1856,7 +1814,46 @@ class TKqpExecuterBase : public TActorBootstrapped { } protected: + // Introduced separate method from `PassAway()` - to not get confused with expectations from other actors, + // that `PassAway()` should kill actor immediately. + virtual void Shutdown() { + PassAway(); + } + void PassAway() override { + YQL_ENSURE(AlreadyReplied && ResponseEv); + + // Fill response stats + { + auto& response = *ResponseEv->Record.MutableResponse(); + + YQL_ENSURE(Stats); + + ReportEventElapsedTime(); + + Stats->FinishTs = TInstant::Now(); + Stats->Finish(); + + if (Stats->CollectStatsByLongTasks || CollectFullStats(Request.StatsMode)) { + response.MutableResult()->MutableStats()->ClearTxPlansWithStats(); + for (ui32 txId = 0; txId < Request.Transactions.size(); ++txId) { + const auto& tx = Request.Transactions[txId].Body; + auto planWithStats = AddExecStatsToTxPlan(tx->GetPlan(), response.GetResult().GetStats()); + response.MutableResult()->MutableStats()->AddTxPlansWithStats(planWithStats); + } + } + + if (Stats->CollectStatsByLongTasks) { + const auto& txPlansWithStats = response.GetResult().GetStats().GetTxPlansWithStats(); + if (!txPlansWithStats.empty()) { + LOG_N("Full stats: " << response.GetResult().GetStats()); + } + } + } + + Request.Transactions.crop(0); + this->Send(Target, ResponseEv.release()); + for (auto channelPair: ResultChannelProxies) { LOG_D("terminate result channel " << channelPair.first << " proxy at " << channelPair.second->SelfId()); @@ -1877,12 +1874,11 @@ class TKqpExecuterBase : public TActorBootstrapped { if (KqpTableResolverId) { this->Send(KqpTableResolverId, new TEvents::TEvPoison); - this->Send(this->SelfId(), new TEvents::TEvPoison); - LOG_T("Terminate, become ZombieState"); - this->Become(&TKqpExecuterBase::ZombieState); - } else { - IActor::PassAway(); } + + this->Send(this->SelfId(), new TEvents::TEvPoison); + LOG_T("Terminate, become ZombieState"); + this->Become(&TKqpExecuterBase::ZombieState); } STATEFN(ZombieState) { @@ -1945,8 +1941,6 @@ class TKqpExecuterBase : public TActorBootstrapped { TActorId KqpShardsResolverId; THashMap ExtraData; - TVector LastStats; - TInstant StartResolveTime; TInstant LastResourceUsageUpdate; diff --git a/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp b/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp index 33fbb0a00156..a59366868edb 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp +++ b/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp @@ -449,6 +449,10 @@ void TQueryExecutionStats::AddComputeActorStats(ui32 /* nodeId */, NYql::NDqProt } // checking whether the task is long + + // TODO(ilezhankin): investigate - for some reason `task.FinishTimeMs` may be large (or small?) + // enough to result in an enormous duration - triggering the "long tasks" mode. + auto taskDuration = TDuration::MilliSeconds(task.GetFinishTimeMs() - task.GetStartTimeMs()); bool longTask = taskDuration > collectLongTaskStatsTimeout; if (longTask) { @@ -784,7 +788,7 @@ void TQueryExecutionStats::Finish() { Result->SetCpuTimeUs(Result->GetCpuTimeUs() + ExecuterCpuTime.MicroSeconds()); Result->SetDurationUs(FinishTs.MicroSeconds() - StartTs.MicroSeconds()); - // Result->Result* feilds are (temporary?) commented out in proto due to lack of use + // Result->Result* fields are (temporary?) commented out in proto due to lack of use // // Result->SetResultBytes(ResultBytes); // Result->SetResultRows(ResultRows); diff --git a/ydb/core/kqp/executer_actor/kqp_planner.cpp b/ydb/core/kqp/executer_actor/kqp_planner.cpp index 696b8732c1d1..9c7988ba68f7 100644 --- a/ydb/core/kqp/executer_actor/kqp_planner.cpp +++ b/ydb/core/kqp/executer_actor/kqp_planner.cpp @@ -443,12 +443,7 @@ TString TKqpPlanner::ExecuteDataComputeTask(ui64 taskId, ui32 computeTasksSize) TActorId* actorId = std::get_if(&startResult); Y_ABORT_UNLESS(actorId); - task.ComputeActorId = *actorId; - - LOG_D("Executing task: " << taskId << " on compute actor: " << task.ComputeActorId); - - auto result = PendingComputeActors.emplace(task.ComputeActorId, TProgressStat()); - YQL_ENSURE(result.second); + AcknowledgeCA(taskId, *actorId, nullptr); return TString(); } @@ -533,8 +528,6 @@ std::unique_ptr TKqpPlanner::PlanExecution() { if (!result.empty()) { return MakeActorStartFailureError(ExecuterId, result); } - - PendingComputeTasks.erase(taskId); } } } @@ -576,11 +569,93 @@ void TKqpPlanner::Unsubscribe() { } } -THashMap& TKqpPlanner::GetPendingComputeActors() { +bool TKqpPlanner::AcknowledgeCA(ui64 taskId, TActorId computeActor, const NYql::NDqProto::TEvComputeActorState* state) { + auto& task = TasksGraph.GetTask(taskId); + if (!task.ComputeActorId) { + task.ComputeActorId = computeActor; + PendingComputeTasks.erase(taskId); + auto [it, success] = PendingComputeActors.try_emplace(computeActor); + YQL_ENSURE(success); + if (state && state->HasStats()) { + it->second.Set(state->GetStats()); + } + + return true; + } + + YQL_ENSURE(task.ComputeActorId == computeActor); + auto it = PendingComputeActors.find(computeActor); + if (!task.Meta.Completed) { + YQL_ENSURE(it != PendingComputeActors.end()); + } + + if (it != PendingComputeActors.end() && state && state->HasStats()) { + it->second.Set(state->GetStats()); + } + + return false; +} + +bool TKqpPlanner::CompletedCA(ui64 taskId, TActorId computeActor) { + auto& task = TasksGraph.GetTask(taskId); + if (task.Meta.Completed) { + YQL_ENSURE(!PendingComputeActors.contains(computeActor)); + return false; + } + + task.Meta.Completed = true; + auto it = PendingComputeActors.find(computeActor); + YQL_ENSURE(it != PendingComputeActors.end()); + LastStats.emplace_back(std::move(it->second)); + PendingComputeActors.erase(it); + + LOG_I("Compute actor has finished execution: " << computeActor.ToString()); + + return true; +} + +void TKqpPlanner::TaskNotStarted(ui64 taskId) { + // NOTE: should be invoked only while shutting down - when node is disconnected. + + auto& task = TasksGraph.GetTask(taskId); + + YQL_ENSURE(!task.ComputeActorId); + YQL_ENSURE(!task.Meta.Completed); + + PendingComputeTasks.erase(taskId); +} + +TProgressStat::TEntry TKqpPlanner::CalculateConsumptionUpdate() { + TProgressStat::TEntry consumption; + + for (const auto& p : PendingComputeActors) { + const auto& t = p.second.GetLastUsage(); + consumption += t; + } + + for (const auto& p : LastStats) { + const auto& t = p.GetLastUsage(); + consumption += t; + } + + return consumption; +} + +void TKqpPlanner::ShiftConsumption() { + for (auto& p : PendingComputeActors) { + p.second.Update(); + } + + for (auto& p : LastStats) { + p.Update(); + } +} + +const THashMap& TKqpPlanner::GetPendingComputeActors() { return PendingComputeActors; } -THashSet& TKqpPlanner::GetPendingComputeTasks() { +const THashSet& TKqpPlanner::GetPendingComputeTasks() { return PendingComputeTasks; } diff --git a/ydb/core/kqp/executer_actor/kqp_planner.h b/ydb/core/kqp/executer_actor/kqp_planner.h index 03ce07758cf5..ab05969f7fd1 100644 --- a/ydb/core/kqp/executer_actor/kqp_planner.h +++ b/ydb/core/kqp/executer_actor/kqp_planner.h @@ -72,12 +72,17 @@ class TKqpPlanner { bool SendStartKqpTasksRequest(ui32 requestId, const TActorId& target); std::unique_ptr PlanExecution(); std::unique_ptr AssignTasksToNodes(); + bool AcknowledgeCA(ui64 taskId, TActorId computeActor, const NYql::NDqProto::TEvComputeActorState* state); + bool CompletedCA(ui64 taskId, TActorId computeActor); + void TaskNotStarted(ui64 taskId); + TProgressStat::TEntry CalculateConsumptionUpdate(); + void ShiftConsumption(); void Submit(); ui32 GetCurrentRetryDelay(ui32 requestId); void Unsubscribe(); - THashMap& GetPendingComputeActors(); - THashSet& GetPendingComputeTasks(); + const THashMap& GetPendingComputeActors(); + const THashSet& GetPendingComputeTasks(); ui32 GetnScanTasks(); ui32 GetnComputeTasks(); @@ -134,6 +139,7 @@ class TKqpPlanner { std::shared_ptr ResourceManager_; std::shared_ptr CaFactory_; TIntrusivePtr TxInfo; + TVector LastStats; public: static bool UseMockEmptyPlanner; // for tests: if true then use TKqpMockEmptyPlanner that leads to the error diff --git a/ydb/core/kqp/executer_actor/kqp_scan_executer.cpp b/ydb/core/kqp/executer_actor/kqp_scan_executer.cpp index e98361797ca1..a4b7363fe7a2 100644 --- a/ydb/core/kqp/executer_actor/kqp_scan_executer.cpp +++ b/ydb/core/kqp/executer_actor/kqp_scan_executer.cpp @@ -112,7 +112,7 @@ class TKqpScanExecuter : public TKqpExecuterBaseGetTypeRewrite()) { - hFunc(TEvDqCompute::TEvState, HandleComputeStats); + hFunc(TEvDqCompute::TEvState, HandleComputeState); hFunc(TEvDqCompute::TEvChannelData, HandleChannelData); // from CA hFunc(TEvKqpExecuter::TEvStreamDataAck, HandleStreamAck); hFunc(TEvKqp::TEvAbortExecution, HandleAbortExecution); @@ -274,34 +274,11 @@ class TKqpScanExecuter : public TKqpExecuterBaseRecord.MutableResponse(); - - response.SetStatus(status); - - if (Stats) { - ReportEventElapsedTime(); - - Stats->FinishTs = TInstant::Now(); - Stats->Finish(); - - if (Stats->CollectStatsByLongTasks || CollectFullStats(Request.StatsMode)) { - const auto& tx = Request.Transactions[0].Body; - auto planWithStats = AddExecStatsToTxPlan(tx->GetPlan(), response.GetResult().GetStats()); - response.MutableResult()->MutableStats()->AddTxPlansWithStats(planWithStats); - } - - if (Stats->CollectStatsByLongTasks) { - const auto& txPlansWithStats = response.GetResult().GetStats().GetTxPlansWithStats(); - if (!txPlansWithStats.empty()) { - LOG_N("Full stats: " << txPlansWithStats); - } - } - } - } - void Finalize() { - FillResponseStats(Ydb::StatusIds::SUCCESS); + YQL_ENSURE(!AlreadyReplied); + AlreadyReplied = true; + + ResponseEv->Record.MutableResponse()->SetStatus(Ydb::StatusIds::SUCCESS); LWTRACK(KqpScanExecuterFinalize, ResponseEv->Orbit, TxId, LastTaskId, LastComputeActorId, ResponseEv->ResultsSize()); @@ -309,8 +286,6 @@ class TKqpScanExecuter : public TKqpExecuterBase TaskParams; // Params for sources/sinks TVector ReadRanges; // Partitioning for sources THashMap SecureParams; diff --git a/ydb/core/kqp/host/kqp_explain_prepared.cpp b/ydb/core/kqp/host/kqp_explain_prepared.cpp index 91ee362da0f1..f76ce851140c 100644 --- a/ydb/core/kqp/host/kqp_explain_prepared.cpp +++ b/ydb/core/kqp/host/kqp_explain_prepared.cpp @@ -66,7 +66,7 @@ class TKqpExplainPreparedTransformer : public NYql::TGraphTransformerBase { PhyQuerySetTxPlans(query, TKqpPhysicalQuery(TransformCtx->ExplainTransformerInput), std::move(TxResults), ctx, Cluster, TransformCtx->Tables, TransformCtx->Config, TypeCtx, OptimizeCtx); - query.SetQueryAst(KqpExprToPrettyString(*TransformCtx->ExplainTransformerInput, ctx)); + query.SetQueryAst(KqpExprToPrettyString(*input, ctx)); TransformCtx->ExplainTransformerInput = nullptr; return TStatus::Ok; diff --git a/ydb/core/kqp/opt/kqp_opt_build_txs.cpp b/ydb/core/kqp/opt/kqp_opt_build_txs.cpp index f30bf59da597..e43f958d1c13 100644 --- a/ydb/core/kqp/opt/kqp_opt_build_txs.cpp +++ b/ydb/core/kqp/opt/kqp_opt_build_txs.cpp @@ -56,12 +56,12 @@ class TKqpBuildTxTransformer : public TSyncTransformerBase { TKqpBuildTxTransformer() : QueryType(EKikimrQueryType::Unspecified) , IsPrecompute(false) - , IsBlocks(false) {} + { + } - void Init(EKikimrQueryType queryType, bool isPrecompute, bool isBlocks) { + void Init(EKikimrQueryType queryType, bool isPrecompute) { QueryType = queryType; IsPrecompute = isPrecompute; - IsBlocks = isBlocks; } TStatus DoTransform(TExprNode::TPtr inputExpr, TExprNode::TPtr& outputExpr, TExprContext& ctx) final { @@ -267,23 +267,6 @@ class TKqpBuildTxTransformer : public TSyncTransformerBase { stages.emplace_back(collectStage); } - if (IsBlocks) { - const auto* tupleOutputType = resultStage.Ref().GetTypeAnn()->Cast(); - YQL_ENSURE(tupleOutputType->GetSize() == 1); - const auto* structExprType = tupleOutputType->GetItems()[0]->Cast()->GetItemType()->Cast(); - - TDqPhyStage beforeCollectStage = collectStage; - collectStage = ConvertResultStageFromBlocksToScalar(collectStage, structExprType, ctx); - if (collectStage.Ptr() != beforeCollectStage.Ptr()) { - for (auto& stage : stages) { - if (stage.Ptr() == beforeCollectStage.Ptr()) { - stage = collectStage; - break; - } - } - } - } - auto newResult = Build(ctx, results.Pos()) .Output() .Stage(collectStage) @@ -317,60 +300,6 @@ class TKqpBuildTxTransformer : public TSyncTransformerBase { .Done(); } - static TDqPhyStage ConvertResultStageFromBlocksToScalar(TDqPhyStage& stage, const TStructExprType* rowType, TExprContext& ctx) { - TCoLambda program(ctx.DeepCopyLambda(stage.Program().Ref())); - - TVector args; - args.reserve(rowType->GetSize()); - for (ui32 i = 0; i < rowType->GetSize(); ++i) { - args.push_back(TCoArgument(ctx.NewArgument(stage.Pos(), "arg"))); - } - - TVector structItems; - structItems.reserve(args.size()); - for (ui32 i = 0; i < args.size(); ++i) { - structItems.emplace_back( - Build(ctx, stage.Pos()) - .Name().Build(rowType->GetItems()[i]->GetName()) - .Value(args[i]) - .Done()); - } - - auto resultStream = Build(ctx, program.Body().Pos()) - .Input() - .Input() - .Input() - .Input(program.Body()) - .Build() - .Build() - .Lambda() - .Args(args) - .Body() - .Add(structItems) - .Build() - .Build() - .Build() - .Done(); - - - auto finalChannelSettings = TDqStageSettings::Parse(stage); - finalChannelSettings.WideChannels = false; - finalChannelSettings.OutputNarrowType = nullptr; - finalChannelSettings.BlockStatus = NYql::NDq::TDqStageSettings::EBlockStatus::None; - - auto output = Build(ctx, stage.Pos()) - .InitFrom(stage) - .Program() - .Args(program.Args()) - .Body(resultStream.Ptr()) - .Build() - .Settings(finalChannelSettings.BuildNode(ctx, stage.Pos())) - .Outputs(stage.Outputs()) - .Done().Ptr(); - - return TDqPhyStage(output); - } - static TMaybeNode ExtractParamsFromStage(const TDqPhyStage& stage, const TNodeOnNodeOwnedMap& stagesMap, TMap& bindingsMap, TExprContext& ctx) { @@ -530,7 +459,6 @@ class TKqpBuildTxTransformer : public TSyncTransformerBase { private: EKikimrQueryType QueryType; bool IsPrecompute; - bool IsBlocks; }; TVector PrecomputeInputs(const TDqStage& stage) { @@ -563,7 +491,6 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { TTypeAnnotationContext& typesCtx, TKikimrConfiguration::TPtr& config) : KqpCtx(kqpCtx) , BuildCtx(buildCtx) - , TypesCtx(typesCtx) , TypeAnnTransformer(std::move(typeAnnTransformer)) { BuildTxTransformer = new TKqpBuildTxTransformer(); @@ -578,6 +505,7 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { .Add(*TypeAnnTransformer, "TypeAnnotation") .AddPostTypeAnnotation(/* forSubgraph */ true) .Add(CreateKqpBuildPhyStagesTransformer(enableSpillingGenericQuery, typesCtx, config->BlockChannelsMode), "BuildPhysicalStages") + // TODO(ilezhankin): "BuildWideBlockChannels" transformer is required only for BLOCK_CHANNELS_FORCE mode. .Add(CreateKqpBuildWideBlockChannelsTransformer(typesCtx, config->BlockChannelsMode), "BuildWideBlockChannels") .Add(*BuildTxTransformer, "BuildPhysicalTx") .Add(CreateKqpTxPeepholeTransformer( @@ -610,7 +538,7 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { } if (!query.Results().Empty()) { - auto tx = BuildTx(query.Results().Ptr(), ctx, false, TypesCtx.BlockEngineMode == EBlockEngineMode::Force); + auto tx = BuildTx(query.Results().Ptr(), ctx, false); if (!tx) { return TStatus::Error; } @@ -632,7 +560,7 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { } if (!query.Effects().Empty()) { - auto tx = BuildTx(query.Effects().Ptr(), ctx, /* isPrecompute */ false, TypesCtx.BlockEngineMode == EBlockEngineMode::Force); + auto tx = BuildTx(query.Effects().Ptr(), ctx, /* isPrecompute */ false); if (!tx) { return TStatus::Error; } @@ -824,7 +752,7 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { .Add(phaseResults) .Done(); - auto tx = BuildTx(phaseResultsNode.Ptr(), ctx, /* isPrecompute */ true, TypesCtx.BlockEngineMode == EBlockEngineMode::Force); + auto tx = BuildTx(phaseResultsNode.Ptr(), ctx, /* isPrecompute */ true); if (!tx.IsValid()) { return TStatus::Error; @@ -850,14 +778,14 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { return TStatus(TStatus::Repeat, true); } - TMaybeNode BuildTx(const TExprNode::TPtr& result, TExprContext& ctx, bool isPrecompute, bool isBlocks) { + TMaybeNode BuildTx(const TExprNode::TPtr& result, TExprContext& ctx, bool isPrecompute) { YQL_CLOG(TRACE, ProviderKqp) << "[BuildTx] " << KqpExprToPrettyString(*result, ctx) - << ", isPrecompute: " << isPrecompute << ", isBlocks: " << isBlocks; + << ", isPrecompute: " << isPrecompute; auto& transformer = KqpCtx->IsScanQuery() ? *ScanTxTransformer : *DataTxTransformer; transformer.Rewind(); - BuildTxTransformer->Init(KqpCtx->QueryCtx->Type, isPrecompute, isBlocks); + BuildTxTransformer->Init(KqpCtx->QueryCtx->Type, isPrecompute); auto expr = result; while (true) { @@ -875,7 +803,6 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { private: TIntrusivePtr KqpCtx; TIntrusivePtr BuildCtx; - TTypeAnnotationContext& TypesCtx; TAutoPtr TypeAnnTransformer; TAutoPtr BuildTxTransformer; TAutoPtr DataTxTransformer; diff --git a/ydb/core/kqp/opt/kqp_query_plan.cpp b/ydb/core/kqp/opt/kqp_query_plan.cpp index 256df95a9aaa..038b0edda6ad 100644 --- a/ydb/core/kqp/opt/kqp_query_plan.cpp +++ b/ydb/core/kqp/opt/kqp_query_plan.cpp @@ -94,7 +94,7 @@ struct TSerializerCtx { const TIntrusivePtr tablesData, const TKikimrConfiguration::TPtr config, ui32 txCount, TVector> pureTxResults, - TTypeAnnotationContext& typeCtx, + TTypeAnnotationContext& typeCtx, TIntrusivePtr optCtx) : ExprCtx(exprCtx) , Cluster(cluster) @@ -348,7 +348,7 @@ class TxPlanSerializer { writer.WriteKey("Inputs"); writer.BeginList(); - + for (const auto& input : op.Inputs) { if (std::holds_alternative(input)) { @@ -461,6 +461,19 @@ class TxPlanSerializer { } void FillConnectionPlanNode(const TDqConnection& connection, TQueryPlanNode& planNode) { + TDqStageSettings settings = TDqStageSettings::Parse(connection.Output().Stage()); + auto GetNarrowColumnName = [&](const TString& wideColumnName) { + ui32 idx; + if (!TryFromString(wideColumnName, idx)) { + return wideColumnName; + } + + YQL_ENSURE(idx < settings.OutputNarrowType->GetSize(), + "Failed to lookup column name for index " << idx << " in type " << settings.OutputNarrowType->ToString()); + + return TString(settings.OutputNarrowType->GetItems()[idx]->GetName()); + }; + planNode.Type = EPlanNodeType::Connection; if (connection.Maybe()) { @@ -473,15 +486,23 @@ class TxPlanSerializer { planNode.TypeName = "HashShuffle"; auto& keyColumns = planNode.NodeInfo["KeyColumns"]; for (const auto& column : hashShuffle.Cast().KeyColumns()) { - keyColumns.AppendValue(TString(column.Value())); + if (settings.WideChannels) { + keyColumns.AppendValue(GetNarrowColumnName(TString(column.Value()))); + } else { + keyColumns.AppendValue(TString(column.Value())); + } } } else if (auto merge = connection.Maybe()) { planNode.TypeName = "Merge"; auto& sortColumns = planNode.NodeInfo["SortColumns"]; for (const auto& sortColumn : merge.Cast().SortColumns()) { TStringBuilder sortColumnDesc; - sortColumnDesc << sortColumn.Column().Value() << " (" - << sortColumn.SortDirection().Value() << ")"; + if (settings.WideChannels) { + sortColumnDesc << GetNarrowColumnName(TString(sortColumn.Column().Value())); + } else { + sortColumnDesc << sortColumn.Column().Value(); + } + sortColumnDesc << " (" << sortColumn.SortDirection().Value() << ")"; sortColumns.AppendValue(sortColumnDesc); } @@ -1444,7 +1465,7 @@ class TxPlanSerializer { TOperator op; op.Properties["Name"] = name; - + return AddOperator(planNode, name, std::move(op)); } @@ -1796,8 +1817,8 @@ class TxPlanSerializer { template void AddReadTableSettings( - TOperator& op, - const TReadTableSettings& readTableSettings, + TOperator& op, + const TReadTableSettings& readTableSettings, TTableRead& readInfo ) { auto settings = NYql::TKqpReadTableSettings::Parse(readTableSettings); @@ -1819,9 +1840,9 @@ class TxPlanSerializer { } if (settings.SequentialInFlight) { - op.Properties["Scan"] = "Sequential"; + op.Properties["Scan"] = "Sequential"; } else { - op.Properties["Scan"] = "Parallel"; + op.Properties["Scan"] = "Parallel"; } } @@ -2021,8 +2042,8 @@ TVector RemoveRedundantNodes(NJson::TJsonValue& plan, const T return {plan}; } -NJson::TJsonValue ReconstructQueryPlanRec(const NJson::TJsonValue& plan, - int operatorIndex, +NJson::TJsonValue ReconstructQueryPlanRec(const NJson::TJsonValue& plan, + int operatorIndex, const THashMap& planIndex, const THashMap& precomputes, int& nodeCounter) { @@ -2080,9 +2101,9 @@ NJson::TJsonValue ReconstructQueryPlanRec(const NJson::TJsonValue& plan, newPlans.AppendValue(ReconstructQueryPlanRec( plan.GetMapSafe().at("Plans").GetArraySafe()[0], - 0, - planIndex, - precomputes, + 0, + planIndex, + precomputes, nodeCounter)); newPlans.AppendValue(lookupPlan); @@ -2113,14 +2134,14 @@ NJson::TJsonValue ReconstructQueryPlanRec(const NJson::TJsonValue& plan, NJson::TJsonValue newOps; NJson::TJsonValue op; - op["Name"] = "TableLookup"; + op["Name"] = "TableLookup"; op["Columns"] = plan.GetMapSafe().at("Columns"); op["LookupKeyColumns"] = plan.GetMapSafe().at("LookupKeyColumns"); op["Table"] = plan.GetMapSafe().at("Table"); if (plan.GetMapSafe().contains("E-Cost")) { op["E-Cost"] = plan.GetMapSafe().at("E-Cost"); - } + } if (plan.GetMapSafe().contains("E-Rows")) { op["E-Rows"] = plan.GetMapSafe().at("E-Rows"); } @@ -2195,8 +2216,8 @@ NJson::TJsonValue ReconstructQueryPlanRec(const NJson::TJsonValue& plan, op.GetMapSafe().erase("Inputs"); } - if (op.GetMapSafe().contains("Input") - || op.GetMapSafe().contains("ToFlow") + if (op.GetMapSafe().contains("Input") + || op.GetMapSafe().contains("ToFlow") || op.GetMapSafe().contains("Member") || op.GetMapSafe().contains("AssumeSorted") || op.GetMapSafe().contains("Iterator")) { @@ -2269,7 +2290,7 @@ double ComputeCpuTimes(NJson::TJsonValue& plan) { } void ComputeTotalRows(NJson::TJsonValue& plan) { - + if (plan.GetMapSafe().contains("Plans")) { for (auto& p : plan.GetMapSafe().at("Plans").GetArraySafe()) { ComputeTotalRows(p); @@ -2321,7 +2342,7 @@ NJson::TJsonValue SimplifyQueryPlan(NJson::TJsonValue& plan) { "ToFlow", "Member", "AssumeSorted" - }; + }; THashMap planIndex; THashMap precomputes; @@ -2357,7 +2378,7 @@ TString AddSimplifiedPlan(const TString& planText, TIntrusivePtrJoinsCount; optimizerStats["EquiJoinsCount"] = optCtx->EquiJoinsCount; simplifiedPlan["OptimizerStats"] = optimizerStats; - } + } planJson["SimplifiedPlan"] = simplifiedPlan; return planJson.GetStringRobust(); diff --git a/ydb/core/kqp/opt/peephole/kqp_opt_peephole.cpp b/ydb/core/kqp/opt/peephole/kqp_opt_peephole.cpp index 808af2b3262e..1471d3f631df 100644 --- a/ydb/core/kqp/opt/peephole/kqp_opt_peephole.cpp +++ b/ydb/core/kqp/opt/peephole/kqp_opt_peephole.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -12,6 +13,7 @@ #include #include #include +#include #include #include @@ -153,7 +155,7 @@ class TKqpPeepholeTransformer : public TOptimizeTransformerBase { struct TKqpPeepholePipelineConfigurator : IPipelineConfigurator { TKqpPeepholePipelineConfigurator( - TKikimrConfiguration::TPtr config, + TKikimrConfiguration::TPtr config, TSet disabledOpts ) : Config(config) @@ -213,6 +215,96 @@ struct TKqpPeepholePipelineFinalConfigurator : IPipelineConfigurator { const TKikimrConfiguration::TPtr Config; }; +// Sort stages in topological order by their inputs, so that we optimize the ones without inputs first. +TVector TopSortStages(const TDqPhyStageList& stages) { + TVector topSortedStages; + topSortedStages.reserve(stages.Size()); + std::function topSort; + THashSet visitedStages; + + // Assume there is no cycles. + topSort = [&](const TDqPhyStage& stage) { + if (visitedStages.contains(stage.Ref().UniqueId())) { + return; + } + + for (const auto& input : stage.Inputs()) { + if (auto connection = input.Maybe()) { + // NOTE: somehow `Output()` is actually an input. + if (auto phyStage = connection.Cast().Output().Stage().Maybe()) { + topSort(phyStage.Cast()); + } + } + } + + visitedStages.insert(stage.Ref().UniqueId()); + topSortedStages.push_back(stage); + }; + + for (const auto& stage : stages) { + topSort(stage); + } + + return topSortedStages; +} + +// TODO: copy-paste from https://github.com/ydb-platform/ydb/blob/122f053354c5df4fc559bf06fe0302f92d813032/ydb/library/yql/dq/opt/dq_opt_build.cpp#L444 +bool IsCompatibleWithBlocks(TPositionHandle pos, const TStructExprType& type, TExprContext& ctx, TTypeAnnotationContext& typesCtx) { + TVector types; + for (auto& item : type.GetItems()) { + types.emplace_back(item->GetItemType()); + } + + auto resolveStatus = typesCtx.ArrowResolver->AreTypesSupported(ctx.GetPosition(pos), types, ctx); + YQL_ENSURE(resolveStatus != IArrowResolver::ERROR); + return resolveStatus == IArrowResolver::OK; +} + +// TODO: composite copy-paste from https://github.com/ydb-platform/ydb/blob/122f053354c5df4fc559bf06fe0302f92d813032/ydb/library/yql/dq/opt/dq_opt_build.cpp#L388 +bool CanPropagateWideBlockThroughChannel( + const TDqOutput& output, + const THashMap& programs, + const TDqStageSettings& stageSettings, + TExprContext& ctx, + TTypeAnnotationContext& typesCtx) +{ + const auto& program = programs.at(output.Stage().Ref().UniqueId()); + + ui32 index = FromString(output.Index().Value()); + if (index != 0) { + // stage has multiple outputs + return false; + } + + if (!stageSettings.WideChannels) { + return false; + } + + YQL_ENSURE(stageSettings.OutputNarrowType); + + if (!IsCompatibleWithBlocks(program.Pos(), *stageSettings.OutputNarrowType, ctx, typesCtx)) { + return false; + } + + // Ensure that stage has blocks on top level (i.e. FromFlow(WideFromBlocks(...))) + if (!program.Lambda().Body().Maybe() || + !program.Lambda().Body().Cast().Input().Maybe()) + { + return false; + } + + auto typeAnnotation = program.Lambda().Ref().GetTypeAnn(); + + YQL_ENSURE(typeAnnotation, "Program for stage " << output.Stage().Ref().UniqueId() << " doesn't have type annotation"); + + if (IsWideBlockType(*typeAnnotation->Cast()->GetItemType())) { + // output is already wide block + return false; + } + + return true; +} + TStatus PeepHoleOptimize(const TExprBase& program, TExprNode::TPtr& newProgram, TExprContext& ctx, IGraphTransformer& typeAnnTransformer, TTypeAnnotationContext& typesCtx, TKikimrConfiguration::TPtr config, bool allowNonDeterministicFunctions, bool withFinalStageRules, TSet disabledOpts) @@ -244,30 +336,112 @@ TMaybeNode PeepholeOptimize(const TKqpPhysicalTx& tx, TExprConte IGraphTransformer& typeAnnTransformer, TTypeAnnotationContext& typesCtx, THashSet& optimizedStages, TKikimrConfiguration::TPtr config, bool withFinalStageRules, TSet disabledOpts) { - TVector stages; - stages.reserve(tx.Stages().Size()); - TNodeOnNodeOwnedMap stagesMap; - TVector bindings(tx.ParamBindings().begin(), tx.ParamBindings().end()); + THashMap programs; THashMap nonDetParamBindings; - for (const auto& stage : tx.Stages()) { + const auto topSortedStages = TopSortStages(tx.Stages()); + for (const auto& stage : topSortedStages) { YQL_ENSURE(!optimizedStages.contains(stage.Ref().UniqueId())); + TCoLambda lambda = stage.Program(); + TVector newArgs; + newArgs.reserve(stage.Inputs().Size()); + + // Propagate "WideFromBlock" through connections. + // TODO(ilezhankin): this peephole optimization should be implemented instead as + // the original whole-graph transformer |CreateDqBuildWideBlockChannelsTransformer|. + if (config->BlockChannelsMode == NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_AUTO) { + TNodeOnNodeOwnedMap argsMap; + + YQL_ENSURE(stage.Inputs().Size() == stage.Program().Args().Size()); + + for (size_t i = 0; i < stage.Inputs().Size(); ++i) { + auto oldArg = stage.Program().Args().Arg(i); + auto newArg = TCoArgument(ctx.NewArgument(oldArg.Pos(), oldArg.Name())); + newArg.MutableRef().SetTypeAnn(oldArg.Ref().GetTypeAnn()); + newArgs.emplace_back(newArg); + + if (auto connection = stage.Inputs().Item(i).Maybe(); connection && + CanPropagateWideBlockThroughChannel(connection.Cast().Output(), programs, TDqStageSettings::Parse(stage), ctx, typesCtx)) + { + TExprNode::TPtr newArgNode = ctx.Builder(oldArg.Pos()) + .Callable("FromFlow") + .Callable(0, "WideFromBlocks") + .Callable(0, "ToFlow") + .Add(0, newArg.Ptr()) + .Seal() + .Seal() + .Seal() + .Build(); + argsMap.emplace(oldArg.Raw(), newArgNode); + + auto stageUid = connection.Cast().Output().Stage().Ref().UniqueId(); + + // Update input program with: FromFlow(WideFromBlocks($1)) → FromFlow($1) + if (const auto& inputProgram = programs.at(stageUid); inputProgram.Lambda().Body().Maybe() && + inputProgram.Lambda().Body().Cast().Input().Maybe()) + { + auto newBody = Build(ctx, inputProgram.Lambda().Body().Cast().Pos()) + .Input(inputProgram.Lambda().Body().Cast().Input().Cast().Input()) + .Done(); + + auto newInputProgram = Build(ctx, inputProgram.Pos()) + .Lambda() + .Args(inputProgram.Lambda().Args()) + .Body(newBody) + .Build() + .ArgsType(inputProgram.ArgsType()) + .Done(); + + // Run the peephole optimization on new program again to update type annotations. + // TODO(ilezhankin): refactor to run only the update of type annotations - not the whole optimization. + bool allowNonDeterministicFunctions = !newInputProgram.Lambda().Body().Maybe(); + TExprNode::TPtr newInputProgramNode; + + auto status = PeepHoleOptimize(newInputProgram, newInputProgramNode, ctx, typeAnnTransformer, typesCtx, config, + allowNonDeterministicFunctions, withFinalStageRules, disabledOpts); + if (status != TStatus::Ok) { + ctx.AddError(TIssue(ctx.GetPosition(stage.Pos()), "Peephole optimization failed for KQP transaction")); + return {}; + } + + programs.at(stageUid) = TKqpProgram(newInputProgramNode); + } + + // Update the type annotation for an argument with return value of the input program. + newArg.MutableRef().SetTypeAnn(programs.at(stageUid).Lambda().Body().Ref().GetTypeAnn()); + } else { + argsMap.emplace(oldArg.Raw(), newArg.Ptr()); + } + } + + // Rebuild lambda with new arguments. + lambda = Build(ctx, lambda.Pos()) + .Args(newArgs) + .Body(ctx.ReplaceNodes(stage.Program().Body().Ptr(), argsMap)) + .Done(); + } else { + for (size_t i = 0; i < stage.Inputs().Size(); ++i) { + auto oldArg = stage.Program().Args().Arg(i); + auto newArg = TCoArgument(ctx.NewArgument(oldArg.Pos(), oldArg.Name())); + newArg.MutableRef().SetTypeAnn(oldArg.Ref().GetTypeAnn()); + newArgs.emplace_back(newArg); + } + } + TVector argTypes; - for (const auto& arg : stage.Program().Args()) { + for (const auto& arg : newArgs) { YQL_ENSURE(arg.Ref().GetTypeAnn()); argTypes.push_back(arg.Ref().GetTypeAnn()); } // TODO: get rid of TKqpProgram-callable (YQL-10078) - TNodeOnNodeOwnedMap tmp; auto program = Build(ctx, stage.Pos()) - //.Lambda(ctx.DeepCopy(stage.Program().Ref(), ctx, tmp, true /* internStrings */, false /* copyTypes */)) - .Lambda(stage.Program()) + .Lambda(lambda) .ArgsType(ExpandType(stage.Pos(), *ctx.MakeType(argTypes), ctx)) .Done(); - bool allowNonDeterministicFunctions = !stage.Program().Body().Maybe(); + const bool allowNonDeterministicFunctions = !program.Lambda().Body().Maybe(); TExprNode::TPtr newProgram; auto status = PeepHoleOptimize(program, newProgram, ctx, typeAnnTransformer, typesCtx, config, @@ -287,26 +461,34 @@ TMaybeNode PeepholeOptimize(const TKqpPhysicalTx& tx, TExprConte } } - auto newStage = Build(ctx, stage.Pos()) - .Inputs(ctx.ReplaceNodes(stage.Inputs().Ptr(), stagesMap)) - .Program(ctx.DeepCopyLambda(TKqpProgram(newProgram).Lambda().Ref())) - .Settings(stage.Settings()) - .Outputs(stage.Outputs()) - .Done(); - - stages.emplace_back(newStage); - stagesMap.emplace(stage.Raw(), newStage.Ptr()); - optimizedStages.emplace(stage.Ref().UniqueId()); + YQL_ENSURE(programs.emplace(stage.Ref().UniqueId(), TKqpProgram(newProgram)).second); } + TVector bindings(tx.ParamBindings().begin(), tx.ParamBindings().end()); + for (const auto& [_, binding] : nonDetParamBindings) { bindings.emplace_back(std::move(binding)); } + TVector newStages; + TNodeOnNodeOwnedMap stagesMap; + + // Rebuild stages only after all new programs are ready. + for (const auto& stage : topSortedStages) { + auto newStage = Build(ctx, stage.Pos()) + .InitFrom(stage) + .Inputs(ctx.ReplaceNodes(stage.Inputs().Ptr(), stagesMap)) + .Program(ctx.DeepCopyLambda(programs.at(stage.Ref().UniqueId()).Lambda().Ref())) + .Done(); + + newStages.emplace_back(newStage); + stagesMap.emplace(stage.Raw(), newStage.Ptr()); + } + return Build(ctx, tx.Pos()) .Stages() - .Add(stages) + .Add(newStages) .Build() .Results(ctx.ReplaceNodes(tx.Results().Ptr(), stagesMap)) .ParamBindings().Add(bindings).Build() @@ -318,7 +500,7 @@ class TKqpTxPeepholeTransformer : public TSyncTransformerBase { public: TKqpTxPeepholeTransformer( IGraphTransformer* typeAnnTransformer, - TTypeAnnotationContext& typesCtx, + TTypeAnnotationContext& typesCtx, TKikimrConfiguration::TPtr config, bool withFinalStageRules, TSet disabledOpts @@ -444,8 +626,8 @@ class TKqpTxsPeepholeTransformer : public TSyncTransformerBase { TAutoPtr CreateKqpTxPeepholeTransformer( NYql::IGraphTransformer* typeAnnTransformer, - TTypeAnnotationContext& typesCtx, - const TKikimrConfiguration::TPtr& config, + TTypeAnnotationContext& typesCtx, + const TKikimrConfiguration::TPtr& config, bool withFinalStageRules, TSet disabledOpts ) @@ -455,7 +637,7 @@ TAutoPtr CreateKqpTxPeepholeTransformer( TAutoPtr CreateKqpTxsPeepholeTransformer( TAutoPtr typeAnnTransformer, - TTypeAnnotationContext& typesCtx, + TTypeAnnotationContext& typesCtx, const TKikimrConfiguration::TPtr& config ) { diff --git a/ydb/core/kqp/query_data/kqp_prepared_query.cpp b/ydb/core/kqp/query_data/kqp_prepared_query.cpp index 9817c4b3ce37..a46ec53d9068 100644 --- a/ydb/core/kqp/query_data/kqp_prepared_query.cpp +++ b/ydb/core/kqp/query_data/kqp_prepared_query.cpp @@ -154,7 +154,7 @@ const NKikimr::NKqp::TStagePredictor& TKqpPhyTxHolder::GetCalculationPredictor(c } TPreparedQueryHolder::TPreparedQueryHolder(NKikimrKqp::TPreparedQuery* proto, - const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry) + const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry, bool noFillTables) : Proto(proto) , Alloc(nullptr) , TableConstInfoById(MakeIntrusive()) @@ -164,6 +164,11 @@ TPreparedQueryHolder::TPreparedQueryHolder(NKikimrKqp::TPreparedQuery* proto, Alloc = std::make_shared(functionRegistry); } + // In case of some compilation failures filling tables may produce new problems which may replace original error messages. + if (noFillTables) { + return; + } + THashSet tablesSet; const auto& phyQuery = Proto->GetPhysicalQuery(); Transactions.reserve(phyQuery.TransactionsSize()); diff --git a/ydb/core/kqp/query_data/kqp_prepared_query.h b/ydb/core/kqp/query_data/kqp_prepared_query.h index a92a79063137..b7cd6ab1b167 100644 --- a/ydb/core/kqp/query_data/kqp_prepared_query.h +++ b/ydb/core/kqp/query_data/kqp_prepared_query.h @@ -143,7 +143,10 @@ class TPreparedQueryHolder { public: - TPreparedQueryHolder(NKikimrKqp::TPreparedQuery* proto, const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry); + TPreparedQueryHolder( + NKikimrKqp::TPreparedQuery* proto, + const NKikimr::NMiniKQL::IFunctionRegistry* functionRegistry, + bool noFillTables = false); ~TPreparedQueryHolder(); using TConstPtr = std::shared_ptr; diff --git a/ydb/core/kqp/rm_service/kqp_rm_service.h b/ydb/core/kqp/rm_service/kqp_rm_service.h index c4cf4ba60f91..213387b2a702 100644 --- a/ydb/core/kqp/rm_service/kqp_rm_service.h +++ b/ydb/core/kqp/rm_service/kqp_rm_service.h @@ -114,10 +114,10 @@ class TTxState : public TAtomicRefCount { {} TString ToString() const { - return TStringBuilder() << "TxResourcesInfo{ " + return TStringBuilder() << "TxResourcesInfo { " << "TxId: " << TxId << ", memory initially granted resources: " << TxExternalDataQueryMemory.load() - << ", extra allocations " << TxScanQueryMemory.load() + << ", tx total allocations " << TxScanQueryMemory.load() << ", execution units: " << TxExecutionUnits.load() << ", started at: " << CreatedAt << " }"; diff --git a/ydb/core/kqp/session_actor/kqp_session_actor.cpp b/ydb/core/kqp/session_actor/kqp_session_actor.cpp index fc857ea6fe59..714a4dc67ffd 100644 --- a/ydb/core/kqp/session_actor/kqp_session_actor.cpp +++ b/ydb/core/kqp/session_actor/kqp_session_actor.cpp @@ -352,8 +352,9 @@ class TKqpSessionActor : public TActorBootstrapped { auto abortEv = TEvKqp::TEvAbortExecution::Aborted("Client lost"); // any status code can be here Send(ExecuterId, abortEv.Release()); + } else { + Cleanup(); } - Cleanup(); } void Handle(TEvKqp::TEvQueryRequest::TPtr& ev) { @@ -1427,6 +1428,12 @@ class TKqpSessionActor : public TActorBootstrapped { ExecuterId = TActorId{}; + auto& executerResults = *response->MutableResult(); + if (executerResults.HasStats()) { + QueryState->QueryStats.Executions.emplace_back(); + QueryState->QueryStats.Executions.back().Swap(executerResults.MutableStats()); + } + if (response->GetStatus() != Ydb::StatusIds::SUCCESS) { const auto executionType = ev->ExecutionType; @@ -1482,16 +1489,10 @@ class TKqpSessionActor : public TActorBootstrapped { QueryState->TxCtx->Locks.LockHandle = std::move(ev->LockHandle); } - auto& executerResults = *response->MutableResult(); if (!MergeLocksWithTxResult(executerResults)) { return; } - if (executerResults.HasStats()) { - QueryState->QueryStats.Executions.emplace_back(); - QueryState->QueryStats.Executions.back().Swap(executerResults.MutableStats()); - } - if (!response->GetIssues().empty()){ NYql::IssuesFromMessage(response->GetIssues(), QueryState->Issues); } @@ -1585,6 +1586,10 @@ class TKqpSessionActor : public TActorBootstrapped { void FillStats(NKikimrKqp::TEvQueryResponse* record) { YQL_ENSURE(QueryState); + // workaround to ensure that request was not transfered to worker. + if (WorkerId || !QueryState->RequestEv) { + return; + } FillSystemViewQueryStats(record); @@ -2224,6 +2229,8 @@ class TKqpSessionActor : public TActorBootstrapped { } } + FillStats(&QueryResponse->Record.GetRef()); + if (issues) { for (auto& i : *issues) { response->AddQueryIssues()->Swap(&i); diff --git a/ydb/core/kqp/ut/olap/helpers/aggregation.h b/ydb/core/kqp/ut/olap/helpers/aggregation.h index e83dcb32fd87..de9ab275fa79 100644 --- a/ydb/core/kqp/ut/olap/helpers/aggregation.h +++ b/ydb/core/kqp/ut/olap/helpers/aggregation.h @@ -81,12 +81,7 @@ class TAggregationTestCase { bool UseLlvm = true; public: void FillExpectedAggregationGroupByPlanOptions() { -#if SSA_RUNTIME_VERSION >= 2U - // AddExpectedPlanOptions("TKqpOlapAgg"); AddExpectedPlanOptions("WideCombiner"); -#else - AddExpectedPlanOptions("CombineCore"); -#endif } TString GetFixedQuery() const { TStringBuilder queryFixed; @@ -177,7 +172,8 @@ void CheckPlanForAggregatePushdown( const TString& query, TClient& client, const std::vector& expectedPlanNodes, - const std::string& readNodeType) { + const std::string& readNodeType) +{ auto res = StreamExplainQuery(query, client); UNIT_ASSERT_C(res.IsSuccess(), res.GetIssues().ToString()); @@ -242,4 +238,4 @@ void WriteTestDataForTableWithNulls(TKikimrRunner& kikimr, TString testTable); void TestTableWithNulls(const std::vector& cases, const bool genericQuery = false); -} \ No newline at end of file +} diff --git a/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp b/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp index 862aa0428bc6..f08611afd340 100644 --- a/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp +++ b/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp @@ -2515,7 +2515,7 @@ Y_UNIT_TEST_SUITE(KqpOlap) { UNIT_ASSERT_C(result.GetStatus() == NYdb::EStatus::SUCCESS, result.GetIssues().ToString()); auto client = kikimr.GetQueryClient(); - { + { auto prepareResult = client.ExecuteQuery(R"( REPLACE INTO `/Root/ColumnShard` (a, b, c) VALUES (1u, 1, 5), @@ -2545,7 +2545,9 @@ Y_UNIT_TEST_SUITE(KqpOlap) { UNIT_ASSERT_C(plan.QueryStats->Getquery_ast().Contains("return (FromFlow (NarrowMap (WideFromBlocks"), plan.QueryStats->Getquery_ast()); break; case NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_AUTO: - UNIT_ASSERT_C(plan.QueryStats->Getquery_ast().Contains("return (FromFlow (ExpandMap (NarrowMap (WideFromBlocks"), plan.QueryStats->Getquery_ast()); + UNIT_ASSERT_C(plan.QueryStats->Getquery_ast().Contains("(FromFlow (WideFromBlocks"), plan.QueryStats->Getquery_ast()); + UNIT_ASSERT_C(!plan.QueryStats->Getquery_ast().Contains("WideToBlocks"), plan.QueryStats->Getquery_ast()); + UNIT_ASSERT_EQUAL_C(plan.QueryStats->Getquery_ast().find("WideFromBlocks"), plan.QueryStats->Getquery_ast().rfind("WideFromBlocks"), plan.QueryStats->Getquery_ast()); break; case NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_FORCE: UNIT_ASSERT_C(plan.QueryStats->Getquery_ast().Contains("return (FromFlow (WideMap"), plan.QueryStats->Getquery_ast()); @@ -2553,7 +2555,7 @@ Y_UNIT_TEST_SUITE(KqpOlap) { } } - { + { auto it = client.StreamExecuteQuery(R"( SELECT b, COUNT(*), SUM(a) @@ -2566,6 +2568,68 @@ Y_UNIT_TEST_SUITE(KqpOlap) { TString output = StreamResultToYson(it); CompareYson(output, R"([[[1];1u;1u];[[2];2u;6u]])"); } + + { + NYdb::NQuery::TExecuteQuerySettings scanSettings; + scanSettings.ExecMode(NYdb::NQuery::EExecMode::Explain); + auto it = client.StreamExecuteQuery(R"( + PRAGMA ydb.CostBasedOptimizationLevel='0'; + + $select1 = ( + SELECT b AS a1, COUNT(*) AS b1, SUM(a) AS c1 + FROM `/Root/ColumnShard` + WHERE c = 5 + GROUP BY b + ); + + $select2 = ( + SELECT (b1 + 1ul) AS a2, COUNT(*) AS b2, SUM(a1) AS c2 + FROM $select1 + WHERE c1 = 5 + GROUP BY b1 + ); + + $select3 = ( + SELECT b1 AS a3, COUNT(*) AS b3, MAX(a1) AS c3 + FROM $select1 + WHERE b1 = 6 + GROUP BY b1 + ); + + SELECT a2, b2 + FROM $select2 AS table2 + JOIN $select3 AS table3 + ON table2.a2 = table3.a3 + ORDER BY b2 + LIMIT 10 + ; + + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), scanSettings).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); + auto plan = CollectStreamResult(it); + + // auto CountSubstr = [](const TString& str, const TString& sub) -> ui64 { + // ui64 count = 0; + // for (auto pos = str.find(sub); pos != TString::npos; pos = str.find(sub, pos + sub.size())) { + // ++count; + // } + // return count; + // }; + + switch (blockChannelsMode) { + case NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_SCALAR: + // TODO: implement checks? + break; + case NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_AUTO: + // TODO: test fails because of some stages don't get wide channels. + // UNIT_ASSERT_EQUAL_C(CountSubstr(plan.QueryStats->Getquery_ast(), "WideFromBlocks"), 2, plan.QueryStats->Getquery_ast()); + // UNIT_ASSERT_C(!plan.QueryStats->Getquery_ast().Contains("WideToBlocks"), plan.QueryStats->Getquery_ast()); + break; + case NKikimrConfig::TTableServiceConfig_EBlockChannelsMode_BLOCK_CHANNELS_FORCE: + // TODO: implement checks? + break; + } + } } Y_UNIT_TEST(BlockChannelScalar) { diff --git a/ydb/core/kqp/ut/query/kqp_limits_ut.cpp b/ydb/core/kqp/ut/query/kqp_limits_ut.cpp index 4a65ed26cf46..5da239d6306c 100644 --- a/ydb/core/kqp/ut/query/kqp_limits_ut.cpp +++ b/ydb/core/kqp/ut/query/kqp_limits_ut.cpp @@ -1,7 +1,10 @@ #include +#include #include +#include #include +#include #include @@ -11,6 +14,36 @@ namespace NKqp { using namespace NYdb; using namespace NYdb::NTable; +NKikimrResourceBroker::TResourceBrokerConfig MakeResourceBrokerTestConfig(ui32 multiplier = 1) { + NKikimrResourceBroker::TResourceBrokerConfig config; + + auto queue = config.AddQueues(); + queue->SetName("queue_default"); + queue->SetWeight(5); + queue->MutableLimit()->AddResource(4); + + queue = config.AddQueues(); + queue->SetName("queue_kqp_resource_manager"); + queue->SetWeight(20); + queue->MutableLimit()->AddResource(4); + queue->MutableLimit()->AddResource(33554453 * multiplier); + + auto task = config.AddTasks(); + task->SetName("unknown"); + task->SetQueueName("queue_default"); + task->SetDefaultDuration(TDuration::Seconds(5).GetValue()); + + task = config.AddTasks(); + task->SetName(NLocalDb::KqpResourceManagerTaskName); + task->SetQueueName("queue_kqp_resource_manager"); + task->SetDefaultDuration(TDuration::Seconds(5).GetValue()); + + config.MutableResourceLimit()->AddResource(10); + config.MutableResourceLimit()->AddResource(100'000); + + return config; +} + namespace { bool IsRetryable(const EStatus& status) { return status == EStatus::OVERLOADED; @@ -147,6 +180,37 @@ 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) { + auto app = NKikimrConfig::TAppConfig(); + app.MutableTableServiceConfig()->MutableResourceManager()->SetMkqlLightProgramMemoryLimit(10); + app.MutableTableServiceConfig()->MutableResourceManager()->SetQueryMemoryLimit(2000); + + app.MutableResourceBrokerConfig()->CopyFrom(MakeResourceBrokerTestConfig(4)); + + TKikimrRunner kikimr(app); + CreateLargeTable(kikimr, 0, 0, 0); + + kikimr.GetTestServer().GetRuntime()->SetLogPriority(NKikimrServices::KQP_SLOW_LOG, NActors::NLog::PRI_ERROR); + + auto db = kikimr.GetQueryClient(); + NYdb::NQuery::TExecuteQuerySettings querySettings; + querySettings.StatsMode(NYdb::NQuery::EStatsMode::Full); + + auto result = db.ExecuteQuery(Q1_(R"( + SELECT * FROM `/Root/LargeTable`; + )"), NQuery::TTxControl::BeginTx().CommitTx(), querySettings).ExtractValueSync(); + result.GetIssues().PrintTo(Cerr); + + auto stats = result.GetStats(); + + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::OVERLOADED); + UNIT_ASSERT_C(result.GetIssues().ToString().Contains("Mkql memory limit exceeded"), result.GetIssues().ToString()); + UNIT_ASSERT(stats.Defined()); + + Cerr << stats->ToString(true) << Endl; } Y_UNIT_TEST(DatashardProgramSize) { @@ -765,6 +829,120 @@ Y_UNIT_TEST_SUITE(KqpLimits) { UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::TIMEOUT); } + /* Scenario: + - prepare and run query + - observe first EvState event from CA to Executer and replace it with EvAbortExecution + - count all EvState events from all CAs + - wait for final event EvTxResponse from Executer + - expect it to happen strictly after all EvState events + */ + Y_UNIT_TEST(WaitCAsStateOnAbort) { + TKikimrRunner kikimr(TKikimrSettings().SetUseRealThreads(false)); + auto db = kikimr.RunCall([&] { return kikimr.GetTableClient(); } ); + auto session = kikimr.RunCall([&] { return db.CreateSession().GetValueSync().GetSession(); } ); + + auto prepareResult = kikimr.RunCall([&] { return session.PrepareDataQuery(Q_(R"( + SELECT COUNT(*) FROM `/Root/TwoShard`; + )")).GetValueSync(); + }); + UNIT_ASSERT_VALUES_EQUAL_C(prepareResult.GetStatus(), EStatus::SUCCESS, prepareResult.GetIssues().ToString()); + auto dataQuery = prepareResult.GetQuery(); + + bool firstEvState = false; + ui32 totalEvState = 0; + TActorId executerId; + ui32 actorCount = 3; // TODO: get number of actors properly. + + auto& runtime = *kikimr.GetTestServer().GetRuntime(); + runtime.SetObserverFunc([&](TAutoPtr& ev) { + if (ev->GetTypeRewrite() == NYql::NDq::TEvDqCompute::TEvState::EventType) { + ++totalEvState; + if (!firstEvState) { + executerId = ev->Recipient; + ev = new IEventHandle(ev->Recipient, ev->Sender, + new NKikimr::NKqp::TEvKqp::TEvAbortExecution(NYql::NDqProto::StatusIds::UNSPECIFIED, NYql::TIssues())); + firstEvState = true; + } + } else if (ev->GetTypeRewrite() == NKikimr::NKqp::TEvKqpExecuter::TEvTxResponse::EventType && ev->Sender == executerId) { + UNIT_ASSERT_C(totalEvState == actorCount*2, "Executer sent response before waiting for CAs"); + } + + return TTestActorRuntime::EEventAction::PROCESS; + }); + + auto settings = TExecDataQuerySettings().OperationTimeout(TDuration::MilliSeconds(500)); + kikimr.RunInThreadPool([&] { return dataQuery.Execute(TTxControl::BeginTx().CommitTx(), settings).GetValueSync(); }); + + TDispatchOptions opts; + opts.FinalEvents.emplace_back([&](IEventHandle& ev) { + return ev.GetTypeRewrite() == NKikimr::NKqp::TEvKqpExecuter::TEvTxResponse::EventType + && ev.Sender == executerId && totalEvState == actorCount*2; + }); + + UNIT_ASSERT(runtime.DispatchEvents(opts)); + } + + /* Scenario: + - prepare and run query + - observe first EvState event from CA to Executer and replace it with EvAbortExecution + - count all EvState events from all CAs + - drop final EvState event from last CA + - wait for final event EvTxResponse from Executer after timeout poison + - expect it to happen strictly after all EvState events + */ + Y_UNIT_TEST(WaitCAsTimeout) { + TKikimrRunner kikimr(TKikimrSettings().SetUseRealThreads(false)); + auto db = kikimr.RunCall([&] { return kikimr.GetTableClient(); } ); + auto session = kikimr.RunCall([&] { return db.CreateSession().GetValueSync().GetSession(); } ); + + auto prepareResult = kikimr.RunCall([&] { return session.PrepareDataQuery(Q_(R"( + SELECT COUNT(*) FROM `/Root/TwoShard`; + )")).GetValueSync(); + }); + UNIT_ASSERT_VALUES_EQUAL_C(prepareResult.GetStatus(), EStatus::SUCCESS, prepareResult.GetIssues().ToString()); + auto dataQuery = prepareResult.GetQuery(); + + bool firstEvState = false; + bool timeoutPoison = false; + ui32 totalEvState = 0; + TActorId executerId; + ui32 actorCount = 3; // TODO: get number of actors properly. + + auto& runtime = *kikimr.GetTestServer().GetRuntime(); + runtime.SetObserverFunc([&](TAutoPtr& ev) { + if (ev->GetTypeRewrite() == NYql::NDq::TEvDqCompute::TEvState::EventType) { + ++totalEvState; + if (!firstEvState) { + executerId = ev->Recipient; + ev = new IEventHandle(ev->Recipient, ev->Sender, + new NKikimr::NKqp::TEvKqp::TEvAbortExecution(NYql::NDqProto::StatusIds::UNSPECIFIED, NYql::TIssues())); + firstEvState = true; + } else { + return TTestActorRuntime::EEventAction::DROP; + } + } else if (ev->GetTypeRewrite() == TEvents::TEvPoison::EventType && totalEvState == actorCount*2 && + ev->Sender == executerId && ev->Recipient == executerId) + { + timeoutPoison = true; + } else if (ev->GetTypeRewrite() == NKikimr::NKqp::TEvKqpExecuter::TEvTxResponse::EventType && ev->Sender == executerId) { + UNIT_ASSERT_C(timeoutPoison, "Executer sent response before waiting for CAs"); + } + + return TTestActorRuntime::EEventAction::PROCESS; + }); + + auto settings = TExecDataQuerySettings().OperationTimeout(TDuration::MilliSeconds(500)); + kikimr.RunInThreadPool([&] { return dataQuery.Execute(TTxControl::BeginTx().CommitTx(), settings).GetValueSync(); }); + + TDispatchOptions opts; + opts.FinalEvents.emplace_back([&](IEventHandle& ev) { + return ev.GetTypeRewrite() == NKikimr::NKqp::TEvKqpExecuter::TEvTxResponse::EventType + && ev.Sender == executerId && totalEvState == actorCount*2 && timeoutPoison; + }); + + UNIT_ASSERT(runtime.DispatchEvents(opts)); + } + Y_UNIT_TEST(ReplySizeExceeded) { auto kikimr = DefaultKikimrRunner(); auto db = kikimr.GetTableClient(); @@ -1007,6 +1185,7 @@ Y_UNIT_TEST_SUITE(KqpLimits) { UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); UNIT_ASSERT(result.GetStats()); + Cerr << result.GetStats()->ToString(true) << Endl; UNIT_ASSERT(result.GetStats()->GetPlan()); NJson::TJsonValue plan; @@ -1014,7 +1193,7 @@ Y_UNIT_TEST_SUITE(KqpLimits) { UNIT_ASSERT_VALUES_EQUAL(plan["Plan"]["Node Type"].GetStringSafe(), "Query"); UNIT_ASSERT_VALUES_EQUAL(plan["Plan"]["Plans"][0]["Node Type"].GetStringSafe(), "ResultSet"); - UNIT_ASSERT_VALUES_EQUAL(plan["Plan"]["Plans"][0]["Plans"][0]["Node Type"].GetStringSafe(), "Collect"); + UNIT_ASSERT_VALUES_EQUAL(plan["Plan"]["Plans"][0]["Plans"][0]["Node Type"].GetStringSafe(), "Stage"); UNIT_ASSERT_VALUES_EQUAL(plan["Plan"]["Plans"][0]["Plans"][0]["Plans"][0]["Node Type"].GetStringSafe(), "Merge"); UNIT_ASSERT_VALUES_EQUAL(plan["Plan"]["Plans"][0]["Plans"][0]["Plans"][0]["SortColumns"].GetArraySafe()[0], "Key (Asc)"); diff --git a/ydb/core/kqp/ut/query/kqp_stats_ut.cpp b/ydb/core/kqp/ut/query/kqp_stats_ut.cpp index 0f1221423669..1192c399f334 100644 --- a/ydb/core/kqp/ut/query/kqp_stats_ut.cpp +++ b/ydb/core/kqp/ut/query/kqp_stats_ut.cpp @@ -451,7 +451,7 @@ Y_UNIT_TEST_TWIN(StreamLookupStats, StreamLookupJoin) { }); } -Y_UNIT_TEST(SysViewTimeout) { +Y_UNIT_TEST(SysViewClientLost) { TKikimrRunner kikimr; CreateLargeTable(kikimr, 500000, 10, 100, 5000, 1); @@ -490,12 +490,13 @@ Y_UNIT_TEST(SysViewTimeout) { auto settings = TStreamExecScanQuerySettings(); settings.ClientTimeout(TDuration::MilliSeconds(50)); - TStringStream request; - request << R"( + TStringStream timeoutedRequestStream; + timeoutedRequestStream << R"( SELECT COUNT(*) FROM `/Root/LargeTable` WHERE SUBSTRING(DataText, 50, 5) = "22222"; )"; + TString timeoutedRequest = timeoutedRequestStream.Str(); - auto result = db.StreamExecuteScanQuery(request.Str(), settings).GetValueSync(); + auto result = db.StreamExecuteScanQuery(timeoutedRequest, settings).GetValueSync(); if (result.IsSuccess()) { try { @@ -510,7 +511,13 @@ Y_UNIT_TEST(SysViewTimeout) { UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), NYdb::EStatus::CLIENT_DEADLINE_EXCEEDED); } + ui32 timeoutedCount = 0; + ui32 iterations = 10; + while (timeoutedCount == 0 && iterations > 0) { + iterations--; + Sleep(TDuration::Seconds(1)); + TStringStream request; request << "SELECT * FROM `/Root/.sys/top_queries_by_read_bytes_one_hour` ORDER BY Duration"; @@ -518,7 +525,6 @@ Y_UNIT_TEST(SysViewTimeout) { UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); ui64 queryCount = 0; - ui64 rowsCount = 0; for (;;) { auto streamPart = it.ReadNext().GetValueSync(); if (!streamPart.IsSuccess()) { @@ -533,17 +539,16 @@ Y_UNIT_TEST(SysViewTimeout) { while (parser.TryNextRow()) { auto value = parser.ColumnParser("QueryText").GetOptionalUtf8(); UNIT_ASSERT(value); - if (*value == request.Str()) { + if (*value == timeoutedRequest) { queryCount++; } - rowsCount++; } } } - - UNIT_ASSERT(queryCount == 1); - UNIT_ASSERT(rowsCount == 2); + timeoutedCount = queryCount; } + + UNIT_ASSERT(timeoutedCount == 1); } Y_UNIT_TEST(SysViewCancelled) { @@ -582,9 +587,9 @@ Y_UNIT_TEST(SysViewCancelled) { UNIT_ASSERT(rowsCount == 1); } - auto prepareResult = session.PrepareDataQuery(Q_(R"( - SELECT COUNT(*) FROM `/Root/LargeTable` WHERE SUBSTRING(DataText, 50, 5) = "33333"; - )")).GetValueSync(); + TStringStream cancelledRequest; + cancelledRequest << "SELECT COUNT(*) FROM `/Root/LargeTable` WHERE SUBSTRING(DataText, 50, 5) = \"33333\""; + auto prepareResult = session.PrepareDataQuery(cancelledRequest.Str()).GetValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(prepareResult.GetStatus(), NYdb::EStatus::SUCCESS, prepareResult.GetIssues().ToString()); auto dataQuery = prepareResult.GetQuery(); @@ -619,7 +624,7 @@ Y_UNIT_TEST(SysViewCancelled) { while (parser.TryNextRow()) { auto value = parser.ColumnParser("QueryText").GetOptionalUtf8(); UNIT_ASSERT(value); - if (*value == request.Str()) { + if (*value == cancelledRequest.Str()) { queryCount++; } rowsCount++; @@ -628,7 +633,7 @@ Y_UNIT_TEST(SysViewCancelled) { } UNIT_ASSERT(queryCount == 1); - UNIT_ASSERT(rowsCount == 2); + UNIT_ASSERT(rowsCount == 3); } } diff --git a/ydb/core/protos/table_service_config.proto b/ydb/core/protos/table_service_config.proto index ce437818af6a..cce377de6039 100644 --- a/ydb/core/protos/table_service_config.proto +++ b/ydb/core/protos/table_service_config.proto @@ -297,7 +297,7 @@ message TTableServiceConfig { BLOCK_CHANNELS_AUTO = 1; BLOCK_CHANNELS_FORCE = 2; } - optional EBlockChannelsMode BlockChannelsMode = 61 [default = BLOCK_CHANNELS_SCALAR]; + optional EBlockChannelsMode BlockChannelsMode = 61 [default = BLOCK_CHANNELS_AUTO]; optional uint64 QueryReplayCacheUploadTTLSec = 62 [default = 36000]; 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 4baded583b14..a9bc116f0d0b 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 @@ -313,12 +313,12 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped void OnMemoryLimitExceptionHandler() { TString memoryConsumptionDetails = MemoryLimits.MemoryQuotaManager->MemoryConsumptionDetails(); TStringBuilder failureReason = TStringBuilder() - << "Mkql memory limit exceeded, limit: " << GetMkqlMemoryLimit() + << "Mkql memory limit exceeded, allocated by task " << Task.GetId() << ": " << GetMkqlMemoryLimit() << ", host: " << HostName() << ", canAllocateExtraMemory: " << CanAllocateExtraMemory; if (!memoryConsumptionDetails.empty()) { - failureReason << ", memory manager details: " << memoryConsumptionDetails; + failureReason << ", memory manager details for current node: " << memoryConsumptionDetails; } InternalError(NYql::NDqProto::StatusIds::OVERLOADED, TIssuesIds::KIKIMR_PRECONDITION_FAILED, failureReason); @@ -554,7 +554,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped } } - void ReportStateAndMaybeDie(NYql::NDqProto::StatusIds::StatusCode statusCode, const TIssues& issues) + void ReportStateAndMaybeDie(NYql::NDqProto::StatusIds::StatusCode statusCode, const TIssues& issues, bool forceTerminate = false) { auto execEv = MakeHolder(); auto& record = execEv->Record; @@ -575,7 +575,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped this->Send(ExecuterId, execEv.Release()); - if (Checkpoints && State == NDqProto::COMPUTE_STATE_FINISHED) { + if (!forceTerminate && Checkpoints && State == NDqProto::COMPUTE_STATE_FINISHED) { // checkpointed CAs must not self-destroy return; } @@ -1028,10 +1028,6 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped auto tag = (EEvWakeupTag) ev->Get()->Tag; switch (tag) { case EEvWakeupTag::TimeoutTag: { - auto abortEv = MakeHolder(NYql::NDqProto::StatusIds::TIMEOUT, TStringBuilder() - << "Timeout event from compute actor " << this->SelfId() - << ", TxId: " << TxId << ", task: " << Task.GetId()); - if (ComputeActorSpan) { ComputeActorSpan.EndError( TStringBuilder() @@ -1040,10 +1036,8 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped ); } - this->Send(ExecuterId, abortEv.Release()); - - TerminateSources("timeout exceeded", false); - Terminate(false, "timeout exceeded"); + State = NDqProto::COMPUTE_STATE_FAILURE; + ReportStateAndMaybeDie(NYql::NDqProto::StatusIds::TIMEOUT, {TIssue("timeout exceeded")}, true); break; } case EEvWakeupTag::PeriodicStatsTag: { @@ -1067,8 +1061,9 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped switch (lostEventType) { case TEvDqCompute::TEvState::EventType: { CA_LOG_E("Handle undelivered TEvState event, abort execution"); - this->TerminateSources("executer lost", false); - Terminate(false, "executer lost"); + + TerminateSources("executer lost", false); + Terminate(false, "executer lost"); // Executer lost - no need to report state break; } default: { @@ -1114,14 +1109,17 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped InternalError(NYql::NDqProto::StatusIds::INTERNAL_ERROR, *ev->Get()->GetIssues().begin()); return; } + TIssues issues = ev->Get()->GetIssues(); CA_LOG_E("Handle abort execution event from: " << ev->Sender << ", status: " << NYql::NDqProto::StatusIds_StatusCode_Name(ev->Get()->Record.GetStatusCode()) << ", reason: " << issues.ToOneLineString()); - bool success = ev->Get()->Record.GetStatusCode() == NYql::NDqProto::StatusIds::SUCCESS; - - this->TerminateSources(issues, success); + if (ev->Get()->Record.GetStatusCode() == NYql::NDqProto::StatusIds::SUCCESS) { + State = NDqProto::COMPUTE_STATE_FINISHED; + } else { + State = NDqProto::COMPUTE_STATE_FAILURE; + } if (ev->Sender != ExecuterId) { if (ComputeActorSpan) { @@ -1131,7 +1129,7 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped NActors::TActivationContext::Send(ev->Forward(ExecuterId)); } - Terminate(success, issues); + ReportStateAndMaybeDie(ev->Get()->Record.GetStatusCode(), issues, true); } void HandleExecuteBase(NActors::TEvInterconnect::TEvNodeDisconnected::TPtr& ev) { @@ -1811,6 +1809,8 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped } } } + } else { + // TODO: what should happen in this case? } static_cast(this)->FillExtraStats(dst, last); diff --git a/ydb/library/yql/dq/actors/protos/dq_stats.proto b/ydb/library/yql/dq/actors/protos/dq_stats.proto index 5bf22125bbf4..d03a02c31c2d 100644 --- a/ydb/library/yql/dq/actors/protos/dq_stats.proto +++ b/ydb/library/yql/dq/actors/protos/dq_stats.proto @@ -357,7 +357,7 @@ message TDqExecutionStats { uint64 CpuTimeUs = 1; // total cpu time, executer + compute actors + ... uint64 DurationUs = 2; // execution wall time - // these feilds are never used, needs to be reviewed + // these fields are never used, needs to be reviewed reserved 3; // uint64 ResultRows = 3; reserved 4; // uint64 ResultBytes = 4; diff --git a/ydb/library/yql/dq/opt/dq_opt_build.cpp b/ydb/library/yql/dq/opt/dq_opt_build.cpp index 0c59fd22df87..c58ac6536fc6 100644 --- a/ydb/library/yql/dq/opt/dq_opt_build.cpp +++ b/ydb/library/yql/dq/opt/dq_opt_build.cpp @@ -3,6 +3,7 @@ #include "dq_opt_phy_finalizing.h" #include +#include #include #include #include @@ -385,6 +386,24 @@ class TDqBuildPhysicalStagesTransformer : public TSyncTransformerBase { } }; +using TStageOptimizer = std::function; + +THolder CreateDqPhyStageTransformer(const TStageOptimizer& func, TTypeAnnotationContext* typesCtx) { + return CreateFunctorTransformer([=](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) -> IGraphTransformer::TStatus { + TOptimizeExprSettings settings(typesCtx); + settings.CustomInstantTypeTransformer = typesCtx->CustomInstantTypeTransformer.Get(); + settings.VisitLambdas = false; + + return OptimizeExpr(input, output, [&func](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { + TMaybeNode maybeStage(node); + if (!maybeStage) { + return node; + } + return func(maybeStage.Cast(), ctx).Ptr(); + }, ctx, settings); + }); +} + bool CanRebuildForWideChannelOutput(const TDqPhyStage& stage) { auto stageSettings = TDqStageSettings::Parse(stage); if (stageSettings.WideChannels) { @@ -414,7 +433,12 @@ bool CanRebuildForWideChannelOutput(const TDqOutput& output) { } bool IsSupportedForWide(const TDqConnection& conn) { - if (conn.Maybe() || conn.Maybe()) { + if (!(conn.Maybe() || + conn.Maybe() || + conn.Maybe() || + conn.Maybe() || + conn.Maybe())) + { return false; } @@ -427,12 +451,9 @@ bool IsSupportedForWide(const TDqConnection& conn) { return true; } -bool CanRebuildForWideChannelOutput(const TDqConnection& conn) { - if (!IsSupportedForWide(conn)) { - return false; - } - - return CanRebuildForWideChannelOutput(conn.Output().Stage().Cast()); +bool IsSupportedForWideBlocks(const TDqConnection& conn) { + // currently all connections supporting wide channels also support wide block channels + return IsSupportedForWide(conn); } const TStructExprType* GetStageOutputItemType(const TDqPhyStage& stage) { @@ -452,9 +473,53 @@ bool IsCompatibleWithBlocks(TPositionHandle pos, const TStructExprType& type, TE return resolveStatus == IArrowResolver::OK; } +TDqPhyStage RebuildStageOutputAsWide(const TDqPhyStage& stage, const TStructExprType& outputItemType, TExprContext& ctx) +{ + TCoLambda program(ctx.DeepCopyLambda(stage.Program().Ref())); + + auto stageSettings = TDqStageSettings::Parse(stage); + YQL_CLOG(INFO, CoreDq) << "Enabled wide channels for stage with logical id = " << stageSettings.LogicalId; + + // convert stream to wide stream + auto resultStream = ctx.Builder(program.Body().Pos()) + .Callable("FromFlow") + .Callable(0, "ExpandMap") + .Callable(0, "ToFlow") + .Add(0, program.Body().Ptr()) + .Seal() + .Lambda(1) + .Param("item") + .Do([&](TExprNodeBuilder& lambda) -> TExprNodeBuilder& { + ui32 i = 0U; + for (const auto& item : outputItemType.GetItems()) { + lambda.Callable(i++, "Member") + .Arg(0, "item") + .Atom(1, item->GetName()) + .Seal(); + } + return lambda; + }) + .Seal() + .Seal() + .Seal() + .Build(); + + return Build(ctx, stage.Pos()) + .InitFrom(stage) + .Program() + .Args(program.Args()) + .Body(resultStream) + .Build() + .Settings(TDqStageSettings::New(stage).SetWideChannels(outputItemType).BuildNode(ctx, stage.Pos())) + .Outputs(stage.Outputs()) + .Done(); +} + TDqPhyStage RebuildStageInputsAsWide(const TDqPhyStage& stage, TExprContext& ctx) { TVector newArgs; + TExprNodeList newInputs; newArgs.reserve(stage.Inputs().Size()); + newInputs.reserve(stage.Inputs().Size()); TNodeOnNodeOwnedMap argsMap; YQL_ENSURE(stage.Inputs().Size() == stage.Program().Args().Size()); @@ -468,7 +533,7 @@ TDqPhyStage RebuildStageInputsAsWide(const TDqPhyStage& stage, TExprContext& ctx auto maybeConn = stage.Inputs().Item(i).Maybe(); - if (maybeConn && CanRebuildForWideChannelOutput(maybeConn.Cast().Output())) { + if (maybeConn && IsSupportedForWide(maybeConn.Cast()) && CanRebuildForWideChannelOutput(maybeConn.Cast().Output())) { needRebuild = true; auto itemType = arg.Ref().GetTypeAnn()->Cast()->GetItemType()->Cast(); TExprNode::TPtr newArgNode = newArg.Ptr(); @@ -500,85 +565,10 @@ TDqPhyStage RebuildStageInputsAsWide(const TDqPhyStage& stage, TExprContext& ctx .Build(); argsMap.emplace(arg.Raw(), argReplace); - } else { - argsMap.emplace(arg.Raw(), newArg.Ptr()); - } - } - - if (!needRebuild) { - return stage; - } - - return Build(ctx, stage.Pos()) - .InitFrom(stage) - .Program() - .Args(newArgs) - .Body(ctx.ReplaceNodes(stage.Program().Body().Ptr(), argsMap)) - .Build() - .Done(); -} - -TDqPhyStage RebuildStageOutputAsWide(const TDqPhyStage& stage, const TStructExprType& outputItemType, TExprContext& ctx) -{ - TCoLambda program(ctx.DeepCopyLambda(stage.Program().Ref())); - - // convert stream to wide stream - auto resultStream = ctx.Builder(program.Body().Pos()) - .Callable("FromFlow") - .Callable(0, "ExpandMap") - .Callable(0, "ToFlow") - .Add(0, program.Body().Ptr()) - .Seal() - .Lambda(1) - .Param("item") - .Do([&](TExprNodeBuilder& lambda) -> TExprNodeBuilder& { - ui32 i = 0U; - for (const auto& item : outputItemType.GetItems()) { - lambda.Callable(i++, "Member") - .Arg(0, "item") - .Atom(1, item->GetName()) - .Seal(); - } - return lambda; - }) - .Seal() - .Seal() - .Seal() - .Build(); + const TDqConnection& conn = maybeConn.Cast(); - return Build(ctx, stage.Pos()) - .InitFrom(stage) - .Program() - .Args(program.Args()) - .Body(resultStream) - .Build() - .Settings(TDqStageSettings::New(stage).SetWideChannels(outputItemType).BuildNode(ctx, stage.Pos())) - .Outputs(stage.Outputs()) - .Done(); -} - -TDqPhyStage RebuildStageAsWide(const TDqPhyStage& stage, TExprContext& ctx) { - const TStructExprType* outputItemType = GetStageOutputItemType(stage); - return RebuildStageOutputAsWide(RebuildStageInputsAsWide(stage, ctx), *outputItemType, ctx); -} - -IGraphTransformer::TStatus DqEnableWideChannels(TExprNode::TPtr input, TExprNode::TPtr& output, TExprContext& ctx) -{ - output = input; - TNodeOnNodeOwnedMap replaces; - TNodeSet processedStages; - VisitExpr(input, [&](const TExprNode::TPtr& node) { - if (node->IsLambda()) { - return false; - } - - TExprBase expr{node}; - auto maybeConn = expr.Maybe(); - if (maybeConn && CanRebuildForWideChannelOutput(maybeConn.Cast())) { - auto conn = maybeConn.Cast(); - processedStages.insert(conn.Output().Stage().Raw()); - auto newStage = RebuildStageAsWide(conn.Output().Stage().Cast(), ctx); auto outputItemType = GetStageOutputItemType(conn.Output().Stage().Cast()); + auto newStage = RebuildStageOutputAsWide(conn.Output().Stage().Cast(), *outputItemType, ctx); if (conn.Maybe()) { auto shuffle = conn.Maybe().Cast(); @@ -588,13 +578,13 @@ IGraphTransformer::TStatus DqEnableWideChannels(TExprNode::TPtr input, TExprNode builder.Add().Build(ToString(idx)); } - replaces[conn.Raw()] = Build(ctx, conn.Pos()) + newInputs.push_back(Build(ctx, conn.Pos()) .Output() .InitFrom(conn.Output()) .Stage(newStage) .Build() .KeyColumns(builder.Build().Value()) - .Done().Ptr(); + .Done().Ptr()); } else if (conn.Maybe()) { auto merge = conn.Maybe().Cast(); auto builder = Build(ctx, merge.SortColumns().Pos()); @@ -606,44 +596,52 @@ IGraphTransformer::TStatus DqEnableWideChannels(TExprNode::TPtr input, TExprNode .Build(); } - replaces[conn.Raw()] = Build(ctx, conn.Pos()) + newInputs.push_back(Build(ctx, conn.Pos()) .Output() .InitFrom(conn.Output()) .Stage(newStage) .Build() .SortColumns(builder.Build().Value()) - .Done().Ptr(); + .Done().Ptr()); } else { auto newOutput = Build(ctx, conn.Output().Pos()) .InitFrom(conn.Output()) .Stage(newStage) .Done(); - replaces[conn.Raw()] = ctx.ChangeChild(conn.Ref(), TDqConnection::idx_Output, newOutput.Ptr()); - } - } else if (expr.Maybe()) { - auto stage = expr.Maybe().Cast(); - if (!processedStages.contains(stage.Raw())) { - processedStages.insert(stage.Raw()); - auto newStage = RebuildStageInputsAsWide(stage, ctx); - if (newStage.Raw() != stage.Raw()) { - replaces[stage.Raw()] = newStage.Ptr(); - } + newInputs.push_back(ctx.ChangeChild(conn.Ref(), TDqConnection::idx_Output, newOutput.Ptr())); } + } else { + argsMap.emplace(arg.Raw(), newArg.Ptr()); + newInputs.push_back(stage.Inputs().Item(i).Ptr()); } + } - return true; - }); - - if (replaces.empty()) { - return IGraphTransformer::TStatus::Ok; + if (!needRebuild) { + return stage; } - YQL_CLOG(INFO, CoreDq) << "[DQ/Build/EnableWideChannels] " << "Enabled wide channels for " << replaces.size() << " stages"; - TOptimizeExprSettings settings{nullptr}; - settings.VisitLambdas = false; - auto status = RemapExpr(input, output, replaces, ctx, settings); - YQL_CLOG(TRACE, CoreDq) << "[DQ/Build/EnableWideChannels] " << "Dump: " << NCommon::ExprToPrettyString(ctx, *output); - return status; + YQL_ENSURE(argsMap.size() == stage.Inputs().Size()); + YQL_ENSURE(newInputs.size() == stage.Inputs().Size()); + + return Build(ctx, stage.Pos()) + .InitFrom(stage) + .Inputs() + .Add(newInputs) + .Build() + .Program() + .Args(newArgs) + .Body(ctx.ReplaceNodes(stage.Program().Body().Ptr(), argsMap)) + .Build() + .Done(); +} + +TDqPhyStage DqEnableWideChannelsInputForStage(const TDqPhyStage& stage, TExprContext& ctx) { + auto stageSettings = TDqStageSettings::Parse(stage); + if (stageSettings.WideChannels) { + // Optimization: If current stage is already wide, assume all its inputs already rebuilt for wide channels + return stage; + } + return RebuildStageInputsAsWide(stage, ctx); } bool CanPullReplicateScalars(const TDqPhyStage& stage) { @@ -666,7 +664,7 @@ bool CanPullReplicateScalars(const TDqOutput& output) { } bool CanPullReplicateScalars(const TDqConnection& conn) { - if (!IsSupportedForWide(conn)) { + if (!IsSupportedForWideBlocks(conn)) { return false; } @@ -729,11 +727,15 @@ TDqPhyStage DqPullReplicateScalarsFromInputs(const TDqPhyStage& stage, TExprCont } } + YQL_ENSURE(argsMap.size() == stage.Inputs().Size()); + YQL_ENSURE(newInputs.size() == stage.Inputs().Size()); + if (!pulled) { return stage; } - YQL_CLOG(INFO, CoreDq) << "Pulled " << pulled << " ReplicateScalars from stage inputs"; + auto stageSettings = TDqStageSettings::Parse(stage); + YQL_CLOG(INFO, CoreDq) << "Pulled ReplicateScalars from " << pulled << " inputs of stage with logical id = " << stageSettings.LogicalId; return Build(ctx, stage.Pos()) .InitFrom(stage) .Inputs() @@ -786,28 +788,33 @@ bool CanRebuildForWideBlockChannelOutput(bool forceBlocks, const TDqOutput& outp return CanRebuildForWideBlockChannelOutput(forceBlocks, output.Stage().Cast(), ctx, typesCtx); } -bool CanRebuildForWideBlockChannelOutput(bool forceBlocks, const TDqConnection& conn, TExprContext& ctx, TTypeAnnotationContext& typesCtx) { - if (conn.Maybe() || conn.Maybe()) { - return false; - } - - ui32 index = FromString(conn.Output().Index().Value()); - if (index != 0) { - // stage has multiple outputs - return false; - } - - return CanRebuildForWideBlockChannelOutput(forceBlocks, conn.Output().Stage().Cast(), ctx, typesCtx); +TDqPhyStage RebuildStageOutputAsWideBlock(const TDqPhyStage& stage, TExprContext& ctx) +{ + return Build(ctx, stage.Pos()) + .InitFrom(stage) + .Program() + .Args(stage.Program().Args()) + .Body() + .Input() + .Input() + .Input(stage.Program().Body()) + .Build() + .Build() + .Build() + .Build() + .Done(); } TDqPhyStage RebuildStageInputsAsWideBlock(bool forceBlocks, const TDqPhyStage& stage, TExprContext& ctx, TTypeAnnotationContext& typesCtx) { TVector newArgs; + TExprNodeList newInputs; newArgs.reserve(stage.Inputs().Size()); + newInputs.reserve(stage.Inputs().Size()); TNodeOnNodeOwnedMap argsMap; YQL_ENSURE(stage.Inputs().Size() == stage.Program().Args().Size()); - bool needRebuild = false; + size_t blockInputs = 0; for (size_t i = 0; i < stage.Inputs().Size(); ++i) { TCoArgument arg = stage.Program().Args().Arg(i); @@ -816,8 +823,8 @@ TDqPhyStage RebuildStageInputsAsWideBlock(bool forceBlocks, const TDqPhyStage& s auto maybeConn = stage.Inputs().Item(i).Maybe(); - if (maybeConn && CanRebuildForWideBlockChannelOutput(forceBlocks, maybeConn.Cast().Output(), ctx, typesCtx)) { - needRebuild = true; + if (maybeConn && IsSupportedForWideBlocks(maybeConn.Cast()) && CanRebuildForWideBlockChannelOutput(forceBlocks, maybeConn.Cast().Output(), ctx, typesCtx)) { + ++blockInputs; // input will actually be wide block stream - convert it to wide stream first TExprNode::TPtr newArgNode = ctx.Builder(arg.Pos()) .Callable("FromFlow") @@ -829,17 +836,34 @@ TDqPhyStage RebuildStageInputsAsWideBlock(bool forceBlocks, const TDqPhyStage& s .Seal() .Build(); argsMap.emplace(arg.Raw(), newArgNode); + + const TDqConnection& conn = maybeConn.Cast(); + auto newOutput = Build(ctx, conn.Output().Pos()) + .InitFrom(conn.Output()) + .Stage(RebuildStageOutputAsWideBlock(conn.Output().Stage().Cast(), ctx)) + .Done(); + newInputs.push_back(ctx.ChangeChild(conn.Ref(), TDqConnection::idx_Output, newOutput.Ptr())); } else { argsMap.emplace(arg.Raw(), newArg.Ptr()); + newInputs.push_back(stage.Inputs().Item(i).Ptr()); } } - if (!needRebuild) { + YQL_ENSURE(argsMap.size() == stage.Inputs().Size()); + YQL_ENSURE(newInputs.size() == stage.Inputs().Size()); + + if (!blockInputs) { return stage; } + auto stageSettings = TDqStageSettings::Parse(stage); + YQL_CLOG(INFO, CoreDq) << "Enabled " << blockInputs << " block inputs for stage with logical id = " << stageSettings.LogicalId; + return Build(ctx, stage.Pos()) .InitFrom(stage) + .Inputs() + .Add(newInputs) + .Build() .Program() .Args(newArgs) .Body(ctx.ReplaceNodes(stage.Program().Body().Ptr(), argsMap)) @@ -847,79 +871,9 @@ TDqPhyStage RebuildStageInputsAsWideBlock(bool forceBlocks, const TDqPhyStage& s .Done(); } -TDqPhyStage RebuildStageOutputAsWideBlock(const TDqPhyStage& stage, TExprContext& ctx) -{ - return Build(ctx, stage.Pos()) - .InitFrom(stage) - .Program() - .Args(stage.Program().Args()) - .Body() - .Input() - .Input() - .Input(stage.Program().Body()) - .Build() - .Build() - .Build() - .Build() - .Done(); -} - -TDqPhyStage RebuildStageAsWideBlock(bool forceBlocks, const TDqPhyStage& stage, TExprContext& ctx, TTypeAnnotationContext& typesCtx) { - return RebuildStageOutputAsWideBlock(RebuildStageInputsAsWideBlock(forceBlocks, stage, ctx, typesCtx), ctx); -} - -IGraphTransformer::TStatus DqEnableWideBlockChannels(bool forceBlocks, TExprNode::TPtr input, TExprNode::TPtr& output, - TExprContext& ctx, TTypeAnnotationContext& typesCtx) -{ - output = input; - TNodeOnNodeOwnedMap replaces; - TNodeSet processedStages; - VisitExpr(input, [&](const TExprNode::TPtr& node) { - if (node->IsLambda()) { - return false; - } - - TExprBase expr{node}; - auto maybeConn = expr.Maybe(); - if (maybeConn && CanRebuildForWideBlockChannelOutput(forceBlocks, maybeConn.Cast(), ctx, typesCtx)) { - auto conn = maybeConn.Cast(); - processedStages.insert(conn.Output().Stage().Raw()); - auto newStage = RebuildStageAsWideBlock(forceBlocks, conn.Output().Stage().Cast(), ctx, typesCtx); - auto newOutput = Build(ctx, conn.Output().Pos()) - .InitFrom(conn.Output()) - .Stage(newStage) - .Done(); - replaces[conn.Raw()] = ctx.ChangeChild(conn.Ref(), TDqConnection::idx_Output, newOutput.Ptr()); - } else if (expr.Maybe()) { - auto stage = expr.Maybe().Cast(); - if (!processedStages.contains(stage.Raw())) { - processedStages.insert(stage.Raw()); - auto newStage = RebuildStageInputsAsWideBlock(forceBlocks, stage, ctx, typesCtx); - if (newStage.Raw() != stage.Raw()) { - replaces[stage.Raw()] = newStage.Ptr(); - } - } - } - - return true; - }); - - if (replaces.empty()) { - return IGraphTransformer::TStatus::Ok; - } - - YQL_CLOG(INFO, CoreDq) << "[DQ/Build/EnableWideBlockChannels] " << "Enabled block channels for " << replaces.size() << " stages"; - TOptimizeExprSettings settings{nullptr}; - settings.VisitLambdas = false; - auto status = RemapExpr(input, output, replaces, ctx, settings); - YQL_CLOG(TRACE, CoreDq) << "[DQ/Build/EnableWideBlockChannels] " << "Dump: " << NCommon::ExprToPrettyString(ctx, *output); - return status; -} - } // namespace TAutoPtr CreateDqBuildPhyStagesTransformer(bool allowDependantConsumers, TTypeAnnotationContext& typesCtx, EChannelMode mode) { - Y_UNUSED(typesCtx); TVector transformers; transformers.push_back(TTransformStage(CreateFunctorTransformer( @@ -940,7 +894,7 @@ TAutoPtr CreateDqBuildPhyStagesTransformer(bool allowDependan TIssuesIds::DEFAULT_ERROR)); if (mode != CHANNEL_SCALAR) { - transformers.push_back(TTransformStage(CreateFunctorTransformer(&DqEnableWideChannels), + transformers.push_back(TTransformStage(CreateDqPhyStageTransformer(&DqEnableWideChannelsInputForStage, &typesCtx), "EnableWideChannels", TIssuesIds::DEFAULT_ERROR)); } @@ -952,25 +906,14 @@ TAutoPtr CreateDqBuildWideBlockChannelsTransformer(TTypeAnnot TVector transformers; if (mode == CHANNEL_WIDE_AUTO_BLOCK || mode == CHANNEL_WIDE_FORCE_BLOCK) { - transformers.push_back(TTransformStage(CreateFunctorTransformer( - [mode, &typesCtx](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { + transformers.push_back(TTransformStage(CreateDqPhyStageTransformer( + [mode, &typesCtx](const TDqPhyStage& stage, TExprContext& ctx) { const bool forceBlocks = mode == CHANNEL_WIDE_FORCE_BLOCK; - return DqEnableWideBlockChannels(forceBlocks, input, output, ctx, typesCtx); - }), + return RebuildStageInputsAsWideBlock(forceBlocks, stage, ctx, typesCtx); + }, &typesCtx), "EnableBlockChannels", TIssuesIds::DEFAULT_ERROR)); - transformers.push_back(TTransformStage(CreateFunctorTransformer( - [&typesCtx](const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx) { - TOptimizeExprSettings optSettings{&typesCtx}; - optSettings.VisitLambdas = false; - return OptimizeExpr(input, output, [](const TExprNode::TPtr& node, TExprContext& ctx) -> TExprNode::TPtr { - TExprBase expr{node}; - if (auto stage = expr.Maybe()) { - return DqPullReplicateScalarsFromInputs(stage.Cast(), ctx).Ptr(); - } - return node; - }, ctx, optSettings); - }), + transformers.push_back(TTransformStage(CreateDqPhyStageTransformer(&DqPullReplicateScalarsFromInputs, &typesCtx), "PullReplicateScalars", TIssuesIds::DEFAULT_ERROR)); } diff --git a/ydb/library/yql/dq/runtime/dq_input_producer.cpp b/ydb/library/yql/dq/runtime/dq_input_producer.cpp index 05c80f11921b..78aeeeb75ccb 100644 --- a/ydb/library/yql/dq/runtime/dq_input_producer.cpp +++ b/ydb/library/yql/dq/runtime/dq_input_producer.cpp @@ -24,11 +24,11 @@ template class TDqInputUnionStreamValue : public TComputationValue> { using TBase = TComputationValue>; public: - TDqInputUnionStreamValue(TMemoryUsageInfo* memInfo, TVector&& inputs, TDqMeteringStats::TInputStatsMeter stats) + TDqInputUnionStreamValue(TMemoryUsageInfo* memInfo, const NKikimr::NMiniKQL::TType* type, TVector&& inputs, TDqMeteringStats::TInputStatsMeter stats) : TBase(memInfo) , Inputs(std::move(inputs)) , Alive(Inputs.size()) - , Batch(Inputs.empty() ? nullptr : Inputs.front()->GetInputType()) + , Batch(type) , Stats(stats) {} @@ -114,13 +114,15 @@ template class TDqInputMergeStreamValue : public TComputationValue> { using TBase = TComputationValue>; public: - TDqInputMergeStreamValue(TMemoryUsageInfo* memInfo, TVector&& inputs, + TDqInputMergeStreamValue(TMemoryUsageInfo* memInfo, const NKikimr::NMiniKQL::TType* type, TVector&& inputs, TVector&& sortCols, TDqMeteringStats::TInputStatsMeter stats) : TBase(memInfo) , Inputs(std::move(inputs)) + , Width(type->IsMulti() ? static_cast(type)->GetElementsCount() : TMaybe()) , SortCols(std::move(sortCols)) , Stats(stats) { + YQL_ENSURE(!IsWide ^ Width.Defined()); CurrentBuffers.reserve(Inputs.size()); CurrentItemIndexes.reserve(Inputs.size()); for (ui32 idx = 0; idx < Inputs.size(); ++idx) { @@ -216,7 +218,7 @@ class TDqInputMergeStreamValue : public TComputationValueGetInputWidth() == width); + YQL_ENSURE(*Width == width); CopyResult(result, width); if (Stats) { Stats.Add(result, width); @@ -300,6 +302,7 @@ class TDqInputMergeStreamValue : public TComputationValue Inputs; + const TMaybe Width; TVector SortCols; TVector CurrentBuffers; TVector> CurrentItemIndexes; @@ -308,20 +311,6 @@ class TDqInputMergeStreamValue : public TComputationValue& inputs) { - NKikimr::NMiniKQL::TType* type = nullptr; - bool isWide = false; - for (auto& input : inputs) { - if (!type) { - type = input->GetInputType(); - isWide = input->GetInputWidth().Defined(); - } else { - YQL_ENSURE(type->IsSameType(*input->GetInputType())); - } - } - return isWide; -} - TVector ExtractBlockItemTypes(const NKikimr::NMiniKQL::TType* type) { TVector result; @@ -390,18 +379,17 @@ TVector MakeComparators(const TVector { using TBase = TComputationValue; public: - TDqInputMergeBlockStreamValue(TMemoryUsageInfo* memInfo, TVector&& inputs, + TDqInputMergeBlockStreamValue(TMemoryUsageInfo* memInfo, const NKikimr::NMiniKQL::TType* type, TVector&& inputs, TVector&& sortCols, const NKikimr::NMiniKQL::THolderFactory& factory, TDqMeteringStats::TInputStatsMeter stats) : TBase(memInfo) , SortCols_(std::move(sortCols)) - , ItemTypes_(ExtractBlockItemTypes(inputs.front()->GetInputType())) + , ItemTypes_(ExtractBlockItemTypes(type)) , MaxOutputBlockLen_(CalcMaxBlockLength(ItemTypes_.begin(), ItemTypes_.end(), TTypeInfoHelper())) , Comparators_(MakeComparators(SortCols_, ItemTypes_)) , Builders_(MakeBuilders(MaxOutputBlockLen_, ItemTypes_)) , Factory_(factory) , Stats_(stats) { - YQL_ENSURE(!inputs.empty()); YQL_ENSURE(MaxOutputBlockLen_ > 0); InputData_.reserve(inputs.size()); for (auto& input : inputs) { @@ -697,6 +685,15 @@ class TDqInputMergeBlockStreamValue : public TComputationValue& inputs) { + YQL_ENSURE(type); + for (size_t i = 0; i < inputs.size(); ++i) { + auto inputType = inputs[i]->GetInputType(); + YQL_ENSURE(inputType); + YQL_ENSURE(type->IsSameType(*inputType), "Unexpected type for input #" << i << ": expected " << *type << ", got " << *inputType); + } +} + } // namespace void TDqMeteringStats::TInputStatsMeter::Add(const NKikimr::NUdf::TUnboxedValue& val) { @@ -737,31 +734,33 @@ void TDqMeteringStats::TInputStatsMeter::Add(const NKikimr::NUdf::TUnboxedValue* } } -NUdf::TUnboxedValue CreateInputUnionValue(TVector&& inputs, +NUdf::TUnboxedValue CreateInputUnionValue(const NKikimr::NMiniKQL::TType* type, TVector&& inputs, const NMiniKQL::THolderFactory& factory, TDqMeteringStats::TInputStatsMeter stats) { - if (IsWideInputs(inputs)) { - return factory.Create>(std::move(inputs), stats); + ValidateInputTypes(type, inputs); + if (type->IsMulti()) { + return factory.Create>(type, std::move(inputs), stats); } - return factory.Create>(std::move(inputs), stats); + return factory.Create>(type, std::move(inputs), stats); } -NKikimr::NUdf::TUnboxedValue CreateInputMergeValue(TVector&& inputs, +NKikimr::NUdf::TUnboxedValue CreateInputMergeValue(const NKikimr::NMiniKQL::TType* type, TVector&& inputs, TVector&& sortCols, const NKikimr::NMiniKQL::THolderFactory& factory, TDqMeteringStats::TInputStatsMeter stats) { + ValidateInputTypes(type, inputs); YQL_ENSURE(!inputs.empty()); - if (IsWideInputs(inputs)) { + if (type->IsMulti()) { if (AnyOf(sortCols, [](const auto& sortCol) { return sortCol.IsBlockOrScalar(); })) { // we can ignore scalar columns, since all they have exactly the same value in all inputs EraseIf(sortCols, [](const auto& sortCol) { return *sortCol.IsScalar; }); if (sortCols.empty()) { - return factory.Create>(std::move(inputs), stats); + return factory.Create>(type, std::move(inputs), stats); } - return factory.Create(std::move(inputs), std::move(sortCols), factory, stats); + return factory.Create(type, std::move(inputs), std::move(sortCols), factory, stats); } - return factory.Create>(std::move(inputs), std::move(sortCols), stats); + return factory.Create>(type, std::move(inputs), std::move(sortCols), stats); } - return factory.Create>(std::move(inputs), std::move(sortCols), stats); + return factory.Create>(type, std::move(inputs), std::move(sortCols), stats); } } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/runtime/dq_input_producer.h b/ydb/library/yql/dq/runtime/dq_input_producer.h index d4e96b2aebb5..8c4ac669c89d 100644 --- a/ydb/library/yql/dq/runtime/dq_input_producer.h +++ b/ydb/library/yql/dq/runtime/dq_input_producer.h @@ -28,10 +28,10 @@ struct TDqMeteringStats { } }; -NKikimr::NUdf::TUnboxedValue CreateInputUnionValue(TVector&& inputs, +NKikimr::NUdf::TUnboxedValue CreateInputUnionValue(const NKikimr::NMiniKQL::TType* type, TVector&& inputs, const NKikimr::NMiniKQL::THolderFactory& holderFactory, TDqMeteringStats::TInputStatsMeter = {}); -NKikimr::NUdf::TUnboxedValue CreateInputMergeValue(TVector&& inputs, +NKikimr::NUdf::TUnboxedValue CreateInputMergeValue(const NKikimr::NMiniKQL::TType* type, TVector&& inputs, TVector&& sortCols, const NKikimr::NMiniKQL::THolderFactory& factory, TDqMeteringStats::TInputStatsMeter = {}); diff --git a/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp b/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp index f3c60fe3771f..2c04e33985b0 100644 --- a/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp +++ b/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp @@ -146,14 +146,14 @@ NUdf::TUnboxedValue DqBuildInputValue(const NDqProto::TTaskInput& inputDesc, con Y_ABORT_UNLESS(inputs.size() == 1); [[fallthrough]]; case NYql::NDqProto::TTaskInput::kUnionAll: - return CreateInputUnionValue(std::move(inputs), holderFactory, stats); + return CreateInputUnionValue(type, std::move(inputs), holderFactory, stats); case NYql::NDqProto::TTaskInput::kMerge: { const auto& protoSortCols = inputDesc.GetMerge().GetSortColumns(); TVector sortColsInfo; GetSortColumnsInfo(type, protoSortCols, sortColsInfo); YQL_ENSURE(!sortColsInfo.empty()); - return CreateInputMergeValue(std::move(inputs), std::move(sortColsInfo), holderFactory, stats); + return CreateInputMergeValue(type, std::move(inputs), std::move(sortColsInfo), holderFactory, stats); } default: YQL_ENSURE(false, "Unknown input type: " << (ui32) inputDesc.GetTypeCase()); @@ -576,7 +576,7 @@ class TDqTaskRunner : public IDqTaskRunner { inputs.clear(); inputs.emplace_back(transform->TransformOutput); entryNode->SetValue(AllocatedHolder->ProgramParsed.CompGraph->GetContext(), - CreateInputUnionValue(std::move(inputs), holderFactory, + CreateInputUnionValue(transform->TransformOutput->GetInputType(), std::move(inputs), holderFactory, {&inputStats, transform->TransformOutputType})); } else { entryNode->SetValue(AllocatedHolder->ProgramParsed.CompGraph->GetContext(), diff --git a/ydb/tests/functional/canonical/canondata/test_sql.TestCanonicalFolder1.test_case_join_group_by_lookup.script-script_/join_group_by_lookup.script.plan b/ydb/tests/functional/canonical/canondata/test_sql.TestCanonicalFolder1.test_case_join_group_by_lookup.script-script_/join_group_by_lookup.script.plan index 9887e7d9bf98..996c1e57a388 100644 --- a/ydb/tests/functional/canonical/canondata/test_sql.TestCanonicalFolder1.test_case_join_group_by_lookup.script-script_/join_group_by_lookup.script.plan +++ b/ydb/tests/functional/canonical/canondata/test_sql.TestCanonicalFolder1.test_case_join_group_by_lookup.script-script_/join_group_by_lookup.script.plan @@ -73,8 +73,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 15 @@ -87,7 +87,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 13, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_1.plan index af38e68204c4..e3c05525b29f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_1.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_2_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_2_0" } ], "PlanNodeId": 27 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 25, "Plans": [ { @@ -207,8 +207,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -221,7 +221,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_2.plan index 24ffa29e9d78..6076b6f27244 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_2.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_3.plan index 942794d1037b..b93e04639064 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_3.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_4.plan index c2105796cb77..a7b411e3df49 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_4.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_5.plan index 1f2e04ddcfe7..6e871c56abdb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_coalesce-and-join.test_/query_5.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_2.plan index 5097d28ccc74..71d98871320e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_2.plan @@ -91,8 +91,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -105,7 +105,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_3.plan index 0dacb2f32678..519ee1298087 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_3.plan @@ -91,8 +91,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -105,7 +105,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_4.plan index a6724a213ff4..e7540c66dd46 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_4.plan @@ -116,7 +116,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 5, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_5.plan index dbadb21402f2..3ab001e1a0da 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join-group-by-with-null.test_/query_5.plan @@ -125,7 +125,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_1.plan index b43fd89b3ed2..ddf85108605d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_1.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_10.plan index 75a2e67f7727..ee8e05543498 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_10.plan @@ -79,7 +79,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_11.plan index f69640b3d8ad..cb5eb4f1c342 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_11.plan @@ -76,7 +76,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_12.plan index b77301e9a83e..d4027692262d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_12.plan @@ -61,7 +61,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -170,7 +170,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_13.plan index 6da1649d210d..61a88e925416 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_13.plan @@ -61,7 +61,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 9, "Plans": [ { @@ -93,7 +93,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -171,14 +171,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "ConstantExpr", - "Operators": [ - { - "Inputs": [], - "Iterator": "[{join_key: 0},{join_key: 1}]", - "Name": "Iterator" - } - ], + "Node Type": "Stage", "PlanNodeId": 11 } ] diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_14.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_14.plan index b8ac79d0caae..26678b20cf69 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_14.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_14.plan @@ -61,7 +61,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { @@ -95,7 +95,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -160,14 +160,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "ConstantExpr", - "Operators": [ - { - "Inputs": [], - "Iterator": "[{id: 1}]", - "Name": "Iterator" - } - ], + "Node Type": "Stage", "PlanNodeId": 9 } ] diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_15.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_15.plan index 5e89e72639ad..ea93b761a245 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_15.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_15.plan @@ -61,7 +61,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_16.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_16.plan index 069755e91d5a..c83a3763d9ea 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_16.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_16.plan @@ -75,7 +75,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -120,7 +120,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_3.plan index 185704f342fd..d12bcc33cd5f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_3.plan @@ -77,7 +77,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_4.plan index 489d5cec033d..6eb677bb0873 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_4.plan @@ -82,7 +82,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_5.plan index 8edc39b7ec5e..d40252a5b254 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_5.plan @@ -71,7 +71,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -139,7 +139,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { @@ -185,7 +185,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_6.plan index 36c62b1c6e60..a957c7bac748 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_6.plan @@ -84,8 +84,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 31 @@ -98,7 +98,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 29, "Plans": [ { @@ -226,7 +226,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 15, "Plans": [ { @@ -258,7 +258,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 9, "Plans": [ { @@ -320,7 +320,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_7.plan index a24ce1f1ce7e..04551b81acd3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_7.plan @@ -68,7 +68,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 9, "Plans": [ { @@ -109,7 +109,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_8.plan index a32e28168e1d..81d34e06428c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_8.plan @@ -68,7 +68,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 9, "Plans": [ { @@ -109,7 +109,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_9.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_9.plan index 1d9a0061c04b..a65d84624058 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_9.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join0.test_/query_9.plan @@ -77,7 +77,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_10.plan index 17ef64e01f55..21de390cb90f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_10.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_11.plan index f8325b2f07e4..2dc3f758529c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_11.plan @@ -73,7 +73,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -119,7 +119,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_12.plan index 750e3105a2ec..905cdf1f706e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_12.plan @@ -73,7 +73,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -119,7 +119,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_13.plan index 4adffb441a31..d2c923142e83 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_13.plan @@ -84,8 +84,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -98,7 +98,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_14.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_14.plan index 082b468b877c..d297d6eae6ce 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_14.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_14.plan @@ -66,8 +66,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -80,7 +80,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_4.plan index 95cd0db0fb97..24958b49745a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_4.plan @@ -71,7 +71,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_6.plan index f17ed34ac242..75a461a10299 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_6.plan @@ -71,7 +71,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_7.plan index a2304eca8039..2efa5993b693 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_7.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_8.plan index a2304eca8039..2efa5993b693 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_8.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_9.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_9.plan index 17ef64e01f55..21de390cb90f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_9.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join1.test_/query_9.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_1.plan index 1bc6e8308f19..03f42f71e58a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_1.plan @@ -79,7 +79,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { @@ -113,7 +113,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { @@ -159,7 +159,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -213,7 +213,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 15, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_10.plan index d32c817823c9..c20e664c5b21 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_10.plan @@ -117,8 +117,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 17 @@ -131,7 +131,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 15, "Plans": [ { @@ -192,7 +192,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_11.plan index 9ed4119c17da..7a0bae093329 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_11.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 17 @@ -140,7 +140,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 15, "Plans": [ { @@ -201,7 +201,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_12.plan index 79d56cf922ad..f33c83023a1c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_12.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 17 @@ -140,7 +140,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 15, "Plans": [ { @@ -201,7 +201,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_3.plan index 5f16bb6e61af..e1a4cda39684 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_3.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_4.plan index ae9e57d22592..7c3038547cfe 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_4.plan @@ -76,7 +76,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -122,7 +122,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 7, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_7.plan index 18f794f75c7e..cb22e35ef976 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_7.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_8.plan index b0f0c307ec9a..f36c67acae81 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_8.plan @@ -85,8 +85,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -99,7 +99,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_9.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_9.plan index b91c3f1b8702..1503ae8bc5d7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_9.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join2.test_/query_9.plan @@ -117,8 +117,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 17 @@ -131,7 +131,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 15, "Plans": [ { @@ -192,7 +192,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_1.plan index 5578f0099498..0e8e0df01ea2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_1.plan @@ -74,7 +74,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_2.plan index 9a84265c6c9f..b774ed5b2ce1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_2.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_3.plan index 42b88afc1857..d2977f0bcae7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_3.plan @@ -71,7 +71,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_4.plan index 0f5b7034949b..771e8d692273 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_4.plan @@ -66,8 +66,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_1_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_1_0" } ], "PlanNodeId": 13 @@ -80,7 +80,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_5.plan index 70476973f580..11a2170ea7f7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_5.plan @@ -161,8 +161,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_1_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_1_0" } ], "PlanNodeId": 13 @@ -175,7 +175,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_6.plan index 336e511ca549..1683f3924116 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_6.plan @@ -81,8 +81,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -95,7 +95,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_7.plan index 20dfd41339b8..184af88cd4ed 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_7.plan @@ -78,8 +78,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -92,7 +92,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_8.plan index eedad10c7124..a5fb1b40fa5f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join3.test_/query_8.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { @@ -117,14 +117,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "ConstantExpr", - "Operators": [ - { - "Inputs": [], - "Iterator": "[{x: 1}]", - "Name": "Iterator" - } - ], + "Node Type": "Stage", "PlanNodeId": 5 } ] diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_1.plan index 7537c798742e..6e973a32a5f8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_1.plan @@ -70,7 +70,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 9, "Plans": [ { @@ -102,7 +102,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_13.plan index 9a239c3bba3b..b1ab09241e92 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_13.plan @@ -67,8 +67,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -81,7 +81,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_4.plan index 3ed3d093e2a7..f4431c8daadc 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_4.plan @@ -98,7 +98,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { @@ -130,7 +130,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 5, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_5.plan index 3ae40ca9e9d7..f37485f8d42c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_5.plan @@ -66,8 +66,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -80,7 +80,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_6.plan index f5dca5021d3e..f66ef2d6521e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_6.plan @@ -66,8 +66,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_2_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_2_0" } ], "PlanNodeId": 27 @@ -80,7 +80,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 25, "Plans": [ { @@ -203,8 +203,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 13 @@ -217,7 +217,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 11, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_8.plan index e483063285b0..d09a5aaa03b0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_8.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_9.plan b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_9.plan index 20a2e73c41c6..a3cb9a94411f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_9.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_postgres.TestPGSQL.test_sql_suite_plan-jointest_join4.test_/query_9.plan @@ -62,7 +62,7 @@ "PlanNodeType": "Connection", "Plans": [ { - "Node Type": "Collect", + "Node Type": "Stage", "PlanNodeId": 3, "Plans": [ { diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_101.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_101.plan index 933eb44641ad..1dbc02ebf5d1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_101.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_101.plan @@ -164,8 +164,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_103.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_103.plan index c6625ebede65..3820577bc7ac 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_103.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_103.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_11.plan index 9ab1537010d4..2c28de5e7674 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_11.plan @@ -201,8 +201,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_114.plan index e70e69265d33..ff08b327eda5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_114.plan @@ -131,8 +131,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_115.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_115.plan index 63696aa8c501..a0c21d38a281 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_115.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_115.plan @@ -192,8 +192,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_121.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_121.plan index 9ba61702ac21..882207d5dc29 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_121.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_121.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_13.plan index 4bcb021c4522..d8aa59a15eee 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_13.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_16.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_16.plan index 32488d482534..69eeb1c66956 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_16.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_16.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_17.plan index 447566aa85f4..57ed3f3ff542 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_17.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_18.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_18.plan index 546ec384e628..884930fa746c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_18.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_18.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_2.plan index e81eb10f02c3..68a33f6abdfa 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_2.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_20.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_20.plan index a628e5897be9..0f58983d7a96 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_20.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_20.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_25.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_25.plan index 8091d516791b..4ff02d7d9cce 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_25.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_25.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_31.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_31.plan index 7cb6fb1f8998..137d0634fd3e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_31.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_31.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_49.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_49.plan index 73c6f1361cb9..9f9527e2e54b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_49.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_49.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_74.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_74.plan index e81eb10f02c3..68a33f6abdfa 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_74.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_74.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_76.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_76.plan index c3b080c37cd3..b14724a0205a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_76.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_76.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_8.plan index 5d24181c028f..c2827282bfab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_8.plan @@ -167,8 +167,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_82.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_82.plan index 79933459c99d..7427ea385d16 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_82.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_82.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_85.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_85.plan index 40e84c34beb5..b361499ec0b8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_85.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-1.test_/query_85.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_102.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_102.plan index 4b89d71aa86d..e80460ba38d2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_102.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_102.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_110.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_110.plan index b9e2fba886ef..6d784d46846f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_110.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_110.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_112.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_112.plan index bf4b6932b93c..5da42ef64d6e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_112.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_112.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_113.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_113.plan index 4b4479d2dbc0..c174c5dd3806 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_113.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_113.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_130.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_130.plan index 32b2a07cf610..3f6bf48e4453 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_130.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_130.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_15.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_15.plan index bb72f5045d72..cad9018787e5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_15.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_15.plan @@ -152,8 +152,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_2.plan index c2fbd2062a7c..540594842dc1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_2.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_42.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_42.plan index 4d9a68c35284..f9aebeaf42be 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_42.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_42.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_44.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_44.plan index 8da756475c30..1cb62aac31af 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_44.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_44.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_54.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_54.plan index 4cdf4145c8f9..7f42eb9be336 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_54.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_54.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_55.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_55.plan index 5f8351a2d132..8a163b606620 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_55.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_55.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_60.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_60.plan index 60888ffb8637..a4074d503147 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_60.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_60.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_69.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_69.plan index 033af4d5837e..62db4e6368e3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_69.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_69.plan @@ -158,8 +158,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_7.plan index e8e96bc79749..54280e7fdae0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_7.plan @@ -131,8 +131,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_77.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_77.plan index 6595437cc3f4..219aba82be67 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_77.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_77.plan @@ -131,8 +131,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_8.plan index 52bb7470f1ee..414a03001b41 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_8.plan @@ -131,8 +131,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_83.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_83.plan index 118eed39c348..1851aa325953 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_83.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_83.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_89.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_89.plan index af89cead5ce6..f8a0526ef373 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_89.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_89.plan @@ -160,8 +160,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_90.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_90.plan index 182f8ce11643..76af5a2e04d6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_90.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_90.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_91.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_91.plan index 4c402af2eb3b..db6919d6a719 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_91.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_91.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_92.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_92.plan index 8216a1007e7e..fd808620c504 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_92.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_92.plan @@ -128,8 +128,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_97.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_97.plan index aa0ff23181af..06fb6c95fcaf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_97.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_97.plan @@ -113,8 +113,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_99.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_99.plan index f0bbd1db3fe2..74cce0089391 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_99.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-2.test_/query_99.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_1.plan index 844eb4db7dc3..a852d1656845 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_1.plan @@ -228,8 +228,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_108.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_108.plan index e609f44a03a7..509b7891531d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_108.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_108.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_110.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_110.plan index de6861109b9e..d827867fa087 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_110.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_110.plan @@ -113,8 +113,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_111.plan index f2c4a56831a5..71305790ff0d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_111.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_112.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_112.plan index a4f356012cbe..679481ddddca 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_112.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_112.plan @@ -146,8 +146,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_113.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_113.plan index 2494ba6d9e4d..36eb2cb41151 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_113.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_113.plan @@ -155,8 +155,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_114.plan index da9f37f520c8..df8e6385c3a9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_114.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_119.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_119.plan index 56c8255144a8..21aaad3a3ea1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_119.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_119.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_120.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_120.plan index 2ef0c49dc235..c83428eae6ad 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_120.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_120.plan @@ -210,8 +210,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_127.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_127.plan index 6b59522b9fc4..4a5b57fd9772 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_127.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_127.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_128.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_128.plan index ab6ab09c0890..400acb869b7f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_128.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_128.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_13.plan index a8db0ff28da5..5f6f23498926 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_13.plan @@ -134,8 +134,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_132.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_132.plan index 713c2605e2bf..48f9aa70a9bf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_132.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_132.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_137.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_137.plan index 27da9473e4d9..08282ad9bc9e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_137.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_137.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_141.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_141.plan index f34b377bedfd..b9ab27ecc685 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_141.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_141.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_19.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_19.plan index 9bbd8245a9d8..afcd8d24e6a2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_19.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_19.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_25.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_25.plan index ee92fdecbc33..e3ecfe909eb9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_25.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_25.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_3.plan index 603844d68c48..61d1404ea6c4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_3.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_33.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_33.plan index 73a63249887d..837151112f33 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_33.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_33.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_41.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_41.plan index b9b0a37c2af0..c77985929ef6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_41.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_41.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_49.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_49.plan index 88b4a1d7b286..b04993278680 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_49.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_49.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_52.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_52.plan index 938da2c89b8e..d03797d25fc0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_52.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_52.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_59.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_59.plan index cba1fde64fab..ff8a89e9f134 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_59.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_59.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_6.plan index 80274d8d4708..1972ab71d5ba 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_6.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_63.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_63.plan index d9a03ded19e6..5dea891a5f37 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_63.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_63.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_64.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_64.plan index 77dc6d18d8e4..bf8d2e839402 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_64.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_64.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_7.plan index 5e9d8ade10fe..d2b10a5fa830 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_7.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_72.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_72.plan index ee8a79587a54..5fc0bec74e33 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_72.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_72.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_8.plan index 0d2b660af217..34a886b4dd4a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_8.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_87.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_87.plan index 3a7734a8d322..6a131956b500 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_87.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_87.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_92.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_92.plan index a961742abcc6..7999290b1d8b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_92.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-3.test_/query_92.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_107.plan index 15a8d082b316..212cdd83e319 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_107.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_110.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_110.plan index 22a7c2c4001b..b994138fe9d2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_110.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_110.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_111.plan index 035c6988d371..c88b27f71004 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_111.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_114.plan index 41c057219929..53b3f6cc964b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_114.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_115.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_115.plan index 4421ffac449e..60a56385c2ae 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_115.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_115.plan @@ -207,8 +207,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_116.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_116.plan index 3f96c700593f..42ef801d4beb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_116.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_116.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_12.plan index 5d29f55dd7a0..024f2b68e344 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_12.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_123.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_123.plan index 495d6dce981f..411845d24e59 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_123.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_123.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_126.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_126.plan index 58f2ed9221f2..d7e6ae80b3b7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_126.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_126.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_133.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_133.plan index a03eb2baefb2..faebf3e794ab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_133.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_133.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_134.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_134.plan index 9b269dc97102..0fedf4d79c31 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_134.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_134.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_2.plan index 2c72fb8996bb..c826d8509b80 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_2.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_21.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_21.plan index 9042ac65ab4b..9497b7490945 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_21.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_21.plan @@ -210,8 +210,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_24.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_24.plan index 54595d68e97d..fc0bbe71050d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_24.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_24.plan @@ -148,8 +148,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_3.plan index e533dc3dcdb9..20cc33f9f5d3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_3.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_31.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_31.plan index 989e7a547e29..fb4b70305395 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_31.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_31.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_40.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_40.plan index 3b984e33a166..20527809ccee 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_40.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_40.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_44.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_44.plan index 9ba29210a7b6..ee6fab962b08 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_44.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_44.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_5.plan index 98165e7acfeb..070172db1682 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_5.plan @@ -133,8 +133,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_55.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_55.plan index 84fac8557c06..7dc3239130c5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_55.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_55.plan @@ -216,8 +216,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_60.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_60.plan index 9724b0069587..5667af156047 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_60.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_60.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_65.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_65.plan index bd13d05011ab..bf286b2013e6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_65.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_65.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_81.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_81.plan index 14a5c9c28cdb..f088dc7ab335 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_81.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_81.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_84.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_84.plan index d85ac5dc4769..950d62f31d3a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_84.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_84.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_89.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_89.plan index 4a4db31b83da..44d48eab199b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_89.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_89.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_90.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_90.plan index 2fe489e260c7..bde1e01ec0c4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_90.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_90.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_91.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_91.plan index 9206f7ff8c1d..433b773e289f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_91.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_91.plan @@ -131,8 +131,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_95.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_95.plan index 2411cc03b72a..3f551d324398 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_95.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_95.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_98.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_98.plan index 8ec2ddd79d75..ea83e26e6dda 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_98.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-4.test_/query_98.plan @@ -131,8 +131,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_11.plan index 1a3286639a63..9f384eddf21b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_11.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_24.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_24.plan index 6510d3aff5b3..ac300f9c6b69 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_24.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_24.plan @@ -137,8 +137,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_26.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_26.plan index fe05152e56f4..9a04deb58b9a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_26.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_26.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_29.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_29.plan index 7d15e8796032..bd50f6b0c1a2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_29.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_29.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_30.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_30.plan index 558032b65576..33e3ce64d669 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_30.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_30.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_34.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_34.plan index b4d6866be294..b7b3760f5a09 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_34.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_34.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_8.plan index b24c9f2b9c1f..1fe814290d60 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select1-5.test_/query_8.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_1.plan index 9f0d24adce2e..b846eb20f117 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_1.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_10.plan index 870a2517421b..1fc53b7d732d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_10.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_107.plan index 9f0d24adce2e..b846eb20f117 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_107.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_109.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_109.plan index ebc5c2c42ad7..d1e37727b62f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_109.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_109.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_11.plan index 5490f1335752..01b9d93686d3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_11.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_111.plan index 3d2b11a60259..016a734ac71d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_111.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_116.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_116.plan index feba8a57d2f0..20edcc37e4a0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_116.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_116.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_122.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_122.plan index 1aed039a13f2..4f3c29fc831f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_122.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_122.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_137.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_137.plan index a9f2ef2007a5..c1b342c2a302 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_137.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_137.plan @@ -128,8 +128,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_20.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_20.plan index 0da644093ab1..ebc16adc8b4c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_20.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_20.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_29.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_29.plan index 70b17675afd1..16bc422ee3d8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_29.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_29.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_40.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_40.plan index 3059b0d37034..892c6b717e3b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_40.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_40.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_48.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_48.plan index fcecb1bc4233..a658d5d1de21 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_48.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_48.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_51.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_51.plan index 734333e24d69..15b0a55c3942 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_51.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_51.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_52.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_52.plan index 1614a8fd7862..75f58158593c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_52.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_52.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_60.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_60.plan index 1a8850959263..46eceb0e6219 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_60.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_60.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_65.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_65.plan index e0f6510b40e2..6c96b772a3bf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_65.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_65.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_8.plan index 39aa6a093429..c8f8e9164218 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_8.plan @@ -216,8 +216,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_89.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_89.plan index 22e926924151..bd7908564f6d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_89.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_89.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_91.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_91.plan index d8e671425f12..878a89ab5ce0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_91.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_91.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_94.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_94.plan index 9eb4f9e944f0..44144e6aa275 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_94.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-1.test_/query_94.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_1.plan index 5ecb2a699322..6f5b6573236e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_1.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_102.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_102.plan index ce16bb6d87c3..243f9050c809 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_102.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_102.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_119.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_119.plan index 20555e5b0da2..6d96e8839573 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_119.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_119.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_121.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_121.plan index 2922bd524c0d..50acea9e1587 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_121.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_121.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_127.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_127.plan index d952bc0c5d45..8af08099a1b1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_127.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_127.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_133.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_133.plan index 6e1cade22d31..37e3b1a86754 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_133.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_133.plan @@ -158,8 +158,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_137.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_137.plan index acf2ce016e6e..79054c725d93 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_137.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_137.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_139.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_139.plan index e99d9e10afc6..757c4adf0985 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_139.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_139.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_143.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_143.plan index a253da15148d..323a007d5f36 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_143.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_143.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_155.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_155.plan index c60acf2d4ac9..6ace5116ac0d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_155.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_155.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_157.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_157.plan index 8a9a7a551c8b..14835060b8ea 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_157.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_157.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_2.plan index 398d1ea12663..37ef850eb886 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_2.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_25.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_25.plan index 435bea1b180e..08ad697e347b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_25.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_25.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_38.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_38.plan index 3c905a40b5b8..8a614bbf4dfe 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_38.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_38.plan @@ -134,8 +134,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_4.plan index ae35e6ba402e..f84e811a6a71 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_4.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_40.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_40.plan index 3d386596cbb8..72156b3bef39 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_40.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_40.plan @@ -192,8 +192,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_55.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_55.plan index 2c8afde25238..5422d6eb7b26 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_55.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_55.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_57.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_57.plan index 1062dbd6bf62..68e3442b4a78 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_57.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_57.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_61.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_61.plan index 69039d4861cf..03bb05826578 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_61.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_61.plan @@ -167,8 +167,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_67.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_67.plan index 5731af3146f3..bcf02a05554f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_67.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_67.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_75.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_75.plan index 0cf2bf83f192..ece5dcd046e5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_75.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_75.plan @@ -158,8 +158,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_77.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_77.plan index afff8aae6821..381d8fdb5102 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_77.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_77.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_85.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_85.plan index 6605d19e6153..1707edc684c2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_85.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_85.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_89.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_89.plan index 428fe410da52..7ca365f378c4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_89.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_89.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_91.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_91.plan index 85db4bd2e3f5..c32c54d6958f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_91.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_91.plan @@ -192,8 +192,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_97.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_97.plan index 69b91121a545..a0e0264027be 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_97.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_97.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_98.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_98.plan index c1bbbb82927e..936b02dc006f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_98.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-2.test_/query_98.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_10.plan index 2027f609c3e9..5d28837c3d03 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_10.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_107.plan index 25ab50637dca..ee7be1c3b574 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_107.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_111.plan index 765d12459d1a..7764dcbb2e14 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_111.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_112.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_112.plan index 26e8362ab395..8175dc9a894a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_112.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_112.plan @@ -174,8 +174,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_114.plan index 44995e3f2edb..b4353052c8e5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_114.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_117.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_117.plan index 4f09f7b097fc..99325c42c919 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_117.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_117.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_13.plan index 414b7c16d9fa..7c740da465f6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_13.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_135.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_135.plan index 5a69fe6f0c2c..36ed568b7e68 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_135.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_135.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_140.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_140.plan index d0bbad3417f4..b74ddd9bd969 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_140.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_140.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_144.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_144.plan index a6dd66f4b5bc..f9b4a496d8c3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_144.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_144.plan @@ -128,8 +128,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_145.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_145.plan index 8dcaac6a8140..5635fc83a11f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_145.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_145.plan @@ -128,8 +128,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_147.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_147.plan index c25206a9d0e3..2da48c081364 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_147.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_147.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_17.plan index 4e611aa00833..2f7f3f774380 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_17.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_2.plan index a8a4ba7db8f9..04f454ef3ad9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_2.plan @@ -192,8 +192,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_27.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_27.plan index 3f5bf1b0f0d3..cc231993d9e2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_27.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_27.plan @@ -128,8 +128,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_31.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_31.plan index c79aa477cb65..7dd2a6b8be5c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_31.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_31.plan @@ -130,8 +130,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_44.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_44.plan index 880b8745d195..92a7a274b609 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_44.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_44.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_48.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_48.plan index 5c321161dbbb..121b7a421695 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_48.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_48.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_5.plan index c19eb131a89c..1c1caf0d6eca 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_5.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_54.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_54.plan index fa3b7472f0c2..1027f596c2cb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_54.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_54.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_59.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_59.plan index 4dc1ff3bb593..223d69182c26 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_59.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_59.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_6.plan index e97d89b102e7..687def04bb72 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_6.plan @@ -149,8 +149,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_69.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_69.plan index faedc0e3dea7..dd4e4806cbd7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_69.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_69.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_70.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_70.plan index 99dd8a50e5d1..d8b90829c6ee 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_70.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_70.plan @@ -149,8 +149,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_74.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_74.plan index 3ec7193e006a..52b20670c38c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_74.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_74.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_78.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_78.plan index 8a829d73b13a..b250adee1a7c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_78.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_78.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_79.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_79.plan index 48ca58369787..a3f86defe195 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_79.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_79.plan @@ -201,8 +201,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_8.plan index 83c5edd9b0b1..5c555aceed8e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_8.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_82.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_82.plan index 108bf2f36971..0bcf5b48d514 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_82.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_82.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_9.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_9.plan index d9437f1fc873..abc3dde2721e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_9.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-3.test_/query_9.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_10.plan index 1cf477ae8a0a..7d1dee6ea1f2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_10.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_102.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_102.plan index 338d859e9c4f..93f6668e4b6f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_102.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_102.plan @@ -192,8 +192,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_107.plan index 0d8143eb1a39..67ea1fb8971b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_107.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_111.plan index 9855e878d82f..4791ff516186 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_111.plan @@ -225,8 +225,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_114.plan index ebde0145f577..88278816c03a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_114.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_134.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_134.plan index 6ba91c9b6119..8255d73eb01f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_134.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_134.plan @@ -192,8 +192,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_14.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_14.plan index 5badc00536de..e3422488664b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_14.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_14.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_19.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_19.plan index d7b86f75a199..86c645eb2a69 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_19.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_19.plan @@ -146,8 +146,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_2.plan index 4e67ff990e9b..4b32d6a56a59 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_2.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_20.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_20.plan index e595226e1b29..2c008673ad1a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_20.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_20.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_28.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_28.plan index 9bc9dba7d027..678723d478c0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_28.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_28.plan @@ -137,8 +137,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_33.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_33.plan index 76b7adcbd6ce..32409792e0be 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_33.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_33.plan @@ -155,8 +155,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_37.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_37.plan index d263d5eb442e..5efa747129ae 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_37.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_37.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_39.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_39.plan index 762df907a97d..2c9a0f74bcf8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_39.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_39.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_45.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_45.plan index b16dddab55a9..37afdd262e41 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_45.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_45.plan @@ -158,8 +158,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_52.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_52.plan index 2bbcf32087e9..d346ca3f461e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_52.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_52.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_61.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_61.plan index 86a320092d5e..eb358d0f8901 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_61.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_61.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_63.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_63.plan index 5101a7d1438e..0f435688c120 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_63.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_63.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_67.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_67.plan index 615e87aeb8c6..045e8695ab05 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_67.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_67.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_68.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_68.plan index 04cc7709183a..773c117e9381 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_68.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_68.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_70.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_70.plan index eff5d368a987..f4cc41e2ad91 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_70.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_70.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_71.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_71.plan index 6286e80287cc..52c60322f4b1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_71.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_71.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_74.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_74.plan index 95b22ddf9ed4..066464be001e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_74.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_74.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_8.plan index cfe4b8956f34..32e87264f53f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_8.plan @@ -207,8 +207,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_82.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_82.plan index 22cac872302d..47dde9202c3d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_82.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_82.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_84.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_84.plan index 96d83c7fe272..53055cc20358 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_84.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_84.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_86.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_86.plan index 68e92ef0d42f..f9b15249ee16 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_86.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_86.plan @@ -120,8 +120,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_89.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_89.plan index df96ed0acac4..db4bb46cb21c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_89.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_89.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_91.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_91.plan index 6ea985c56cb8..e349a94fcd42 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_91.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_91.plan @@ -170,8 +170,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_98.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_98.plan index d344d90ed1e0..231f9630deaa 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_98.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select2-4.test_/query_98.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_1.plan index 79e1ee84fc26..1134d7b78630 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_1.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_131.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_131.plan index 5ce4a84238aa..6356d13f72be 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_131.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_131.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_132.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_132.plan index 5ce4a84238aa..6356d13f72be 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_132.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_132.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_17.plan index 6f8b9dfd567e..c4e4b1c33236 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_17.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_18.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_18.plan index 3c39f10ea1da..9da2b8305964 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_18.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_18.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_19.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_19.plan index ce8913cf6001..beec11816c69 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_19.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_19.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_2.plan index 79e1ee84fc26..1134d7b78630 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_2.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_20.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_20.plan index 45717297f368..5939cee60f7d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_20.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_20.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_39.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_39.plan index 962d19d22524..2aa1ddb66ad6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_39.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_39.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_40.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_40.plan index 607134d6b34b..3611a616f622 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_40.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_40.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_83.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_83.plan index 7a49966772aa..3f51f3eaeb5a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_83.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_83.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_84.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_84.plan index cb892c9bf22e..4572e3d44372 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_84.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-1.test_/query_84.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_11.plan index f006e5a4c9d4..964109ee7e61 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_11.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_12.plan index e8e67f420537..1a07badac397 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_12.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_23.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_23.plan index a30baa1950a4..f36c96c9f459 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_23.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_23.plan @@ -174,8 +174,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_24.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_24.plan index ff83a7ee30c0..b7357e753a3d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_24.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_24.plan @@ -174,8 +174,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_69.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_69.plan index dbe7e642c18b..85e5bc088429 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_69.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_69.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_70.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_70.plan index 93d3dcca8048..caf5c15693e3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_70.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_70.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_75.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_75.plan index 2c09fdecd405..149bb8afbe8d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_75.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_75.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_76.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_76.plan index 93c7e6780cd0..dadca6934322 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_76.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_76.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_93.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_93.plan index ac1602ffa99f..3cef525f5b9c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_93.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_93.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_94.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_94.plan index a34600937fac..4469d7e5fcd6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_94.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_94.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_97.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_97.plan index 53f2f1bca94c..d025851bf7bc 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_97.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_97.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_98.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_98.plan index 54450afe62d3..615a27f61321 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_98.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-10.test_/query_98.plan @@ -171,8 +171,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_105.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_105.plan index 53f588388a2d..d23ce041ebd1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_105.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_105.plan @@ -155,8 +155,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_106.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_106.plan index 53f588388a2d..d23ce041ebd1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_106.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_106.plan @@ -155,8 +155,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_19.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_19.plan index 769663913034..eb2541b8d22e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_19.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_19.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_20.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_20.plan index 769663913034..eb2541b8d22e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_20.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_20.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_31.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_31.plan index eff01bbae90e..9a08fa2fb3c6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_31.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_31.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_32.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_32.plan index eff01bbae90e..9a08fa2fb3c6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_32.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_32.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_43.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_43.plan index 37eac4400ecb..eada1e1b40e7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_43.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_43.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_44.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_44.plan index d84191ed2e47..d55553c6a9aa 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_44.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_44.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_45.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_45.plan index 9135b6b9af5c..c6b302cd33bb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_45.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_45.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_46.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_46.plan index 9135b6b9af5c..c6b302cd33bb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_46.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_46.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_47.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_47.plan index 6086129bee73..a5ce5d9232bb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_47.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_47.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_48.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_48.plan index 37b4c2c317b2..5ebf726d8d13 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_48.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_48.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_5.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_5.plan index 07bb06791ea2..a4b74defe84f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_5.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_5.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_6.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_6.plan index f76f16620f52..41d5071be1b8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_6.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-11.test_/query_6.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_105.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_105.plan index 436ecd04d5ae..738e77b13722 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_105.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_105.plan @@ -201,8 +201,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_106.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_106.plan index f6ce570a2b8b..dbe1c74b6fbe 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_106.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_106.plan @@ -201,8 +201,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_107.plan index 537f43984efe..1a4dce4a97ad 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_107.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_108.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_108.plan index bdd12bfbad0a..e8e961d44a41 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_108.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_108.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_109.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_109.plan index 6353ed509578..3ef43160a758 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_109.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_109.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_110.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_110.plan index cb566a8bd09e..553abefb3fd6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_110.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_110.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_111.plan index c9c77fb105a8..2e434d99ceb4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_111.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_112.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_112.plan index 62348532928d..e39a41a86b77 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_112.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_112.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_139.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_139.plan index f436fb383336..fcf8ef38d35e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_139.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_139.plan @@ -213,8 +213,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_140.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_140.plan index 28963eaaa3c5..aea0d3eb1d43 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_140.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_140.plan @@ -213,8 +213,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_37.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_37.plan index 579ea1309477..87a8bd671144 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_37.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_37.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_38.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_38.plan index 44d3031ce41a..cfd6af781888 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_38.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_38.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_45.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_45.plan index 7e3e56aaae99..77fe0f4bf5ab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_45.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_45.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_46.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_46.plan index 7e3e56aaae99..77fe0f4bf5ab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_46.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_46.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_47.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_47.plan index 2892911f3f30..c9b32aa6618f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_47.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_47.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_48.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_48.plan index 08a3cb67835d..b0560c05d647 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_48.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_48.plan @@ -126,8 +126,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_73.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_73.plan index b2b4b62894df..f7f3095aceb9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_73.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_73.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_74.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_74.plan index fe2aa0036d0a..327cd91646e9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_74.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_74.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_77.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_77.plan index 83dcbaca1785..c7330332292a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_77.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_77.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_78.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_78.plan index 83dcbaca1785..c7330332292a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_78.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_78.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_79.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_79.plan index 881f4d613da9..556a55bfc2a4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_79.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_79.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_80.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_80.plan index caa9a6fbb951..64e637c8d9c8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_80.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_80.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_81.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_81.plan index ef4ff90cc87d..14c7b26b82ff 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_81.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_81.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_82.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_82.plan index 9f3f283a5109..a8a48d099abc 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_82.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_82.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_85.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_85.plan index be178fabd65c..99fcd8ed4025 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_85.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_85.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_86.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_86.plan index 205ded097c7d..4037d57a57bb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_86.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-12.test_/query_86.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_10.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_10.plan index e5b8687c31c6..4270e739a42c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_10.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_10.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_11.plan index 037f7f82aa3b..bed264815314 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_11.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_12.plan index 4bc826f72a7e..46a0ecc65748 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_12.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_13.plan index e158dff44e35..3bdc1c086e6b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_13.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_14.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_14.plan index 219f99cc395c..f5eb73415817 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_14.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_14.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_15.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_15.plan index fadd40577cdc..06f3e6623602 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_15.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_15.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_16.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_16.plan index 67381a32079b..c614546de76a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_16.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_16.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_17.plan index 8a0b3be7357b..d3e1f070e653 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_17.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_24.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_24.plan index 4857e589f283..9d6a447a72b4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_24.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_24.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_25.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_25.plan index b7eea74580af..a45799359ae9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_25.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_25.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_26.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_26.plan index fea42cb346e7..3ac024a64c3c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_26.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_26.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_27.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_27.plan index fea42cb346e7..3ac024a64c3c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_27.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_27.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_34.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_34.plan index 32fbb55105e7..85a5b36c2de4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_34.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_34.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_35.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_35.plan index 107f8b5dfd58..eaa38382edc8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_35.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_35.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_36.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_36.plan index 33ec7bfd9d9c..c84f65f7b7f6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_36.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_36.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_37.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_37.plan index 917597715f67..32357cc86ad4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_37.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_37.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_38.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_38.plan index 06369954e45c..67922b37c044 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_38.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_38.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_39.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_39.plan index 8789e9875ff5..6c490a6f721f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_39.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_39.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_62.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_62.plan index 3fe81f781904..d7ef808dc630 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_62.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_62.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_63.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_63.plan index 3fe81f781904..d7ef808dc630 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_63.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_63.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_8.plan index cab385839d93..56e8b3001d06 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_8.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_9.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_9.plan index 0647bb30fc95..b379187c8c8f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_9.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-13.test_/query_9.plan @@ -195,8 +195,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_22.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_22.plan index 1fdb66fd0a0b..71b951fe582e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_22.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_22.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_23.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_23.plan index 2ebf81a8fc16..0e9b3375d7f4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_23.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_23.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_28.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_28.plan index 906b3e661b34..fe547fa469fc 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_28.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_28.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_29.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_29.plan index 469d7bcac3a0..8c1ca8a69b58 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_29.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_29.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_30.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_30.plan index 00495e6c2abe..d6da6660fd96 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_30.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_30.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_31.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_31.plan index aa4d6288eacd..fa847496fff6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_31.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_31.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_60.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_60.plan index 25fb0670bbfd..0892b83677e1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_60.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_60.plan @@ -160,8 +160,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_61.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_61.plan index 53a9706dcd27..9331f19d3f00 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_61.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_61.plan @@ -160,8 +160,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_76.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_76.plan index d956827e354d..d612a234304f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_76.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_76.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_77.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_77.plan index 54e3ca9d2fe7..bd9729522432 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_77.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_77.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_78.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_78.plan index 74c27bc8903a..effd06261646 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_78.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_78.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_79.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_79.plan index 44b4a1a9e954..ce9a16c8c08f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_79.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_79.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_80.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_80.plan index d427422734a0..2e08be4758ac 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_80.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_80.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_81.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_81.plan index bb83b83546c9..ea8010c785d4 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_81.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_81.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_88.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_88.plan index 408f9737f3f9..f064b47fa599 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_88.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_88.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_89.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_89.plan index abec8ee0fe48..5f65f114fd42 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_89.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-14.test_/query_89.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_11.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_11.plan index bff75eff7aba..19a1b546a9a7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_11.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_11.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_12.plan index 6b0262ceddc9..1275597a53a3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_12.plan @@ -132,8 +132,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_17.plan index 379c824593cc..084f33fcc494 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_17.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_18.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_18.plan index 379c824593cc..084f33fcc494 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_18.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_18.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_19.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_19.plan index a1921234dfb4..0222dbc754c0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_19.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_19.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_20.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_20.plan index a1921234dfb4..0222dbc754c0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_20.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_20.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_3.plan index 82a46ea485e4..fd832ba9a4cf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_3.plan @@ -115,8 +115,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_4.plan index 6128ba371cee..a42cfb8a011d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-15.test_/query_4.plan @@ -115,8 +115,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_100.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_100.plan index 0a8d2cdf7933..837f85004edf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_100.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_100.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_101.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_101.plan index 9ab421e666b2..113657471c08 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_101.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_101.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_12.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_12.plan index d4e4e66cc844..882cf4814267 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_12.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_12.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_13.plan index c42795ec2364..159ec334e45a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_13.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_14.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_14.plan index cfaf4904d519..11ec36e51f22 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_14.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_14.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_140.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_140.plan index 6beeb9c758d4..e8dcad44e2e6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_140.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_140.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_141.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_141.plan index b49db1caf363..d95ec127cfc9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_141.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_141.plan @@ -168,8 +168,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_15.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_15.plan index c68d91ffcd5f..e8622f189845 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_15.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_15.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_16.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_16.plan index f937c16a0c14..b0a5e6861cfa 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_16.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_16.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_17.plan index 130bdcc53436..27eaa25b0561 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_17.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_22.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_22.plan index cc671b9da63b..bac9c0037725 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_22.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_22.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_23.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_23.plan index ddbb5cfc9583..ea89d0485eac 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_23.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_23.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_24.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_24.plan index 2779b8b42579..874b47ad280d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_24.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_24.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_25.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_25.plan index 95f144f7a6a6..bcba40d8f23a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_25.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_25.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_40.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_40.plan index 4720d7a1534c..939dfecdb949 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_40.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_40.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_41.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_41.plan index 26f61382871c..204a04e117ee 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_41.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_41.plan @@ -177,8 +177,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_44.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_44.plan index b3df32f11600..d7c3a0c0d19b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_44.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_44.plan @@ -142,8 +142,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_45.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_45.plan index d817fecfb9d8..df86cac29d96 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_45.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_45.plan @@ -142,8 +142,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_54.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_54.plan index ea3f91471ba8..fc74491e7913 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_54.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_54.plan @@ -137,8 +137,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_55.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_55.plan index aacc413db17e..18e81a196d5d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_55.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_55.plan @@ -137,8 +137,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_64.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_64.plan index 62f9dd439641..acf32139fb1a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_64.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_64.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_65.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_65.plan index 5ee0dff84599..5fb2d25e3fda 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_65.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_65.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_66.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_66.plan index b0d6a89572f0..512159a9fc1b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_66.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_66.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_67.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_67.plan index cc1256308961..bdfa7f90fddf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_67.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_67.plan @@ -135,8 +135,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_72.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_72.plan index 0cc98db2dfc2..23656ea06507 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_72.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_72.plan @@ -204,8 +204,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_73.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_73.plan index 0af550795b6f..0c630753b4f0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_73.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_73.plan @@ -204,8 +204,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_94.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_94.plan index faf3c36d3f7c..e4d846129c50 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_94.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_94.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_95.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_95.plan index 608d0f678b65..8fc833213467 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_95.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_95.plan @@ -122,8 +122,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_98.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_98.plan index 4a23b56f0662..2477278cc7c3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_98.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_98.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_99.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_99.plan index 78e3346b16fd..26ac9f17390e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_99.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-2.test_/query_99.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_1.plan index f3d7915c15d6..fb0da1f98b0a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_1.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_100.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_100.plan index 21bf4e1fac42..61cb9fa8ca95 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_100.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_100.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_101.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_101.plan index 53277ed47562..9e776f51c095 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_101.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_101.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_102.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_102.plan index 814e6e8f35ef..ee264a4e0faf 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_102.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_102.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_107.plan index 1165594a9f0b..8271fed19f7f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_107.plan @@ -134,8 +134,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_108.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_108.plan index 2e2910fa4060..db86e0d673fc 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_108.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_108.plan @@ -134,8 +134,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_119.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_119.plan index 5525a5db82e2..d1e07635028f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_119.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_119.plan @@ -146,8 +146,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_120.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_120.plan index cd7fa3f3c1bb..fc2417ad454c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_120.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_120.plan @@ -146,8 +146,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_2.plan index f56e3a23149b..f8cc1543e7ee 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_2.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_29.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_29.plan index dc6c05f77c22..3b5e02814265 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_29.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_29.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_3.plan index 0403c0143494..6ef8047e9233 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_3.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_30.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_30.plan index 72e42aef06bb..6105d50b2651 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_30.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_30.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_4.plan index 777872ad5b8a..3adddc490a14 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_4.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_43.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_43.plan index bb4524c5f025..61151eadf110 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_43.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_43.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_44.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_44.plan index 574817788295..38b00c1d7175 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_44.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_44.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_45.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_45.plan index 2d07c8c52a11..49e80ddc4e52 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_45.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_45.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_46.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_46.plan index 5fd98e8df46c..e149e8892031 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_46.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_46.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_97.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_97.plan index 3321417b22ed..8d204f113cc0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_97.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_97.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_98.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_98.plan index 11d095b30c1e..2ecd97e866f8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_98.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_98.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_99.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_99.plan index 2398ffbce8a4..d58b854634c5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_99.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-3.test_/query_99.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_16.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_16.plan index 3fb2a6e46539..8a567f926264 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_16.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_16.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_17.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_17.plan index 12aff0ad4a2c..e362941cf8db 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_17.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_17.plan @@ -180,8 +180,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_28.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_28.plan index d2d158748825..857e2afa5e18 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_28.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_28.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_29.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_29.plan index a88c0ed07bdc..3fc5bd2a121a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_29.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_29.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_40.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_40.plan index 6ed3d53450f8..9063cc0d078a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_40.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_40.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_41.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_41.plan index 20ed65c4a3df..482753503971 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_41.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_41.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_42.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_42.plan index 1ea1e96e1382..b40b055f9f11 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_42.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_42.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_43.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_43.plan index 1ea1e96e1382..b40b055f9f11 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_43.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_43.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_68.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_68.plan index da446ef3ddc2..edd334fe0bcb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_68.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_68.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_69.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_69.plan index 32fc554e6962..bb7539a10090 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_69.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_69.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_70.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_70.plan index 1e8324a685dd..4b32df084b96 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_70.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_70.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_71.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_71.plan index ab8ef707dd15..926ce8906212 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_71.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_71.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_76.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_76.plan index eb91b41631f5..bfc081a42181 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_76.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_76.plan @@ -113,8 +113,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_77.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_77.plan index eb91b41631f5..bfc081a42181 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_77.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-4.test_/query_77.plan @@ -113,8 +113,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_13.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_13.plan index b858f7e4c551..ff42855af199 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_13.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_13.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_14.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_14.plan index b858f7e4c551..ff42855af199 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_14.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_14.plan @@ -125,8 +125,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_21.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_21.plan index 8570d8222f06..111f5da4a0ff 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_21.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_21.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_22.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_22.plan index 8570d8222f06..111f5da4a0ff 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_22.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_22.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_27.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_27.plan index cd24b4b241b6..e51307aa24ab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_27.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_27.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_28.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_28.plan index 5f9ad6e3a6dc..0b1a50308df9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_28.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_28.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_55.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_55.plan index 261d81bc2516..c293f7009bf9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_55.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_55.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_56.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_56.plan index 261d81bc2516..c293f7009bf9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_56.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_56.plan @@ -109,8 +109,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_75.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_75.plan index 9f16937fd9de..df9eb5723582 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_75.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_75.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_76.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_76.plan index fde9f0a61a6f..ba2a83011260 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_76.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-5.test_/query_76.plan @@ -150,8 +150,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_103.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_103.plan index 7b0376800fd4..b3103254e94a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_103.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_103.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_104.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_104.plan index e0d9f8953ba0..a2c5d3ec51a8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_104.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_104.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_107.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_107.plan index 989c8efd0dbd..909bcfb2e8e2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_107.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_107.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_108.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_108.plan index 311d94acb667..50d5844bd4e5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_108.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_108.plan @@ -140,8 +140,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_131.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_131.plan index e3ea34b424c7..24eb7909b832 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_131.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_131.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_132.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_132.plan index d270ca479b8b..6b9144ca94a6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_132.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_132.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_133.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_133.plan index 2fcbf5334981..6520c17e204d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_133.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_133.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_134.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_134.plan index ab9406fd13f6..cc94947f58fe 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_134.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_134.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_33.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_33.plan index 80c5e5c65cc3..f37795b79cb9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_33.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_33.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_34.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_34.plan index 80c5e5c65cc3..f37795b79cb9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_34.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_34.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_35.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_35.plan index 4e823491a485..e697b3f35088 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_35.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_35.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_36.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_36.plan index 4e823491a485..e697b3f35088 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_36.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_36.plan @@ -119,8 +119,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_49.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_49.plan index 8ef04c798aa2..32d88798322e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_49.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_49.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_50.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_50.plan index f9cf7cb30969..10c78b88db32 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_50.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-6.test_/query_50.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_1.plan index cf5aad507104..1fce0ad617fa 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_1.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_109.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_109.plan index 328d206cd4dd..aa061423f706 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_109.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_109.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_110.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_110.plan index f58363eee1f8..7b64f844f43f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_110.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_110.plan @@ -129,8 +129,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_113.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_113.plan index 8d88c8122469..c10539c42e8d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_113.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_113.plan @@ -174,8 +174,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_114.plan index 5329db1441f2..492736734ea8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_114.plan @@ -174,8 +174,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_119.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_119.plan index f551667b38d4..a12fa1b3b4f9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_119.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_119.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_120.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_120.plan index 4ef9db2ea81d..bcb9899e2cb7 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_120.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_120.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_121.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_121.plan index 7346df23aaaf..5b329fad61e6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_121.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_121.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_122.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_122.plan index ba7093eae6e9..a8478bf3e72d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_122.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_122.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_123.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_123.plan index 9686c6f86b32..037e659433d9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_123.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_123.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_2.plan index 0d4f59b5e889..47486283fc65 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-7.test_/query_2.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_1.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_1.plan index 55e35743c05d..05ff54260c09 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_1.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_1.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_108.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_108.plan index e55997b1a370..7edbebba7fb5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_108.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_108.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_109.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_109.plan index 4fc3a6acca14..3710b360fdcb 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_109.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_109.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_110.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_110.plan index 0d9137a80126..266691b77967 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_110.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_110.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_111.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_111.plan index e9a6c64bf1f6..bf12cc981f1e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_111.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_111.plan @@ -159,8 +159,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_114.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_114.plan index 599382dcc569..64cf7d1c78b3 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_114.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_114.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_115.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_115.plan index e14b39bb0123..cbc8a6e8aa5e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_115.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_115.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_116.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_116.plan index 1102731bdc60..1dd3cb37bab6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_116.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_116.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_117.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_117.plan index c1e14b5c836b..314b0dd4ff43 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_117.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_117.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_120.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_120.plan index 7e8a63b503ce..3035fca6334e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_120.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_120.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_121.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_121.plan index 1c82dfae3d7c..e326d2ed0cf5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_121.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_121.plan @@ -123,8 +123,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_2.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_2.plan index 6c3e42b12988..ddc8c5ac8b50 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_2.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_2.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_3.plan index 9aed80fdb836..c8b51a5e5f32 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_3.plan @@ -183,8 +183,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_58.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_58.plan index 5fe3bc4405fe..7f8517ee1b4f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_58.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_58.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_59.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_59.plan index 8bffa731413d..3fbcb034d3b2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_59.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_59.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_60.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_60.plan index 8305566dcfa1..84626403aaec 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_60.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_60.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_61.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_61.plan index 0f62f7828a9c..731ed2c38f2f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_61.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_61.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_62.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_62.plan index 71201a966be5..34606fcd9fc1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_62.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_62.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_63.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_63.plan index c556cf3ee785..f4c438060c94 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_63.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_63.plan @@ -198,8 +198,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_80.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_80.plan index 45c29f354bc8..adfb59336e73 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_80.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_80.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_81.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_81.plan index 0d882425f315..f31fcbdf238a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_81.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_81.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_82.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_82.plan index 8952327df665..dd91d70cb627 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_82.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_82.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_83.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_83.plan index 65c8789773bb..25d4cd161590 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_83.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_83.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_84.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_84.plan index fd789e7841cd..882c5a4842f2 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_84.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_84.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_85.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_85.plan index f3daf5c72e50..9f7065025488 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_85.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_85.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_86.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_86.plan index 4d232025d633..fbdcc1f83857 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_86.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_86.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_87.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_87.plan index cf84d2dccd9c..a1bbf7e180ca 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_87.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_87.plan @@ -141,8 +141,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_92.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_92.plan index 95ee2e080984..0cbe4595f642 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_92.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_92.plan @@ -142,8 +142,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_93.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_93.plan index 817c67c2b756..1293a4a67ae9 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_93.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-8.test_/query_93.plan @@ -142,8 +142,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_100.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_100.plan index 9bfe0fc0b506..0ba06f0b7cca 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_100.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_100.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_101.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_101.plan index 90831f7cd2f3..879ca776713d 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_101.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_101.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_102.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_102.plan index 8c760e465ffc..b5f363d106b8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_102.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_102.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_129.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_129.plan index 221aba0a53ed..93f05636f6ab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_129.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_129.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_130.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_130.plan index 60d6a2565fad..472b36117f85 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_130.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_130.plan @@ -147,8 +147,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_133.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_133.plan index 664871e4e28f..e30a22c742a5 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_133.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_133.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_134.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_134.plan index ca844aa6fce0..412147c0a91c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_134.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_134.plan @@ -165,8 +165,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_21.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_21.plan index cea68cfae8cc..6aebbf2c0c9f 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_21.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_21.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_22.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_22.plan index 540d824e8fda..1b9da334848a 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_22.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_22.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_23.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_23.plan index bbf55ad6f145..da569fc5d8f0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_23.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_23.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_24.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_24.plan index 49d93fa31d35..50d7520a538e 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_24.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_24.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_27.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_27.plan index e4c917bda14b..2bdae9ffbc73 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_27.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_27.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_28.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_28.plan index b4feff65be87..405b94998758 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_28.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_28.plan @@ -186,8 +186,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_29.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_29.plan index 9620f79335df..808755d4d289 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_29.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_29.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_3.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_3.plan index 40f6f7c16826..3bdf066e8ef6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_3.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_3.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_30.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_30.plan index b96ad64f7a41..e9696af33423 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_30.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_30.plan @@ -138,8 +138,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_31.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_31.plan index e03c45be7532..24c514a221b0 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_31.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_31.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_32.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_32.plan index dca2fc06f9db..11b147a4e883 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_32.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_32.plan @@ -118,8 +118,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_4.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_4.plan index 6320d9bd9764..9322ba3278f1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_4.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_4.plan @@ -153,8 +153,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_63.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_63.plan index 837448b9a2d3..2df9b6a37239 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_63.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_63.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_64.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_64.plan index c2503eab5a57..88a86d3b8951 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_64.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_64.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_65.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_65.plan index 7adb29285863..2ed43c85b8da 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_65.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_65.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_66.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_66.plan index b60bf16fdaec..7bba6c9d9fe6 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_66.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_66.plan @@ -144,8 +144,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_7.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_7.plan index 6d7bc3f8ceaa..a8d811514f9c 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_7.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_7.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_73.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_73.plan index b7dfa5138f75..1e98049688c8 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_73.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_73.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_74.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_74.plan index 028d43d28f0e..a433751930b1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_74.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_74.plan @@ -162,8 +162,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_8.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_8.plan index bdf2c19c2f5b..289c4968502b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_8.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_8.plan @@ -189,8 +189,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_91.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_91.plan index 4dcce72a41ed..09d8f8b84dab 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_91.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_91.plan @@ -116,8 +116,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_92.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_92.plan index 51e751011005..a62a408c63c1 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_92.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_92.plan @@ -116,8 +116,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_93.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_93.plan index 1af87a58651a..c37059d29688 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_93.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_93.plan @@ -173,8 +173,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_94.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_94.plan index a4aa19fb7884..6c976f49744b 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_94.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_94.plan @@ -173,8 +173,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6 diff --git a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_99.plan b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_99.plan index 58a8f91709ec..82bb2cf90c27 100644 --- a/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_99.plan +++ b/ydb/tests/functional/suite_tests/canondata/test_sql_logic.TestSQLLogic.test_sql_suite_plan-select3-9.test_/query_99.plan @@ -156,8 +156,8 @@ "Operators": [ { "Inputs": [], - "Iterator": "precompute_0_0", - "Name": "Iterator" + "Name": "ToFlow", + "ToFlow": "precompute_0_0" } ], "PlanNodeId": 6