diff options
author | hor911 <hor911@ydb.tech> | 2023-10-19 13:40:22 +0300 |
---|---|---|
committer | hor911 <hor911@ydb.tech> | 2023-10-19 14:12:16 +0300 |
commit | bc9bf98d406f1786bc879a38d42edd98b1be12cc (patch) | |
tree | 82c54bc63192ba80c73a4bf7b548439fe1b8cf8e | |
parent | ebcfb905dddb333a8d15a9bc2cc56eb78e7ad311 (diff) | |
download | ydb-bc9bf98d406f1786bc879a38d42edd98b1be12cc.tar.gz |
New DQ statistics
1. Выравниваю уровни KQP и DQ - добавляю DQ_STATS_MODE_FULL
2. При запуске графа DQ (TDqExecuter) можно указать требуемый уровень сбора статистики (сейчас он почти всегда PROFILE)
3. Унифицирую метрики всех входов и выходов TaskRunner по (1) трафику и (2) таймингам
4. Оптимизирую собираемую статистику исходя из концепции: NONE - нет вообще, BASIC - per query/graph, FULL - агрегированная per stage, PROFILE - исходные метрики от каждой task
94 files changed, 2821 insertions, 1631 deletions
diff --git a/ydb/core/fq/libs/actors/run_actor.cpp b/ydb/core/fq/libs/actors/run_actor.cpp index 93c9f4c417..4195a17ee3 100644 --- a/ydb/core/fq/libs/actors/run_actor.cpp +++ b/ydb/core/fq/libs/actors/run_actor.cpp @@ -57,6 +57,7 @@ #include <ydb/core/fq/libs/common/compression.h> #include <ydb/core/fq/libs/common/entity_id.h> #include <ydb/core/fq/libs/compute/common/pinger.h> +#include <ydb/core/fq/libs/compute/common/utils.h> #include <ydb/core/fq/libs/control_plane_storage/control_plane_storage.h> #include <ydb/core/fq/libs/control_plane_storage/events/events.h> #include <ydb/core/fq/libs/control_plane_storage/util.h> @@ -337,6 +338,25 @@ public: , Compressor(Params.Config.GetCommon().GetQueryArtifactsCompressionMethod(), Params.Config.GetCommon().GetQueryArtifactsCompressionMinSize()) { QueryCounters.SetUptimePublicAndServiceCounter(0); + + switch (Params.Config.GetControlPlaneStorage().GetStatsMode()) { + case Ydb::Query::StatsMode::STATS_MODE_NONE: + StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_NONE; + break; + case Ydb::Query::StatsMode::STATS_MODE_BASIC: + StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_BASIC; + break; + case Ydb::Query::StatsMode::STATS_MODE_FULL: + StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_FULL; + break; + case Ydb::Query::StatsMode::STATS_MODE_PROFILE: + StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_PROFILE; + break; + case Ydb::Query::StatsMode::STATS_MODE_UNSPECIFIED: + default: + StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_FULL; + break; + } } static constexpr char ActorName[] = "YQ_RUN_ACTOR"; @@ -928,6 +948,11 @@ private: } void Handle(TEvDqStats::TPtr& ev) { + + if (!CollectBasic()) { + return; + } + auto& proto = ev->Get()->Record; TString GraphKey; @@ -1050,26 +1075,26 @@ private: if (Children.empty()) { if (Name.EndsWith("Us")) { // TDuration writer.OnKeyedItem("sum"); - writer.OnStringScalar(TDuration::MicroSeconds(Sum).ToString()); + writer.OnStringScalar(FormatDurationUs(Sum)); writer.OnKeyedItem("count"); writer.OnInt64Scalar(Count); writer.OnKeyedItem("avg"); - writer.OnStringScalar(TDuration::MicroSeconds(Avg).ToString()); + writer.OnStringScalar(FormatDurationUs(Avg)); writer.OnKeyedItem("max"); - writer.OnStringScalar(TDuration::MicroSeconds(Max).ToString()); + writer.OnStringScalar(FormatDurationUs(Max)); writer.OnKeyedItem("min"); - writer.OnStringScalar(TDuration::MicroSeconds(Min).ToString()); + writer.OnStringScalar(FormatDurationUs(Min)); } else if (Name.EndsWith("Ms")) { // TInstant writer.OnKeyedItem("sum"); writer.OnStringScalar("N/A"); writer.OnKeyedItem("count"); writer.OnInt64Scalar(Count); writer.OnKeyedItem("avg"); - writer.OnStringScalar(TInstant::MilliSeconds(Avg).FormatLocalTime("%H:%M:%S.%ms")); + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(Avg))); writer.OnKeyedItem("max"); - writer.OnStringScalar(TInstant::MilliSeconds(Max).FormatLocalTime("%H:%M:%S.%ms")); + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(Max))); writer.OnKeyedItem("min"); - writer.OnStringScalar(TInstant::MilliSeconds(Min).FormatLocalTime("%H:%M:%S.%ms")); + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(Min))); } else { writer.OnKeyedItem("sum"); writer.OnInt64Scalar(Sum); @@ -1195,9 +1220,11 @@ private: QueryStateUpdateRequest.mutable_result_id()->set_value(Params.ResultId); - TString statistics; - if (SaveAndPackStatistics("Graph=" + ToString(DqGraphIndex), result.metric(), statistics)) { - QueryStateUpdateRequest.set_statistics(statistics); + if (CollectBasic()) { + TString statistics; + if (SaveAndPackStatistics("Graph=" + ToString(DqGraphIndex), result.metric(), statistics)) { + QueryStateUpdateRequest.set_statistics(statistics); + } } KillExecuter(); } @@ -1249,9 +1276,11 @@ private: queryResult.SetSuccess(); } - TString statistics; - if (SaveAndPackStatistics("Precompute=" + ToString(it->second.Index), result.metric(), statistics)) { - QueryStateUpdateRequest.set_statistics(statistics); + if (CollectBasic()) { + TString statistics; + if (SaveAndPackStatistics("Precompute=" + ToString(it->second.Index), result.metric(), statistics)) { + QueryStateUpdateRequest.set_statistics(statistics); + } } queryResult.AddIssues(issues); @@ -1553,6 +1582,8 @@ private: auto& commonTaskParams = *request.MutableCommonTaskParams(); commonTaskParams["fq.job_id"] = Params.JobId; commonTaskParams["fq.restart_count"] = ToString(Params.RestartCount); + request.SetStatsMode(StatsMode); + NTasksPacker::UnPack(*request.MutableTask(), dqGraphParams.GetTasks(), dqGraphParams.GetStageProgram()); Send(ExecuterId, new NYql::NDqs::TEvGraphRequest(request, ControlId, resultId)); LOG_D("Executer: " << ExecuterId << ", Controller: " << ControlId << ", ResultIdActor: " << resultId); @@ -2171,6 +2202,10 @@ private: << " }"); } + bool CollectBasic() { + return StatsMode >= NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_BASIC; + } + private: TActorId FetcherId; TActorId ProgramRunnerId; @@ -2202,8 +2237,10 @@ private: const ui64 MaxTasksPerOperation; const TCompressor Compressor; - // Consumers creation + NYql::NDqProto::EDqStatsMode StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_NONE; TMap<TString, TString> Statistics; + + // Consumers creation NActors::TActorId ReadRulesCreatorId; // Rate limiter resource creation diff --git a/ydb/core/fq/libs/compute/common/utils.cpp b/ydb/core/fq/libs/compute/common/utils.cpp index 2c5ad65a74..c8514927de 100644 --- a/ydb/core/fq/libs/compute/common/utils.cpp +++ b/ydb/core/fq/libs/compute/common/utils.cpp @@ -5,65 +5,192 @@ namespace NFq { -void WriteNamedNode(NYson::TYsonWriter& writer, NJson::TJsonValue& node, const TString& name) { +using TAggregates = std::map<TString, std::optional<ui64>>; + +struct TTotalStatistics { + ui64 MaxMemoryUsage = 0; + ui64 CpuTimeUs = 0; + ui64 SourceCpuTimeUs = 0; + ui64 TotalInputRows = 0; + ui64 TotalInputBytes = 0; + ui64 TotalOutputRows = 0; + ui64 TotalOutputBytes = 0; + ui64 TotalIngressBytes = 0; + TAggregates Aggregates; +}; + +TString FormatDurationMs(ui64 durationMs) { + TStringBuilder builder; + + auto seconds = durationMs / 1'000; + if (seconds >= 60) { + builder << (seconds / 60) << "m " << (seconds % 60) << "s"; + } else { + auto hundredths = (durationMs % 1'000) / 10; + builder << seconds << "."; + if (hundredths < 10) { + builder << '0'; + } + builder << hundredths << "s"; + } + + return builder; +} + +TString FormatDurationUs(ui64 durationUs) { + return FormatDurationMs(durationUs / 1000); +} + +TString FormatInstant(TInstant instant) { + TStringBuilder builder; + builder << instant.FormatLocalTime("%H:%M:%S."); + auto msd = (instant.MilliSeconds() % 1000) / 10; + if (msd < 10) { + builder << '0'; + } + builder << msd << 's'; + return builder; +} + +void WriteNamedNode(NYson::TYsonWriter& writer, NJson::TJsonValue& node, const TString& name, TTotalStatistics& totals) { switch (node.GetType()) { case NJson::JSON_INTEGER: case NJson::JSON_DOUBLE: case NJson::JSON_UINTEGER: if (name) { - auto v = node.GetIntegerRobust(); + auto sum = node.GetIntegerSafe(); + if (name == "TotalInputRows") { + totals.TotalInputRows += sum; + } else if (name == "TotalInputBytes") { + totals.TotalInputBytes += sum; + } else if (name == "TotalOutputRows") { + totals.TotalOutputRows += sum; + } else if (name == "TotalOutputBytes") { + totals.TotalOutputBytes += sum; + } writer.OnKeyedItem(name); writer.OnBeginMap(); writer.OnKeyedItem("sum"); - writer.OnInt64Scalar(v); + if (name.EndsWith("Us")) { + writer.OnStringScalar(FormatDurationUs(sum)); + } else if (name.EndsWith("Ms")) { + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(sum))); + } else { + writer.OnInt64Scalar(sum); + } writer.OnKeyedItem("count"); writer.OnInt64Scalar(1); - writer.OnKeyedItem("avg"); - writer.OnInt64Scalar(v); - writer.OnKeyedItem("max"); - writer.OnInt64Scalar(v); - writer.OnKeyedItem("min"); - writer.OnInt64Scalar(v); writer.OnEndMap(); } break; case NJson::JSON_ARRAY: for (auto item : node.GetArray()) { if (auto* subNode = item.GetValueByPath("Name")) { - WriteNamedNode(writer, item, name + "=" + subNode->GetStringRobust()); + WriteNamedNode(writer, item, name + "=" + subNode->GetStringSafe(), totals); } - } - break; - case NJson::JSON_MAP: - if (auto* subNode = node.GetValueByPath("Sum")) { - auto sum = subNode->GetIntegerRobust(); - auto count = 1; - if (auto* subNode = node.GetValueByPath("Count")) { - count = subNode->GetIntegerRobust(); - if (count <= 1) { - count = 1; + if (name == "Ingress") { + if (auto* ingressNode = item.GetValueByPath("Ingress.Bytes.Sum")) { + totals.TotalIngressBytes += ingressNode->GetIntegerSafe(); } } - auto min = sum; - if (auto* subNode = node.GetValueByPath("Min")) { - min = subNode->GetIntegerRobust(); + } + break; + case NJson::JSON_MAP: { + std::optional<ui64> count; + std::optional<ui64> sum; + std::optional<ui64> min; + std::optional<ui64> max; + + if (auto* subNode = node.GetValueByPath("Count")) { + count = subNode->GetIntegerSafe(); + if (*count <= 1) { + *count = 1; } - auto max = sum; - if (auto* subNode = node.GetValueByPath("Max")) { - max = subNode->GetIntegerRobust(); + } + if (auto* subNode = node.GetValueByPath("Sum")) { + sum = subNode->GetIntegerSafe(); + if (name == "MaxMemoryUsage") { + totals.MaxMemoryUsage += *sum; + } else if (name == "CpuTimeUs") { + totals.CpuTimeUs += *sum; + } else if (name == "SourceCpuTimeUs") { + totals.SourceCpuTimeUs += *sum; } + } + if (auto* subNode = node.GetValueByPath("Min")) { + min = subNode->GetIntegerSafe(); + } + if (auto* subNode = node.GetValueByPath("Max")) { + max = subNode->GetIntegerSafe(); + } + + if (count || sum || min || max) { writer.OnKeyedItem(name); writer.OnBeginMap(); - writer.OnKeyedItem("sum"); - writer.OnInt64Scalar(sum); - writer.OnKeyedItem("count"); - writer.OnInt64Scalar(count); - writer.OnKeyedItem("avg"); - writer.OnInt64Scalar(sum / count); - writer.OnKeyedItem("max"); - writer.OnInt64Scalar(max); - writer.OnKeyedItem("min"); - writer.OnInt64Scalar(min); + if (name.EndsWith("Us")) { // TDuration + if (sum) { + writer.OnKeyedItem("sum"); + writer.OnStringScalar(FormatDurationUs(*sum)); + } + if (count) { + writer.OnKeyedItem("count"); + writer.OnInt64Scalar(*count); + } + if (sum && count && *count) { + writer.OnKeyedItem("avg"); + writer.OnStringScalar(FormatDurationUs(*sum / *count)); + } + if (max) { + writer.OnKeyedItem("max"); + writer.OnStringScalar(FormatDurationUs(*max)); + } + if (min) { + writer.OnKeyedItem("min"); + writer.OnStringScalar(FormatDurationUs(*min)); + } + } else if (name.EndsWith("Ms")) { // TInstant + if (sum) { // sum of timestamps has no meaning + writer.OnKeyedItem("avg"); + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds((count && *count) ? (*sum / *count) : *sum))); + } + if (min && max) { // render duration here + writer.OnKeyedItem("sum"); + writer.OnStringScalar(FormatDurationMs(*max - *min)); + } + if (count) { + writer.OnKeyedItem("count"); + writer.OnInt64Scalar(*count); + } + if (max) { + writer.OnKeyedItem("max"); + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(*max))); + } + if (min) { + writer.OnKeyedItem("min"); + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(*min))); + } + } else { + if (sum) { + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(*sum); + } + if (count) { + writer.OnKeyedItem("count"); + writer.OnInt64Scalar(*count); + } + if (sum && count && *count) { + writer.OnKeyedItem("avg"); + writer.OnInt64Scalar(*sum / *count); + } + if (max) { + writer.OnKeyedItem("max"); + writer.OnInt64Scalar(*max); + } + if (min) { + writer.OnKeyedItem("min"); + writer.OnInt64Scalar(*min); + } + } writer.OnEndMap(); } else { if (name) { @@ -71,42 +198,48 @@ void WriteNamedNode(NYson::TYsonWriter& writer, NJson::TJsonValue& node, const T writer.OnBeginMap(); } for (auto& [key, value] : node.GetMapSafe()) { - WriteNamedNode(writer, value, key); + WriteNamedNode(writer, value, key, totals); } if (name) { writer.OnEndMap(); } } break; + } default: break; } } -void EnumeratePlans(NYson::TYsonWriter& writer, NJson::TJsonValue& value) { +void EnumeratePlans(NYson::TYsonWriter& writer, NJson::TJsonValue& value, ui32& stageViewIndex, TTotalStatistics& totals) { if (auto* subNode = value.GetValueByPath("Plans")) { for (auto plan : subNode->GetArray()) { - EnumeratePlans(writer, plan); + EnumeratePlans(writer, plan, stageViewIndex, totals); } } if (auto* statNode = value.GetValueByPath("Stats")) { - TString nodeType = ""; - if (auto* typeNode = value.GetValueByPath("Node Type")) { - nodeType = TString("_") + typeNode->GetStringRobust(); + + TStringBuilder builder; + stageViewIndex++; + if (stageViewIndex < 10) { + builder << '0'; } - ui64 nodeId = 0; + builder << stageViewIndex; if (auto* idNode = value.GetValueByPath("PlanNodeId")) { - nodeId = idNode->GetIntegerRobust(); + builder << '_' << idNode->GetIntegerSafe(); } - writer.OnKeyedItem("Stage_" + ToString(nodeId) + nodeType); + if (auto* typeNode = value.GetValueByPath("Node Type")) { + builder << '_' << typeNode->GetStringSafe(); + } + + writer.OnKeyedItem(builder); writer.OnBeginMap(); - WriteNamedNode(writer, *statNode, ""); + WriteNamedNode(writer, *statNode, "", totals); writer.OnEndMap(); } } TString GetV1StatFromV2Plan(const TString& plan) { - Y_UNUSED(plan); TStringStream out; NYson::TYsonWriter writer(&out); writer.OnBeginMap(); @@ -117,10 +250,356 @@ TString GetV1StatFromV2Plan(const TString& plan) { if (auto* subNode = topNode->GetValueByPath("Plans")) { for (auto plan : subNode->GetArray()) { if (auto* typeNode = plan.GetValueByPath("Node Type")) { - auto nodeType = typeNode->GetStringRobust(); + auto nodeType = typeNode->GetStringSafe(); + TTotalStatistics totals; + ui32 stageViewIndex = 0; + writer.OnKeyedItem(nodeType); + writer.OnBeginMap(); + EnumeratePlans(writer, plan, stageViewIndex, totals); + if (totals.MaxMemoryUsage) { + writer.OnKeyedItem("MaxMemoryUsage"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(totals.MaxMemoryUsage); + writer.OnEndMap(); + } + if (totals.CpuTimeUs) { + writer.OnKeyedItem("CpuTimeUs"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnStringScalar(FormatDurationUs(totals.CpuTimeUs)); + writer.OnEndMap(); + } + if (totals.SourceCpuTimeUs) { + writer.OnKeyedItem("SourceCpuTimeUs"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnStringScalar(FormatDurationUs(totals.SourceCpuTimeUs)); + writer.OnEndMap(); + } + if (totals.TotalInputRows) { + writer.OnKeyedItem("TotalInputRows"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(totals.TotalInputRows); + writer.OnEndMap(); + } + if (totals.TotalInputBytes) { + writer.OnKeyedItem("TotalInputBytes"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(totals.TotalInputBytes); + writer.OnEndMap(); + } + if (totals.TotalOutputRows) { + writer.OnKeyedItem("TotalOutputRows"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(totals.TotalOutputRows); + writer.OnEndMap(); + } + if (totals.TotalOutputBytes) { + writer.OnKeyedItem("TotalOutputBytes"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(totals.TotalOutputBytes); + writer.OnEndMap(); + } + if (totals.TotalIngressBytes) { + writer.OnKeyedItem("TotalIngressBytes"); + writer.OnBeginMap(); + writer.OnKeyedItem("sum"); + writer.OnInt64Scalar(totals.TotalIngressBytes); + writer.OnEndMap(); + } + writer.OnEndMap(); + } + } + } + } + } + writer.OnEndMap(); + return NJson2Yson::ConvertYson2Json(out.Str()); +} + +std::optional<ui64> WriteMetric(NYson::TYsonWriter& writer, NJson::TJsonValue& node, const TString& column, const TString& name, const TString& tag) { + std::optional<ui64> value; + if (auto* subNode = node.GetValueByPath(name)) { + auto t = tag; + if (t == "") { + if (column == "first" || column == "pause") t = "Min"; + else if (column == "resume" || column == "last") t = "Max"; + else t = "Sum"; + } + + if (t == "Avg") { + if (auto* metricNode = subNode->GetValueByPath("Sum")) { + value = metricNode->GetIntegerSafe(); + if (auto* metricNode = subNode->GetValueByPath("Count")) { + auto count = metricNode->GetIntegerSafe(); + if (count) { + *value /= count; + } + } + } + } else if (auto* metricNode = subNode->GetValueByPath(t)) { + value = metricNode->GetIntegerSafe(); + } + + if (value) { + writer.OnKeyedItem(column); + if (t == "Count") { + writer.OnInt64Scalar(*value); + } else { + if (name.EndsWith("Us")) { + writer.OnStringScalar(FormatDurationUs(*value)); + } else if (name.EndsWith("Ms")) { + writer.OnStringScalar(t == "Sum" ? "-" : FormatInstant(TInstant::MilliSeconds(*value))); + } else { + writer.OnInt64Scalar(*value); + } + } + } + } + return value; +} + +std::vector<std::pair<TString, TString>> columns = { + std::make_pair<TString, TString>("id", ""), + std::make_pair<TString, TString>("cpu", ""), + std::make_pair<TString, TString>("scpu", ""), + std::make_pair<TString, TString>("mem", ""), + std::make_pair<TString, TString>("first", "FirstMessageMs"), + std::make_pair<TString, TString>("pause", "PauseMessageMs"), + std::make_pair<TString, TString>("resume", "PauseMessageMs"), + std::make_pair<TString, TString>("last", "LastMessageMs"), + std::make_pair<TString, TString>("active", "ActiveTimeUs"), + std::make_pair<TString, TString>("wait", "WaitTimeUs"), + std::make_pair<TString, TString>("bytes", "Bytes"), + std::make_pair<TString, TString>("rows", "Rows") +}; + +void WriteAggregates(NYson::TYsonWriter& writer, TAggregates& aggregates) { + for (auto& p : aggregates) { + if (*p.second) { + writer.OnKeyedItem(p.first); + if (p.first == "first" || p.first == "pause" || p.first == "resume" || p.first == "last") { + writer.OnStringScalar(FormatInstant(TInstant::MilliSeconds(*p.second))); + } else if (p.first == "active" || p.first == "wait") { + writer.OnStringScalar(FormatDurationUs(*p.second)); + } else { + writer.OnInt64Scalar(*p.second); + } + } + } +} + +void MergeAggregates(TAggregates& parentAggregates, TAggregates& aggregates) { + for (auto& p : aggregates) { + if (*p.second) { + auto& aggr = parentAggregates[p.first]; + if (!aggr) { + aggr = *p.second; + } else if (p.first == "first" || p.first == "pause") { + aggr = std::min(*aggr, *p.second); + } else if (p.first == "resume" || p.first == "last") { + aggr = std::max(*aggr, *p.second); + } else { + *aggr += *p.second; + } + } + } +} + +void WriteAsyncStatNode(NYson::TYsonWriter& writer, NJson::TJsonValue& node, const TString& name, TAggregates& aggregates) { + writer.OnKeyedItem(name); + writer.OnBeginMap(); + for (auto& p : columns) { + if (p.second) { + auto value = WriteMetric(writer, node, p.first, p.second, ""); + if (value) { + auto& aggr = aggregates[p.first]; + if (!aggr) { + aggr = *value; + } else if (p.first == "first" || p.first == "pause") { + aggr = std::min(*aggr, *value); + } else if (p.first == "resume" || p.first == "last") { + aggr = std::max(*aggr, *value); + } else { + *aggr += *value; + } + } + } + } + writer.OnKeyedItem("1_Min"); + writer.OnBeginMap(); + for (auto& p : columns) { + if (p.second) { + WriteMetric(writer, node, p.first, p.second, "Min"); + } + } + writer.OnEndMap(); + writer.OnKeyedItem("2_Avg"); + writer.OnBeginMap(); + for (auto& p : columns) { + if (p.second) { + WriteMetric(writer, node, p.first, p.second, "Avg"); + } + } + writer.OnEndMap(); + writer.OnKeyedItem("3_Max"); + writer.OnBeginMap(); + for (auto& p : columns) { + if (p.second) { + WriteMetric(writer, node, p.first, p.second, "Max"); + } + } + writer.OnEndMap(); + writer.OnKeyedItem("4_Sum"); + writer.OnBeginMap(); + for (auto& p : columns) { + if (p.second) { + WriteMetric(writer, node, p.first, p.second, "Sum"); + } + } + writer.OnEndMap(); + writer.OnKeyedItem("5_Count"); + writer.OnBeginMap(); + for (auto& p : columns) { + if (p.second) { + WriteMetric(writer, node, p.first, p.second, "Count"); + } + } + writer.OnEndMap(); + writer.OnEndMap(); +} + +void WriteAsyncIoNode(NYson::TYsonWriter& writer, NJson::TJsonValue& node, const TString& prefix, TAggregates& parentAggregates) { + if (node.GetType() == NJson::JSON_ARRAY) { + for (auto item : node.GetArray()) { + if (auto* subNode = item.GetValueByPath("Name")) { + writer.OnKeyedItem(prefix + "_" + subNode->GetStringSafe()); + writer.OnBeginMap(); + TAggregates aggregates; + if (auto* subNode = item.GetValueByPath("Ingress")) { + WriteAsyncStatNode(writer, *subNode, "1_Ingress", aggregates); + } + if (auto* subNode = item.GetValueByPath("Push")) { + WriteAsyncStatNode(writer, *subNode, "2_Push", aggregates); + } + if (auto* subNode = item.GetValueByPath("Pop")) { + WriteAsyncStatNode(writer, *subNode, "3_Pop", aggregates); + } + if (auto* subNode = item.GetValueByPath("Egress")) { + WriteAsyncStatNode(writer, *subNode, "4_Egress", aggregates); + } + WriteAggregates(writer, aggregates); + MergeAggregates(parentAggregates, aggregates); + writer.OnEndMap(); + } + } + } +} + +void EnumeratePlansV2(NYson::TYsonWriter& writer, NJson::TJsonValue& value, ui32& stageViewIndex, TTotalStatistics& totals) { + if (auto* subNode = value.GetValueByPath("Plans")) { + for (auto plan : subNode->GetArray()) { + EnumeratePlansV2(writer, plan, stageViewIndex, totals); + } + } + if (auto* statNode = value.GetValueByPath("Stats")) { + + TStringBuilder builder; + stageViewIndex++; + if (stageViewIndex < 10) { + builder << '0'; + } + builder << stageViewIndex; + if (auto* idNode = value.GetValueByPath("PlanNodeId")) { + builder << '_' << idNode->GetIntegerSafe(); + } + if (auto* typeNode = value.GetValueByPath("Node Type")) { + builder << '_' << typeNode->GetStringSafe(); + } + + writer.OnKeyedItem(builder); + writer.OnBeginMap(); + TAggregates aggregates; + if (auto* subNode = statNode->GetValueByPath("Ingress")) { + WriteAsyncIoNode(writer, *subNode, "1_Ingress", aggregates); + } + if (auto* subNode = statNode->GetValueByPath("Push")) { + WriteAsyncIoNode(writer, *subNode, "2_Push", aggregates); + } + if (auto* subNode = statNode->GetValueByPath("Pop")) { + WriteAsyncIoNode(writer, *subNode, "3_Pop", aggregates); + } + if (auto* subNode = statNode->GetValueByPath("Egress")) { + WriteAsyncIoNode(writer, *subNode, "4_Egress", aggregates); + } + WriteAggregates(writer, aggregates); + MergeAggregates(totals.Aggregates, aggregates); + if (auto* subNode = statNode->GetValueByPath("MaxMemoryUsage.Sum")) { + auto sum = subNode->GetIntegerSafe(); + totals.MaxMemoryUsage += sum; + writer.OnKeyedItem("mem"); + writer.OnInt64Scalar(sum); + } + if (auto* subNode = statNode->GetValueByPath("CpuTimeUs.Sum")) { + auto sum = subNode->GetIntegerSafe(); + totals.CpuTimeUs += sum; + writer.OnKeyedItem("cpu"); + writer.OnStringScalar(FormatDurationUs(sum)); + } + if (auto* subNode = statNode->GetValueByPath("SourceCpuTimeUs.Sum")) { + auto sum = subNode->GetIntegerSafe(); + totals.SourceCpuTimeUs += sum; + writer.OnKeyedItem("scpu"); + writer.OnStringScalar(FormatDurationUs(sum)); + } + writer.OnEndMap(); + } +} + +TString GetV1StatFromV2PlanV2(const TString& plan) { + TStringStream out; + NYson::TYsonWriter writer(&out); + writer.OnBeginMap(); + + writer.OnKeyedItem("Columns"); + writer.OnBeginList(); + for (auto& p : columns) { + writer.OnListItem(); + writer.OnStringScalar(p.first); + } + writer.OnEndList(); + + NJson::TJsonReaderConfig jsonConfig; + NJson::TJsonValue stat; + if (NJson::ReadJsonTree(plan, &jsonConfig, &stat)) { + if (auto* topNode = stat.GetValueByPath("Plan")) { + if (auto* subNode = topNode->GetValueByPath("Plans")) { + for (auto plan : subNode->GetArray()) { + if (auto* typeNode = plan.GetValueByPath("Node Type")) { + auto nodeType = typeNode->GetStringSafe(); + TTotalStatistics totals; + ui32 stageViewIndex = 0; writer.OnKeyedItem(nodeType); writer.OnBeginMap(); - EnumeratePlans(writer, plan); + EnumeratePlansV2(writer, plan, stageViewIndex, totals); + WriteAggregates(writer, totals.Aggregates); + if (totals.MaxMemoryUsage) { + writer.OnKeyedItem("mem"); + writer.OnInt64Scalar(totals.MaxMemoryUsage); + } + if (totals.CpuTimeUs) { + writer.OnKeyedItem("cpu"); + writer.OnStringScalar(FormatDurationUs(totals.CpuTimeUs)); + } + if (totals.SourceCpuTimeUs) { + writer.OnKeyedItem("scpu"); + writer.OnStringScalar(FormatDurationUs(totals.SourceCpuTimeUs)); + } writer.OnEndMap(); } } diff --git a/ydb/core/fq/libs/compute/common/utils.h b/ydb/core/fq/libs/compute/common/utils.h index 3ee72cb19a..dae8624eaf 100644 --- a/ydb/core/fq/libs/compute/common/utils.h +++ b/ydb/core/fq/libs/compute/common/utils.h @@ -25,5 +25,10 @@ inline std::shared_ptr<NYdb::NTable::TTableClient> CreateNewTableClient(const TS } TString GetV1StatFromV2Plan(const TString& plan); +TString GetV1StatFromV2PlanV2(const TString& plan); + +TString FormatDurationMs(ui64 durationMs); +TString FormatDurationUs(ui64 durationUs); +TString FormatInstant(TInstant instant); } // namespace NFq diff --git a/ydb/core/fq/libs/compute/ydb/ydb_connector_actor.cpp b/ydb/core/fq/libs/compute/ydb/ydb_connector_actor.cpp index f1590e7238..b1fb68170a 100644 --- a/ydb/core/fq/libs/compute/ydb/ydb_connector_actor.cpp +++ b/ydb/core/fq/libs/compute/ydb/ydb_connector_actor.cpp @@ -23,7 +23,12 @@ public: : YqSharedResources(params.YqSharedResources) , CredentialsProviderFactory(params.CredentialsProviderFactory) , ComputeConnection(params.ComputeConnection) - {} + { + StatsMode = params.Config.GetControlPlaneStorage().GetStatsMode(); + if (StatsMode == Ydb::Query::StatsMode::STATS_MODE_UNSPECIFIED) { + StatsMode = Ydb::Query::StatsMode::STATS_MODE_FULL; + } + } void Bootstrap() { auto querySettings = NFq::GetClientSettings<NYdb::NQuery::TClientSettings>(ComputeConnection, CredentialsProviderFactory); @@ -49,7 +54,7 @@ public: settings.OperationTimeout(event.OperationTimeout); settings.Syntax(event.Syntax); settings.ExecMode(event.ExecMode); - settings.StatsMode(Ydb::Query::StatsMode::STATS_MODE_FULL); + settings.StatsMode(StatsMode); settings.TraceId(event.TraceId); QueryClient ->ExecuteScript(event.Sql, settings) @@ -135,6 +140,7 @@ private: NConfig::TYdbStorageConfig ComputeConnection; std::unique_ptr<NYdb::NQuery::TQueryClient> QueryClient; std::unique_ptr<NYdb::NOperation::TOperationClient> OperationClient; + Ydb::Query::StatsMode StatsMode; }; std::unique_ptr<NActors::IActor> CreateConnectorActor(const TRunActorParams& params) { diff --git a/ydb/core/fq/libs/config/protos/control_plane_storage.proto b/ydb/core/fq/libs/config/protos/control_plane_storage.proto index b8c8faa556..fa8ed94fec 100644 --- a/ydb/core/fq/libs/config/protos/control_plane_storage.proto +++ b/ydb/core/fq/libs/config/protos/control_plane_storage.proto @@ -7,6 +7,7 @@ option java_package = "ru.yandex.kikimr.proto"; import "ydb/core/fq/libs/config/protos/storage.proto"; import "ydb/library/yql/dq/actors/protos/dq_status_codes.proto"; +import "ydb/public/api/protos/ydb_query.proto"; //////////////////////////////////////////////////////////// @@ -71,4 +72,5 @@ message TControlPlaneStorageConfig { string DbReloadPeriod = 31; bool DumpRawStatistics = 32; bool IgnorePrivateSources = 33; + Ydb.Query.StatsMode StatsMode = 34; } diff --git a/ydb/core/fq/libs/control_plane_storage/internal/utils.cpp b/ydb/core/fq/libs/control_plane_storage/internal/utils.cpp index a35d5a2001..e782aff1ce 100644 --- a/ydb/core/fq/libs/control_plane_storage/internal/utils.cpp +++ b/ydb/core/fq/libs/control_plane_storage/internal/utils.cpp @@ -144,7 +144,7 @@ std::vector<TString> GetMeteringRecords(const TString& statistics, bool billable if (auto* ingressNode = stage.second.GetValueByPath("IngressBytes=S3Source.sum")) { // raw old ingress += ingressNode->GetIntegerSafe(); - } else if (auto* ingressNode = stage.second.GetValueByPath("IngressBytes=S3Source.Input.Bytes.sum")) { + } else if (auto* ingressNode = stage.second.GetValueByPath("IngressBytes=S3Source.Ingress.Bytes.sum")) { // raw new ingress += ingressNode->GetIntegerSafe(); } @@ -279,6 +279,12 @@ TString GetPrettyStatistics(const TString& statistics) { NJson::TJsonReaderConfig jsonConfig; NJson::TJsonValue stat; if (NJson::ReadJsonTree(statistics, &jsonConfig, &stat)) { + + // EXP + if (stat.GetValueByPath("Columns")) { + return statistics; + } + for (const auto& p : stat.GetMap()) { // YQv1 if (p.first.StartsWith("Graph=") || p.first.StartsWith("Precompute=")) { @@ -304,8 +310,8 @@ TString GetPrettyStatistics(const TString& statistics) { writer.OnBeginMap(); AggregateNode(writer, p.second, "TotalTasks", "TasksCount"); AggregateNode(writer, p.second, "TotalCpuTimeUs", "CpuTimeUs"); - AggregateNode(writer, p.second, "IngressBytes=S3Source", "IngressObjectStorageBytes"); - AggregateNode(writer, p.second, "EgressBytes=S3Sink", "EgressObjectStorageBytes"); + AggregateNode(writer, p.second, "Ingress=S3Source.Ingress.Bytes", "IngressObjectStorageBytes"); + AggregateNode(writer, p.second, "Egress=S3Sink.Egress.Bytes", "EgressObjectStorageBytes"); writer.OnEndMap(); } } diff --git a/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp b/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp index 7da21db094..f3868ca3bd 100644 --- a/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp +++ b/ydb/core/kqp/compute_actor/kqp_pure_compute_actor.cpp @@ -49,8 +49,7 @@ void TKqpComputeActor::DoBootstrap() { execCtx.PatternCache = GetKqpResourceManager()->GetPatternCache(); TDqTaskRunnerSettings settings; - settings.CollectBasicStats = RuntimeSettings.StatsMode >= NYql::NDqProto::DQ_STATS_MODE_BASIC; - settings.CollectProfileStats = RuntimeSettings.StatsMode >= NYql::NDqProto::DQ_STATS_MODE_PROFILE; + settings.StatsMode = RuntimeSettings.StatsMode; settings.OptLLVM = (GetTask().HasUseLlvm() && GetTask().GetUseLlvm()) ? "--compile-options=disable-opt" : "OFF"; settings.UseCacheForLLVM = AppData()->FeatureFlags.GetEnableLLVMCache(); diff --git a/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp b/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp index 8e09164f26..3358d032cd 100644 --- a/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp +++ b/ydb/core/kqp/compute_actor/kqp_scan_compute_actor.cpp @@ -193,8 +193,7 @@ void TKqpScanComputeActor::DoBootstrap() { const TActorSystem* actorSystem = TlsActivationContext->ActorSystem(); NDq::TDqTaskRunnerSettings settings; - settings.CollectBasicStats = GetStatsMode() >= NYql::NDqProto::DQ_STATS_MODE_BASIC; - settings.CollectProfileStats = GetStatsMode() >= NYql::NDqProto::DQ_STATS_MODE_PROFILE; + settings.StatsMode = GetStatsMode(); settings.OptLLVM = (GetTask().HasUseLlvm() && GetTask().GetUseLlvm()) ? "--compile-options=disable-opt" : "OFF"; settings.UseCacheForLLVM = AppData()->FeatureFlags.GetEnableLLVMCache(); diff --git a/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp b/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp index e667be9be0..066df07b1a 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp +++ b/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp @@ -31,6 +31,49 @@ void UpdateAggr(NDqProto::TDqStatsAggr* aggr, ui64 value) noexcept { aggr->SetCnt(aggr->GetCnt() + 1); } +struct TAsyncGroupStat { + ui64 Bytes = 0; + ui64 Rows = 0; + ui64 Chunks = 0; + ui64 Splits = 0; + ui64 FirstMessageMs = 0; + ui64 PauseMessageMs = 0; + ui64 ResumeMessageMs = 0; + ui64 LastMessageMs = 0; + ui64 WaitTimeUs = 0; + ui64 WaitPeriods = 0; + ui64 Count = 0; +}; + +void UpdateAsyncAggr(NDqProto::TDqAsyncStatsAggr& asyncAggr, const NDqProto::TDqAsyncBufferStats& asyncStat) noexcept { + UpdateAggr(asyncAggr.MutableBytes(), asyncStat.GetBytes()); + UpdateAggr(asyncAggr.MutableRows(), asyncStat.GetRows()); + UpdateAggr(asyncAggr.MutableChunks(), asyncStat.GetChunks()); + UpdateAggr(asyncAggr.MutableSplits(), asyncStat.GetSplits()); + + auto firstMessageMs = asyncStat.GetFirstMessageMs(); + if (firstMessageMs) { + UpdateAggr(asyncAggr.MutableFirstMessageMs(), firstMessageMs); + } + if (asyncStat.GetPauseMessageMs()) { + UpdateAggr(asyncAggr.MutablePauseMessageMs(), asyncStat.GetPauseMessageMs()); + } + if (asyncStat.GetResumeMessageMs()) { + UpdateAggr(asyncAggr.MutableResumeMessageMs(), asyncStat.GetResumeMessageMs()); + } + auto lastMessageMs = asyncStat.GetLastMessageMs(); + if (lastMessageMs) { + UpdateAggr(asyncAggr.MutableLastMessageMs(), lastMessageMs); + } + + UpdateAggr(asyncAggr.MutableWaitTimeUs(), asyncStat.GetWaitTimeUs()); + UpdateAggr(asyncAggr.MutableWaitPeriods(), asyncStat.GetWaitPeriods()); + + if (firstMessageMs && lastMessageMs >= firstMessageMs) { + UpdateAggr(asyncAggr.MutableActiveTimeUs(), (lastMessageMs - firstMessageMs) * 1000); + } +} + void UpdateMinMax(NDqProto::TDqStatsMinMax* minMax, ui64 value) noexcept { if (minMax->GetMin() == 0) { minMax->SetMin(value); @@ -79,8 +122,9 @@ NYql::NDqProto::EDqStatsMode GetDqStatsMode(Ydb::Table::QueryStatsCollection::Mo // Always collect basic stats for system views / request unit computation. case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_NONE: case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_BASIC: - case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_FULL: return NYql::NDqProto::DQ_STATS_MODE_BASIC; + case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_FULL: + return NYql::NDqProto::DQ_STATS_MODE_FULL; case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_PROFILE: return NYql::NDqProto::DQ_STATS_MODE_PROFILE; default: @@ -94,8 +138,9 @@ NYql::NDqProto::EDqStatsMode GetDqStatsModeShard(Ydb::Table::QueryStatsCollectio case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_NONE: return NYql::NDqProto::DQ_STATS_MODE_NONE; case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_BASIC: - case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_FULL: return NYql::NDqProto::DQ_STATS_MODE_BASIC; + case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_FULL: + return NYql::NDqProto::DQ_STATS_MODE_FULL; case Ydb::Table::QueryStatsCollection::STATS_COLLECTION_PROFILE: return NYql::NDqProto::DQ_STATS_MODE_PROFILE; default: @@ -161,26 +206,39 @@ void TQueryExecutionStats::AddComputeActorStats(ui32 /* nodeId */, NYql::NDqProt auto* stageStats = GetOrCreateStageStats(task, *TasksGraph, *Result); stageStats->SetTotalTasksCount(stageStats->GetTotalTasksCount() + 1); + UpdateAggr(stageStats->MutableMaxMemoryUsage(), stats.GetMaxMemoryUsage()); // only 1 task per CA now UpdateAggr(stageStats->MutableCpuTimeUs(), task.GetCpuTimeUs()); + UpdateAggr(stageStats->MutableSourceCpuTimeUs(), task.GetSourceCpuTimeUs()); UpdateAggr(stageStats->MutableInputRows(), task.GetInputRows()); UpdateAggr(stageStats->MutableInputBytes(), task.GetInputBytes()); UpdateAggr(stageStats->MutableOutputRows(), task.GetOutputRows()); UpdateAggr(stageStats->MutableOutputBytes(), task.GetOutputBytes()); - UpdateMinMax(stageStats->MutableStartTimeMs(), task.GetStartTimeMs()); - UpdateMinMax(stageStats->MutableFirstRowTimeMs(), task.GetFirstRowTimeMs()); - UpdateMinMax(stageStats->MutableFinishTimeMs(), task.GetFinishTimeMs()); + UpdateMinMax(stageStats->MutableStartTimeMs(), task.GetStartTimeMs()); // to be reviewed + UpdateMinMax(stageStats->MutableFirstRowTimeMs(), task.GetFirstRowTimeMs()); // to be reviewed + UpdateMinMax(stageStats->MutableFinishTimeMs(), task.GetFinishTimeMs()); // to be reviewed stageStats->SetDurationUs((stageStats->GetFinishTimeMs().GetMax() - stageStats->GetStartTimeMs().GetMin()) * 1'000); - for (auto ingressStat : task.GetIngress()) { - UpdateAggr(&(*stageStats->MutableIngressBytes())[ingressStat.GetName()], ingressStat.GetBytes()); + for (auto& sourcesStat : task.GetSources()) { + UpdateAsyncAggr(*(*stageStats->MutableIngress())[sourcesStat.GetIngressName()].MutableIngress(), sourcesStat.GetIngress()); + UpdateAsyncAggr(*(*stageStats->MutableIngress())[sourcesStat.GetIngressName()].MutablePush(), sourcesStat.GetPush()); + UpdateAsyncAggr(*(*stageStats->MutableIngress())[sourcesStat.GetIngressName()].MutablePop(), sourcesStat.GetPop()); + } + for (auto& inputChannelStat : task.GetInputChannels()) { + UpdateAsyncAggr(*(*stageStats->MutableInput())[inputChannelStat.GetSrcStageId()].MutablePush(), inputChannelStat.GetPush()); + UpdateAsyncAggr(*(*stageStats->MutableInput())[inputChannelStat.GetSrcStageId()].MutablePop(), inputChannelStat.GetPop()); } - for (auto egressStat : task.GetEgress()) { - UpdateAggr(&(*stageStats->MutableEgressBytes())[egressStat.GetName()], egressStat.GetBytes()); + for (auto& outputChannelStat : task.GetOutputChannels()) { + UpdateAsyncAggr(*(*stageStats->MutableOutput())[outputChannelStat.GetDstStageId()].MutablePush(), outputChannelStat.GetPush()); + UpdateAsyncAggr(*(*stageStats->MutableOutput())[outputChannelStat.GetDstStageId()].MutablePop(), outputChannelStat.GetPop()); + } + for (auto& sinksStat : task.GetSinks()) { + UpdateAsyncAggr(*(*stageStats->MutableEgress())[sinksStat.GetEgressName()].MutablePush(), sinksStat.GetPush()); + UpdateAsyncAggr(*(*stageStats->MutableEgress())[sinksStat.GetEgressName()].MutablePop(), sinksStat.GetPop()); + UpdateAsyncAggr(*(*stageStats->MutableEgress())[sinksStat.GetEgressName()].MutableEgress(), sinksStat.GetEgress()); } } - } if (CollectProfileStats(StatsMode)) { @@ -255,9 +313,9 @@ void TQueryExecutionStats::AddDatashardStats(NYql::NDqProto::TDqComputeActorStat UpdateAggr(stageStats->MutableOutputRows(), task.GetOutputRows()); UpdateAggr(stageStats->MutableOutputBytes(), task.GetOutputBytes()); - UpdateMinMax(stageStats->MutableStartTimeMs(), task.GetStartTimeMs()); - UpdateMinMax(stageStats->MutableFirstRowTimeMs(), task.GetFirstRowTimeMs()); - UpdateMinMax(stageStats->MutableFinishTimeMs(), task.GetFinishTimeMs()); + UpdateMinMax(stageStats->MutableStartTimeMs(), task.GetStartTimeMs()); // to be reviewed + UpdateMinMax(stageStats->MutableFirstRowTimeMs(), task.GetFirstRowTimeMs()); // to be reviewed + UpdateMinMax(stageStats->MutableFinishTimeMs(), task.GetFinishTimeMs()); // to be reviewed stageStats->SetDurationUs((stageStats->GetFinishTimeMs().GetMax() - stageStats->GetStartTimeMs().GetMin()) * 1'000); diff --git a/ydb/core/kqp/executer_actor/kqp_literal_executer.cpp b/ydb/core/kqp/executer_actor/kqp_literal_executer.cpp index 762a5b4512..04aabbd48c 100644 --- a/ydb/core/kqp/executer_actor/kqp_literal_executer.cpp +++ b/ydb/core/kqp/executer_actor/kqp_literal_executer.cpp @@ -53,8 +53,7 @@ std::unique_ptr<TDqTaskRunnerContext> CreateTaskRunnerContext(NMiniKQL::TKqpComp TDqTaskRunnerSettings CreateTaskRunnerSettings(Ydb::Table::QueryStatsCollection::Mode statsMode) { TDqTaskRunnerSettings settings; // Always collect basic stats for system views / request unit computation. - settings.CollectBasicStats = true; - settings.CollectProfileStats = CollectProfileStats(statsMode); + settings.StatsMode = GetDqStatsMode(statsMode); settings.OptLLVM = "OFF"; settings.TerminateOnError = false; return settings; @@ -273,7 +272,7 @@ public: auto taskCpuTime = stats->BuildCpuTime + stats->ComputeCpuTime; executerCpuTime -= taskCpuTime; NYql::NDq::FillTaskRunnerStats(taskRunner->GetTaskId(), TaskId2StageId[taskRunner->GetTaskId()], - *stats, fakeComputeActorStats.AddTasks(), CollectProfileStats(Request.StatsMode)); + *stats, fakeComputeActorStats.AddTasks(), StatsModeToCollectStatsLevel(GetDqStatsMode(Request.StatsMode))); fakeComputeActorStats.SetCpuTimeUs(fakeComputeActorStats.GetCpuTimeUs() + taskCpuTime.MicroSeconds()); } diff --git a/ydb/core/kqp/executer_actor/kqp_tasks_graph.cpp b/ydb/core/kqp/executer_actor/kqp_tasks_graph.cpp index b9b98e4934..3221fae39a 100644 --- a/ydb/core/kqp/executer_actor/kqp_tasks_graph.cpp +++ b/ydb/core/kqp/executer_actor/kqp_tasks_graph.cpp @@ -748,6 +748,8 @@ void FillEndpointDesc(NDqProto::TEndpoint& endpoint, const TTask& task) { void FillChannelDesc(const TKqpTasksGraph& tasksGraph, NDqProto::TChannel& channelDesc, const TChannel& channel, const NKikimrConfig::TTableServiceConfig::EChannelTransportVersion chanTransportVersion) { channelDesc.SetId(channel.Id); + channelDesc.SetSrcStageId(channel.SrcStageId.StageId); + channelDesc.SetDstStageId(channel.DstStageId.StageId); channelDesc.SetSrcTaskId(channel.SrcTask); channelDesc.SetDstTaskId(channel.DstTask); diff --git a/ydb/core/kqp/opt/kqp_query_plan.cpp b/ydb/core/kqp/opt/kqp_query_plan.cpp index f072ae20da..61419143e3 100644 --- a/ydb/core/kqp/opt/kqp_query_plan.cpp +++ b/ydb/core/kqp/opt/kqp_query_plan.cpp @@ -1857,14 +1857,79 @@ void PhyQuerySetTxPlans(NKqpProto::TKqpPhyQuery& queryProto, const TKqpPhysicalQ queryProto.SetQueryPlan(SerializeTxPlans(txPlans, writer.Str())); } +void FillAggrStat(NJson::TJsonValue& node, const NYql::NDqProto::TDqStatsAggr& aggr, const TString& name) { + auto min = aggr.GetMin(); + auto max = aggr.GetMax(); + auto sum = aggr.GetSum(); + if (min || max || sum) { // do not fill empty metrics + auto& aggrStat = node.InsertValue(name, NJson::JSON_MAP); + aggrStat["Min"] = min; + aggrStat["Max"] = max; + aggrStat["Sum"] = sum; + aggrStat["Count"] = aggr.GetCnt(); + } +} + +void FillAsyncAggrStat(NJson::TJsonValue& node, const NYql::NDqProto::TDqAsyncStatsAggr& asyncAggr) { + if (asyncAggr.HasBytes()) { + FillAggrStat(node, asyncAggr.GetBytes(), "Bytes"); + } + if (asyncAggr.HasRows()) { + FillAggrStat(node, asyncAggr.GetRows(), "Rows"); + } + if (asyncAggr.HasChunks()) { + FillAggrStat(node, asyncAggr.GetChunks(), "Chunks"); + } + if (asyncAggr.HasSplits()) { + FillAggrStat(node, asyncAggr.GetSplits(), "Splits"); + } + + if (asyncAggr.HasFirstMessageMs()) { + FillAggrStat(node, asyncAggr.GetFirstMessageMs(), "FirstMessageMs"); + } + if (asyncAggr.HasPauseMessageMs()) { + FillAggrStat(node, asyncAggr.GetPauseMessageMs(), "PauseMessageMs"); + } + if (asyncAggr.HasResumeMessageMs()) { + FillAggrStat(node, asyncAggr.GetResumeMessageMs(), "ResumeMessageMs"); + } + if (asyncAggr.HasLastMessageMs()) { + FillAggrStat(node, asyncAggr.GetLastMessageMs(), "LastMessageMs"); + } + + if (asyncAggr.HasWaitTimeUs()) { + FillAggrStat(node, asyncAggr.GetWaitTimeUs(), "WaitTimeUs"); + } + if (asyncAggr.HasWaitPeriods()) { + FillAggrStat(node, asyncAggr.GetWaitPeriods(), "WaitPeriods"); + } + if (asyncAggr.HasActiveTimeUs()) { + FillAggrStat(node, asyncAggr.GetActiveTimeUs(), "ActiveTimeUs"); + } + if (asyncAggr.HasFirstMessageMs() && asyncAggr.HasLastMessageMs()) { + auto& aggrStat = node.InsertValue("ActiveMessageMs", NJson::JSON_MAP); + aggrStat["Min"] = asyncAggr.GetFirstMessageMs().GetMin(); + aggrStat["Max"] = asyncAggr.GetLastMessageMs().GetMax(); + aggrStat["Count"] = asyncAggr.GetFirstMessageMs().GetCnt(); + } + if (asyncAggr.HasPauseMessageMs() && asyncAggr.HasResumeMessageMs()) { + auto& aggrStat = node.InsertValue("WaitMessageMs", NJson::JSON_MAP); + aggrStat["Min"] = asyncAggr.GetPauseMessageMs().GetMin(); + aggrStat["Max"] = asyncAggr.GetResumeMessageMs().GetMax(); + aggrStat["Count"] = asyncAggr.GetPauseMessageMs().GetCnt(); + } +} + TString AddExecStatsToTxPlan(const TString& txPlanJson, const NYql::NDqProto::TDqExecutionStats& stats) { if (txPlanJson.empty()) { return {}; } THashMap<TProtoStringType, const NYql::NDqProto::TDqStageStats*> stages; + THashMap<ui32, TString> stageIdToGuid; for (const auto& stage : stats.GetStages()) { stages[stage.GetStageGuid()] = &stage; + stageIdToGuid[stage.GetStageId()] = stage.GetStageGuid(); } NJson::TJsonValue root; @@ -1872,20 +1937,22 @@ TString AddExecStatsToTxPlan(const TString& txPlanJson, const NYql::NDqProto::TD auto fillInputStats = [](NJson::TJsonValue& node, const NYql::NDqProto::TDqInputChannelStats& inputStats) { node["ChannelId"] = inputStats.GetChannelId(); + node["SrcStageId"] = inputStats.GetSrcStageId(); - SetNonZero(node, "Bytes", inputStats.GetBytes()); - SetNonZero(node, "Rows", inputStats.GetRowsIn()); + SetNonZero(node, "Bytes", inputStats.GetPush().GetBytes()); + SetNonZero(node, "Rows", inputStats.GetPush().GetRows()); - SetNonZero(node, "WaitTimeUs", inputStats.GetWaitTimeUs()); + SetNonZero(node, "WaitTimeUs", inputStats.GetPush().GetWaitTimeUs()); }; auto fillOutputStats = [](NJson::TJsonValue& node, const NYql::NDqProto::TDqOutputChannelStats& outputStats) { node["ChannelId"] = outputStats.GetChannelId(); + node["DstStageId"] = outputStats.GetDstStageId(); - SetNonZero(node, "Bytes", outputStats.GetBytes()); - SetNonZero(node, "Rows", outputStats.GetRowsOut()); + SetNonZero(node, "Bytes", outputStats.GetPop().GetBytes()); + SetNonZero(node, "Rows", outputStats.GetPop().GetRows()); - SetNonZero(node, "WritesBlockedNoSpace", outputStats.GetBlockedByCapacity()); + SetNonZero(node, "WaitTimeUs", outputStats.GetPop().GetWaitTimeUs()); SetNonZero(node, "SpilledBytes", outputStats.GetSpilledBytes()); }; @@ -1901,15 +1968,16 @@ TString AddExecStatsToTxPlan(const TString& txPlanJson, const NYql::NDqProto::TD // equals to max if there was no first row if(taskStats.GetFirstRowTimeMs() != std::numeric_limits<ui64>::max()) { - SetNonZero(node, "FirstRowTimeMs", taskStats.GetFirstRowTimeMs()); + SetNonZero(node, "FirstRowTimeMs", taskStats.GetFirstRowTimeMs()); // need to be reviewed } - SetNonZero(node, "StartTimeMs", taskStats.GetStartTimeMs()); - SetNonZero(node, "FinishTimeMs", taskStats.GetFinishTimeMs()); + SetNonZero(node, "StartTimeMs", taskStats.GetStartTimeMs()); // need to be reviewed + SetNonZero(node, "FinishTimeMs", taskStats.GetFinishTimeMs()); // need to be reviewed SetNonZero(node, "ComputeTimeUs", taskStats.GetComputeCpuTimeUs()); - SetNonZero(node, "WaitTimeUs", taskStats.GetWaitTimeUs()); - SetNonZero(node, "PendingInputTimeUs", taskStats.GetPendingInputTimeUs()); - SetNonZero(node, "PendingOutputTimeUs", taskStats.GetPendingOutputTimeUs()); + + SetNonZero(node, "WaitTimeUs", taskStats.GetWaitTimeUs()); // need to be reviewed + SetNonZero(node, "PendingInputTimeUs", taskStats.GetPendingInputTimeUs()); // need to be reviewed + SetNonZero(node, "PendingOutputTimeUs", taskStats.GetPendingOutputTimeUs()); // need to be reviewed NKqpProto::TKqpTaskExtraStats taskExtraStats; if (taskStats.GetExtra().UnpackTo(&taskExtraStats)) { @@ -1939,6 +2007,16 @@ TString AddExecStatsToTxPlan(const TString& txPlanJson, const NYql::NDqProto::TD } }; + THashMap<TString, ui32> guidToPlaneId; + + auto collectPlanNodeId = [&](NJson::TJsonValue& node) { + if (auto stageGuid = node.GetMapSafe().FindPtr("StageGuid")) { + if (auto planNodeId = node.GetMapSafe().FindPtr("PlanNodeId")) { + guidToPlaneId[stageGuid->GetStringSafe()] = planNodeId->GetIntegerSafe(); + } + } + }; + auto addStatsToPlanNode = [&](NJson::TJsonValue& node) { if (auto stageGuid = node.GetMapSafe().FindPtr("StageGuid")) { if (auto stat = stages.FindPtr(stageGuid->GetStringSafe())) { @@ -1957,30 +2035,85 @@ TString AddExecStatsToTxPlan(const TString& txPlanJson, const NYql::NDqProto::TD stats["TotalOutputRows"] = (*stat)->GetOutputRows().GetSum(); stats["TotalOutputBytes"] = (*stat)->GetOutputBytes().GetSum(); - if (!(*stat)->GetIngressBytes().empty()) { - auto& ingressStats = stats.InsertValue("IngressBytes", NJson::JSON_ARRAY); - for (auto ingressBytes : (*stat)->GetIngressBytes()) { + if ((*stat)->HasCpuTimeUs()) { + FillAggrStat(stats, (*stat)->GetCpuTimeUs(), "CpuTimeUs"); + } + + if ((*stat)->HasSourceCpuTimeUs()) { + FillAggrStat(stats, (*stat)->GetSourceCpuTimeUs(), "SourceCpuTimeUs"); + } + + if ((*stat)->HasMaxMemoryUsage()) { + FillAggrStat(stats, (*stat)->GetMaxMemoryUsage(), "MaxMemoryUsage"); + } + + if (!(*stat)->GetIngress().empty()) { + auto& ingressStats = stats.InsertValue("Ingress", NJson::JSON_ARRAY); + for (auto ingress : (*stat)->GetIngress()) { auto& ingressInfo = ingressStats.AppendValue(NJson::JSON_MAP); - ingressInfo["Name"] = ingressBytes.first; - ingressInfo["Min"] = ingressBytes.second.GetMin(); - ingressInfo["Max"] = ingressBytes.second.GetMax(); - ingressInfo["Sum"] = ingressBytes.second.GetSum(); - ingressInfo["Count"] = ingressBytes.second.GetCnt(); + ingressInfo["Name"] = ingress.first; + if (ingress.second.HasIngress()) { + FillAsyncAggrStat(ingressInfo.InsertValue("Ingress", NJson::JSON_MAP), ingress.second.GetIngress()); + } + if (ingress.second.HasPush()) { + FillAsyncAggrStat(ingressInfo.InsertValue("Push", NJson::JSON_MAP), ingress.second.GetPush()); + } + if (ingress.second.HasPop()) { + FillAsyncAggrStat(ingressInfo.InsertValue("Pop", NJson::JSON_MAP), ingress.second.GetPop()); + } } } - - if (!(*stat)->GetEgressBytes().empty()) { - auto& egressStats = stats.InsertValue("EgressBytes", NJson::JSON_ARRAY); - for (auto egressBytes : (*stat)->GetEgressBytes()) { + if (!(*stat)->GetInput().empty()) { + auto& inputStats = stats.InsertValue("Input", NJson::JSON_ARRAY); + for (auto input : (*stat)->GetInput()) { + auto& inputInfo = inputStats.AppendValue(NJson::JSON_MAP); + auto stageGuid = stageIdToGuid.at(input.first); + auto planNodeId = guidToPlaneId.at(stageGuid); + inputInfo["Name"] = ToString(planNodeId); + if (input.second.HasPush()) { + FillAsyncAggrStat(inputInfo.InsertValue("Push", NJson::JSON_MAP), input.second.GetPush()); + } + if (input.second.HasPop()) { + FillAsyncAggrStat(inputInfo.InsertValue("Pop", NJson::JSON_MAP), input.second.GetPop()); + } + } + } + if (!(*stat)->GetOutput().empty()) { + auto& outputStats = stats.InsertValue("Output", NJson::JSON_ARRAY); + for (auto output : (*stat)->GetOutput()) { + auto& outputInfo = outputStats.AppendValue(NJson::JSON_MAP); + if (output.first == 0) { + outputInfo["Name"] = "RESULT"; + } else { + auto stageGuid = stageIdToGuid.at(output.first); + auto planNodeId = guidToPlaneId.at(stageGuid); + outputInfo["Name"] = ToString(planNodeId); + } + if (output.second.HasPush()) { + FillAsyncAggrStat(outputInfo.InsertValue("Push", NJson::JSON_MAP), output.second.GetPush()); + } + if (output.second.HasPop()) { + FillAsyncAggrStat(outputInfo.InsertValue("Pop", NJson::JSON_MAP), output.second.GetPop()); + } + } + } + if (!(*stat)->GetEgress().empty()) { + auto& egressStats = stats.InsertValue("Egress", NJson::JSON_ARRAY); + for (auto egress : (*stat)->GetEgress()) { auto& egressInfo = egressStats.AppendValue(NJson::JSON_MAP); - egressInfo["Name"] = egressBytes.first; - egressInfo["Min"] = egressBytes.second.GetMin(); - egressInfo["Max"] = egressBytes.second.GetMax(); - egressInfo["Sum"] = egressBytes.second.GetSum(); - egressInfo["Count"] = egressBytes.second.GetCnt(); + egressInfo["Name"] = egress.first; + if (egress.second.HasPush()) { + FillAsyncAggrStat(egressInfo.InsertValue("Push", NJson::JSON_MAP), egress.second.GetPush()); + } + if (egress.second.HasPop()) { + FillAsyncAggrStat(egressInfo.InsertValue("Pop", NJson::JSON_MAP), egress.second.GetPop()); + } + if (egress.second.HasEgress()) { + FillAsyncAggrStat(egressInfo.InsertValue("Egress", NJson::JSON_MAP), egress.second.GetEgress()); + } } } - + NKqpProto::TKqpStageExtraStats kqpStageStats; if ((*stat)->GetExtra().UnpackTo(&kqpStageStats)) { auto& nodesStats = stats.InsertValue("NodesScanShards", NJson::JSON_ARRAY); @@ -1999,6 +2132,7 @@ TString AddExecStatsToTxPlan(const TString& txPlanJson, const NYql::NDqProto::TD } }; + ModifyPlan(root, collectPlanNodeId); ModifyPlan(root, addStatsToPlanNode); NJsonWriter::TBuf txWriter; diff --git a/ydb/core/kqp/runtime/kqp_read_actor.cpp b/ydb/core/kqp/runtime/kqp_read_actor.cpp index 0e4caebd2c..636e040df6 100644 --- a/ydb/core/kqp/runtime/kqp_read_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_read_actor.cpp @@ -401,6 +401,8 @@ public: Y_ABORT_UNLESS(Arena); Y_ABORT_UNLESS(settings->GetArena() == Arena->Get()); + IngressStats.Level = args.StatsLevel; + TableId = TTableId( Settings->GetTable().GetTableId().GetOwnerId(), Settings->GetTable().GetTableId().GetTableId(), @@ -1045,6 +1047,10 @@ public: return InputIndex; } + const NYql::NDq::TDqAsyncStats& GetIngressStats() const override { + return IngressStats; + } + NMiniKQL::TBytesStatistics GetRowSize(const NUdf::TUnboxedValue* row) { NMiniKQL::TBytesStatistics rowStats{0, 0}; size_t columnIndex = 0; @@ -1431,6 +1437,7 @@ private: const TActorId ComputeActorId; const ui64 InputIndex; + NYql::NDq::TDqAsyncStats IngressStats; const NMiniKQL::TTypeEnvironment& TypeEnv; const NMiniKQL::THolderFactory& HolderFactory; std::shared_ptr<NKikimr::NMiniKQL::TScopedAlloc> Alloc; diff --git a/ydb/core/kqp/runtime/kqp_sequencer_actor.cpp b/ydb/core/kqp/runtime/kqp_sequencer_actor.cpp index 688b6d33fa..ebdc0bcc2b 100644 --- a/ydb/core/kqp/runtime/kqp_sequencer_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_sequencer_actor.cpp @@ -99,10 +99,10 @@ class TKqpSequencerActor : public NActors::TActorBootstrapped<TKqpSequencerActor }; public: - TKqpSequencerActor(ui64 inputIndex, const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, - const NMiniKQL::TTypeEnvironment& typeEnv, const NMiniKQL::THolderFactory& holderFactory, - std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, NKikimrKqp::TKqpSequencerSettings&& settings, - TIntrusivePtr<TKqpCounters> counters) + TKqpSequencerActor(ui64 inputIndex, NYql::NDq::TCollectStatsLevel statsLevel, const NUdf::TUnboxedValue& input, + const NActors::TActorId& computeActorId, const NMiniKQL::TTypeEnvironment& typeEnv, + const NMiniKQL::THolderFactory& holderFactory, std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, + NKikimrKqp::TKqpSequencerSettings&& settings, TIntrusivePtr<TKqpCounters> counters) : LogPrefix(TStringBuilder() << "SequencerActor, inputIndex: " << inputIndex << ", CA Id " << computeActorId) , InputIndex(inputIndex) , Input(input) @@ -118,6 +118,7 @@ public: const auto& col = Settings.GetColumns(colId); ColumnSequenceInfo.emplace_back(col); } + IngressStats.Level = statsLevel; } virtual ~TKqpSequencerActor() { @@ -151,6 +152,10 @@ private: return InputIndex; } + const NYql::NDq::TDqAsyncStats& GetIngressStats() const final { + return IngressStats; + } + void PassAway() final { Counters->SequencerActorsCount->Dec(); @@ -324,6 +329,7 @@ private: private: const TString LogPrefix; const ui64 InputIndex; + NYql::NDq::TDqAsyncStats IngressStats; NUdf::TUnboxedValue Input; NKikimr::NMiniKQL::TUnboxedValueBatch UnprocessedBatch; const NActors::TActorId ComputeActorId; @@ -341,11 +347,11 @@ private: } // namespace std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateSequencerActor(ui64 inputIndex, - const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, const NMiniKQL::TTypeEnvironment& typeEnv, - const NMiniKQL::THolderFactory& holderFactory, std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, - NKikimrKqp::TKqpSequencerSettings&& settings, + NYql::NDq::TCollectStatsLevel statsLevel, const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, + const NMiniKQL::TTypeEnvironment& typeEnv, const NMiniKQL::THolderFactory& holderFactory, + std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, NKikimrKqp::TKqpSequencerSettings&& settings, TIntrusivePtr<TKqpCounters> counters) { - auto actor = new TKqpSequencerActor(inputIndex, input, computeActorId, typeEnv, holderFactory, alloc, + auto actor = new TKqpSequencerActor(inputIndex, statsLevel, input, computeActorId, typeEnv, holderFactory, alloc, std::move(settings), counters); return {actor, actor}; } diff --git a/ydb/core/kqp/runtime/kqp_sequencer_actor.h b/ydb/core/kqp/runtime/kqp_sequencer_actor.h index bd68ec82c8..b775382dd9 100644 --- a/ydb/core/kqp/runtime/kqp_sequencer_actor.h +++ b/ydb/core/kqp/runtime/kqp_sequencer_actor.h @@ -8,9 +8,10 @@ namespace NKikimr { namespace NKqp { std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateSequencerActor(ui64 inputIndex, - const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, const NMiniKQL::TTypeEnvironment& typeEnv, - const NMiniKQL::THolderFactory& holderFactory, std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, - NKikimrKqp::TKqpSequencerSettings&& settings, TIntrusivePtr<TKqpCounters>); + NYql::NDq::TCollectStatsLevel statsLevel, const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, + const NMiniKQL::TTypeEnvironment& typeEnv, const NMiniKQL::THolderFactory& holderFactory, + std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, NKikimrKqp::TKqpSequencerSettings&& settings, + TIntrusivePtr<TKqpCounters>); } // namespace NKqp } // namespace NKikimr diff --git a/ydb/core/kqp/runtime/kqp_sequencer_factory.cpp b/ydb/core/kqp/runtime/kqp_sequencer_factory.cpp index fa55a5297b..8b261bc3c9 100644 --- a/ydb/core/kqp/runtime/kqp_sequencer_factory.cpp +++ b/ydb/core/kqp/runtime/kqp_sequencer_factory.cpp @@ -7,7 +7,7 @@ namespace NKqp { void RegisterSequencerActorFactory(NYql::NDq::TDqAsyncIoFactory& factory, TIntrusivePtr<TKqpCounters> counters) { factory.RegisterInputTransform<NKikimrKqp::TKqpSequencerSettings>("SequencerInputTransformer", [counters](NKikimrKqp::TKqpSequencerSettings&& settings, NYql::NDq::TDqAsyncIoFactory::TInputTransformArguments&& args) { - return CreateSequencerActor(args.InputIndex, args.TransformInput, args.ComputeActorId, args.TypeEnv, + return CreateSequencerActor(args.InputIndex, args.StatsLevel, args.TransformInput, args.ComputeActorId, args.TypeEnv, args.HolderFactory, args.Alloc, std::move(settings), counters); }); } diff --git a/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp b/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp index e343ce9ef0..8866befd90 100644 --- a/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_stream_lookup_actor.cpp @@ -25,10 +25,10 @@ static constexpr ui64 MAX_SHARD_RETRIES = 10; class TKqpStreamLookupActor : public NActors::TActorBootstrapped<TKqpStreamLookupActor>, public NYql::NDq::IDqComputeActorAsyncInput { public: - TKqpStreamLookupActor(ui64 inputIndex, const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, - const NMiniKQL::TTypeEnvironment& typeEnv, const NMiniKQL::THolderFactory& holderFactory, - std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, NKikimrKqp::TKqpStreamLookupSettings&& settings, - TIntrusivePtr<TKqpCounters> counters) + TKqpStreamLookupActor(ui64 inputIndex, NYql::NDq::TCollectStatsLevel statsLevel, const NUdf::TUnboxedValue& input, + const NActors::TActorId& computeActorId, const NMiniKQL::TTypeEnvironment& typeEnv, + const NMiniKQL::THolderFactory& holderFactory, std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, + NKikimrKqp::TKqpStreamLookupSettings&& settings, TIntrusivePtr<TKqpCounters> counters) : LogPrefix(TStringBuilder() << "StreamLookupActor, inputIndex: " << inputIndex << ", CA Id " << computeActorId) , InputIndex(inputIndex) , Input(input) @@ -41,7 +41,8 @@ public: , StreamLookupWorker(CreateStreamLookupWorker(std::move(settings), typeEnv, holderFactory)) , Counters(counters) { - }; + IngressStats.Level = statsLevel; + } virtual ~TKqpStreamLookupActor() { if (Alloc) { @@ -152,6 +153,10 @@ private: return InputIndex; } + const NYql::NDq::TDqAsyncStats& GetIngressStats() const final { + return IngressStats; + } + void PassAway() final { Counters->StreamLookupActorsCount->Dec(); { @@ -466,6 +471,7 @@ private: private: const TString LogPrefix; const ui64 InputIndex; + NYql::NDq::TDqAsyncStats IngressStats; NUdf::TUnboxedValue Input; const NActors::TActorId ComputeActorId; const NMiniKQL::TTypeEnvironment& TypeEnv; @@ -492,12 +498,12 @@ private: } // namespace std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateStreamLookupActor(ui64 inputIndex, - const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, const NMiniKQL::TTypeEnvironment& typeEnv, - const NMiniKQL::THolderFactory& holderFactory, std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, - NKikimrKqp::TKqpStreamLookupSettings&& settings, + NYql::NDq::TCollectStatsLevel statsLevel, const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, + const NMiniKQL::TTypeEnvironment& typeEnv, const NMiniKQL::THolderFactory& holderFactory, + std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, NKikimrKqp::TKqpStreamLookupSettings&& settings, TIntrusivePtr<TKqpCounters> counters) { - auto actor = new TKqpStreamLookupActor(inputIndex, input, computeActorId, typeEnv, holderFactory, alloc, - std::move(settings), counters); + auto actor = new TKqpStreamLookupActor(inputIndex, statsLevel, input, computeActorId, typeEnv, holderFactory, + alloc, std::move(settings), counters); return {actor, actor}; } diff --git a/ydb/core/kqp/runtime/kqp_stream_lookup_actor.h b/ydb/core/kqp/runtime/kqp_stream_lookup_actor.h index 29fe5d70d8..6e4b7bad4a 100644 --- a/ydb/core/kqp/runtime/kqp_stream_lookup_actor.h +++ b/ydb/core/kqp/runtime/kqp_stream_lookup_actor.h @@ -8,9 +8,10 @@ namespace NKikimr { namespace NKqp { std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateStreamLookupActor(ui64 inputIndex, - const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, const NMiniKQL::TTypeEnvironment& typeEnv, - const NMiniKQL::THolderFactory& holderFactory, std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, - NKikimrKqp::TKqpStreamLookupSettings&& settings, TIntrusivePtr<TKqpCounters>); + NYql::NDq::TCollectStatsLevel statsLevel, const NUdf::TUnboxedValue& input, const NActors::TActorId& computeActorId, + const NMiniKQL::TTypeEnvironment& typeEnv, const NMiniKQL::THolderFactory& holderFactory, + std::shared_ptr<NMiniKQL::TScopedAlloc>& alloc, NKikimrKqp::TKqpStreamLookupSettings&& settings, + TIntrusivePtr<TKqpCounters>); } // namespace NKqp } // namespace NKikimr diff --git a/ydb/core/kqp/runtime/kqp_stream_lookup_factory.cpp b/ydb/core/kqp/runtime/kqp_stream_lookup_factory.cpp index 59ee432d95..98661a47b3 100644 --- a/ydb/core/kqp/runtime/kqp_stream_lookup_factory.cpp +++ b/ydb/core/kqp/runtime/kqp_stream_lookup_factory.cpp @@ -7,7 +7,7 @@ namespace NKqp { void RegisterStreamLookupActorFactory(NYql::NDq::TDqAsyncIoFactory& factory, TIntrusivePtr<TKqpCounters> counters) { factory.RegisterInputTransform<NKikimrKqp::TKqpStreamLookupSettings>("StreamLookupInputTransformer", [counters](NKikimrKqp::TKqpStreamLookupSettings&& settings, NYql::NDq::TDqAsyncIoFactory::TInputTransformArguments&& args) { - return CreateStreamLookupActor(args.InputIndex, args.TransformInput, args.ComputeActorId, args.TypeEnv, + return CreateStreamLookupActor(args.InputIndex, args.StatsLevel, args.TransformInput, args.ComputeActorId, args.TypeEnv, args.HolderFactory, args.Alloc, std::move(settings), counters); }); } diff --git a/ydb/core/kqp/ut/common/kqp_ut_common.cpp b/ydb/core/kqp/ut/common/kqp_ut_common.cpp index f0db5fea88..88a7dd8e18 100644 --- a/ydb/core/kqp/ut/common/kqp_ut_common.cpp +++ b/ydb/core/kqp/ut/common/kqp_ut_common.cpp @@ -934,12 +934,11 @@ void FindPlanNodesImpl(const NJson::TJsonValue& node, const TString& key, std::v return; } - auto map = node.GetMap(); - if (map.contains(key)) { - results.push_back(map.at(key)); + if (auto* valueNode = node.GetValueByPath(key)) { + results.push_back(*valueNode); } - for (const auto& [_, value]: map) { + for (const auto& [_, value]: node.GetMap()) { FindPlanNodesImpl(value, key, results); } } diff --git a/ydb/core/kqp/ut/scan/kqp_flowcontrol_ut.cpp b/ydb/core/kqp/ut/scan/kqp_flowcontrol_ut.cpp index bcf8e7a03e..4367166de9 100644 --- a/ydb/core/kqp/ut/scan/kqp_flowcontrol_ut.cpp +++ b/ydb/core/kqp/ut/scan/kqp_flowcontrol_ut.cpp @@ -101,12 +101,12 @@ void DoFlowControlTest(ui64 limit, bool hasBlockedByCapacity) { NJson::ReadJsonTree(*res.PlanJson, &plan, true); ui32 writesBlockedNoSpace = 0; - auto nodes = FindPlanNodes(plan, "WritesBlockedNoSpace"); + auto nodes = FindPlanNodes(plan, "Pop.WaitTimeUs.Sum"); for (auto& node : nodes) { writesBlockedNoSpace += node.GetIntegerSafe(); } - UNIT_ASSERT_EQUAL(hasBlockedByCapacity, writesBlockedNoSpace > 0); + UNIT_ASSERT_EQUAL_C(hasBlockedByCapacity, writesBlockedNoSpace > 0, *res.PlanJson); } Y_UNIT_TEST(FlowControl_Unlimited) { diff --git a/ydb/core/tx/datashard/datashard__engine_host.cpp b/ydb/core/tx/datashard/datashard__engine_host.cpp index ad5c3f7974..92184e0057 100644 --- a/ydb/core/tx/datashard/datashard__engine_host.cpp +++ b/ydb/core/tx/datashard/datashard__engine_host.cpp @@ -1278,13 +1278,9 @@ NKqp::TKqpTasksRunner& TEngineBay::GetKqpTasksRunner(NKikimrTxDataShard::TKqpTra NYql::NDq::TDqTaskRunnerSettings settings; if (tx.HasRuntimeSettings() && tx.GetRuntimeSettings().HasStatsMode()) { - auto statsMode = tx.GetRuntimeSettings().GetStatsMode(); - // Always collect basic stats for system views / request unit computation. - settings.CollectBasicStats = true; - settings.CollectProfileStats = statsMode >= NYql::NDqProto::DQ_STATS_MODE_PROFILE; + settings.StatsMode = tx.GetRuntimeSettings().GetStatsMode(); } else { - settings.CollectBasicStats = false; - settings.CollectProfileStats = false; + settings.StatsMode = NYql::NDqProto::DQ_STATS_MODE_NONE; } settings.OptLLVM = "OFF"; diff --git a/ydb/core/tx/datashard/datashard_kqp.cpp b/ydb/core/tx/datashard/datashard_kqp.cpp index 85b3ca8669..7c63dbd15b 100644 --- a/ydb/core/tx/datashard/datashard_kqp.cpp +++ b/ydb/core/tx/datashard/datashard_kqp.cpp @@ -1040,10 +1040,6 @@ void KqpFillStats(TDataShard& dataShard, const NKqp::TKqpTasksRunner& tasksRunne Y_ABORT_UNLESS(dataShard.GetUserTables().size() == 1, "TODO: Fix handling of collocated tables"); auto tableInfo = dataShard.GetUserTables().begin(); - // Directly use StatsMode instead of bool flag, too much is reported for STATS_COLLECTION_BASIC mode. - bool withBasicStats = statsMode >= NYql::NDqProto::DQ_STATS_MODE_BASIC; - bool withProfileStats = statsMode >= NYql::NDqProto::DQ_STATS_MODE_PROFILE; - auto& computeActorStats = *result.Record.MutableComputeActorStats(); ui64 minFirstRowTimeMs = std::numeric_limits<ui64>::max(); @@ -1063,19 +1059,19 @@ void KqpFillStats(TDataShard& dataShard, const NKqp::TKqpTasksRunner& tasksRunne protoTable->SetWriteBytes(taskTableStats.UpdateRowBytes); protoTable->SetEraseRows(taskTableStats.NEraseRow); - if (!withBasicStats) { + if (statsMode <= NYql::NDqProto::DQ_STATS_MODE_NONE) { // UNSPECIFIED === NONE continue; } auto stageId = tasksRunner.GetTask(taskId).GetStageId(); - NYql::NDq::FillTaskRunnerStats(taskId, stageId, *taskStats, protoTask, withProfileStats); + NYql::NDq::FillTaskRunnerStats(taskId, stageId, *taskStats, protoTask, NYql::NDq::StatsModeToCollectStatsLevel(statsMode)); - minFirstRowTimeMs = std::min(minFirstRowTimeMs, protoTask->GetFirstRowTimeMs()); - maxFinishTimeMs = std::max(maxFinishTimeMs, protoTask->GetFinishTimeMs()); + // minFirstRowTimeMs = std::min(minFirstRowTimeMs, protoTask->GetFirstRowTimeMs()); + // maxFinishTimeMs = std::max(maxFinishTimeMs, protoTask->GetFinishTimeMs()); computeActorStats.SetCpuTimeUs(computeActorStats.GetCpuTimeUs() + protoTask->GetCpuTimeUs()); - if (Y_UNLIKELY(withProfileStats)) { + if (Y_UNLIKELY(statsMode >= NYql::NDqProto::DQ_STATS_MODE_FULL)) { NKqpProto::TKqpShardTableExtraStats tableExtraStats; tableExtraStats.SetShardId(dataShard.TabletID()); protoTable->MutableExtra()->PackFrom(tableExtraStats); diff --git a/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp b/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp index a605f93c19..09a6db5e29 100644 --- a/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_async_compute_actor.cpp @@ -184,14 +184,12 @@ private: WaitingForStateResponse.clear(); } - const TDqAsyncOutputBufferStats* GetSinkStats(ui64 outputIdx, const TAsyncOutputInfoBase& sinkInfo) const override { - Y_UNUSED(sinkInfo); - return TaskRunnerStats.GetSinkStats(outputIdx); + const IDqAsyncOutputBuffer* GetSink(ui64 outputIdx, const TAsyncOutputInfoBase&) const override { + return TaskRunnerStats.GetSink(outputIdx); } - const TDqAsyncInputBufferStats* GetInputTransformStats(ui64 inputIdx, const TAsyncInputTransformInfo& inputTransformInfo) const override { - Y_UNUSED(inputTransformInfo); - return TaskRunnerStats.GetInputTransformStats(inputIdx); + const IDqAsyncInputBuffer* GetInputTransform(ui64 inputIdx, const TAsyncInputTransformInfo&) const override { + return TaskRunnerStats.GetInputTransform(inputIdx); } void DrainOutputChannel(TOutputChannelInfo& outputChannel) override { @@ -746,7 +744,7 @@ private: return MkqlMemoryLimit; } - const TDqMemoryQuota::TProfileStats* GetProfileStats() const override { + const TDqMemoryQuota::TProfileStats* GetMemoryProfileStats() const override { return &ProfileStats; } diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor.h b/ydb/library/yql/dq/actors/compute/dq_compute_actor.h index a3203b35d2..a4788cddec 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor.h +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor.h @@ -5,6 +5,7 @@ #include <ydb/library/yql/dq/actors/protos/dq_events.pb.h> #include <ydb/library/yql/dq/common/dq_common.h> #include <ydb/library/yql/dq/proto/dq_checkpoint.pb.h> +#include <ydb/library/yql/dq/runtime/dq_async_stats.h> #include <ydb/library/yql/dq/runtime/dq_tasks_runner.h> #include <ydb/library/yql/dq/runtime/dq_transport.h> #include <ydb/library/yql/public/issue/yql_issue.h> @@ -214,6 +215,13 @@ struct TReportStatsSettings { TDuration MaxInterval; }; +inline TCollectStatsLevel StatsModeToCollectStatsLevel(NDqProto::EDqStatsMode statsMode) { + if (statsMode >= NDqProto::DQ_STATS_MODE_PROFILE) return TCollectStatsLevel::Profile; + else if (statsMode >= NDqProto::DQ_STATS_MODE_FULL) return TCollectStatsLevel::Full; + else if (statsMode >= NDqProto::DQ_STATS_MODE_BASIC) return TCollectStatsLevel::Basic; + else return TCollectStatsLevel::None; +} + struct TComputeRuntimeSettings { TMaybe<TDuration> Timeout; NDqProto::EDqStatsMode StatsMode = NDqProto::DQ_STATS_MODE_NONE; @@ -230,6 +238,26 @@ struct TComputeRuntimeSettings { std::function<void(bool success, const TIssues& issues)> TerminateHandler; TMaybe<NDqProto::TRlPath> RlPath; + + inline bool CollectNone() const { + return StatsMode <= NDqProto::DQ_STATS_MODE_NONE; + } + + inline bool CollectBasic() const { + return StatsMode >= NDqProto::DQ_STATS_MODE_BASIC; + } + + inline bool CollectFull() const { + return StatsMode >= NDqProto::DQ_STATS_MODE_FULL; + } + + inline bool CollectProfile() const { + return StatsMode >= NDqProto::DQ_STATS_MODE_PROFILE; + } + + inline TCollectStatsLevel GetCollectStatsLevel() const { + return StatsModeToCollectStatsLevel(StatsMode); + } }; struct TGuaranteeQuotaManager : public IMemoryQuotaManager { @@ -239,6 +267,7 @@ struct TGuaranteeQuotaManager : public IMemoryQuotaManager { Y_ABORT_UNLESS(Limit >= Guarantee); Y_ABORT_UNLESS(Limit >= Quota); Y_ABORT_UNLESS((Step ^ ~Step) + 1 == 0); + MaxMemorySize = Limit; } bool AllocateQuota(ui64 memorySize) override { @@ -252,6 +281,9 @@ struct TGuaranteeQuotaManager : public IMemoryQuotaManager { } Limit += delta; + if (MaxMemorySize < Limit) { + MaxMemorySize = Limit; + } } Quota += memorySize; @@ -274,6 +306,10 @@ struct TGuaranteeQuotaManager : public IMemoryQuotaManager { return Quota; } + ui64 GetMaxMemorySize() const override { + return MaxMemorySize; + }; + virtual bool AllocateExtraQuota(ui64) { return false; } @@ -285,6 +321,7 @@ struct TGuaranteeQuotaManager : public IMemoryQuotaManager { ui64 Guarantee; // do not free memory below this value even if Quota == 0 ui64 Step; // allocation chunk size ui64 Quota; // current value + ui64 MaxMemorySize; // usage peak for statistics }; struct TChainedQuotaManager : public TGuaranteeQuotaManager { @@ -321,8 +358,10 @@ using TTaskRunnerFactory = std::function< TIntrusivePtr<IDqTaskRunner>(const TDqTaskSettings& task, const TLogFunc& logFunc) >; +void FillAsyncStats(NDqProto::TDqAsyncBufferStats& proto, TDqAsyncStats stats); + void FillTaskRunnerStats(ui64 taskId, ui32 stageId, const TTaskRunnerStatsBase& taskStats, - NDqProto::TDqTaskStats* protoTask, bool withProfileStats, const THashMap<ui64, ui64>& ingressBytesMap = {}); + NDqProto::TDqTaskStats* protoTask, TCollectStatsLevel level); NActors::IActor* CreateDqComputeActor(const NActors::TActorId& executerId, const TTxId& txId, NDqProto::TDqTask* task, IDqAsyncIoFactory::TPtr asyncIoFactory, diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h b/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h index 39b20c49d8..b553e7fc20 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h @@ -38,6 +38,7 @@ struct IMemoryQuotaManager { virtual bool AllocateQuota(ui64 memorySize) = 0; virtual void FreeQuota(ui64 memorySize) = 0; virtual ui64 GetCurrentQuota() const = 0; + virtual ui64 GetMaxMemorySize() const = 0; }; // Source/transform. @@ -79,6 +80,8 @@ struct IDqComputeActorAsyncInput { virtual ui64 GetInputIndex() const = 0; + virtual const TDqAsyncStats& GetIngressStats() const = 0; + // Gets data and returns space used by filled data batch. // Watermark will be returned if source watermark was moved forward. Watermark should be handled AFTER data. // Method should be called under bound mkql allocator. @@ -94,10 +97,6 @@ struct IDqComputeActorAsyncInput { virtual void CommitState(const NDqProto::TCheckpoint& checkpoint) = 0; // Apply side effects related to this checkpoint. virtual void LoadState(const NDqProto::TSourceState& state) = 0; - virtual ui64 GetIngressBytes() { - return 0; - } - virtual TDuration GetCpuTime() { return TDuration::Zero(); } @@ -153,6 +152,8 @@ struct IDqComputeActorAsyncOutput { virtual i64 GetFreeSpace() const = 0; + virtual const TDqAsyncStats& GetEgressStats() const = 0; + // Sends data. // Method shoud be called under bound mkql allocator. // Could throw YQL errors. @@ -165,10 +166,6 @@ struct IDqComputeActorAsyncOutput { virtual void CommitState(const NDqProto::TCheckpoint& checkpoint) = 0; // Apply side effects related to this checkpoint. virtual void LoadState(const NDqProto::TSinkState& state) = 0; - virtual ui64 GetEgressBytes() { - return 0; - } - virtual void PassAway() = 0; // The same signature as IActor::PassAway() virtual ~IDqComputeActorAsyncOutput() = default; @@ -181,6 +178,7 @@ public: struct TSourceArguments { const NDqProto::TTaskInput& InputDesc; ui64 InputIndex; + TCollectStatsLevel StatsLevel; TTxId TxId; ui64 TaskId; const THashMap<TString, TString>& SecureParams; @@ -199,6 +197,7 @@ public: struct TSinkArguments { const NDqProto::TTaskOutput& OutputDesc; ui64 OutputIndex; + TCollectStatsLevel StatsLevel; TTxId TxId; ui64 TaskId; IDqComputeActorAsyncOutput::ICallbacks* Callback; @@ -212,6 +211,7 @@ public: struct TInputTransformArguments { const NDqProto::TTaskInput& InputDesc; const ui64 InputIndex; + TCollectStatsLevel StatsLevel; TTxId TxId; ui64 TaskId; const NUdf::TUnboxedValue TransformInput; @@ -227,6 +227,7 @@ public: struct TOutputTransformArguments { const NDqProto::TTaskOutput& OutputDesc; const ui64 OutputIndex; + TCollectStatsLevel StatsLevel; TTxId TxId; ui64 TaskId; const IDqOutputConsumer::TPtr TransformOutput; diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.cpp b/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.cpp index d60aa2f81e..7d2add26b5 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.cpp @@ -111,14 +111,6 @@ void TDqComputeActorChannels::HandleWork(TEvDqCompute::TEvChannelData::TPtr& ev) TInputChannelState& inputChannel = InCh(channelId); - if (Y_UNLIKELY(inputChannel.Stats)) { - auto now = Now(); - if (inputChannel.Stats->FirstMessageTs == TInstant::Zero()) { - inputChannel.Stats->FirstMessageTs = now; - } - inputChannel.Stats->LastMessageTs = now; - } - LOG_T("Received input for channelId: " << channelId << ", seqNo: " << record.GetSeqNo() << ", size: " << channelData.Proto.GetData().GetRaw().size() @@ -160,19 +152,6 @@ void TDqComputeActorChannels::HandleWork(TEvDqCompute::TEvChannelData::TPtr& ev) << " messages: " << InFlightMessagesStr(inputChannel.InFlight)); } - if (inputChannel.PollRequest && inputChannel.PollRequest->SeqNo <= record.GetSeqNo()) { - if (Y_UNLIKELY(inputChannel.Stats)) { - auto waitTime = TInstant::Now() - *inputChannel.StartPollTime; - if (inputChannel.Stats->FirstMessageTs == TInstant::Zero()) { - inputChannel.Stats->IdleTime += waitTime; - } else { - inputChannel.Stats->WaitTime += waitTime; - } - inputChannel.StartPollTime.reset(); - } - inputChannel.PollRequest.reset(); - } - if (channelData.Proto.GetFinished()) { LOG_D("Finish input channelId: " << channelId << ", from: " << *inputChannel.Peer); inputChannel.Finished = true; @@ -258,15 +237,6 @@ void TDqComputeActorChannels::HandleWork(TEvDqCompute::TEvRetryChannelData::TPtr LOG_E("Output channelId: " << msg->ChannelId << " has unexpected inflight message seqNo: " << seqNo); } - if (Y_UNLIKELY(outputChannel.Stats)) { - outputChannel.Stats->ResentMessages++; - auto now = Now(); - if (outputChannel.Stats->FirstMessageTs == TInstant::Zero()) { - outputChannel.Stats->FirstMessageTs = now; - } - outputChannel.Stats->LastMessageTs = now; - } - auto retryEv = MakeHolder<TEvDqCompute::TEvChannelData>(); retryEv->Record.SetSeqNo(seqNo); retryEv->Record.SetSendTime(now.MilliSeconds()); @@ -586,14 +556,6 @@ void TDqComputeActorChannels::SendChannelData(TChannelDataOOB&& channelData, con << ", seqNo: " << seqNo << ", finished: " << finished); - if (Y_UNLIKELY(outputChannel.Stats)) { - auto now = Now(); - if (outputChannel.Stats->FirstMessageTs == TInstant::Zero()) { - outputChannel.Stats->FirstMessageTs = now; - } - outputChannel.Stats->LastMessageTs = now; - } - auto dataEv = MakeHolder<TEvDqCompute::TEvChannelData>(); dataEv->Record.SetSeqNo(seqNo); dataEv->Record.SetSendTime(Now().MilliSeconds()); diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.h b/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.h index c00fe1ab25..078c5bb023 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.h +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_channels.h @@ -74,16 +74,10 @@ public: struct TInputChannelStats { ui64 PollRequests = 0; ui64 ResentMessages = 0; - TDuration IdleTime; // wait time until 1st message received - TDuration WaitTime; // wait time after 1st message received - TInstant FirstMessageTs; - TInstant LastMessageTs; }; struct TOutputChannelStats { ui64 ResentMessages = 0; - TInstant FirstMessageTs; - TInstant LastMessageTs; }; public: 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 b7b5b48fdb..8c433ee435 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 @@ -206,9 +206,6 @@ protected: , Running(!Task.GetCreateSuspended()) , PassExceptions(passExceptions) { - if (RuntimeSettings.StatsMode >= NDqProto::DQ_STATS_MODE_BASIC) { - BasicStats = std::make_unique<TBasicStats>(); - } InitMonCounters(taskCounters); InitializeTask(); if (ownMemoryQuota) { @@ -226,9 +223,9 @@ protected: } void ReportEventElapsedTime() { - if (BasicStats) { + if (RuntimeSettings.CollectBasic()) { ui64 elapsedMicros = NActors::TlsActivationContext->GetCurrentEventTicksAsSeconds() * 1'000'000ull; - BasicStats->CpuTime += TDuration::MicroSeconds(elapsedMicros); + CpuTime += TDuration::MicroSeconds(elapsedMicros); } } @@ -294,7 +291,7 @@ protected: MemoryLimits, TxId, Task.GetId(), - RuntimeSettings.StatsMode >= NDqProto::DQ_STATS_MODE_PROFILE, + RuntimeSettings.CollectFull(), CanAllocateExtraMemory, NActors::TActivationContext::ActorSystem()); } @@ -1370,7 +1367,7 @@ protected: } private: - virtual const TDqMemoryQuota::TProfileStats* GetProfileStats() const { + virtual const TDqMemoryQuota::TProfileStats* GetMemoryProfileStats() const { Y_ABORT_UNLESS(MemoryQuota); return MemoryQuota->GetProfileStats(); } @@ -1566,6 +1563,7 @@ protected: channel.Channel = TaskRunner->GetInputChannel(channelId); } } + auto collectStatsLevel = StatsModeToCollectStatsLevel(RuntimeSettings.StatsMode); for (auto& [inputIndex, source] : SourcesMap) { if (TaskRunner) { source.Buffer = TaskRunner->GetSource(inputIndex); Y_ABORT_UNLESS(source.Buffer);} Y_ABORT_UNLESS(AsyncIoFactory); @@ -1581,6 +1579,7 @@ protected: IDqAsyncIoFactory::TSourceArguments { .InputDesc = inputDesc, .InputIndex = inputIndex, + .StatsLevel = collectStatsLevel, .TxId = TxId, .SecureParams = secureParams, .TaskParams = taskParams, @@ -1612,6 +1611,7 @@ protected: IDqAsyncIoFactory::TInputTransformArguments { .InputDesc = inputDesc, .InputIndex = inputIndex, + .StatsLevel = collectStatsLevel, .TxId = TxId, .TaskId = Task.GetId(), .TransformInput = transform.InputBuffer, @@ -1647,6 +1647,7 @@ protected: IDqAsyncIoFactory::TOutputTransformArguments { .OutputDesc = outputDesc, .OutputIndex = outputIndex, + .StatsLevel = collectStatsLevel, .TxId = TxId, .TransformOutput = transform.OutputBuffer, .Callback = static_cast<TOutputTransformCallbacks*>(this), @@ -1675,6 +1676,7 @@ protected: IDqAsyncIoFactory::TSinkArguments { .OutputDesc = outputDesc, .OutputIndex = outputIndex, + .StatsLevel = collectStatsLevel, .TxId = TxId, .Callback = static_cast<TSinkCallbacks*>(this), .SecureParams = secureParams, @@ -1944,14 +1946,12 @@ private: return TaskRunner->GetStats(); } - virtual const TDqAsyncOutputBufferStats* GetSinkStats(ui64 outputIdx, const TAsyncOutputInfoBase& sinkInfo) const { - Y_UNUSED(outputIdx); - return sinkInfo.Buffer ? sinkInfo.Buffer->GetStats() : nullptr; + virtual const IDqAsyncOutputBuffer* GetSink(ui64, const TAsyncOutputInfoBase& sinkInfo) const { + return sinkInfo.Buffer.Get(); } - virtual const TDqAsyncInputBufferStats* GetInputTransformStats(ui64 inputIdx, const TAsyncInputTransformInfo& inputTransformInfo) const { - Y_UNUSED(inputIdx); - return inputTransformInfo.Buffer ? inputTransformInfo.Buffer->GetStats() : nullptr; + virtual const IDqAsyncInputBuffer* GetInputTransform(ui64, const TAsyncInputTransformInfo& inputTransformInfo) const { + return inputTransformInfo.Buffer.Get(); } public: @@ -1972,7 +1972,7 @@ public: } void FillStats(NDqProto::TDqComputeActorStats* dst, bool last) { - if (!BasicStats) { + if (RuntimeSettings.CollectNone()) { return; } @@ -1980,12 +1980,13 @@ public: ReportEventElapsedTime(); } - dst->SetCpuTimeUs(BasicStats->CpuTime.MicroSeconds()); + dst->SetCpuTimeUs(CpuTime.MicroSeconds()); + dst->SetMaxMemoryUsage(MemoryLimits.MemoryQuotaManager->GetMaxMemorySize()); - if (GetProfileStats()) { - dst->SetMkqlMaxMemoryUsage(GetProfileStats()->MkqlMaxUsedMemory); - dst->SetMkqlExtraMemoryBytes(GetProfileStats()->MkqlExtraMemoryBytes); - dst->SetMkqlExtraMemoryRequests(GetProfileStats()->MkqlExtraMemoryRequests); + if (auto memProfileStats = GetMemoryProfileStats(); memProfileStats) { + dst->SetMkqlMaxMemoryUsage(memProfileStats->MkqlMaxUsedMemory); + dst->SetMkqlExtraMemoryBytes(memProfileStats->MkqlExtraMemoryBytes); + dst->SetMkqlExtraMemoryRequests(memProfileStats->MkqlExtraMemoryRequests); } if (Stat) { // for task_runner_actor @@ -2005,67 +2006,88 @@ public: } else if (auto* taskStats = GetTaskRunnerStats()) { // for task_runner_actor_local auto* protoTask = dst->AddTasks(); - THashMap<TString, ui64> Ingress; - THashMap<TString, ui64> Egress; - - THashMap<ui64, ui64> ingressBytesMap; for (auto& [inputIndex, sourceInfo] : SourcesMap) { if (auto* source = sourceInfo.AsyncInput) { - auto ingressBytes = sourceInfo.AsyncInput->GetIngressBytes(); - ingressBytesMap.emplace(inputIndex, ingressBytes); - Ingress[sourceInfo.Type] = Ingress.Value(sourceInfo.Type, 0) + ingressBytes; // TODO: support async CA source->FillExtraStats(protoTask, last, TaskRunner ? TaskRunner->GetMeteringStats() : nullptr); } } - FillTaskRunnerStats(Task.GetId(), Task.GetStageId(), *taskStats, protoTask, (bool) GetProfileStats(), ingressBytesMap); + FillTaskRunnerStats(Task.GetId(), Task.GetStageId(), *taskStats, protoTask, RuntimeSettings.GetCollectStatsLevel()); // More accurate cpu time counter: if (TDerived::HasAsyncTaskRunner) { - protoTask->SetCpuTimeUs(BasicStats->CpuTime.MicroSeconds() + taskStats->ComputeCpuTime.MicroSeconds() + taskStats->BuildCpuTime.MicroSeconds()); + protoTask->SetCpuTimeUs(CpuTime.MicroSeconds() + taskStats->ComputeCpuTime.MicroSeconds() + taskStats->BuildCpuTime.MicroSeconds()); } protoTask->SetSourceCpuTimeUs(SourceCpuTime.MicroSeconds()); - for (auto& [outputIndex, sinkInfo] : SinksMap) { - - ui64 egressBytes = sinkInfo.AsyncOutput ? sinkInfo.AsyncOutput->GetEgressBytes() : 0; + THashMap<TString, TDqAsyncStats> ingressStats; + THashMap<TString, TDqAsyncStats> egressStats; + TDqAsyncStats pushStats; - if (auto* sinkStats = GetSinkStats(outputIndex, sinkInfo)) { - protoTask->SetOutputRows(protoTask->GetOutputRows() + sinkStats->RowsIn); - protoTask->SetOutputBytes(protoTask->GetOutputBytes() + sinkStats->Bytes); - Egress[sinkInfo.Type] = Egress.Value(sinkInfo.Type, 0) + egressBytes; - - if (GetProfileStats()) { - auto* protoSink = protoTask->AddSinks(); - protoSink->SetOutputIndex(outputIndex); - - protoSink->SetChunks(sinkStats->Chunks); - protoSink->SetBytes(sinkStats->Bytes); - protoSink->SetRowsIn(sinkStats->RowsIn); - protoSink->SetRowsOut(sinkStats->RowsOut); - protoSink->SetEgressBytes(egressBytes); + if (RuntimeSettings.CollectFull()) { + // in full/profile mode enumerate existing protos + for (auto& protoSource : *protoTask->MutableSources()) { + if (auto* sourceInfoPtr = SourcesMap.FindPtr(protoSource.GetInputIndex())) { + auto& sourceInfo = *sourceInfoPtr; + protoSource.SetIngressName(sourceInfo.Type); + FillAsyncStats(*protoSource.MutableIngress(), sourceInfo.AsyncInput->GetIngressStats()); + ingressStats[sourceInfo.Type].MergeData(sourceInfo.AsyncInput->GetIngressStats()); + } + } + } else { + // in basic mode enum sources directly + for (auto& [inputIndex, sourceInfo] : SourcesMap) { + ingressStats[sourceInfo.Type].MergeData(sourceInfo.AsyncInput->GetIngressStats()); + } + }; - protoSink->SetMaxMemoryUsage(sinkStats->MaxMemoryUsage); - protoSink->SetErrorsCount(sinkInfo.IssuesBuffer.GetAllAddedIssuesCount()); + for (auto& [outputIndex, sinkInfo] : SinksMap) { + if (auto* sink = GetSink(outputIndex, sinkInfo)) { + if (RuntimeSettings.CollectFull()) { + auto& protoSink = *protoTask->AddSinks(); + protoSink.SetOutputIndex(outputIndex); + protoSink.SetEgressName(sinkInfo.Type); + FillAsyncStats(*protoSink.MutablePush(), sink->GetPushStats()); + FillAsyncStats(*protoSink.MutablePop(), sink->GetPopStats()); + FillAsyncStats(*protoSink.MutableEgress(), sinkInfo.AsyncOutput->GetEgressStats()); + protoSink.SetMaxMemoryUsage(sink->GetPopStats().MaxMemoryUsage); + protoSink.SetErrorsCount(sinkInfo.IssuesBuffer.GetAllAddedIssuesCount()); } + egressStats[sinkInfo.Type].MergeData(sinkInfo.AsyncOutput->GetEgressStats()); + pushStats.MergeData(sink->GetPushStats()); + // p.s. sink == sinkInfo.Buffer } } + for (auto& [name, stats] : ingressStats) { + auto& protoIngress = *protoTask->AddIngress(); + protoIngress.SetName(name); + protoIngress.SetBytes(stats.Bytes); + protoIngress.SetRows(stats.Rows); + protoIngress.SetChunks(stats.Chunks); + protoIngress.SetSplits(stats.Splits); + } + for (auto& [name, stats] : egressStats) { + auto& protoEgress = *protoTask->AddEgress(); + protoEgress.SetName(name); + protoEgress.SetBytes(stats.Bytes); + protoEgress.SetRows(stats.Rows); + protoEgress.SetChunks(stats.Chunks); + protoEgress.SetSplits(stats.Splits); + } + // add egress to output channel stats + protoTask->SetOutputRows(protoTask->GetOutputRows() + pushStats.Rows); + protoTask->SetOutputBytes(protoTask->GetOutputBytes() + pushStats.Bytes); + for (auto& [inputIndex, transformInfo] : InputTransformsMap) { - auto* transformStats = GetInputTransformStats(inputIndex, transformInfo); - if (transformStats && GetProfileStats()) { - YQL_ENSURE(transformStats); - ui64 ingressBytes = transformInfo.AsyncInput ? transformInfo.AsyncInput->GetIngressBytes() : 0; - - auto* protoTransform = protoTask->AddInputTransforms(); - protoTransform->SetInputIndex(inputIndex); - protoTransform->SetChunks(transformStats->Chunks); - protoTransform->SetBytes(transformStats->Bytes); - protoTransform->SetRowsIn(transformStats->RowsIn); - protoTransform->SetRowsOut(transformStats->RowsOut); - protoTransform->SetIngressBytes(ingressBytes); - - protoTransform->SetMaxMemoryUsage(transformStats->MaxMemoryUsage); + auto* transform = GetInputTransform(inputIndex, transformInfo); + if (transform && RuntimeSettings.CollectFull()) { + // TODO: Ingress clarification + auto& protoTransform = *protoTask->AddInputTransforms(); + protoTransform.SetInputIndex(inputIndex); + FillAsyncStats(*protoTransform.MutablePush(), transform->GetPushStats()); + FillAsyncStats(*protoTransform.MutablePop(), transform->GetPopStats()); + protoTransform.SetMaxMemoryUsage(transform->PushStats.MaxMemoryUsage); } if (auto* transform = transformInfo.AsyncInput) { @@ -2074,51 +2096,57 @@ public: } } - for (auto& [name, bytes] : Egress) { - auto* egressStats = protoTask->AddEgress(); - egressStats->SetName(name); - egressStats->SetBytes(bytes); - } - for (auto& [name, bytes] : Ingress) { - auto* ingressStats = protoTask->AddIngress(); - ingressStats->SetName(name); - ingressStats->SetBytes(bytes); - } - - if (GetProfileStats()) { + if (RuntimeSettings.CollectFull()) { for (auto& protoSource : *protoTask->MutableSources()) { if (auto* sourceInfo = SourcesMap.FindPtr(protoSource.GetInputIndex())) { protoSource.SetErrorsCount(sourceInfo->IssuesBuffer.GetAllAddedIssuesCount()); } } - for (auto& protoInputChannelStats : *protoTask->MutableInputChannels()) { - if (auto* caChannelStats = Channels->GetInputChannelStats(protoInputChannelStats.GetChannelId())) { - protoInputChannelStats.SetPollRequests(caChannelStats->PollRequests); - protoInputChannelStats.SetWaitTimeUs(caChannelStats->WaitTime.MicroSeconds()); - protoInputChannelStats.SetResentMessages(caChannelStats->ResentMessages); - protoInputChannelStats.SetFirstMessageMs(caChannelStats->FirstMessageTs.MilliSeconds()); - protoInputChannelStats.SetLastMessageMs(caChannelStats->LastMessageTs.MilliSeconds()); - } - - if (auto* inputInfo = InputChannelsMap.FindPtr(protoInputChannelStats.GetChannelId())) { - protoInputChannelStats.SetSrcStageId(inputInfo->SrcStageId); + for (auto& protoChannel : *protoTask->MutableInputChannels()) { + if (auto channelId = protoChannel.GetChannelId()) { // Profile or Full Single + if (auto* channelStats = Channels->GetInputChannelStats(channelId)) { + protoChannel.SetPollRequests(channelStats->PollRequests); + protoChannel.SetResentMessages(channelStats->ResentMessages); + } + } else if (auto srcStageId = protoChannel.GetSrcStageId()) { // Full Aggregated + // TODO Optimize + ui64 pollRequests = 0; + ui64 resentMessages = 0; + for (const auto& [channelId, channel] : InputChannelsMap) { + if (channel.SrcStageId == srcStageId) { + if (auto* channelStats = Channels->GetInputChannelStats(channelId)) { + pollRequests += channelStats->PollRequests; + resentMessages += channelStats->ResentMessages; + } + } + } + if (pollRequests) { + protoChannel.SetPollRequests(pollRequests); + } + if (resentMessages) { + protoChannel.SetResentMessages(resentMessages); + } } } - for (auto& protoOutputChannelStats : *protoTask->MutableOutputChannels()) { - if (auto* caChannelStats = Channels->GetOutputChannelStats(protoOutputChannelStats.GetChannelId())) { - protoOutputChannelStats.SetResentMessages(caChannelStats->ResentMessages); - protoOutputChannelStats.SetFirstMessageMs(caChannelStats->FirstMessageTs.MilliSeconds()); - protoOutputChannelStats.SetLastMessageMs(caChannelStats->LastMessageTs.MilliSeconds()); - } - - if (auto* outputInfo = OutputChannelsMap.FindPtr(protoOutputChannelStats.GetChannelId())) { - protoOutputChannelStats.SetDstStageId(outputInfo->DstStageId); - if (auto *outputChannelStats = outputInfo->Stats.Get()) { - protoOutputChannelStats.SetBlockedTimeUs(outputChannelStats->BlockedTime.MicroSeconds()); - protoOutputChannelStats.SetBlockedByCapacity(outputChannelStats->BlockedByCapacity); - protoOutputChannelStats.SetNoDstActorId(outputChannelStats->NoDstActorId); + for (auto& protoChannel : *protoTask->MutableOutputChannels()) { + if (auto channelId = protoChannel.GetChannelId()) { // Profile or Full Single + if (auto* channelStats = Channels->GetOutputChannelStats(channelId)) { + protoChannel.SetResentMessages(channelStats->ResentMessages); + } + } else if (auto dstStageId = protoChannel.GetDstStageId()) { // Full Aggregated + // TODO Optimize + ui64 resentMessages = 0; + for (const auto& [channelId, channel] : OutputChannelsMap) { + if (channel.DstStageId == dstStageId) { + if (auto* channelStats = Channels->GetOutputChannelStats(channelId)) { + resentMessages += channelStats->ResentMessages; + } + } + } + if (resentMessages) { + protoChannel.SetResentMessages(resentMessages); } } } @@ -2127,14 +2155,9 @@ public: static_cast<TDerived*>(this)->FillExtraStats(dst, last); - if (last) { - BasicStats.reset(); - } if (last && MemoryQuota) { MemoryQuota->ResetProfileStats(); } - - // Cerr << "STAAT\n" << dst->DebugString() << Endl; } protected: @@ -2198,11 +2221,7 @@ protected: bool ResumeEventScheduled = false; NDqProto::EComputeState State; TIntrusivePtr<NYql::NDq::TRequestContext> RequestContext; - - struct TBasicStats { - TDuration CpuTime; - }; - std::unique_ptr<TBasicStats> BasicStats; + TDuration CpuTime; struct TProcessOutputsState { int Inflight = 0; diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp b/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp index d6fd13f729..e86e3b11f6 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_stats.cpp @@ -6,30 +6,52 @@ namespace NYql { namespace NDq { +void FillAsyncStats(NDqProto::TDqAsyncBufferStats& proto, TDqAsyncStats stats) { + if (stats.CollectBasic()) { + proto.SetBytes(stats.Bytes); + proto.SetRows(stats.Rows); + proto.SetChunks(stats.Chunks); + proto.SetSplits(stats.Splits); + if (stats.CollectFull()) { + proto.SetFirstMessageMs(stats.FirstMessageTs.MilliSeconds()); + proto.SetPauseMessageMs(stats.PauseMessageTs.MilliSeconds()); + proto.SetResumeMessageMs(stats.ResumeMessageTs.MilliSeconds()); + proto.SetLastMessageMs(stats.LastMessageTs.MilliSeconds()); + proto.SetWaitTimeUs(stats.WaitTime.MicroSeconds()); + proto.SetWaitPeriods(stats.WaitPeriods); + } + } +} + void FillTaskRunnerStats(ui64 taskId, ui32 stageId, const TTaskRunnerStatsBase& taskStats, - NDqProto::TDqTaskStats* protoTask, bool withProfileStats, const THashMap<ui64, ui64>& ingressBytesMap) + NDqProto::TDqTaskStats* protoTask, TCollectStatsLevel level) { + if (StatsLevelCollectNone(level)) { + return; + } + protoTask->SetTaskId(taskId); protoTask->SetStageId(stageId); protoTask->SetCpuTimeUs(taskStats.ComputeCpuTime.MicroSeconds() + taskStats.BuildCpuTime.MicroSeconds()); - protoTask->SetFinishTimeMs(taskStats.FinishTs.MilliSeconds()); - protoTask->SetStartTimeMs(taskStats.StartTs.MilliSeconds()); - // Cerr << (TStringBuilder() << "FillTaskRunnerStats: " << taskStats.ComputeCpuTime << ", " << taskStats.BuildCpuTime << Endl); - if (Y_UNLIKELY(withProfileStats)) { + protoTask->SetFinishTimeMs(taskStats.FinishTs.MilliSeconds()); // to be reviewed + protoTask->SetStartTimeMs(taskStats.StartTs.MilliSeconds()); // to be reviewed + + if (StatsLevelCollectProfile(level)) { if (NActors::TlsActivationContext && NActors::TlsActivationContext->ActorSystem()) { protoTask->SetNodeId(NActors::TlsActivationContext->ActorSystem()->NodeId); } protoTask->SetHostName(HostName()); protoTask->SetComputeCpuTimeUs(taskStats.ComputeCpuTime.MicroSeconds()); protoTask->SetBuildCpuTimeUs(taskStats.BuildCpuTime.MicroSeconds()); - protoTask->SetWaitTimeUs(taskStats.WaitTime.MicroSeconds()); - protoTask->SetWaitOutputTimeUs(taskStats.WaitOutputTime.MicroSeconds()); + + protoTask->SetWaitTimeUs(taskStats.WaitTime.MicroSeconds()); // to be reviewed + protoTask->SetWaitOutputTimeUs(taskStats.WaitOutputTime.MicroSeconds()); // to be reviewed // All run statuses metrics - protoTask->SetPendingInputTimeUs(taskStats.RunStatusTimeMetrics[ERunStatus::PendingInput].MicroSeconds()); - protoTask->SetPendingOutputTimeUs(taskStats.RunStatusTimeMetrics[ERunStatus::PendingOutput].MicroSeconds()); - protoTask->SetFinishTimeUs(taskStats.RunStatusTimeMetrics[ERunStatus::Finished].MicroSeconds()); + protoTask->SetPendingInputTimeUs(taskStats.RunStatusTimeMetrics[ERunStatus::PendingInput].MicroSeconds()); // to be reviewed + protoTask->SetPendingOutputTimeUs(taskStats.RunStatusTimeMetrics[ERunStatus::PendingOutput].MicroSeconds()); // to be reviewed + protoTask->SetFinishTimeUs(taskStats.RunStatusTimeMetrics[ERunStatus::Finished].MicroSeconds()); // to be reviewed static_assert(TRunStatusTimeMetrics::StatusesCount == 3); // Add all statuses here if (taskStats.ComputeCpuTimeByRun) { @@ -49,89 +71,152 @@ void FillTaskRunnerStats(ui64 taskId, ui32 stageId, const TTaskRunnerStatsBase& } } - ui64 firstRowTs = std::numeric_limits<ui64>::max(); - - for (auto& [channelId, inputChannelStats] : taskStats.InputChannels) { - if (!inputChannelStats) { - continue; - } - - if (inputChannelStats->FirstRowTs) { - firstRowTs = std::min(firstRowTs, inputChannelStats->FirstRowTs.MilliSeconds()); - } - - protoTask->SetInputRows(protoTask->GetInputRows() + inputChannelStats->RowsOut); // Yes, OutputRows of input channel, i.e. rows that were visible by the task runner - protoTask->SetInputBytes(protoTask->GetInputBytes() + inputChannelStats->Bytes); // should be bytes out... - - if (Y_UNLIKELY(withProfileStats)) { - auto protoChannel = protoTask->AddInputChannels(); - protoChannel->SetChannelId(channelId); - protoChannel->SetChunks(inputChannelStats->Chunks); - protoChannel->SetBytes(inputChannelStats->Bytes); - protoChannel->SetRowsIn(inputChannelStats->RowsIn); - protoChannel->SetRowsOut(inputChannelStats->RowsOut); - protoChannel->SetMaxMemoryUsage(inputChannelStats->MaxMemoryUsage); - protoChannel->SetDeserializationTimeUs(inputChannelStats->DeserializationTime.MicroSeconds()); + TDqAsyncStats taskPushStats; + + for (auto& [srcStageId, inputChannels] : taskStats.InputChannels) { + switch (level) { + case TCollectStatsLevel::None: + break; + case TCollectStatsLevel::Basic: + for (auto& [channelId, inputChannel] : inputChannels) { + taskPushStats.MergeData(inputChannel->GetPushStats()); + } + break; + case TCollectStatsLevel::Full: + { + TDqInputChannelStats pushStats; + TDqAsyncStats popStats; + bool firstChannelInStage = true; + for (auto& [channelId, inputChannel] : inputChannels) { + taskPushStats.MergeData(inputChannel->GetPushStats()); + if (firstChannelInStage) { + pushStats = inputChannel->GetPushStats(); + popStats = inputChannel->GetPopStats(); + firstChannelInStage = false; + } else { + pushStats.Merge(inputChannel->GetPushStats()); + pushStats.DeserializationTime += inputChannel->GetPushStats().DeserializationTime; + pushStats.MaxMemoryUsage += inputChannel->GetPushStats().MaxMemoryUsage; + popStats.Merge(inputChannel->GetPopStats()); + } + } + if (inputChannels.size() > 1) { + pushStats.WaitTime /= inputChannels.size(); + popStats.WaitTime /= inputChannels.size(); + } + { + auto& protoChannel = *protoTask->AddInputChannels(); + protoChannel.SetSrcStageId(srcStageId); + FillAsyncStats(*protoChannel.MutablePush(), pushStats); + FillAsyncStats(*protoChannel.MutablePop(), popStats); + protoChannel.SetDeserializationTimeUs(pushStats.DeserializationTime.MicroSeconds()); + protoChannel.SetMaxMemoryUsage(pushStats.MaxMemoryUsage); + } + } + break; + case TCollectStatsLevel::Profile: + for (auto& [channelId, inputChannel] : inputChannels) { + taskPushStats.MergeData(inputChannel->GetPushStats()); + auto& protoChannel = *protoTask->AddInputChannels(); + protoChannel.SetChannelId(channelId); + protoChannel.SetSrcStageId(srcStageId); + FillAsyncStats(*protoChannel.MutablePush(), inputChannel->GetPushStats()); + FillAsyncStats(*protoChannel.MutablePop(), inputChannel->GetPopStats()); + protoChannel.SetDeserializationTimeUs(inputChannel->GetPushStats().DeserializationTime.MicroSeconds()); + protoChannel.SetMaxMemoryUsage(inputChannel->GetPushStats().MaxMemoryUsage); + } + break; } } - for (auto& [inputIndex, sourceStats] : taskStats.Sources) { - if (!sourceStats) { - continue; - } - - if (sourceStats->FirstRowTs) { - firstRowTs = std::min(firstRowTs, sourceStats->FirstRowTs.MilliSeconds()); - } - - protoTask->SetInputRows(protoTask->GetInputRows() + sourceStats->RowsOut); // the same comment here ... ^^^ - protoTask->SetInputBytes(protoTask->GetInputBytes() + sourceStats->Bytes); - - if (Y_UNLIKELY(withProfileStats)) { - auto* protoSource = protoTask->AddSources(); - protoSource->SetInputIndex(inputIndex); - protoSource->SetChunks(sourceStats->Chunks); - protoSource->SetBytes(sourceStats->Bytes); - protoSource->SetRowsIn(sourceStats->RowsIn); - protoSource->SetRowsOut(sourceStats->RowsOut); - protoSource->SetIngressBytes(ingressBytesMap.Value(inputIndex, 0)); - - protoSource->SetMaxMemoryUsage(sourceStats->MaxMemoryUsage); + protoTask->SetInputRows(taskPushStats.Rows); + protoTask->SetInputBytes(taskPushStats.Bytes); + + // + // task runner is not aware of ingress/egress stats, fill in in CA + // + for (auto& [inputIndex, sources] : taskStats.Sources) { + if (StatsLevelCollectFull(level)) { + auto& protoSource = *protoTask->AddSources(); + protoSource.SetInputIndex(inputIndex); + FillAsyncStats(*protoSource.MutablePush(), sources->GetPushStats()); + FillAsyncStats(*protoSource.MutablePop(), sources->GetPopStats()); + protoSource.SetMaxMemoryUsage(sources->GetPushStats().MaxMemoryUsage); } } - for (auto& [channelId, outputChannelStats] : taskStats.OutputChannels) { - if (!outputChannelStats) { - continue; - } - - if (outputChannelStats->FirstRowIn) { - firstRowTs = std::min(firstRowTs, outputChannelStats->FirstRowIn.MilliSeconds()); - } - - protoTask->SetOutputRows(protoTask->GetOutputRows() + outputChannelStats->RowsIn); - protoTask->SetOutputBytes(protoTask->GetOutputBytes() + outputChannelStats->Bytes); - - if (Y_UNLIKELY(withProfileStats)) { - auto* protoChannel = protoTask->AddOutputChannels(); - protoChannel->SetChannelId(channelId); - - protoChannel->SetChunks(outputChannelStats->Chunks); - protoChannel->SetBytes(outputChannelStats->Bytes); - protoChannel->SetRowsIn(outputChannelStats->RowsIn); - protoChannel->SetRowsOut(outputChannelStats->RowsOut); - - protoChannel->SetMaxMemoryUsage(outputChannelStats->MaxMemoryUsage); - protoChannel->SetMaxRowsInMemory(outputChannelStats->MaxRowsInMemory); - protoChannel->SetSerializationTimeUs(outputChannelStats->SerializationTime.MicroSeconds()); - - protoChannel->SetSpilledBytes(outputChannelStats->SpilledBytes); - protoChannel->SetSpilledRows(outputChannelStats->SpilledRows); - protoChannel->SetSpilledBlobs(outputChannelStats->SpilledBlobs); + TDqAsyncStats taskPopStats; + + for (auto& [dstStageId, outputChannels] : taskStats.OutputChannels) { + switch (level) { + case TCollectStatsLevel::None: + break; + case TCollectStatsLevel::Basic: + for (auto& [channelId, outputChannel] : outputChannels) { + taskPopStats.MergeData(outputChannel->GetPopStats()); + } + break; + case TCollectStatsLevel::Full: + { + TDqAsyncStats pushStats; + TDqOutputChannelStats popStats; + bool firstChannelInStage = true; + for (auto& [channelId, outputChannel] : outputChannels) { + taskPopStats.MergeData(outputChannel->GetPopStats()); + if (firstChannelInStage) { + pushStats = outputChannel->GetPushStats(); + popStats = outputChannel->GetPopStats(); + firstChannelInStage = false; + } else { + pushStats.Merge(outputChannel->GetPushStats()); + popStats.Merge(outputChannel->GetPopStats()); + popStats.MaxMemoryUsage += outputChannel->GetPopStats().MaxMemoryUsage; + popStats.MaxRowsInMemory += outputChannel->GetPopStats().MaxRowsInMemory; + popStats.SerializationTime += outputChannel->GetPopStats().SerializationTime; + popStats.SpilledBytes += outputChannel->GetPopStats().SpilledBytes; + popStats.SpilledRows += outputChannel->GetPopStats().SpilledRows; + popStats.SpilledBlobs += outputChannel->GetPopStats().SpilledBlobs; + } + } + if (outputChannels.size() > 1) { + pushStats.WaitTime /= outputChannels.size(); + popStats.WaitTime /= outputChannels.size(); + } + { + auto& protoChannel = *protoTask->AddOutputChannels(); + protoChannel.SetDstStageId(dstStageId); + FillAsyncStats(*protoChannel.MutablePush(), pushStats); + FillAsyncStats(*protoChannel.MutablePop(), popStats); + protoChannel.SetMaxMemoryUsage(popStats.MaxMemoryUsage); + protoChannel.SetMaxRowsInMemory(popStats.MaxRowsInMemory); + protoChannel.SetSerializationTimeUs(popStats.SerializationTime.MicroSeconds()); + protoChannel.SetSpilledBytes(popStats.SpilledBytes); + protoChannel.SetSpilledRows(popStats.SpilledRows); + protoChannel.SetSpilledBlobs(popStats.SpilledBlobs); + } + } + break; + case TCollectStatsLevel::Profile: + for (auto& [channelId, outputChannel] : outputChannels) { + taskPopStats.MergeData(outputChannel->GetPopStats()); + auto& protoChannel = *protoTask->AddOutputChannels(); + protoChannel.SetChannelId(channelId); + protoChannel.SetDstStageId(dstStageId); + FillAsyncStats(*protoChannel.MutablePush(), outputChannel->GetPushStats()); + FillAsyncStats(*protoChannel.MutablePop(), outputChannel->GetPopStats()); + protoChannel.SetMaxMemoryUsage(outputChannel->GetPopStats().MaxMemoryUsage); + protoChannel.SetMaxRowsInMemory(outputChannel->GetPopStats().MaxRowsInMemory); + protoChannel.SetSerializationTimeUs(outputChannel->GetPopStats().SerializationTime.MicroSeconds()); + protoChannel.SetSpilledBytes(outputChannel->GetPopStats().SpilledBytes); + protoChannel.SetSpilledRows(outputChannel->GetPopStats().SpilledRows); + protoChannel.SetSpilledBlobs(outputChannel->GetPopStats().SpilledBlobs); + } + break; } } - protoTask->SetFirstRowTimeMs(firstRowTs); + protoTask->SetOutputRows(taskPopStats.Rows); + protoTask->SetOutputBytes(taskPopStats.Bytes); } } // namespace NDq diff --git a/ydb/library/yql/dq/actors/protos/dq_stats.proto b/ydb/library/yql/dq/actors/protos/dq_stats.proto index f0051319fa..48f16bbf0f 100644 --- a/ydb/library/yql/dq/actors/protos/dq_stats.proto +++ b/ydb/library/yql/dq/actors/protos/dq_stats.proto @@ -8,96 +8,127 @@ import "google/protobuf/any.proto"; enum EDqStatsMode { DQ_STATS_MODE_UNSPECIFIED = 0; - DQ_STATS_MODE_NONE = 10; - DQ_STATS_MODE_BASIC = 20; - DQ_STATS_MODE_PROFILE = 30; + DQ_STATS_MODE_NONE = 10; // no statistics is expected, it will be ignored anyway + DQ_STATS_MODE_BASIC = 20; // per graph statistics + DQ_STATS_MODE_FULL = 25; // per stage/connection statistics + DQ_STATS_MODE_PROFILE = 30; // per task statistics +} + +// XxxxxMs means time period in ms since epoch + +message TDqAsyncBufferStats { + // Data + uint64 Bytes = 1; // physical bytes + uint64 Rows = 2; // logical rows (if applicable) + uint64 Chunks = 3; // chunk is group of sent/received bytes in single batch + uint64 Splits = 4; // logical partitioning (if applicable) + // Time + uint64 FirstMessageMs = 5; // first message processed + uint64 PauseMessageMs = 6; // first blocking time on empty/overflow condition + uint64 ResumeMessageMs = 7; // last unblocking time + uint64 LastMessageMs = 8; // last message processed + uint64 WaitTimeUs = 9; // SUM(Resume_i - Pause_i) in us + uint64 WaitPeriods = 10; // COUNT(Resume_i - Pause_i) } message TDqAsyncInputBufferStats { - // basic stats uint64 InputIndex = 1; - uint64 Chunks = 2; - uint64 Bytes = 3; - uint64 RowsIn = 4; - uint64 RowsOut = 5; + string IngressName = 9; + + TDqAsyncBufferStats Ingress = 10; + TDqAsyncBufferStats Push = 11; + TDqAsyncBufferStats Pop = 12; // profile stats uint64 MaxMemoryUsage = 6; uint32 ErrorsCount = 7; - uint64 IngressBytes = 8; - google.protobuf.Any Extra = 100; + + // deprecated + uint64 Chunks_Deprecated = 2; + uint64 Bytes_Deprecated = 3; + uint64 RowsIn_Deprecated = 4; + uint64 RowsOut_Deprecated = 5; + uint64 IngressBytes_Deprecated = 8; } message TDqInputChannelStats { - // basic stats uint64 ChannelId = 1; - uint32 SrcStageId = 20; - uint64 Chunks = 2; - uint64 Bytes = 3; - uint64 RowsIn = 4; - uint64 RowsOut = 5; + uint32 SrcStageId = 12; + + TDqAsyncBufferStats Push = 13; + TDqAsyncBufferStats Pop = 14; // profile stats uint64 DeserializationTimeUs = 6; uint64 MaxMemoryUsage = 7; - uint32 PollRequests = 8; uint32 ResentMessages = 9; - uint64 IdleTimeUs = 11; // wait time until 1st message received - uint64 WaitTimeUs = 10; // wait time after 1st message received - - uint64 FirstMessageMs = 30; - uint64 LastMessageMs = 31; google.protobuf.Any Extra = 100; + + // deprecated + uint64 Chunks_Deprecated = 2; + uint64 Bytes_Deprecated = 3; + uint64 RowsIn_Deprecated = 4; + uint64 RowsOut_Deprecated = 5; + uint64 IdleTimeUs_Deprecated = 11; + uint64 WaitTimeUs_Deprecated = 10; + uint64 FirstMessageMs_Deprecated = 30; + uint64 LastMessageMs_Deprecated = 31; } message TDqAsyncOutputBufferStats { - // basic stats uint64 OutputIndex = 1; - uint64 Chunks = 2; - uint64 Bytes = 3; - uint64 RowsIn = 4; - uint64 RowsOut = 5; + string EgressName = 9; + + TDqAsyncBufferStats Push = 10; + TDqAsyncBufferStats Pop = 11; + TDqAsyncBufferStats Egress = 12; // profile stats uint64 MaxMemoryUsage = 6; uint32 ErrorsCount = 7; - uint64 EgressBytes = 8; - google.protobuf.Any Extra = 100; + + // deprecated + uint64 Chunks_Deprecated = 2; + uint64 Bytes_Deprecated = 3; + uint64 RowsIn_Deprecated = 4; + uint64 RowsOut_Deprecated = 5; + uint64 EgressBytes_Deprecated = 8; } message TDqOutputChannelStats { - // basic stats uint64 ChannelId = 1; uint32 DstStageId = 20; - uint64 Chunks = 2; - uint64 Bytes = 3; - uint64 RowsIn = 4; - uint64 RowsOut = 5; + + TDqAsyncBufferStats Push = 16; + TDqAsyncBufferStats Pop = 17; // profile stats uint64 SerializationTimeUs = 6; - uint32 BlockedByCapacity = 7; uint32 ResentMessages = 8; uint32 NoDstActorId = 9; uint32 MaxRowsInMemory = 10; uint64 MaxMemoryUsage = 11; - uint64 SpilledBytes = 12; uint64 SpilledRows = 13; uint64 SpilledBlobs = 14; - uint64 BlockedTimeUs = 15; - - uint64 FirstMessageMs = 30; - uint64 LastMessageMs = 31; - google.protobuf.Any Extra = 100; + + // deprecated + uint64 Chunks_Deprecated = 2; + uint64 Bytes_Deprecated = 3; + uint64 RowsIn_Deprecated = 4; + uint64 RowsOut_Deprecated = 5; + uint32 BlockedByCapacity_Deprecated = 7; + uint64 BlockedTimeUs_Deprecated = 15; + uint64 FirstMessageMs_Deprecated = 30; + uint64 LastMessageMs_Deprecated = 31; } message TDqTableStats { @@ -122,8 +153,11 @@ message TDqMkqlStat { } message TDqDataProviderStats { - string Name = 1; - uint64 Bytes = 2; + string Name = 1; + uint64 Bytes = 2; + uint64 Rows = 3; + uint64 Chunks = 4; + uint64 Splits = 5; } message TDqTaskStats { @@ -133,45 +167,49 @@ message TDqTaskStats { uint64 TaskId = 1; uint32 StageId = 2; uint64 CpuTimeUs = 3; // total cpu time (build & compute) - uint64 SourceCpuTimeUs = 15; // time consumed in source - uint64 FirstRowTimeMs = 4; // first row time, timestamp in millis - uint64 FinishTimeMs = 5; // task finish time, timestamp in millis + uint64 SourceCpuTimeUs = 15; // time consumed in source(s) + uint64 BuildCpuTimeUs = 103; // prepare task time: build computation graph, prepare parameters, ... + uint64 ComputeCpuTimeUs = 102; // compute time only + + repeated TDqDataProviderStats Ingress = 13; + repeated TDqDataProviderStats Egress = 14; + uint64 InputRows = 6; uint64 InputBytes = 7; uint64 OutputRows = 8; uint64 OutputBytes = 9; - repeated TDqTableStats Tables = 10; - - repeated TDqDataProviderStats Ingress = 13; - repeated TDqDataProviderStats Egress = 14; + // full stats + repeated TDqAsyncInputBufferStats Sources = 150; + repeated TDqInputChannelStats InputChannels = 151; + repeated TDqAsyncOutputBufferStats Sinks = 152; + repeated TDqOutputChannelStats OutputChannels = 153; + repeated TDqAsyncInputBufferStats InputTransforms = 155; // profile stats - uint64 BuildCpuTimeUs = 103; // prepare task time: build computation graph, prepare parameters, ... - uint64 WaitTimeUs = 104; // total wait (input + output) wall time - uint64 WaitOutputTimeUs = 105; // wait output wall time (any output: channels, sinks, ...) - uint64 ComputeCpuTimeUs = 102; // compute time only - uint64 PendingInputTimeUs = 107; // time waiting input data - uint64 PendingOutputTimeUs = 108; // time waiting output data - uint64 FinishTimeUs = 109; // time in finished state // ComputeCpuTimeUs + PendingInputTimeUs + PendingOutputTimeUs + FinishTimeUs == 100% (or == const in aggregated graphs for several stages/tasks) - repeated TDqMkqlStat MkqlStats = 110; // stats from mkql + repeated TDqTableStats Tables = 10; + repeated TDqMkqlStat MkqlStats = 110; // stats from mkql message THistBucket { double Bound = 1; uint64 Value = 2; } repeated THistBucket ComputeCpuTimeByRun = 106; - repeated TDqAsyncInputBufferStats Sources = 150; - repeated TDqInputChannelStats InputChannels = 151; - repeated TDqAsyncOutputBufferStats Sinks = 152; - repeated TDqOutputChannelStats OutputChannels = 153; - repeated TDqAsyncInputBufferStats InputTransforms = 155; string HostName = 156; uint32 NodeId = 157; - uint64 StartTimeMs = 158; google.protobuf.Any Extra = 200; + + // to be reviewed + uint64 StartTimeMs = 158; + uint64 FinishTimeMs = 5; // task finish time, timestamp in millis + uint64 FirstRowTimeMs = 4; // first row time, timestamp in millis + uint64 WaitTimeUs = 104; // total wait (input + output) wall time + uint64 WaitOutputTimeUs = 105; // wait output wall time (any output: channels, sinks, ...) + uint64 PendingInputTimeUs = 107; // time waiting input data + uint64 PendingOutputTimeUs = 108; // time waiting output data + uint64 FinishTimeUs = 109; // time in finished state // ComputeCpuTimeUs + PendingInputTimeUs + PendingOutputTimeUs + FinishTimeUs == 100% (or == const in aggregated graphs for several stages/tasks) } message TDqComputeActorStats { @@ -179,6 +217,7 @@ message TDqComputeActorStats { uint64 CpuTimeUs = 1; // total cpu time: tasks cpu time + self cpu time uint64 DurationUs = 2; // compute actor duration, wall time (from FirstRowTime to FinishTime) repeated TDqTaskStats Tasks = 3; // in the BASIC_MODE only basic fields are used + uint64 MaxMemoryUsage = 4; // profile stats uint64 MkqlMaxMemoryUsage = 102; // MKQL allocations stats @@ -204,6 +243,29 @@ message TDqStatsMinMax { uint64 Max = 2; } +message TDqAsyncStatsAggr { + // Data + TDqStatsAggr Bytes = 1; + TDqStatsAggr Rows = 2; + TDqStatsAggr Chunks = 3; + TDqStatsAggr Splits = 4; + // Time + TDqStatsAggr FirstMessageMs = 5; + TDqStatsAggr PauseMessageMs = 6; + TDqStatsAggr ResumeMessageMs = 7; + TDqStatsAggr LastMessageMs = 8; + TDqStatsAggr WaitTimeUs = 9; + TDqStatsAggr WaitPeriods = 10; + TDqStatsAggr ActiveTimeUs = 11; +} + +message TDqAsyncBufferStatsAggr { + TDqAsyncStatsAggr Ingress = 1; // for Ingress only + TDqAsyncStatsAggr Push = 2; + TDqAsyncStatsAggr Pop = 3; + TDqAsyncStatsAggr Egress = 4; // for Egress only +} + message TDqTableAggrStats { string TablePath = 1; @@ -229,6 +291,7 @@ message TDqStageStats { uint32 FailedTasksCount = 6; TDqStatsAggr CpuTimeUs = 8; + TDqStatsAggr SourceCpuTimeUs = 25; TDqStatsAggr InputRows = 9; TDqStatsAggr InputBytes = 10; TDqStatsAggr OutputRows = 11; @@ -246,9 +309,12 @@ message TDqStageStats { bool UseLlvm = 18; } - // currently only 1 source/sink per stage is supported - map<string, TDqStatsAggr> IngressBytes = 19; // ingress from external source, per provider - map<string, TDqStatsAggr> EgressBytes = 20; // egress to external consumer, per provider + // currently only 1 source/sink per stage is possible, but format is ready for multiples + map<string, TDqAsyncBufferStatsAggr> Ingress = 19; // ingress from external source, per provider + map<string, TDqAsyncBufferStatsAggr> Egress = 20; // egress to external consumer, per provider + map<uint32, TDqAsyncBufferStatsAggr> Input = 22; // input from other stage, per stage + map<uint32, TDqAsyncBufferStatsAggr> Output = 23; // output to other stage, per stage + TDqStatsAggr MaxMemoryUsage = 24; google.protobuf.Any Extra = 100; } diff --git a/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp b/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp index 2312e1e771..b23306faec 100644 --- a/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp +++ b/ydb/library/yql/dq/actors/task_runner/task_runner_actor_local.cpp @@ -84,18 +84,18 @@ public: private: void OnStatisticsRequest(TEvStatistics::TPtr& ev) { TaskRunner->UpdateStats(); - THashMap<ui32, const TDqAsyncOutputBufferStats*> sinkStats; + + THashMap<ui32, const IDqAsyncOutputBuffer*> sinks; for (const auto sinkId : ev->Get()->SinkIds) { - sinkStats[sinkId] = TaskRunner->GetSink(sinkId)->GetStats(); + sinks[sinkId] = TaskRunner->GetSink(sinkId).Get(); } - THashMap<ui32, const TDqAsyncInputBufferStats*> inputTransformStats; + THashMap<ui32, const IDqAsyncInputBuffer*> inputTransforms; for (const auto inputTransformId : ev->Get()->InputTransformIds) { - inputTransformStats[inputTransformId] = TaskRunner->GetInputTransform(inputTransformId).second->GetStats(); + inputTransforms[inputTransformId] = TaskRunner->GetInputTransform(inputTransformId).second.Get(); } - ev->Get()->Stats = TDqTaskRunnerStatsView(TaskRunner->GetStats(), std::move(sinkStats), - std::move(inputTransformStats)); + ev->Get()->Stats = TDqTaskRunnerStatsView(TaskRunner->GetStats(), std::move(sinks), std::move(inputTransforms)); Send( ev->Sender, ev->Release().Release(), @@ -221,17 +221,17 @@ private: auto st = MakeHolder<TEvStatistics>(std::move(ev->Get()->SinkIds), std::move(ev->Get()->InputTransformIds)); TaskRunner->UpdateStats(); - THashMap<ui32, const TDqAsyncOutputBufferStats*> sinkStats; + THashMap<ui32, const IDqAsyncOutputBuffer*> sinks; for (const auto sinkId : st->SinkIds) { - sinkStats[sinkId] = TaskRunner->GetSink(sinkId)->GetStats(); + sinks[sinkId] = TaskRunner->GetSink(sinkId).Get(); } - THashMap<ui32, const TDqAsyncInputBufferStats*> inputTransformStats; + THashMap<ui32, const IDqAsyncInputBuffer*> inputTransforms; for (const auto inputTransformId : st->InputTransformIds) { // TODO - inputTransformStats[inputTransformId] = TaskRunner->GetInputTransform(inputTransformId).second->GetStats(); + inputTransforms[inputTransformId] = TaskRunner->GetInputTransform(inputTransformId).second.Get(); } - st->Stats = TDqTaskRunnerStatsView(TaskRunner->GetStats(), std::move(sinkStats), std::move(inputTransformStats)); + st->Stats = TDqTaskRunnerStatsView(TaskRunner->GetStats(), std::move(sinks), std::move(inputTransforms)); Send(ev->Sender, st.Release()); } @@ -242,7 +242,7 @@ private: std::move(inputChannelFreeSpace), std::move(sourcesFreeSpace), {}, - MemoryQuota ? *MemoryQuota->GetProfileStats() : TDqMemoryQuota::TProfileStats(), + (MemoryQuota && MemoryQuota->GetProfileStats()) ? *MemoryQuota->GetProfileStats() : TDqMemoryQuota::TProfileStats(), MemoryQuota ? MemoryQuota->GetMkqlMemoryLimit() : 0, std::move(mkqlProgramState), watermarkInjectedToOutputs, diff --git a/ydb/library/yql/dq/runtime/dq_async_input.cpp b/ydb/library/yql/dq/runtime/dq_async_input.cpp index b57ef6047f..5acfc3e3d1 100644 --- a/ydb/library/yql/dq/runtime/dq_async_input.cpp +++ b/ydb/library/yql/dq/runtime/dq_async_input.cpp @@ -7,14 +7,28 @@ class TDqAsyncInputBuffer : public TDqInputImpl<TDqAsyncInputBuffer, IDqAsyncInp using TBaseImpl = TDqInputImpl<TDqAsyncInputBuffer, IDqAsyncInputBuffer>; friend TBaseImpl; public: - TDqAsyncInputBuffer(ui64 inputIndex, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, bool collectProfileStats) + TDqAsyncInputBufferStats PushStats; + TDqInputStats PopStats; + + TDqAsyncInputBuffer(ui64 inputIndex, const TString& type, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, TCollectStatsLevel level) : TBaseImpl(inputType, maxBufferBytes) - , InputIndex(inputIndex) - , BasicStats(InputIndex) - , ProfileStats(collectProfileStats ? &BasicStats : nullptr) {} + { + PopStats.Level = level; + PushStats.Level = level; + PushStats.InputIndex = inputIndex; + PushStats.Type = type; + } ui64 GetInputIndex() const override { - return InputIndex; + return PushStats.InputIndex; + } + + const TDqAsyncInputBufferStats& GetPushStats() const override { + return PushStats; + } + + const TDqInputStats& GetPopStats() const override { + return PopStats; } void Push(NKikimr::NMiniKQL::TUnboxedValueBatch&& batch, i64 space) override { @@ -23,21 +37,12 @@ public: AddBatch(std::move(batch), space); } } - - const TDqAsyncInputBufferStats* GetStats() const override { - return &BasicStats; - } - -private: - const ui64 InputIndex; - TDqAsyncInputBufferStats BasicStats; - TDqAsyncInputBufferStats* ProfileStats = nullptr; }; IDqAsyncInputBuffer::TPtr CreateDqAsyncInputBuffer( - ui64 inputIndex, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, bool collectStats) + ui64 inputIndex, const TString& type, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, TCollectStatsLevel level) { - return new TDqAsyncInputBuffer(inputIndex, inputType, maxBufferBytes, collectStats); + return new TDqAsyncInputBuffer(inputIndex, type, inputType, maxBufferBytes, level); } } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/runtime/dq_async_input.h b/ydb/library/yql/dq/runtime/dq_async_input.h index 45c3ef4fc3..3ef75dec31 100644 --- a/ydb/library/yql/dq/runtime/dq_async_input.h +++ b/ydb/library/yql/dq/runtime/dq_async_input.h @@ -5,38 +5,25 @@ namespace NYql::NDq { struct TDqAsyncInputBufferStats : TDqInputStats { ui64 InputIndex = 0; - - explicit TDqAsyncInputBufferStats(ui64 inputIndex) - : InputIndex(inputIndex) {} - - template<typename T> - void FromProto(const T& f) - { - this->InputIndex = f.GetInputIndex(); - this->Chunks = f.GetChunks(); - this->Bytes = f.GetBytes(); - this->RowsIn = f.GetRowsIn(); - this->RowsOut = f.GetRowsOut(); - this->MaxMemoryUsage = f.GetMaxMemoryUsage(); - //s->StartTs = TInstant::MilliSeconds(f.GetStartTs()); - //s->FinishTs = TInstant::MilliSeconds(f.GetFinishTs()); - } + TString Type; + ui64 RowsInMemory = 0; + ui64 MaxMemoryUsage = 0; }; class IDqAsyncInputBuffer : public IDqInput { public: using TPtr = TIntrusivePtr<IDqAsyncInputBuffer>; + TDqAsyncInputBufferStats PushStats; virtual ui64 GetInputIndex() const = 0; + virtual const TDqAsyncInputBufferStats& GetPushStats() const = 0; virtual void Push(NKikimr::NMiniKQL::TUnboxedValueBatch&& batch, i64 space) = 0; virtual void Finish() = 0; - - virtual const TDqAsyncInputBufferStats* GetStats() const = 0; }; -IDqAsyncInputBuffer::TPtr CreateDqAsyncInputBuffer(ui64 inputIndex, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, - bool collectProfileStats); +IDqAsyncInputBuffer::TPtr CreateDqAsyncInputBuffer(ui64 inputIndex, const TString& type, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, + TCollectStatsLevel level); } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/runtime/dq_async_output.cpp b/ydb/library/yql/dq/runtime/dq_async_output.cpp index bfbea45e0a..e2bbf1f089 100644 --- a/ydb/library/yql/dq/runtime/dq_async_output.cpp +++ b/ydb/library/yql/dq/runtime/dq_async_output.cpp @@ -37,15 +37,29 @@ class TDqAsyncOutputBuffer : public IDqAsyncOutputBuffer { }; public: - TDqAsyncOutputBuffer(ui64 outputIndex, NKikimr::NMiniKQL::TType* outputType, ui64 maxStoredBytes, bool collectProfileStats) - : OutputIndex(outputIndex) - , MaxStoredBytes(maxStoredBytes) + TDqOutputStats PushStats; + TDqAsyncOutputBufferStats PopStats; + + TDqAsyncOutputBuffer(ui64 outputIndex, const TString& type, NKikimr::NMiniKQL::TType* outputType, ui64 maxStoredBytes, TCollectStatsLevel level) + : MaxStoredBytes(maxStoredBytes) , OutputType(outputType) - , BasicStats(OutputIndex) - , ProfileStats(collectProfileStats ? &BasicStats : nullptr) {} + { + PushStats.Level = level; + PopStats.Level = level; + PopStats.OutputIndex = outputIndex; + PopStats.Type = type; + } ui64 GetOutputIndex() const override { - return OutputIndex; + return PopStats.OutputIndex; + } + + const TDqOutputStats& GetPushStats() const override { + return PushStats; + } + + const TDqAsyncOutputBufferStats& GetPopStats() const override { + return PopStats; } bool IsFull() const override { @@ -53,10 +67,6 @@ public: } void Push(NUdf::TUnboxedValue&& value) override { - if (!BasicStats.FirstRowIn) { - BasicStats.FirstRowIn = TInstant::Now(); - } - if (ValuesPushed++ % 1000 == 0) { ReestimateRowBytes(value); } @@ -64,7 +74,7 @@ public: Values.emplace_back(std::move(value), EstimatedRowBytes); EstimatedStoredBytes += EstimatedRowBytes; - ReportChunkIn(); + ReportChunkIn(1, EstimatedRowBytes); } void WidePush(NUdf::TUnboxedValue* values, ui32 count) override { @@ -78,7 +88,7 @@ public: Values.emplace_back(std::move(watermark), bytesSize); EstimatedStoredBytes += bytesSize; - ReportChunkIn(); + ReportChunkIn(1, bytesSize); } void Push(NDqProto::TCheckpoint&& checkpoint) override { @@ -86,15 +96,11 @@ public: Values.emplace_back(std::move(checkpoint), bytesSize); EstimatedStoredBytes += bytesSize; - ReportChunkIn(); + ReportChunkIn(1, bytesSize); } void Finish() override { Finished = true; - - if (!BasicStats.FirstRowIn) { - BasicStats.FirstRowIn = TInstant::Now(); - } } ui64 Pop(NKikimr::NMiniKQL::TUnboxedValueBatch& batch, ui64 bytes) override { @@ -102,6 +108,11 @@ public: ui64 valuesCount = 0; ui64 usedBytes = 0; + if (Values.empty()) { + PushStats.TryPause(); + return 0; + } + // Calc values count. for (auto iter = Values.cbegin(), end = Values.cend(); usedBytes < bytes && iter != end && std::holds_alternative<NUdf::TUnboxedValue>(iter->Value); @@ -136,6 +147,7 @@ public: return true; } + PushStats.TryPause(); return false; } @@ -151,6 +163,7 @@ public: return true; } + PushStats.TryPause(); return false; } @@ -175,10 +188,6 @@ public: return OutputType; } - const TDqAsyncOutputBufferStats* GetStats() const override { - return &BasicStats; - } - private: void ReestimateRowBytes(const NUdf::TUnboxedValue& value) { const ui64 valueSize = TDqDataSerializer::EstimateSize(value, OutputType); @@ -192,22 +201,36 @@ private: } } - void ReportChunkIn() { - BasicStats.Bytes += EstimatedRowBytes; - BasicStats.RowsIn++; - if (ProfileStats) { - ProfileStats->MaxMemoryUsage = std::max(ProfileStats->MaxMemoryUsage, EstimatedStoredBytes); - ProfileStats->MaxRowsInMemory = std::max(ProfileStats->MaxRowsInMemory, Values.size()); + void ReportChunkIn(ui64 rows, ui64 bytes) { + if (PushStats.CollectBasic()) { + PushStats.Bytes += bytes; + PushStats.Rows += rows; + PushStats.Chunks++; + PushStats.Resume(); + } + + if (IsFull()) { + PopStats.TryPause(); + } + + if (PopStats.CollectFull()) { + PopStats.MaxMemoryUsage = std::max(PopStats.MaxMemoryUsage, EstimatedStoredBytes); + PopStats.MaxRowsInMemory = std::max(PopStats.MaxRowsInMemory, Values.size()); } } - void ReportChunkOut(ui64 rowsCount, ui64 /* usedBytes */) { - BasicStats.Chunks++; - BasicStats.RowsOut += rowsCount; + void ReportChunkOut(ui64 rows, ui64 bytes) { + if (PopStats.CollectBasic()) { + PopStats.Bytes += bytes; + PopStats.Rows += rows; + PopStats.Chunks++; + if (!IsFull()) { + PopStats.Resume(); + } + } } private: - const ui64 OutputIndex; const ui64 MaxStoredBytes; NKikimr::NMiniKQL::TType* const OutputType; ui64 EstimatedStoredBytes = 0; @@ -215,16 +238,14 @@ private: bool Finished = false; std::deque<TValueDesc> Values; ui64 EstimatedRowBytes = 0; - TDqAsyncOutputBufferStats BasicStats; - TDqAsyncOutputBufferStats* ProfileStats = nullptr; }; } // namespace -IDqAsyncOutputBuffer::TPtr CreateDqAsyncOutputBuffer(ui64 outputIndex, NKikimr::NMiniKQL::TType* outputType, ui64 maxStoredBytes, - bool collectProfileStats) +IDqAsyncOutputBuffer::TPtr CreateDqAsyncOutputBuffer(ui64 outputIndex, const TString& type, NKikimr::NMiniKQL::TType* outputType, ui64 maxStoredBytes, + TCollectStatsLevel level) { - return MakeIntrusive<TDqAsyncOutputBuffer>(outputIndex, outputType, maxStoredBytes, collectProfileStats); + return MakeIntrusive<TDqAsyncOutputBuffer>(outputIndex, type, outputType, maxStoredBytes, level); } } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/runtime/dq_async_output.h b/ydb/library/yql/dq/runtime/dq_async_output.h index c8e8eb8be7..0375b6c9eb 100644 --- a/ydb/library/yql/dq/runtime/dq_async_output.h +++ b/ydb/library/yql/dq/runtime/dq_async_output.h @@ -8,20 +8,10 @@ namespace NYql::NDq { struct TDqAsyncOutputBufferStats : TDqOutputStats { - const ui64 OutputIndex; - - explicit TDqAsyncOutputBufferStats(ui64 outputIndex) - : OutputIndex(outputIndex) {} - - template<typename T> - void FromProto(const T& f) - { - this->Chunks = f.GetChunks(); - this->Bytes = f.GetBytes(); - this->RowsIn = f.GetRowsIn(); - this->RowsOut = f.GetRowsOut(); - this->MaxMemoryUsage = f.GetMaxMemoryUsage(); - } + ui64 OutputIndex = 0; + TString Type; + ui64 MaxMemoryUsage = 0; + ui64 MaxRowsInMemory = 0; }; class IDqAsyncOutputBuffer : public IDqOutput { @@ -29,6 +19,7 @@ public: using TPtr = TIntrusivePtr<IDqAsyncOutputBuffer>; virtual ui64 GetOutputIndex() const = 0; + virtual const TDqAsyncOutputBufferStats& GetPopStats() const = 0; // Pop data to send. Return estimated size of returned data. [[nodiscard]] @@ -39,11 +30,9 @@ public: // Pop chechpoint. Checkpoints may be taken from sink even after it is finished. [[nodiscard]] virtual bool Pop(NDqProto::TCheckpoint& checkpoint) = 0; - - virtual const TDqAsyncOutputBufferStats* GetStats() const = 0; }; -IDqAsyncOutputBuffer::TPtr CreateDqAsyncOutputBuffer(ui64 outputIndex, NKikimr::NMiniKQL::TType* outputType, ui64 maxStoredBytes, - bool collectProfileStats); +IDqAsyncOutputBuffer::TPtr CreateDqAsyncOutputBuffer(ui64 outputIndex, const TString& type, NKikimr::NMiniKQL::TType* outputType, ui64 maxStoredBytes, + TCollectStatsLevel level); } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/runtime/dq_async_stats.h b/ydb/library/yql/dq/runtime/dq_async_stats.h new file mode 100644 index 0000000000..db701e4eae --- /dev/null +++ b/ydb/library/yql/dq/runtime/dq_async_stats.h @@ -0,0 +1,161 @@ +#pragma once + +namespace NYql::NDq { + +/* + + Each async buffer (source/channel/sink) collect pair of TDqAsyncStats: + 1. push (in data), pause/resume/wait describe starvation when pop tries to get data from empty buffer + 2. pop (out data), pause/resume/wait describe backpressure when push is blocked by back pressure (full buffer) + Minimum wait duration is needed to filter out short edge cases + + ---> ---> ---> time ---> ---> ---> ---> + 0 1 2 3 Push: + 012345678902345678901234567890123456789 F=00 P=13 R=21 L=35 W=8 (wait on 06-07 is too short and ignored) + F P=======R L + -----#---------------------######------ (full) + #### ########### + #### ## ########### + ###### #### ############# #### + --------------------------------------- (empty) + F P====R L + 012345678902345678901234567890123456789 Pop: + 0 1 2 3 F=05 P=28 R=33 L=38 W=5 (wait on 05-05 is too short and ignored) + +*/ + +enum TCollectStatsLevel { + None, // collect nothing + Basic, // aggregated per graph, collect bytes/rows/chunks/split, do not collect timings + Full, // agrregated per stage, collect all, data and timings + Profile // like Full but don't aggregate, send full info to the client +}; + +inline bool StatsLevelCollectNone(TCollectStatsLevel level) { + return level == TCollectStatsLevel::None; +} + +inline bool StatsLevelCollectBasic(TCollectStatsLevel level) { + return level != TCollectStatsLevel::None; +} + +inline bool StatsLevelCollectFull(TCollectStatsLevel level) { + return level == TCollectStatsLevel::Full || level == TCollectStatsLevel::Profile; +} + +inline bool StatsLevelCollectProfile(TCollectStatsLevel level) { + return level == TCollectStatsLevel::Profile; +} + +struct TDqAsyncStats { + TCollectStatsLevel Level = TCollectStatsLevel::None; + TDuration MinWaitDuration = TDuration::MicroSeconds(100); + std::optional<TInstant> CurrentPauseTs; + bool MergeWaitPeriod = false; + + // basic stats + ui64 Bytes = 0; + ui64 Rows = 0; + ui64 Chunks = 0; + ui64 Splits = 0; + + // full stats + TInstant FirstMessageTs; + TInstant PauseMessageTs; + TInstant ResumeMessageTs; + TInstant LastMessageTs; + TDuration WaitTime; + ui64 WaitPeriods = 0; + + void MergeData(const TDqAsyncStats& other) { + Bytes += other.Bytes; + Rows += other.Rows; + Chunks += other.Chunks; + Splits += other.Splits; + } + + void MergeTime(const TDqAsyncStats& other) { + if (other.FirstMessageTs && (!FirstMessageTs || FirstMessageTs > other.FirstMessageTs)) { + FirstMessageTs = other.FirstMessageTs; + } + if (other.PauseMessageTs && (!PauseMessageTs || PauseMessageTs > other.PauseMessageTs)) { + PauseMessageTs = other.PauseMessageTs; + } + if (other.ResumeMessageTs && (!ResumeMessageTs || ResumeMessageTs < other.ResumeMessageTs)) { + ResumeMessageTs = other.ResumeMessageTs; + } + if (other.LastMessageTs && (!LastMessageTs || LastMessageTs < other.LastMessageTs)) { + LastMessageTs = other.LastMessageTs; + } + WaitTime += other.WaitTime; + WaitPeriods += other.WaitPeriods; + } + + void Merge(const TDqAsyncStats& other) { + MergeData(other); + MergeTime(other); + } + + inline void TryPause() { + if (CollectFull()) { + if (!CurrentPauseTs) { + auto now = TInstant::Now(); + if (ResumeMessageTs) { + auto delta = now - ResumeMessageTs; + if (delta >= MinWaitDuration) { + CurrentPauseTs = now; + } else { + CurrentPauseTs = ResumeMessageTs; + MergeWaitPeriod = true; + } + } else { + CurrentPauseTs = now; + } + } + } + } + + inline void Resume() { + if (CollectFull()) { + auto now = TInstant::Now(); + if (!FirstMessageTs) { + FirstMessageTs = now; + } + LastMessageTs = now; + if (CurrentPauseTs) { + auto delta = now - *CurrentPauseTs; + if (delta >= MinWaitDuration) { + if (!PauseMessageTs) { + PauseMessageTs = *CurrentPauseTs; + } + ResumeMessageTs = now; + WaitTime += delta; + } + CurrentPauseTs.reset(); + if (MergeWaitPeriod) { + MergeWaitPeriod = false; + } else { + WaitPeriods++; + } + } + } + } + + inline bool CollectNone() const { + return StatsLevelCollectNone(Level); + } + + inline bool CollectBasic() const { + return StatsLevelCollectBasic(Level); + } + + inline bool CollectFull() const { + return StatsLevelCollectFull(Level); + } + + inline bool CollectProfile() const { + return StatsLevelCollectProfile(Level); + } +}; + +} // namespace NYql::NDq
\ No newline at end of file diff --git a/ydb/library/yql/dq/runtime/dq_input.h b/ydb/library/yql/dq/runtime/dq_input.h index d027d1d03a..9628406df2 100644 --- a/ydb/library/yql/dq/runtime/dq_input.h +++ b/ydb/library/yql/dq/runtime/dq_input.h @@ -3,19 +3,12 @@ #include <ydb/library/yql/minikql/computation/mkql_computation_node_holders.h> #include <ydb/library/yql/minikql/mkql_node.h> +#include "dq_async_stats.h" + namespace NYql::NDq { -struct TDqInputStats { - // basic stats - ui64 Chunks = 0; - ui64 Bytes = 0; - ui64 RowsIn = 0; - ui64 RowsOut = 0; - TInstant FirstRowTs; - - // profile stats - ui64 RowsInMemory = 0; - ui64 MaxMemoryUsage = 0; +struct TDqInputStats : public TDqAsyncStats { + }; class IDqInput : public TSimpleRefCount<IDqInput> { @@ -24,6 +17,7 @@ public: virtual ~IDqInput() = default; + virtual const TDqInputStats& GetPopStats() const = 0; virtual i64 GetFreeSpace() const = 0; virtual ui64 GetStoredBytes() const = 0; @@ -35,9 +29,8 @@ public: virtual bool IsFinished() const = 0; - virtual const TDqInputStats* GetStats() const = 0; - virtual NKikimr::NMiniKQL::TType* GetInputType() const = 0; + inline TMaybe<ui32> GetInputWidth() const { auto type = GetInputType(); if (type->IsMulti()) { diff --git a/ydb/library/yql/dq/runtime/dq_input_channel.cpp b/ydb/library/yql/dq/runtime/dq_input_channel.cpp index 9f6bc5965b..322d9c28c5 100644 --- a/ydb/library/yql/dq/runtime/dq_input_channel.cpp +++ b/ydb/library/yql/dq/runtime/dq_input_channel.cpp @@ -15,10 +15,10 @@ private: const size_t rowCount = data.RowCount(); NKikimr::NMiniKQL::TUnboxedValueBatch batch(InputType); - if (Y_UNLIKELY(ProfileStats)) { + if (Y_UNLIKELY(PushStats.CollectProfile())) { auto startTime = TInstant::Now(); DataSerializer.Deserialize(std::move(data), InputType, batch); - ProfileStats->DeserializationTime += (TInstant::Now() - startTime); + PushStats.DeserializationTime += (TInstant::Now() - startTime); } else { DataSerializer.Deserialize(std::move(data), InputType, batch); } @@ -36,18 +36,31 @@ private: } public: - TDqInputChannel(ui64 channelId, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, bool collectProfileStats, + TDqInputChannelStats PushStats; + TDqInputStats PopStats; + + TDqInputChannel(ui64 channelId, ui32 srcStageId, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, TCollectStatsLevel level, const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, const NKikimr::NMiniKQL::THolderFactory& holderFactory, NDqProto::EDataTransportVersion transportVersion) : TBaseImpl(inputType, maxBufferBytes) - , ChannelId(channelId) - , BasicStats(ChannelId) - , ProfileStats(collectProfileStats ? &BasicStats : nullptr) , DataSerializer(typeEnv, holderFactory, transportVersion) - {} + { + PopStats.Level = level; + PushStats.Level = level; + PushStats.ChannelId = channelId; + PushStats.SrcStageId = srcStageId; + } ui64 GetChannelId() const override { - return ChannelId; + return PushStats.ChannelId; + } + + const TDqInputChannelStats& GetPushStats() const override { + return PushStats; + } + + const TDqInputStats& GetPopStats() const override { + return PopStats; } i64 GetFreeSpace() const override { @@ -81,7 +94,7 @@ public: } void Push(TDqSerializedBatch&& data) override { - YQL_ENSURE(!Finished, "input channel " << ChannelId << " already finished"); + YQL_ENSURE(!Finished, "input channel " << PushStats.ChannelId << " already finished"); if (Y_UNLIKELY(data.Proto.GetRows() == 0)) { return; } @@ -89,22 +102,15 @@ public: DataForDeserialize.emplace_back(std::move(data)); } - const TDqInputChannelStats* GetStats() const override { - return &BasicStats; - } - private: - const ui64 ChannelId; - TDqInputChannelStats BasicStats; - TDqInputChannelStats* ProfileStats = nullptr; TDqDataSerializer DataSerializer; }; -IDqInputChannel::TPtr CreateDqInputChannel(ui64 channelId, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, - bool collectProfileStats, const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, +IDqInputChannel::TPtr CreateDqInputChannel(ui64 channelId, ui32 srcStageId, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, + TCollectStatsLevel level, const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, const NKikimr::NMiniKQL::THolderFactory& holderFactory, NDqProto::EDataTransportVersion transportVersion) { - return new TDqInputChannel(channelId, inputType, maxBufferBytes, collectProfileStats, typeEnv, holderFactory, + return new TDqInputChannel(channelId, srcStageId, inputType, maxBufferBytes, level, typeEnv, holderFactory, transportVersion); } diff --git a/ydb/library/yql/dq/runtime/dq_input_channel.h b/ydb/library/yql/dq/runtime/dq_input_channel.h index 12054bcffa..6bfb812656 100644 --- a/ydb/library/yql/dq/runtime/dq_input_channel.h +++ b/ydb/library/yql/dq/runtime/dq_input_channel.h @@ -10,26 +10,10 @@ namespace NYql::NDq { struct TDqInputChannelStats : TDqInputStats { ui64 ChannelId = 0; - - // profile stats + ui32 SrcStageId = 0; + ui64 RowsInMemory = 0; + ui64 MaxMemoryUsage = 0; TDuration DeserializationTime; - - explicit TDqInputChannelStats(ui64 channelId) - : ChannelId(channelId) {} - - template<typename T> - void FromProto(const T& f) - { - this->ChannelId = f.GetChannelId(); - this->Chunks = f.GetChunks(); - this->Bytes = f.GetBytes(); - this->RowsIn = f.GetRowsIn(); - this->RowsOut = f.GetRowsOut(); - this->MaxMemoryUsage = f.GetMaxMemoryUsage(); - //s->StartTs = TInstant::MilliSeconds(f.GetStartTs()); - //s->FinishTs = TInstant::MilliSeconds(f.GetFinishTs()); - this->DeserializationTime = TDuration::MicroSeconds(f.GetDeserializationTimeUs()); - } }; class IDqInputChannel : public IDqInput { @@ -37,16 +21,15 @@ public: using TPtr = TIntrusivePtr<IDqInputChannel>; virtual ui64 GetChannelId() const = 0; + virtual const TDqInputChannelStats& GetPushStats() const = 0; virtual void Push(TDqSerializedBatch&& data) = 0; virtual void Finish() = 0; - - virtual const TDqInputChannelStats* GetStats() const = 0; }; -IDqInputChannel::TPtr CreateDqInputChannel(ui64 channelId, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, - bool collectProfileStats, const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, +IDqInputChannel::TPtr CreateDqInputChannel(ui64 channelId, ui32 srcStageId, NKikimr::NMiniKQL::TType* inputType, ui64 maxBufferBytes, + TCollectStatsLevel level, const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, const NKikimr::NMiniKQL::THolderFactory& holderFactory, NDqProto::EDataTransportVersion transportVersion); } // namespace NYql::NDq diff --git a/ydb/library/yql/dq/runtime/dq_input_impl.h b/ydb/library/yql/dq/runtime/dq_input_impl.h index 2ac6680f2a..d9665f5457 100644 --- a/ydb/library/yql/dq/runtime/dq_input_impl.h +++ b/ydb/library/yql/dq/runtime/dq_input_impl.h @@ -34,16 +34,18 @@ public: StoredBytes += space; StoredRows += batch.RowCount(); - auto& stats = MutableBasicStats(); - stats.Chunks++; - stats.Bytes += space; - stats.RowsIn += batch.RowCount(); - if (!stats.FirstRowTs) { - stats.FirstRowTs = TInstant::Now(); + if (static_cast<TDerived*>(this)->PushStats.CollectBasic()) { + static_cast<TDerived*>(this)->PushStats.Bytes += space; + static_cast<TDerived*>(this)->PushStats.Rows += batch.RowCount(); + static_cast<TDerived*>(this)->PushStats.Chunks++; + static_cast<TDerived*>(this)->PushStats.Resume(); + if (static_cast<TDerived*>(this)->PushStats.CollectFull()) { + static_cast<TDerived*>(this)->PushStats.MaxMemoryUsage = std::max(static_cast<TDerived*>(this)->PushStats.MaxMemoryUsage, StoredBytes); + } } - if (auto* profile = MutableProfileStats()) { - profile->MaxMemoryUsage = std::max(profile->MaxMemoryUsage, StoredBytes); + if (GetFreeSpace() < 0) { + static_cast<TDerived*>(this)->PopStats.TryPause(); } Batches.emplace_back(std::move(batch)); @@ -53,11 +55,15 @@ public: bool Pop(NKikimr::NMiniKQL::TUnboxedValueBatch& batch) override { Y_ABORT_UNLESS(batch.Width() == GetWidth()); if (Empty()) { + static_cast<TDerived*>(this)->PushStats.TryPause(); return false; } batch.clear(); + static_cast<TDerived*>(this)->PopStats.Resume(); //save timing before processing + ui64 popBytes = 0; + if (IsPaused()) { ui64 batchesCount = GetBatchesBeforePause(); Y_ABORT_UNLESS(batchesCount <= Batches.size()); @@ -80,6 +86,8 @@ public: } } + popBytes = StoredBytesBeforePause; + BatchesBeforePause = PauseMask; Y_ABORT_UNLESS(GetBatchesBeforePause() == 0); StoredBytes -= StoredBytesBeforePause; @@ -101,12 +109,19 @@ public: } } + popBytes = StoredBytes; + StoredBytes = 0; StoredRows = 0; Batches.clear(); } - MutableBasicStats().RowsOut += batch.RowCount(); + if (static_cast<TDerived*>(this)->PopStats.CollectBasic()) { + static_cast<TDerived*>(this)->PopStats.Bytes += popBytes; + static_cast<TDerived*>(this)->PopStats.Rows += batch.RowCount(); // may do not match to pushed row count + static_cast<TDerived*>(this)->PopStats.Chunks++; + } + return true; } @@ -122,14 +137,6 @@ public: return InputType; } - auto& MutableBasicStats() { - return static_cast<TDerived*>(this)->BasicStats; - } - - auto* MutableProfileStats() { - return static_cast<TDerived*>(this)->ProfileStats; - } - void Pause() override { Y_ABORT_UNLESS(!IsPaused()); if (!Finished) { diff --git a/ydb/library/yql/dq/runtime/dq_output.h b/ydb/library/yql/dq/runtime/dq_output.h index 23dd7dac29..311624ef87 100644 --- a/ydb/library/yql/dq/runtime/dq_output.h +++ b/ydb/library/yql/dq/runtime/dq_output.h @@ -6,6 +6,8 @@ #include <util/datetime/base.h> #include <util/generic/ptr.h> +#include "dq_async_stats.h" + namespace NYql { namespace NDqProto { @@ -19,14 +21,7 @@ class TUnboxedValue; namespace NDq { -struct TDqOutputStats { - // basic stats - ui64 Chunks = 0; - ui64 Bytes = 0; - ui64 RowsIn = 0; - ui64 RowsOut = 0; - TInstant FirstRowIn; - +struct TDqOutputStats : public TDqAsyncStats { // profile stats ui64 MaxMemoryUsage = 0; ui64 MaxRowsInMemory = 0; @@ -38,6 +33,8 @@ public: virtual ~IDqOutput() = default; + virtual const TDqOutputStats& GetPushStats() const = 0; + // <| producer methods [[nodiscard]] virtual bool IsFull() const = 0; @@ -55,8 +52,6 @@ public: virtual bool IsFinished() const = 0; virtual NKikimr::NMiniKQL::TType* GetOutputType() const = 0; - - virtual const TDqOutputStats* GetStats() const = 0; }; } // namespace NDq diff --git a/ydb/library/yql/dq/runtime/dq_output_channel.cpp b/ydb/library/yql/dq/runtime/dq_output_channel.cpp index f20cd5b1a0..dc97f8c75f 100644 --- a/ydb/library/yql/dq/runtime/dq_output_channel.cpp +++ b/ydb/library/yql/dq/runtime/dq_output_channel.cpp @@ -11,7 +11,7 @@ namespace NYql::NDq { -#define LOG(s) do { if (Y_UNLIKELY(LogFunc)) { LogFunc(TStringBuilder() << "channelId: " << ChannelId << ". " << s); } } while (0) +#define LOG(s) do { if (Y_UNLIKELY(LogFunc)) { LogFunc(TStringBuilder() << "channelId: " << PopStats.ChannelId << ". " << s); } } while (0) #ifndef NDEBUG #define DLOG(s) LOG(s) @@ -21,26 +21,6 @@ namespace NYql::NDq { namespace { -class TProfileGuard { -public: - TProfileGuard(TDuration* duration) - : Duration(duration) - { - if (Y_UNLIKELY(duration)) { - Start = TInstant::Now(); - } - } - - ~TProfileGuard() { - if (Y_UNLIKELY(Duration)) { - *Duration += TInstant::Now() - Start; - } - } -private: - TInstant Start; - TDuration* Duration; -}; - using namespace NKikimr; using NKikimr::NMiniKQL::TPagedBuffer; @@ -50,13 +30,13 @@ using NKikimr::NMiniKQL::TPagedBuffer; template<bool FastPack> class TDqOutputChannel : public IDqOutputChannel { public: - TDqOutputChannel(ui64 channelId, NMiniKQL::TType* outputType, + TDqOutputStats PushStats; + TDqOutputChannelStats PopStats; + + TDqOutputChannel(ui64 channelId, ui32 dstStageId, NMiniKQL::TType* outputType, const NMiniKQL::THolderFactory& holderFactory, const TDqOutputChannelSettings& settings, const TLogFunc& logFunc, NDqProto::EDataTransportVersion transportVersion) - : ChannelId(channelId) - , OutputType(outputType) - , BasicStats(ChannelId) - , ProfileStats(settings.CollectProfileStats ? &BasicStats : nullptr) + : OutputType(outputType) , Packer(OutputType) , Width(OutputType->IsMulti() ? static_cast<NMiniKQL::TMultiType*>(OutputType)->GetElementsCount() : 1u) , Storage(settings.ChannelStorage) @@ -67,16 +47,28 @@ public: , ChunkSizeLimit(settings.ChunkSizeLimit) , LogFunc(logFunc) { + PopStats.Level = settings.Level; + PushStats.Level = settings.Level; + PopStats.ChannelId = channelId; + PopStats.DstStageId = dstStageId; } ui64 GetChannelId() const override { - return ChannelId; + return PopStats.ChannelId; } ui64 GetValuesCount() const override { return SpilledRowCount + PackedRowCount + ChunkRowCount; } + const TDqOutputStats& GetPushStats() const override { + return PushStats; + } + + const TDqOutputChannelStats& GetPopStats() const override { + return PopStats; + } + [[nodiscard]] bool IsFull() const override { if (!Storage) { @@ -97,11 +89,6 @@ public: } void DoPush(NUdf::TUnboxedValue* values, ui32 width) { - TProfileGuard guard(ProfileStats ? &ProfileStats->SerializationTime : nullptr); - if (!BasicStats.FirstRowIn) { - BasicStats.FirstRowIn = TInstant::Now(); - } - ui64 rowsInMemory = PackedRowCount + ChunkRowCount; LOG("Push request, rows in memory: " << rowsInMemory << ", bytesInMemory: " << (PackedDataSize + Packer.PackedSizeEstimate()) @@ -112,6 +99,12 @@ public: return; } + if (PushStats.CollectBasic()) { + PushStats.Rows++; + PushStats.Chunks++; + PushStats.Resume(); + } + if (OutputType->IsMulti()) { Packer.AddWideItem(values, width); } else { @@ -122,13 +115,14 @@ public: } ChunkRowCount++; - BasicStats.RowsIn++; size_t packerSize = Packer.PackedSizeEstimate(); if (packerSize >= MaxChunkBytes) { Data.emplace_back(); Data.back().Buffer = FinishPackAndCheckSize(); - BasicStats.Bytes += Data.back().Buffer.size(); + if (PushStats.CollectBasic()) { + PushStats.Bytes += Data.back().Buffer.size(); + } PackedDataSize += Data.back().Buffer.size(); PackedRowCount += ChunkRowCount; Data.back().RowCount = ChunkRowCount; @@ -152,18 +146,22 @@ public: SpilledRowCount += head.RowCount; - if (Y_UNLIKELY(ProfileStats)) { - ProfileStats->SpilledRows += head.RowCount; - ProfileStats->SpilledBytes += bufSize + sizeof(head.RowCount); - ProfileStats->SpilledBlobs++; + if (PopStats.CollectFull()) { + PopStats.SpilledRows += head.RowCount; + PopStats.SpilledBytes += bufSize + sizeof(head.RowCount); + PopStats.SpilledBlobs++; } Data.pop_front(); } - if (Y_UNLIKELY(ProfileStats)) { - ProfileStats->MaxMemoryUsage = std::max(ProfileStats->MaxMemoryUsage, PackedDataSize + packerSize); - ProfileStats->MaxRowsInMemory = std::max(ProfileStats->MaxRowsInMemory, PackedRowCount); + if (IsFull() || FirstStoredId < NextStoredId) { + PopStats.TryPause(); + } + + if (PopStats.CollectFull()) { + PopStats.MaxMemoryUsage = std::max(PopStats.MaxMemoryUsage, PackedDataSize + packerSize); + PopStats.MaxRowsInMemory = std::max(PopStats.MaxRowsInMemory, PackedRowCount); } } @@ -182,6 +180,7 @@ public: LOG("Pop request, rows in memory: " << GetValuesCount() << ", finished: " << Finished); if (!HasData()) { + PushStats.TryPause(); if (Finished) { data.Clear(); data.Proto.SetTransportVersion(TransportVersion); @@ -217,8 +216,15 @@ public: DLOG("Took " << data.RowCount() << " rows"); - BasicStats.Chunks++; - BasicStats.RowsOut += data.RowCount(); + if (PopStats.CollectBasic()) { + PopStats.Bytes += data.Payload.size(); + PopStats.Rows += data.RowCount(); + PopStats.Chunks++; + if (!IsFull() || FirstStoredId == NextStoredId) { + PopStats.Resume(); + } + } + return true; } @@ -265,7 +271,6 @@ public: if (ChunkRowCount) { Data.emplace_back(); Data.back().Buffer = FinishPackAndCheckSize(); - BasicStats.Bytes += Data.back().Buffer.size(); PackedDataSize += Data.back().Buffer.size(); PackedRowCount += ChunkRowCount; Data.back().RowCount = ChunkRowCount; @@ -293,6 +298,14 @@ public: data.Proto.SetRows(rows.RowCount()); data.SetPayload(FinishPackAndCheckSize()); + if (PopStats.CollectBasic()) { + PopStats.Bytes += data.Payload.size(); + PopStats.Rows += data.RowCount(); + PopStats.Chunks++; + if (!IsFull() || FirstStoredId == NextStoredId) { + PopStats.Resume(); + } + } YQL_ENSURE(!HasData()); return true; } @@ -300,10 +313,6 @@ public: void Finish() override { LOG("Finish request"); Finished = true; - - if (!BasicStats.FirstRowIn) { - BasicStats.FirstRowIn = TInstant::Now(); - } } TRope FinishPackAndCheckSize() { @@ -337,18 +346,11 @@ public: return OutputType; } - const TDqOutputChannelStats* GetStats() const override { - return &BasicStats; - } - void Terminate() override { } private: - const ui64 ChannelId; NKikimr::NMiniKQL::TType* OutputType; - TDqOutputChannelStats BasicStats; - TDqOutputChannelStats* ProfileStats = nullptr; NKikimr::NMiniKQL::TValuePackerTransport<FastPack> Packer; const ui32 Width; const IDqChannelStorage::TPtr Storage; @@ -383,7 +385,7 @@ private: } // anonymous namespace -IDqOutputChannel::TPtr CreateDqOutputChannel(ui64 channelId, NKikimr::NMiniKQL::TType* outputType, +IDqOutputChannel::TPtr CreateDqOutputChannel(ui64 channelId, ui32 dstStageId, NKikimr::NMiniKQL::TType* outputType, const NKikimr::NMiniKQL::THolderFactory& holderFactory, const TDqOutputChannelSettings& settings, const TLogFunc& logFunc) { @@ -394,10 +396,10 @@ IDqOutputChannel::TPtr CreateDqOutputChannel(ui64 channelId, NKikimr::NMiniKQL:: [[fallthrough]]; case NDqProto::EDataTransportVersion::DATA_TRANSPORT_UV_PICKLE_1_0: case NDqProto::EDataTransportVersion::DATA_TRANSPORT_OOB_PICKLE_1_0: - return new TDqOutputChannel<false>(channelId, outputType, holderFactory, settings, logFunc, transportVersion); + return new TDqOutputChannel<false>(channelId, dstStageId, outputType, holderFactory, settings, logFunc, transportVersion); case NDqProto::EDataTransportVersion::DATA_TRANSPORT_UV_FAST_PICKLE_1_0: case NDqProto::EDataTransportVersion::DATA_TRANSPORT_OOB_FAST_PICKLE_1_0: - return new TDqOutputChannel<true>(channelId, outputType, holderFactory, settings, logFunc, transportVersion); + return new TDqOutputChannel<true>(channelId, dstStageId, outputType, holderFactory, settings, logFunc, transportVersion); default: YQL_ENSURE(false, "Unsupported transport version " << (ui32)transportVersion); } diff --git a/ydb/library/yql/dq/runtime/dq_output_channel.h b/ydb/library/yql/dq/runtime/dq_output_channel.h index 70ce0b3b30..3164f09233 100644 --- a/ydb/library/yql/dq/runtime/dq_output_channel.h +++ b/ydb/library/yql/dq/runtime/dq_output_channel.h @@ -14,31 +14,15 @@ namespace NYql::NDq { -struct TDqOutputChannelStats : TDqOutputStats { +struct TDqOutputChannelStats : public TDqOutputStats { ui64 ChannelId = 0; - - // profile stats + ui32 DstStageId = 0; + ui64 MaxMemoryUsage = 0; + ui64 MaxRowsInMemory = 0; TDuration SerializationTime; - ui64 SpilledBytes = 0; ui64 SpilledRows = 0; ui64 SpilledBlobs = 0; - - explicit TDqOutputChannelStats(ui64 channelId) - : ChannelId(channelId) {} - - template<typename T> - void FromProto(const T& f) - { - this->ChannelId = f.GetChannelId(); - this->Chunks = f.GetChunks(); - this->Bytes = f.GetBytes(); - this->RowsIn = f.GetRowsIn(); - this->RowsOut = f.GetRowsOut(); - this->MaxMemoryUsage = f.GetMaxMemoryUsage(); - //s->StartTs = TInstant::MilliSeconds(f.GetStartTs()); - //s->FinishTs = TInstant::MilliSeconds(f.GetFinishTs()); - } }; class IDqOutputChannel : public IDqOutput { @@ -47,6 +31,7 @@ public: virtual ui64 GetChannelId() const = 0; virtual ui64 GetValuesCount() const = 0; + virtual const TDqOutputChannelStats& GetPopStats() const = 0; // <| consumer methods // can throw TDqChannelStorageException @@ -69,7 +54,6 @@ public: virtual ui64 Drop() = 0; - virtual const TDqOutputChannelStats* GetStats() const = 0; virtual void Terminate() = 0; }; @@ -79,13 +63,13 @@ struct TDqOutputChannelSettings { ui64 ChunkSizeLimit = 48_MB; NDqProto::EDataTransportVersion TransportVersion = NDqProto::EDataTransportVersion::DATA_TRANSPORT_UV_PICKLE_1_0; IDqChannelStorage::TPtr ChannelStorage; - bool CollectProfileStats = false; + TCollectStatsLevel Level = TCollectStatsLevel::None; }; struct TDqOutputChannelChunkSizeLimitExceeded : public yexception { }; -IDqOutputChannel::TPtr CreateDqOutputChannel(ui64 channelId, NKikimr::NMiniKQL::TType* outputType, +IDqOutputChannel::TPtr CreateDqOutputChannel(ui64 channelId, ui32 dstStageId, NKikimr::NMiniKQL::TType* outputType, const NKikimr::NMiniKQL::THolderFactory& holderFactory, const TDqOutputChannelSettings& settings, const TLogFunc& logFunc = {}); diff --git a/ydb/library/yql/dq/runtime/dq_output_channel_ut.cpp b/ydb/library/yql/dq/runtime/dq_output_channel_ut.cpp index fd44005ae2..1bd6beb87e 100644 --- a/ydb/library/yql/dq/runtime/dq_output_channel_ut.cpp +++ b/ydb/library/yql/dq/runtime/dq_output_channel_ut.cpp @@ -176,10 +176,10 @@ void TestSingleRead(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = 1000; settings.MaxChunkBytes = 200; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 10; ++i) { auto row = ctx.CreateRow(i); @@ -187,17 +187,20 @@ void TestSingleRead(TTestContext& ctx) { PushRow(ctx, std::move(row), ch); } - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(10, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(10, ch->GetPushStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(10, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); TDqSerializedBatch data; UNIT_ASSERT(ch->Pop(data)); UNIT_ASSERT_VALUES_EQUAL(10, data.RowCount()); - UNIT_ASSERT_VALUES_EQUAL(1, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(10, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(10, ch->GetStats()->RowsOut); + + UNIT_ASSERT_VALUES_EQUAL(10, ch->GetPushStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(10, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(1, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(10, ch->GetPopStats().Rows); TUnboxedValueBatch buffer(ctx.GetOutputType()); ctx.Ds.Deserialize(std::move(data), ctx.GetOutputType(), buffer); @@ -211,10 +214,10 @@ void TestPartialRead(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = 1000; settings.MaxChunkBytes = 17; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 9; ++i) { auto row = ctx.CreateRow(i); @@ -222,9 +225,10 @@ void TestPartialRead(TTestContext& ctx) { PushRow(ctx, std::move(row), ch); } - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(9, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); int req = 0; ui32 expected[] = {2, 2, 2, 2, 1}; @@ -240,9 +244,9 @@ void TestPartialRead(TTestContext& ctx) { ++req; UNIT_ASSERT_VALUES_EQUAL(v, rowCount); - UNIT_ASSERT_VALUES_EQUAL(++readChunks, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(9, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(readRows + rowCount, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(++readChunks, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(readRows + rowCount, ch->GetPopStats().Rows); TUnboxedValueBatch buffer(ctx.GetOutputType()); ctx.Ds.Deserialize(std::move(data), ctx.GetOutputType(), buffer); @@ -258,10 +262,10 @@ void TestOverflow(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = 30; settings.MaxChunkBytes = 10; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 8; ++i) { auto row = ctx.CreateRow(i); @@ -269,8 +273,8 @@ void TestOverflow(TTestContext& ctx) { PushRow(ctx, std::move(row), ch); } - UNIT_ASSERT_VALUES_EQUAL(8, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(8, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); UNIT_ASSERT(ch->IsFull()); try { @@ -286,10 +290,10 @@ void TestPopAll(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = 1000; settings.MaxChunkBytes = 10; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 50; ++i) { auto row = ctx.CreateRow(i); @@ -297,8 +301,8 @@ void TestPopAll(TTestContext& ctx) { PushRow(ctx, std::move(row), ch); } - UNIT_ASSERT_VALUES_EQUAL(50, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(50, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); TDqSerializedBatch data; TUnboxedValueBatch buffer(ctx.GetOutputType()); @@ -317,10 +321,10 @@ void TestBigRow(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = std::numeric_limits<ui32>::max(); settings.MaxChunkBytes = 2_MB; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); { auto row = ctx.CreateRow(1); @@ -335,18 +339,19 @@ void TestBigRow(TTestContext& ctx) { } } - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(9, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); { TDqSerializedBatch data; UNIT_ASSERT(ch->Pop(data)); UNIT_ASSERT_VALUES_EQUAL(2, data.RowCount()); - UNIT_ASSERT_VALUES_EQUAL(1, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(9, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(2, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(1, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(2, ch->GetPopStats().Rows); TUnboxedValueBatch buffer(ctx.GetOutputType()); ctx.Ds.Deserialize(std::move(data), ctx.GetOutputType(), buffer); @@ -376,9 +381,9 @@ void TestBigRow(TTestContext& ctx) { UNIT_ASSERT(ch->Pop(data)); UNIT_ASSERT_VALUES_EQUAL(1, data.RowCount()); - UNIT_ASSERT_VALUES_EQUAL(i - 1, ch->GetStats()->Chunks); - UNIT_ASSERT_VALUES_EQUAL(9, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(i, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(i - 1, ch->GetPopStats().Chunks); + UNIT_ASSERT_VALUES_EQUAL(9, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(i, ch->GetPopStats().Rows); TUnboxedValueBatch buffer(ctx.GetOutputType()); ctx.Ds.Deserialize(std::move(data), ctx.GetOutputType(), buffer); @@ -403,13 +408,13 @@ void TestSpillWithMockStorage(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = 100; settings.MaxChunkBytes = 20; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; auto storage = MakeIntrusive<TMockChannelStorage>(100'500ul); settings.ChannelStorage = storage; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 35; ++i) { auto row = ctx.CreateRow(i); @@ -419,11 +424,11 @@ void TestSpillWithMockStorage(TTestContext& ctx) { UNIT_ASSERT_VALUES_EQUAL(35, ch->GetValuesCount()); - UNIT_ASSERT_VALUES_EQUAL(35, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); - UNIT_ASSERT_VALUES_EQUAL(18, ch->GetStats()->SpilledRows); - UNIT_ASSERT_VALUES_EQUAL(5, ch->GetStats()->SpilledBlobs); - UNIT_ASSERT(ch->GetStats()->SpilledBytes > 5 * 8); + UNIT_ASSERT_VALUES_EQUAL(35, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(18, ch->GetPopStats().SpilledRows); + UNIT_ASSERT_VALUES_EQUAL(5, ch->GetPopStats().SpilledBlobs); + UNIT_ASSERT(ch->GetPopStats().SpilledBytes > 5 * 8); ui32 loadedRows = 0; @@ -467,13 +472,13 @@ void TestOverflowWithMockStorage(TTestContext& ctx) { TDqOutputChannelSettings settings; settings.MaxStoredBytes = 500; settings.MaxChunkBytes = 10; - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; auto storage = MakeIntrusive<TMockChannelStorage>(500ul); settings.ChannelStorage = storage; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 42; ++i) { auto row = ctx.CreateRow(i); @@ -481,8 +486,8 @@ void TestOverflowWithMockStorage(TTestContext& ctx) { PushRow(ctx, std::move(row), ch); } - UNIT_ASSERT_VALUES_EQUAL(42, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(42, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); // UNIT_ASSERT(ch->IsFull()); it can be false-negative with storage enabled try { @@ -498,11 +503,10 @@ void TestChunkSizeLimit(TTestContext& ctx) { settings.MaxStoredBytes = 500; settings.MaxChunkBytes = 100; settings.ChunkSizeLimit = 100000; - - settings.CollectProfileStats = true; + settings.Level = TCollectStatsLevel::Profile; settings.TransportVersion = ctx.TransportVersion; - auto ch = CreateDqOutputChannel(1, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); + auto ch = CreateDqOutputChannel(1, 1000, ctx.GetOutputType(), ctx.HolderFactory, settings, Log); for (i32 i = 0; i < 10; ++i) { auto row = ctx.CreateRow(i); @@ -510,8 +514,8 @@ void TestChunkSizeLimit(TTestContext& ctx) { PushRow(ctx, std::move(row), ch); } - UNIT_ASSERT_VALUES_EQUAL(10, ch->GetStats()->RowsIn); - UNIT_ASSERT_VALUES_EQUAL(0, ch->GetStats()->RowsOut); + UNIT_ASSERT_VALUES_EQUAL(10, ch->GetPushStats().Rows); + UNIT_ASSERT_VALUES_EQUAL(0, ch->GetPopStats().Rows); try { PushRow(ctx, ctx.CreateBigRow(0, 100'500), ch); diff --git a/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp b/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp index 6d38614bb0..6dd562a088 100644 --- a/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp +++ b/ydb/library/yql/dq/runtime/dq_tasks_runner.cpp @@ -215,6 +215,12 @@ IDqChannelStorage::TPtr TDqTaskRunnerExecutionContext::CreateChannelStorage(ui64 return {}; } +inline TCollectStatsLevel StatsModeToCollectStatsLevel(NDqProto::EDqStatsMode statsMode) { + if (statsMode >= NDqProto::DQ_STATS_MODE_PROFILE) return TCollectStatsLevel::Profile; + else if (statsMode >= NDqProto::DQ_STATS_MODE_FULL) return TCollectStatsLevel::Full; + else if (statsMode >= NDqProto::DQ_STATS_MODE_BASIC) return TCollectStatsLevel::Basic; + else return TCollectStatsLevel::None; +} //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// /// TDqTaskRunner @@ -226,17 +232,13 @@ public: , Settings(settings) , LogFunc(logFunc) , AllocatedHolder(std::make_optional<TAllocatedHolder>()) - , CollectBasicStats(Settings.CollectBasicStats) - , CollectProfileStats(Settings.CollectProfileStats) { - if (CollectBasicStats) { + if (Settings.StatsMode >= NDqProto::DQ_STATS_MODE_BASIC) { Stats = std::make_unique<TDqTaskRunnerStats>(); Stats->StartTs = TInstant::Now(); - if (Y_UNLIKELY(CollectProfileStats)) { + if (Y_UNLIKELY(CollectFull())) { Stats->ComputeCpuTimeByRun = NMonitoring::ExponentialHistogram(6, 10, 10); } - } else { - YQL_ENSURE(!CollectProfileStats, "CollectProfileStats requires CollectBasicStats to be set as well"); } if (!Context.Alloc) { @@ -259,11 +261,16 @@ public: ~TDqTaskRunner() { if (SelfAlloc) { SelfAlloc->Acquire(); + Stats.reset(); AllocatedHolder.reset(); SelfAlloc->Release(); } } + bool CollectFull() const { + return Settings.StatsMode >= NDqProto::DQ_STATS_MODE_FULL; + } + const TDqMeteringStats* GetMeteringStats() const override { return &BillingStats; } @@ -291,7 +298,7 @@ public: return nullptr; }; - if (Y_UNLIKELY(CollectProfileStats && !AllocatedHolder->ProgramParsed.StatsRegistry)) { + if (Y_UNLIKELY(CollectFull() && !AllocatedHolder->ProgramParsed.StatsRegistry)) { AllocatedHolder->ProgramParsed.StatsRegistry = NMiniKQL::CreateDefaultStatsRegistry(); } TComputationPatternOpts opts(alloc.Ref(), typeEnv, taskRunnerFactory, @@ -539,23 +546,23 @@ public: << transformDesc.GetType() << " with input type: " << *transform->TransformInputType << " , output type: " << *outputType); - transform->TransformOutput = CreateDqAsyncInputBuffer(i, outputType, - memoryLimits.ChannelBufferSize, Settings.CollectProfileStats); + transform->TransformOutput = CreateDqAsyncInputBuffer(i, transformDesc.GetType(), outputType, + memoryLimits.ChannelBufferSize, StatsModeToCollectStatsLevel(Settings.StatsMode)); inputType = &transform->TransformInputType; } if (inputDesc.HasSource()) { - auto source = CreateDqAsyncInputBuffer(i, *inputType, - memoryLimits.ChannelBufferSize, Settings.CollectProfileStats); + auto source = CreateDqAsyncInputBuffer(i, inputDesc.GetSource().GetType(), *inputType, + memoryLimits.ChannelBufferSize, StatsModeToCollectStatsLevel(Settings.StatsMode)); auto [_, inserted] = AllocatedHolder->Sources.emplace(i, source); Y_ABORT_UNLESS(inserted); inputs.emplace_back(source); } else { for (auto& inputChannelDesc : inputDesc.GetChannels()) { ui64 channelId = inputChannelDesc.GetId(); - auto inputChannel = CreateDqInputChannel(channelId, *inputType, - memoryLimits.ChannelBufferSize, Settings.CollectProfileStats, typeEnv, holderFactory, + auto inputChannel = CreateDqInputChannel(channelId, inputChannelDesc.GetSrcStageId(), *inputType, + memoryLimits.ChannelBufferSize, StatsModeToCollectStatsLevel(Settings.StatsMode), typeEnv, holderFactory, inputChannelDesc.GetTransportVersion()); auto ret = AllocatedHolder->InputChannels.emplace(channelId, inputChannel); YQL_ENSURE(ret.second, "task: " << TaskId << ", duplicated input channelId: " << channelId); @@ -608,14 +615,14 @@ public: << transformDesc.GetType() << " with input type: " << *inputType << " , output type: " << *transform->TransformOutputType); - transform->TransformInput = CreateDqAsyncOutputBuffer(i, entry->OutputItemTypes[i], memoryLimits.ChannelBufferSize, - Settings.CollectProfileStats); + transform->TransformInput = CreateDqAsyncOutputBuffer(i, transformDesc.GetType(), entry->OutputItemTypes[i], memoryLimits.ChannelBufferSize, + StatsModeToCollectStatsLevel(Settings.StatsMode)); taskOutputType = &transform->TransformOutputType; } if (outputDesc.HasSink()) { - auto sink = CreateDqAsyncOutputBuffer(i, *taskOutputType, memoryLimits.ChannelBufferSize, - Settings.CollectProfileStats); + auto sink = CreateDqAsyncOutputBuffer(i, outputDesc.GetSink().GetType(), *taskOutputType, memoryLimits.ChannelBufferSize, + StatsModeToCollectStatsLevel(Settings.StatsMode)); auto [_, inserted] = AllocatedHolder->Sinks.emplace(i, sink); Y_ABORT_UNLESS(inserted); outputs.emplace_back(sink); @@ -628,13 +635,13 @@ public: settings.MaxChunkBytes = memoryLimits.OutputChunkMaxSize; settings.ChunkSizeLimit = memoryLimits.ChunkSizeLimit; settings.TransportVersion = outputChannelDesc.GetTransportVersion(); - settings.CollectProfileStats = Settings.CollectProfileStats; + settings.Level = StatsModeToCollectStatsLevel(Settings.StatsMode); if (!outputChannelDesc.GetInMemory()) { settings.ChannelStorage = execCtx.CreateChannelStorage(channelId); } - auto outputChannel = CreateDqOutputChannel(channelId, *taskOutputType, holderFactory, settings, LogFunc); + auto outputChannel = CreateDqOutputChannel(channelId, outputChannelDesc.GetDstStageId(), *taskOutputType, holderFactory, settings, LogFunc); auto ret = AllocatedHolder->OutputChannels.emplace(channelId, outputChannel); YQL_ENSURE(ret.second, "task: " << TaskId << ", duplicated output channelId: " << channelId); @@ -682,15 +689,12 @@ public: LOG(TStringBuilder() << "Prepare task: " << TaskId << ", takes " << prepareTime.MicroSeconds() << " us"); if (Stats) { Stats->BuildCpuTime += prepareTime; - - for (auto&[channelId, inputChannel] : AllocatedHolder->InputChannels) { - Stats->InputChannels.emplace(channelId, inputChannel->GetStats()); - } - for (auto&[inputIndex, source] : AllocatedHolder->Sources) { - Stats->Sources.emplace(inputIndex, source->GetStats()); + for (auto& [channelId, inputChannel] : AllocatedHolder->InputChannels) { + Stats->InputChannels[inputChannel->GetPushStats().SrcStageId].emplace(channelId, inputChannel); } - for (auto&[channelId, outputChannel] : AllocatedHolder->OutputChannels) { - Stats->OutputChannels.emplace(channelId, outputChannel->GetStats()); + Stats->Sources = AllocatedHolder->Sources; + for (auto& [channelId, outputChannel] : AllocatedHolder->OutputChannels) { + Stats->OutputChannels[outputChannel->GetPopStats().DstStageId].emplace(channelId, outputChannel); } } } @@ -710,7 +714,7 @@ public: Stats->RunStatusTimeMetrics.SetCurrentStatus(runStatus, RunComputeTime); } - if (Y_UNLIKELY(CollectProfileStats)) { + if (Y_UNLIKELY(CollectFull())) { Stats->ComputeCpuTimeByRun->Collect(RunComputeTime.MilliSeconds()); if (AllocatedHolder->ProgramParsed.StatsRegistry) { @@ -725,14 +729,14 @@ public: if (Stats) { Stats->FinishTs = TInstant::Now(); } - if (Y_UNLIKELY(CollectProfileStats)) { + if (Y_UNLIKELY(CollectFull())) { StopWaiting(Stats->FinishTs); } return ERunStatus::Finished; } - if (Y_UNLIKELY(CollectProfileStats)) { + if (Y_UNLIKELY(CollectFull())) { auto now = TInstant::Now(); StartWaiting(now); if (runStatus == ERunStatus::PendingOutput) { @@ -877,7 +881,7 @@ private: if (Stats) { auto duration = TInstant::Now() - startComputeTime; Stats->ComputeCpuTime += duration; - if (Y_UNLIKELY(CollectProfileStats)) { + if (Y_UNLIKELY(CollectFull())) { RunComputeTime = duration; } } @@ -899,7 +903,7 @@ private: wideBuffer.resize(AllocatedHolder->OutputWideType->GetElementsCount()); } while (!AllocatedHolder->Output->IsFull()) { - if (Y_UNLIKELY(CollectProfileStats)) { + if (Y_UNLIKELY(CollectFull())) { auto now = TInstant::Now(); StopWaitingOutput(now); StopWaiting(now); @@ -992,8 +996,6 @@ private: bool TaskHasEffects = false; - bool CollectBasicStats = false; - bool CollectProfileStats = false; std::unique_ptr<TDqTaskRunnerStats> Stats; TDqMeteringStats BillingStats; TDuration RunComputeTime; @@ -1004,26 +1006,26 @@ private: std::optional<TInstant> StartWaitTime; void StartWaitingOutput(TInstant now) { - if (Y_UNLIKELY(CollectProfileStats) && !StartWaitOutputTime) { + if (Y_UNLIKELY(CollectFull()) && !StartWaitOutputTime) { StartWaitOutputTime = now; } } void StopWaitingOutput(TInstant now) { - if (Y_UNLIKELY(CollectProfileStats) && StartWaitOutputTime) { + if (Y_UNLIKELY(CollectFull()) && StartWaitOutputTime) { Stats->WaitOutputTime += (now - *StartWaitOutputTime); StartWaitOutputTime.reset(); } } void StartWaiting(TInstant now) { - if (Y_UNLIKELY(CollectProfileStats) && !StartWaitTime) { + if (Y_UNLIKELY(CollectFull()) && !StartWaitTime) { StartWaitTime = now; } } void StopWaiting(TInstant now) { - if (Y_UNLIKELY(CollectProfileStats) && StartWaitTime) { + if (Y_UNLIKELY(CollectFull()) && StartWaitTime) { Stats->WaitTime += (now - *StartWaitTime); StartWaitTime.reset(); } diff --git a/ydb/library/yql/dq/runtime/dq_tasks_runner.h b/ydb/library/yql/dq/runtime/dq_tasks_runner.h index 9235084950..22330a5d15 100644 --- a/ydb/library/yql/dq/runtime/dq_tasks_runner.h +++ b/ydb/library/yql/dq/runtime/dq_tasks_runner.h @@ -84,9 +84,9 @@ struct TTaskRunnerStatsBase { NMonitoring::IHistogramCollectorPtr ComputeCpuTimeByRun; // in millis - THashMap<ui64, const TDqInputChannelStats*> InputChannels; // Channel id -> Channel stats - THashMap<ui64, const TDqAsyncInputBufferStats*> Sources; // Input index -> Source stats - THashMap<ui64, const TDqOutputChannelStats*> OutputChannels; // Channel id -> Channel stats + THashMap<ui32, THashMap<ui64, IDqInputChannel::TPtr>> InputChannels; // SrcStageId => {ChannelId => Channel} + THashMap<ui64, IDqAsyncInputBuffer::TPtr> Sources; // InputIndex => Source + THashMap<ui32, THashMap<ui64, IDqOutputChannel::TPtr>> OutputChannels; // DstStageId => {ChannelId => Channel} TVector<TMkqlStat> MkqlStats; @@ -95,57 +95,9 @@ struct TTaskRunnerStatsBase { TTaskRunnerStatsBase& operator=(TTaskRunnerStatsBase&&) = default; virtual ~TTaskRunnerStatsBase() = default; - - template<typename T> - void FromProto(const T& f) - { - this->BuildCpuTime = TDuration::MicroSeconds(f.GetBuildCpuTimeUs()); - this->ComputeCpuTime = TDuration::MicroSeconds(f.GetComputeCpuTimeUs()); - this->RunStatusTimeMetrics.Load(ERunStatus::PendingInput, TDuration::MicroSeconds(f.GetPendingInputTimeUs())); - this->RunStatusTimeMetrics.Load(ERunStatus::PendingOutput, TDuration::MicroSeconds(f.GetPendingOutputTimeUs())); - this->RunStatusTimeMetrics.Load(ERunStatus::Finished, TDuration::MicroSeconds(f.GetFinishTimeUs())); - //s->TotalTime = TDuration::MilliSeconds(f.GetTotalTime()); - this->WaitTime = TDuration::MicroSeconds(f.GetWaitTimeUs()); - this->WaitOutputTime = TDuration::MicroSeconds(f.GetWaitOutputTimeUs()); - - //s->MkqlTotalNodes = f.GetMkqlTotalNodes(); - //s->MkqlCodegenFunctions = f.GetMkqlCodegenFunctions(); - //s->CodeGenTotalInstructions = f.GetCodeGenTotalInstructions(); - //s->CodeGenTotalFunctions = f.GetCodeGenTotalFunctions(); - //s->CodeGenFullTime = f.GetCodeGenFullTime(); - //s->CodeGenFinalizeTime = f.GetCodeGenFinalizeTime(); - //s->CodeGenModulePassTime = f.GetCodeGenModulePassTime(); - - for (const auto& input : f.GetInputChannels()) { - this->MutableInputChannel(input.GetChannelId())->FromProto(input); - } - - for (const auto& output : f.GetOutputChannels()) { - this->MutableOutputChannel(output.GetChannelId())->FromProto(output); - } - - // todo: (whcrc) fill sources and ComputeCpuTimeByRun? - } - -private: - virtual TDqInputChannelStats* MutableInputChannel(ui64 channelId) = 0; - virtual TDqAsyncInputBufferStats* MutableSource(ui64 sourceId) = 0; // todo: (whcrc) unused, not modified by these pointers - virtual TDqOutputChannelStats* MutableOutputChannel(ui64 channelId) = 0; }; struct TDqTaskRunnerStats : public TTaskRunnerStatsBase { - // these stats are owned by TDqTaskRunner - TDqInputChannelStats* MutableInputChannel(ui64 channelId) override { - return const_cast<TDqInputChannelStats*>(InputChannels[channelId]); - } - - TDqAsyncInputBufferStats* MutableSource(ui64 sourceId) override { - return const_cast<TDqAsyncInputBufferStats*>(Sources[sourceId]); - } - - TDqOutputChannelStats* MutableOutputChannel(ui64 channelId) override { - return const_cast<TDqOutputChannelStats*>(OutputChannels[channelId]); - } }; // Provides read access to TTaskRunnerStatsBase @@ -159,12 +111,12 @@ public: , IsDefined(true) { } - TDqTaskRunnerStatsView(const TDqTaskRunnerStats* stats, THashMap<ui32, const TDqAsyncOutputBufferStats*>&& sinkStats, - THashMap<ui32, const TDqAsyncInputBufferStats*>&& inputTransformStats) + TDqTaskRunnerStatsView(const TDqTaskRunnerStats* stats, THashMap<ui32, const IDqAsyncOutputBuffer*>&& sinks, + THashMap<ui32, const IDqAsyncInputBuffer*>&& inputTransforms) : StatsPtr(stats) , IsDefined(true) - , SinkStats(std::move(sinkStats)) - , InputTransformStats(std::move(inputTransformStats)) { + , Sinks(std::move(sinks)) + , InputTransforms(std::move(inputTransforms)) { } const TTaskRunnerStatsBase* Get() { @@ -178,19 +130,19 @@ public: return IsDefined; } - const TDqAsyncOutputBufferStats* GetSinkStats(ui32 sinkId) const { - return SinkStats.at(sinkId); + const IDqAsyncOutputBuffer* GetSink(ui32 sinkId) const { + return Sinks.at(sinkId); } - const TDqAsyncInputBufferStats* GetInputTransformStats(ui32 inputTransformId) const { - return InputTransformStats.at(inputTransformId); + const IDqAsyncInputBuffer* GetInputTransform(ui32 inputTransformId) const { + return InputTransforms.at(inputTransformId); } private: const TDqTaskRunnerStats* StatsPtr; bool IsDefined; - THashMap<ui32, const TDqAsyncOutputBufferStats*> SinkStats; - THashMap<ui32, const TDqAsyncInputBufferStats*> InputTransformStats; + THashMap<ui32, const IDqAsyncOutputBuffer*> Sinks; + THashMap<ui32, const IDqAsyncInputBuffer*> InputTransforms; }; struct TDqTaskRunnerContext { @@ -231,8 +183,7 @@ public: }; struct TDqTaskRunnerSettings { - bool CollectBasicStats = false; - bool CollectProfileStats = false; + NDqProto::EDqStatsMode StatsMode = NDqProto::DQ_STATS_MODE_NONE; bool TerminateOnError = false; bool UseCacheForLLVM = false; TString OptLLVM = ""; diff --git a/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.cpp b/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.cpp index 6dd3638766..de4d79b165 100644 --- a/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.cpp +++ b/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.cpp @@ -49,6 +49,7 @@ struct TEvPrivate { class TClickHouseReadActor : public TActorBootstrapped<TClickHouseReadActor>, public IDqComputeActorAsyncInput { public: TClickHouseReadActor(ui64 inputIndex, + TCollectStatsLevel statsLevel, IHTTPGateway::TPtr gateway, TString&& url, TString&& query, @@ -59,7 +60,9 @@ public: , ActorSystem(TActivationContext::ActorSystem()) , Url(std::move(url)) , Query(std::move(query)) - {} + { + IngressStats.Level = statsLevel; + } void Bootstrap() { Become(&TClickHouseReadActor::StateFunc); @@ -72,7 +75,14 @@ private: void SaveState(const NDqProto::TCheckpoint&, NDqProto::TSourceState&) final {} void LoadState(const NDqProto::TSourceState&) final {} void CommitState(const NDqProto::TCheckpoint&) final {} - ui64 GetInputIndex() const final { return InputIndex; } + + ui64 GetInputIndex() const final { + return InputIndex; + } + + const TDqAsyncStats& GetIngressStats() const final { + return IngressStats; + } STRICT_STFUNC(StateFunc, hFunc(TEvPrivate::TEvReadResult, Handle); @@ -118,6 +128,7 @@ private: const IHTTPGateway::TPtr Gateway; const ui64 InputIndex; + TDqAsyncStats IngressStats; const NActors::TActorId ComputeActorId; TActorSystem* const ActorSystem; @@ -130,6 +141,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateClickHouseReadAc IHTTPGateway::TPtr gateway, NCH::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, const NActors::TActorId& computeActorId, @@ -153,7 +165,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateClickHouseReadAc TStringBuilder url; url << params.GetScheme() << token.substr(one + 1u, two - one - 1u) << ':' << token.substr(two + 1u) << '@' << params.GetEndpoint() << "/?default_format=Native"; - const auto actor = new TClickHouseReadActor(inputIndex, std::move(gateway), std::move(url), params.GetQuery() + part, computeActorId); + const auto actor = new TClickHouseReadActor(inputIndex, statsLevel, std::move(gateway), std::move(url), params.GetQuery() + part, computeActorId); return {actor, actor}; } diff --git a/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.h b/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.h index 9958901969..a41b2645db 100644 --- a/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.h +++ b/ydb/library/yql/providers/clickhouse/actors/yql_ch_read_actor.h @@ -12,6 +12,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateClickHo IHTTPGateway::TPtr gateway, NCH::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, const NActors::TActorId& computeActorId, diff --git a/ydb/library/yql/providers/clickhouse/actors/yql_ch_source_factory.cpp b/ydb/library/yql/providers/clickhouse/actors/yql_ch_source_factory.cpp index 917cef24dd..cd28970708 100644 --- a/ydb/library/yql/providers/clickhouse/actors/yql_ch_source_factory.cpp +++ b/ydb/library/yql/providers/clickhouse/actors/yql_ch_source_factory.cpp @@ -8,7 +8,8 @@ namespace NYql::NDq { void RegisterClickHouseReadActorFactory(TDqAsyncIoFactory& factory, ISecuredServiceAccountCredentialsFactory::TPtr credentialsFactory, IHTTPGateway::TPtr gateway) { factory.RegisterSource<NCH::TSource>("ClickHouseSource", [credentialsFactory, gateway](NCH::TSource&& settings, IDqAsyncIoFactory::TSourceArguments&& args) { - return CreateClickHouseReadActor(gateway, std::move(settings), args.InputIndex, args.SecureParams, args.TaskParams, args.ComputeActorId, credentialsFactory); + return CreateClickHouseReadActor(gateway, std::move(settings), args.InputIndex, args.StatsLevel, + args.SecureParams, args.TaskParams, args.ComputeActorId, credentialsFactory); }); } diff --git a/ydb/library/yql/providers/dq/actors/compute_actor.cpp b/ydb/library/yql/providers/dq/actors/compute_actor.cpp index 084c78dd0d..a944e60b00 100644 --- a/ydb/library/yql/providers/dq/actors/compute_actor.cpp +++ b/ydb/library/yql/providers/dq/actors/compute_actor.cpp @@ -22,7 +22,8 @@ IActor* CreateComputeActor( NYql::NDqProto::TDqTask* task, const TString& computeActorType, const NDq::NTaskRunnerActor::ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory, - ::NMonitoring::TDynamicCounterPtr taskCounters) + ::NMonitoring::TDynamicCounterPtr taskCounters, + NDqProto::EDqStatsMode statsMode) { auto memoryLimits = NDq::TComputeMemoryLimits(); memoryLimits.ChannelBufferSize = 1000000; @@ -38,7 +39,7 @@ IActor* CreateComputeActor( auto computeRuntimeSettings = NDq::TComputeRuntimeSettings(); computeRuntimeSettings.ExtraMemoryAllocationPool = 3; computeRuntimeSettings.FailOnUndelivery = false; - computeRuntimeSettings.StatsMode = NDqProto::DQ_STATS_MODE_PROFILE; + computeRuntimeSettings.StatsMode = (statsMode != NDqProto::DQ_STATS_MODE_UNSPECIFIED) ? statsMode : NDqProto::DQ_STATS_MODE_FULL; // clear fake actorids for (auto& input : *task->MutableInputs()) { diff --git a/ydb/library/yql/providers/dq/actors/compute_actor.h b/ydb/library/yql/providers/dq/actors/compute_actor.h index 25199bd28b..807e2d24c1 100644 --- a/ydb/library/yql/providers/dq/actors/compute_actor.h +++ b/ydb/library/yql/providers/dq/actors/compute_actor.h @@ -10,9 +10,10 @@ NActors::IActor* CreateComputeActor( NDq::IMemoryQuotaManager::TPtr memoryQuotaManager, const NActors::TActorId& executerId, const TString& operationId, - NYql::NDqProto::TDqTask* task, + NDqProto::TDqTask* task, const TString& computeActorType, const NDq::NTaskRunnerActor::ITaskRunnerActorFactory::TPtr& taskRunnerActorFactory, - ::NMonitoring::TDynamicCounterPtr taskCounters = nullptr); + ::NMonitoring::TDynamicCounterPtr taskCounters, + NDqProto::EDqStatsMode statsMode); } // namespace NYql diff --git a/ydb/library/yql/providers/dq/actors/events.cpp b/ydb/library/yql/providers/dq/actors/events.cpp index 85863d5260..51e1558797 100644 --- a/ydb/library/yql/providers/dq/actors/events.cpp +++ b/ydb/library/yql/providers/dq/actors/events.cpp @@ -41,9 +41,10 @@ namespace NYql::NDqs { NActors::ActorIdToProto(resultId, Record.MutableResultId()); } - TEvReadyState::TEvReadyState(NActors::TActorId sourceId, TString type) { + TEvReadyState::TEvReadyState(NActors::TActorId sourceId, TString type, NYql::NDqProto::EDqStatsMode statsMode) { NActors::ActorIdToProto(sourceId, Record.MutableSourceId()); *Record.MutableResultType() = std::move(type); + Record.SetStatsMode(statsMode); } TEvReadyState::TEvReadyState(NDqProto::TReadyState&& proto) { diff --git a/ydb/library/yql/providers/dq/actors/events.h b/ydb/library/yql/providers/dq/actors/events.h index df65b75b32..f3f94f0b70 100644 --- a/ydb/library/yql/providers/dq/actors/events.h +++ b/ydb/library/yql/providers/dq/actors/events.h @@ -45,7 +45,7 @@ namespace NYql::NDqs { struct TEvReadyState : NActors::TEventPB<TEvReadyState, NDqProto::TReadyState, TDqExecuterEvents::ES_READY_TO_PULL> { TEvReadyState() = default; - TEvReadyState(NActors::TActorId sourceId, TString type); + TEvReadyState(NActors::TActorId sourceId, TString type, NYql::NDqProto::EDqStatsMode statsMode); explicit TEvReadyState(NDqProto::TReadyState&& proto); }; diff --git a/ydb/library/yql/providers/dq/actors/executer_actor.cpp b/ydb/library/yql/providers/dq/actors/executer_actor.cpp index 6567d5f58b..77ce1ac57a 100644 --- a/ydb/library/yql/providers/dq/actors/executer_actor.cpp +++ b/ydb/library/yql/providers/dq/actors/executer_actor.cpp @@ -142,6 +142,9 @@ private: }); ControlId = NActors::ActorIdFromProto(ev->Get()->Record.GetControlId()); ResultId = NActors::ActorIdFromProto(ev->Get()->Record.GetResultId()); + if (ev->Get()->Record.GetRequest().GetStatsMode() != NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_UNSPECIFIED) { + StatsMode = ev->Get()->Record.GetRequest().GetStatsMode(); + } // These actors will be killed at exit. AddChild(ControlId); AddChild(ResultId); @@ -191,11 +194,13 @@ private: TraceId, Settings, Counters, enableComputeActor ? tasks : TVector<NYql::NDqProto::TDqTask>(), - computeActorType)); + computeActorType, + StatsMode)); auto allocateRequest = MakeHolder<TEvAllocateWorkersRequest>(workerCount, Username); allocateRequest->Record.SetTraceId(TraceId); allocateRequest->Record.SetCreateComputeActor(enableComputeActor); allocateRequest->Record.SetComputeActorType(computeActorType); + allocateRequest->Record.SetStatsMode(StatsMode); if (enableComputeActor) { ActorIdToProto(ControlId, allocateRequest->Record.MutableResultActorId()); } @@ -224,7 +229,9 @@ private: MergeFilter(filter, pragmaFilter); } - StartCounter("AllocateWorkers"); + if (CollectBasic()) { + StartCounter("AllocateWorkersUs"); + } TActivationContext::Send(new IEventHandle( GwmActorId, @@ -256,11 +263,13 @@ private: if (Finished) { YQL_CLOG(WARN, ProviderDq) << "Re-Finish IGNORED with status=" << static_cast<int>(statusCode); } else { - FlushCounter("ExecutionTime"); TQueryResponse result; + if (CollectBasic()) { + FlushCounter("ExecutionTimeUs"); + FlushCounters(result); + } IssuesToMessage(Issues, result.MutableIssues()); result.SetStatusCode(statusCode); - FlushCounters(result); Send(ControlId, MakeHolder<TEvQueryResponse>(std::move(result))); Finished = true; } @@ -273,7 +282,9 @@ private: << ", status=" << static_cast<int>(ev->Get()->Record.GetStatusCode()) << ", issues size=" << ev->Get()->Record.IssuesSize() << ", sender=" << ev->Sender; - AddCounters(ev->Get()->Record); + if (CollectBasic()) { + AddCounters(ev->Get()->Record); + } if (ev->Get()->Record.IssuesSize()) { TIssues issues; IssuesFromMessage(ev->Get()->Record.GetIssues(), issues); @@ -323,8 +334,10 @@ private: YQL_LOG_CTX_ROOT_SESSION_SCOPE(TraceId); YQL_CLOG(DEBUG, ProviderDq) << "TDqExecuter::TEvAllocateWorkersResponse"; - AddCounters(ev->Get()->Record); - FlushCounter("AllocateWorkers"); + if (CollectBasic()) { + AddCounters(ev->Get()->Record); + FlushCounter("AllocateWorkersUs"); + } auto& response = ev->Get()->Record; switch (response.GetTResponseCase()) { @@ -371,14 +384,16 @@ private: uniqueWorkers.insert(std::make_pair(workerInfo.GetGuid(), workerInfo)); } - AddCounter("UniqueWorkers", uniqueWorkers.size()); + if (CollectBasic()) { + AddCounter("UniqueWorkers", uniqueWorkers.size()); + } } YQL_CLOG(INFO, ProviderDq) << workers.size() << " workers allocated"; YQL_ENSURE(workers.size() == tasks.size()); - auto res = MakeHolder<TEvReadyState>(ExecutionPlanner->GetSourceID(), ExecutionPlanner->GetResultType()); + auto res = MakeHolder<TEvReadyState>(ExecutionPlanner->GetSourceID(), ExecutionPlanner->GetResultType(), StatsMode); if (Settings->EnableComputeActor.Get().GetOrElse(false) == false) { for (size_t i = 0; i < tasks.size(); i++) { @@ -403,7 +418,9 @@ private: WorkersAllocated = true; ExecutionStart = TInstant::Now(); - StartCounter("ExecutionTime"); + if (CollectBasic()) { + StartCounter("ExecutionTimeUs"); + } AllocationHistogram->Collect((ExecutionStart-StartTime).Seconds()); @@ -459,6 +476,10 @@ private: } } + bool CollectBasic() { + return StatsMode >= NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_BASIC; + } + NActors::TActorId GwmActorId; NActors::TActorId PrinterId; TDqConfiguration::TPtr Settings; @@ -492,6 +513,7 @@ private: TIssues Issues; bool CreateTaskSuspended; bool Finished = false; + NYql::NDqProto::EDqStatsMode StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_FULL; }; NActors::IActor* MakeDqExecuter( diff --git a/ydb/library/yql/providers/dq/actors/resource_allocator.cpp b/ydb/library/yql/providers/dq/actors/resource_allocator.cpp index 7342252387..b2aa780918 100644 --- a/ydb/library/yql/providers/dq/actors/resource_allocator.cpp +++ b/ydb/library/yql/providers/dq/actors/resource_allocator.cpp @@ -50,7 +50,8 @@ public: const TDqConfiguration::TPtr settings, const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, const TVector<NYql::NDqProto::TDqTask>& tasks, - const TString& computeActorType) + const TString& computeActorType, + NDqProto::EDqStatsMode statsMode) : TRichActor<TResourceAllocator>(&TResourceAllocator::Handle) , GwmActor(gwmActor) , SenderId(senderId) @@ -64,6 +65,7 @@ public: , RetryCounter(counters->GetSubgroup("component", "ServiceProxyActor")->GetCounter("RetryCreateActor", /*derivative=*/ true)) , Tasks(tasks) , ComputeActorType(computeActorType) + , StatsMode(statsMode) { AllocatedWorkers.resize(workerCount); if (!Tasks.empty()) { @@ -188,7 +190,7 @@ private: if (requestedNode.ClusterName) { labels.emplace("ClusterName", requestedNode.ClusterName); } - QueryStat.AddCounter(QueryStat.GetCounterName("Actor", labels, "ActorCreateTime"), delta); + QueryStat.AddCounter(QueryStat.GetCounterName("Actor", labels, "ActorCreateTimeUs"), delta); } if (AllocatedCount == RequestedCount) { @@ -253,6 +255,7 @@ private: ActorIdToProto(ControlId, request->Record.MutableResultActorId()); *request->Record.AddTask() = node.Task; } + request->Record.SetStatsMode(StatsMode); YQL_CLOG(WARN, ProviderDq) << "Send TEvAllocateWorkersRequest to " << NDqs::NExecutionHelpers::PrettyPrintWorkerInfo(node.WorkerInfo, 0); if (backoff) { TActivationContext::Schedule(backoff, new IEventHandle( @@ -331,6 +334,7 @@ private: TVector<NYql::NDqProto::TDqTask> Tasks; // for compute actor const TString ComputeActorType; + NDqProto::EDqStatsMode StatsMode; }; NActors::IActor* CreateResourceAllocator( @@ -342,9 +346,10 @@ NActors::IActor* CreateResourceAllocator( const TDqConfiguration::TPtr& settings, const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, const TVector<NYql::NDqProto::TDqTask>& tasks, - const TString& computeActorType) + const TString& computeActorType, + NDqProto::EDqStatsMode statsMode) { - return new TResourceAllocator(gwmActor, senderId, controlId, size, traceId, settings, counters, tasks, computeActorType); + return new TResourceAllocator(gwmActor, senderId, controlId, size, traceId, settings, counters, tasks, computeActorType, statsMode); } } // namespace NYql diff --git a/ydb/library/yql/providers/dq/actors/resource_allocator.h b/ydb/library/yql/providers/dq/actors/resource_allocator.h index af9002663c..9c09873811 100644 --- a/ydb/library/yql/providers/dq/actors/resource_allocator.h +++ b/ydb/library/yql/providers/dq/actors/resource_allocator.h @@ -1,5 +1,6 @@ #pragma once +#include <ydb/library/yql/dq/actors/protos/dq_stats.pb.h> #include <ydb/library/yql/dq/proto/dq_tasks.pb.h> #include <ydb/library/yql/providers/dq/common/yql_dq_settings.h> @@ -16,5 +17,6 @@ namespace NYql { const TDqConfiguration::TPtr& settings, const TIntrusivePtr<::NMonitoring::TDynamicCounters>& counters, const TVector<NYql::NDqProto::TDqTask>& tasks = {}, - const TString& computeActorType = "old"); + const TString& computeActorType = "old", + NDqProto::EDqStatsMode statsMode = NDqProto::DQ_STATS_MODE_UNSPECIFIED); } // namespace NYql diff --git a/ydb/library/yql/providers/dq/actors/task_controller_impl.h b/ydb/library/yql/providers/dq/actors/task_controller_impl.h index ca2b3ecfa0..fd3a49f4b0 100644 --- a/ydb/library/yql/providers/dq/actors/task_controller_impl.h +++ b/ydb/library/yql/providers/dq/actors/task_controller_impl.h @@ -6,7 +6,7 @@ #include "executer_actor.h" #include "grouped_issues.h" -#include <ydb/library/yql/providers/dq/counters/counters.h> +#include <ydb/library/yql/providers/dq/counters/task_counters.h> #include <ydb/library/yql/providers/dq/common/yql_dq_common.h> @@ -158,21 +158,22 @@ public: << " PingCookie: " << ev->Cookie << " StatusCode: " << NYql::NDqProto::StatusIds_StatusCode_Name(state.GetStatusCode()); - if (state.HasStats() && TryAddStatsFromExtra(state.GetStats())) { - if (Settings->ExportStats.Get().GetOrElse(TDqSettings::TDefault::ExportStats) && ServiceCounters.Counters && !AggrPeriod) { - ExportStats(TaskStat, taskId); - TrySendNonFinalStat(); - } - } else if (state.HasStats() && state.GetStats().GetTasks().size()) { - YQL_CLOG(TRACE, ProviderDq) << " " << SelfId() << " AddStats " << taskId; - AddStats(state.GetStats()); - if (Settings->ExportStats.Get().GetOrElse(TDqSettings::TDefault::ExportStats) && ServiceCounters.Counters && !AggrPeriod) { - ExportStats(TaskStat, taskId); - TrySendNonFinalStat(); + if (CollectBasic()) { + if (state.HasStats() && TryAddStatsFromExtra(state.GetStats())) { + if (Settings->ExportStats.Get().GetOrElse(TDqSettings::TDefault::ExportStats) && ServiceCounters.Counters && !AggrPeriod) { + ExportStats(TaskStat, taskId); + TrySendNonFinalStat(); + } + } else if (state.HasStats() && state.GetStats().GetTasks().size()) { + YQL_CLOG(TRACE, ProviderDq) << " " << SelfId() << " AddStats " << taskId; + AddStats(state.GetStats()); + if (Settings->ExportStats.Get().GetOrElse(TDqSettings::TDefault::ExportStats) && ServiceCounters.Counters && !AggrPeriod) { + ExportStats(TaskStat, taskId); + TrySendNonFinalStat(); + } } } - TIssues localIssues; // TODO: don't convert issues to string NYql::IssuesFromMessage(state.GetIssues(), localIssues); @@ -230,7 +231,7 @@ public: case AGGR_TIMER_TAG: if (AggrPeriod) { if (Settings->ExportStats.Get().GetOrElse(TDqSettings::TDefault::ExportStats) && ServiceCounters.Counters) { - ExportStats(AggregateQueryStatsByStage(TaskStat, Stages), 0); + ExportStats(AggregateQueryStatsByStage(TaskStat, Stages, CollectFull()), 0); } SendNonFinalStat(); Schedule(AggrPeriod, new TEvents::TEvWakeup(AGGR_TIMER_TAG)); @@ -278,13 +279,17 @@ private: } else if (name == "CpuTimeUs") { publicCounterName = "query.cpu_usage_us"; isDeriv = true; - } else if (name == "Bytes") { + } else if (name == "IngressBytes") { if (labels.count(SourceLabel)) publicCounterName = "query.input_bytes"; - else if (labels.count(SinkLabel)) publicCounterName = "query.output_bytes"; isDeriv = true; - } else if (name == "RowsIn") { + } else if (name == "EgressBytes") { + if (labels.count(SinkLabel)) publicCounterName = "query.output_bytes"; + isDeriv = true; + } else if (name == "OutputRows") { if (labels.count(SourceLabel)) publicCounterName = "query.source_input_records"; - else if (labels.count(SinkLabel)) publicCounterName = "query.sink_output_records"; // RowsIn == RowsOut for Sinks + isDeriv = true; + } else if (name == "InputRows") { + if (labels.count(SinkLabel)) publicCounterName = "query.sink_output_records"; isDeriv = true; } else if (name == "MultiHop_LateThrownEventsCount") { publicCounterName = "query.late_events"; @@ -356,21 +361,21 @@ private: ADD_COUNTER(CpuTimeUs) ADD_COUNTER(ComputeCpuTimeUs) ADD_COUNTER(SourceCpuTimeUs) - ADD_COUNTER(FirstRowTimeMs) - ADD_COUNTER(FinishTimeMs) + // ADD_COUNTER(FirstRowTimeMs) + // ADD_COUNTER(FinishTimeMs) ADD_COUNTER(InputRows) ADD_COUNTER(InputBytes) ADD_COUNTER(OutputRows) ADD_COUNTER(OutputBytes) - ADD_COUNTER(StartTimeMs) + // ADD_COUNTER(StartTimeMs) // profile stats ADD_COUNTER(BuildCpuTimeUs) - ADD_COUNTER(WaitTimeUs) - ADD_COUNTER(WaitOutputTimeUs) - ADD_COUNTER(PendingInputTimeUs) - ADD_COUNTER(PendingOutputTimeUs) - ADD_COUNTER(FinishTimeUs) + // ADD_COUNTER(WaitTimeUs) + // ADD_COUNTER(WaitOutputTimeUs) + // ADD_COUNTER(PendingInputTimeUs) + // ADD_COUNTER(PendingOutputTimeUs) + // ADD_COUNTER(FinishTimeUs) for (const auto& ingress : s.GetIngress()) { TaskStat.SetCounter(TaskStat.GetCounterName("TaskRunner", labels, "Ingress" + ingress.GetName() + "Bytes"), ingress.GetBytes()); @@ -415,92 +420,35 @@ private: auto labels = commonLabels; labels["InputChannel"] = ToString(stats.GetChannelId()); labels["SrcStageId"] = ToString(stats.GetSrcStageId()); - - ADD_COUNTER(Chunks); - ADD_COUNTER(Bytes); - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(MaxMemoryUsage); - ADD_COUNTER(DeserializationTimeUs); - - ADD_COUNTER(IdleTimeUs); - ADD_COUNTER(WaitTimeUs); - ADD_COUNTER(FirstMessageMs); - ADD_COUNTER(LastMessageMs); - - if (stats.GetFirstMessageMs() && stats.GetLastMessageMs()) { - TaskStat.SetCounter(TaskStat.GetCounterName("TaskRunner", labels, "ActiveTimeUs"), - ( TInstant::MilliSeconds(stats.GetLastMessageMs()) - - TInstant::MilliSeconds(stats.GetFirstMessageMs()) ).MicroSeconds() - ); - } + TaskStat.AddAsyncStats(stats.GetPush(), labels, "Push"); + TaskStat.AddAsyncStats(stats.GetPop(), labels, "Pop"); } for (const auto& stats : s.GetOutputChannels()) { auto labels = commonLabels; labels["OutputChannel"] = ToString(stats.GetChannelId()); labels["DstStageId"] = ToString(stats.GetDstStageId()); - - ADD_COUNTER(Chunks) - ADD_COUNTER(Bytes); - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(MaxMemoryUsage); - - ADD_COUNTER(SerializationTimeUs); - ADD_COUNTER(BlockedByCapacity); - - ADD_COUNTER(SpilledBytes); - ADD_COUNTER(SpilledRows); - ADD_COUNTER(SpilledBlobs); - - ADD_COUNTER(BlockedTimeUs); - ADD_COUNTER(FirstMessageMs); - ADD_COUNTER(LastMessageMs); - - if (stats.GetFirstMessageMs() && stats.GetLastMessageMs()) { - TaskStat.SetCounter(TaskStat.GetCounterName("TaskRunner", labels, "ActiveTimeUs"), - ( TInstant::MilliSeconds(stats.GetLastMessageMs()) - - TInstant::MilliSeconds(stats.GetFirstMessageMs()) ).MicroSeconds() - ); - } + TaskStat.AddAsyncStats(stats.GetPush(), labels, "Push"); + TaskStat.AddAsyncStats(stats.GetPop(), labels, "Pop"); } } for (const auto& stats : s.GetSources()) { auto labels = commonLabels; labels["Source"] = ToString(stats.GetInputIndex()); - - ADD_COUNTER(Chunks); - ADD_COUNTER(Bytes); - ADD_COUNTER(IngressBytes) - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(MaxMemoryUsage); - - ADD_COUNTER(ErrorsCount); - -// if (stats.GetFinishTs() >= stats.GetStartTs()) { -// TaskStat.SetCounter(TaskStat.GetCounterName("TaskRunner", labels, "Total"), stats.GetFinishTs() - stats.GetStartTs()); -// } + labels["Name"] = stats.GetIngressName(); + TaskStat.AddAsyncStats(stats.GetIngress(), labels, "Ingress"); + TaskStat.AddAsyncStats(stats.GetPush(), labels, "Push"); + TaskStat.AddAsyncStats(stats.GetPop(), labels, "Pop"); } for (const auto& stats : s.GetSinks()) { auto labels = commonLabels; labels["Sink"] = ToString(stats.GetOutputIndex()); - - ADD_COUNTER(Chunks) - ADD_COUNTER(Bytes); - ADD_COUNTER(EgressBytes) - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(MaxMemoryUsage); - - ADD_COUNTER(ErrorsCount); - -// if (stats.GetFinishTs() >= stats.GetStartTs()) { -// TaskStat.SetCounter(TaskStat.GetCounterName("TaskRunner", labels, "Total"), stats.GetFinishTs() - stats.GetStartTs()); -// } + labels["Name"] = stats.GetEgressName(); + TaskStat.AddAsyncStats(stats.GetPush(), labels, "Push"); + TaskStat.AddAsyncStats(stats.GetPop(), labels, "Pop"); + TaskStat.AddAsyncStats(stats.GetEgress(), labels, "Egress"); } #undef ADD_COUNTER @@ -526,6 +474,10 @@ private: public: void OnReadyState(TEvReadyState::TPtr& ev) { + + if (ev->Get()->Record.GetStatsMode() != NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_UNSPECIFIED) { + StatsMode = ev->Get()->Record.GetStatsMode(); + } YQL_LOG_CTX_ROOT_SESSION_SCOPE(TraceId); TaskStat.AddCounters(ev->Get()->Record); @@ -641,7 +593,7 @@ public: private: void Finish() { if (Settings->ExportStats.Get().GetOrElse(TDqSettings::TDefault::ExportStats) && ServiceCounters.Counters && AggrPeriod) { - ExportStats(AggregateQueryStatsByStage(TaskStat, Stages), 0); // force metrics upload on Finish when Aggregated + ExportStats(AggregateQueryStatsByStage(TaskStat, Stages, CollectFull()), 0); // force metrics upload on Finish when Aggregated } Send(ExecuterId, new TEvGraphFinished()); Finished = true; @@ -659,9 +611,16 @@ public: private: TCounters FinalStat() { - return AggrPeriod ? AggregateQueryStatsByStage(TaskStat, Stages) : TaskStat; + return AggrPeriod ? AggregateQueryStatsByStage(TaskStat, Stages, CollectFull()) : TaskStat; } + bool CollectBasic() { + return StatsMode >= NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_BASIC; + } + + bool CollectFull() { + return StatsMode >= NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_FULL; + } bool ChannelsUpdated = false; TVector<std::pair<NDq::TDqTaskSettings, TActorId>> Tasks; @@ -675,13 +634,14 @@ private: const TString TraceId; TDqConfiguration::TPtr Settings; bool Finished = false; - TCounters TaskStat; + TTaskCounters TaskStat; NYql::NCommon::TServiceCounters ServiceCounters; TDuration PingPeriod = TDuration::Zero(); TDuration AggrPeriod = TDuration::Zero(); NYql::NDq::GroupedIssues Issues; ui64 PingCookie = 0; TInstant LastStatReport; + NYql::NDqProto::EDqStatsMode StatsMode = NYql::NDqProto::EDqStatsMode::DQ_STATS_MODE_FULL; }; } /* namespace NYql */ diff --git a/ydb/library/yql/providers/dq/actors/worker_actor.cpp b/ydb/library/yql/providers/dq/actors/worker_actor.cpp index 9c306215a5..915ca8453b 100644 --- a/ydb/library/yql/providers/dq/actors/worker_actor.cpp +++ b/ydb/library/yql/providers/dq/actors/worker_actor.cpp @@ -75,6 +75,10 @@ class TDummyMemoryQuotaManager: public IMemoryQuotaManager { ui64 GetCurrentQuota() const override { return std::numeric_limits<ui64>::max(); } + + ui64 GetMaxMemorySize() const override { + return std::numeric_limits<ui64>::max(); + } }; class TDqWorker: public TRichActor<TDqWorker> diff --git a/ydb/library/yql/providers/dq/api/protos/dqs.proto b/ydb/library/yql/providers/dq/api/protos/dqs.proto index 4279bf529a..8c4f40c47b 100644 --- a/ydb/library/yql/providers/dq/api/protos/dqs.proto +++ b/ydb/library/yql/providers/dq/api/protos/dqs.proto @@ -4,6 +4,7 @@ option cc_enable_arenas = true; package NYql.NDqProto; import "google/protobuf/any.proto"; +import "ydb/library/yql/dq/actors/protos/dq_stats.proto"; import "ydb/library/yql/dq/actors/protos/dq_status_codes.proto"; import "ydb/library/yql/dq/proto/dq_transport.proto"; import "ydb/library/yql/dq/proto/dq_tasks.proto"; @@ -37,6 +38,7 @@ message TAllocateWorkersRequest { NActorsProto.TActorId ResultActorId = 13; // used for compute actor uint64 FreeWorkerAfterMs = 14; + NYql.NDqProto.EDqStatsMode StatsMode = 16; } message TWorkerGroup { @@ -210,6 +212,7 @@ message TReadyState { repeated TMetric Metric = 3; repeated NYql.NDqProto.TDqTask Task = 4; // used for compute actor repeated NActorsProto.TActorId ActorId = 5; // used for compute actor + NYql.NDqProto.EDqStatsMode StatsMode = 6; } enum EGraphExecutionEventType { diff --git a/ydb/library/yql/providers/dq/api/protos/service.proto b/ydb/library/yql/providers/dq/api/protos/service.proto index 2d68e636cf..194e220127 100644 --- a/ydb/library/yql/providers/dq/api/protos/service.proto +++ b/ydb/library/yql/providers/dq/api/protos/service.proto @@ -3,6 +3,7 @@ option cc_enable_arenas = true; import "ydb/public/api/protos/ydb_operation.proto"; import "ydb/public/api/protos/ydb_value.proto"; +import "ydb/library/yql/dq/actors/protos/dq_stats.proto"; import "ydb/library/yql/dq/proto/dq_tasks.proto"; package Yql.DqsProto; @@ -86,6 +87,7 @@ message ExecuteGraphRequest { string RateLimiter = 14; string RateLimiterResource = 15; map<string, bytes> CommonTaskParams = 16; // to be merged into each task TTaskMeta.TaskParams + NYql.NDqProto.EDqStatsMode StatsMode = 17; } message ExecuteGraphResponse { diff --git a/ydb/library/yql/providers/dq/counters/CMakeLists.darwin-x86_64.txt b/ydb/library/yql/providers/dq/counters/CMakeLists.darwin-x86_64.txt index 3f0ba79d39..0bcccf96f3 100644 --- a/ydb/library/yql/providers/dq/counters/CMakeLists.darwin-x86_64.txt +++ b/ydb/library/yql/providers/dq/counters/CMakeLists.darwin-x86_64.txt @@ -18,4 +18,5 @@ target_link_libraries(providers-dq-counters PUBLIC ) target_sources(providers-dq-counters PRIVATE ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/counters.cpp + ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/task_counters.cpp ) diff --git a/ydb/library/yql/providers/dq/counters/CMakeLists.linux-aarch64.txt b/ydb/library/yql/providers/dq/counters/CMakeLists.linux-aarch64.txt index 3879ce3cf0..077f4e8a5d 100644 --- a/ydb/library/yql/providers/dq/counters/CMakeLists.linux-aarch64.txt +++ b/ydb/library/yql/providers/dq/counters/CMakeLists.linux-aarch64.txt @@ -19,4 +19,5 @@ target_link_libraries(providers-dq-counters PUBLIC ) target_sources(providers-dq-counters PRIVATE ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/counters.cpp + ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/task_counters.cpp ) diff --git a/ydb/library/yql/providers/dq/counters/CMakeLists.linux-x86_64.txt b/ydb/library/yql/providers/dq/counters/CMakeLists.linux-x86_64.txt index 3879ce3cf0..077f4e8a5d 100644 --- a/ydb/library/yql/providers/dq/counters/CMakeLists.linux-x86_64.txt +++ b/ydb/library/yql/providers/dq/counters/CMakeLists.linux-x86_64.txt @@ -19,4 +19,5 @@ target_link_libraries(providers-dq-counters PUBLIC ) target_sources(providers-dq-counters PRIVATE ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/counters.cpp + ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/task_counters.cpp ) diff --git a/ydb/library/yql/providers/dq/counters/CMakeLists.windows-x86_64.txt b/ydb/library/yql/providers/dq/counters/CMakeLists.windows-x86_64.txt index 3f0ba79d39..0bcccf96f3 100644 --- a/ydb/library/yql/providers/dq/counters/CMakeLists.windows-x86_64.txt +++ b/ydb/library/yql/providers/dq/counters/CMakeLists.windows-x86_64.txt @@ -18,4 +18,5 @@ target_link_libraries(providers-dq-counters PUBLIC ) target_sources(providers-dq-counters PRIVATE ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/counters.cpp + ${CMAKE_SOURCE_DIR}/ydb/library/yql/providers/dq/counters/task_counters.cpp ) diff --git a/ydb/library/yql/providers/dq/counters/counters.cpp b/ydb/library/yql/providers/dq/counters/counters.cpp index df3171b004..caddba0e04 100644 --- a/ydb/library/yql/providers/dq/counters/counters.cpp +++ b/ydb/library/yql/providers/dq/counters/counters.cpp @@ -1,163 +1 @@ #include "counters.h" -#include <ydb/library/yql/providers/dq/common/yql_dq_common.h> - -namespace NYql { - -void AddHistogram(THashMap<i64, ui64>& aggregatedHist, const THashMap<i64, ui64>& hist) { - for (const auto& [k, v] : hist) { - aggregatedHist[k] += v; - } -} - -TCounters AggregateQueryStatsByStage(TCounters& queryStat, const THashMap<ui64, ui64>& task2Stage) { - TCounters aggregatedQueryStat; - THashMap<TString, THashSet<ui64>> stage2Tasks; - THashMap<TString, THashSet<ui64>> stage2Output; - THashMap<TString, THashSet<ui64>> stage2Input; - - /* Depends on missing TotalTime statistics - THashMap<ui64, i64> BusyTime; - for (const auto& [k, v] : queryStat.Get()) { - std::map<TString, TString> labels; - TString prefix, name; - if (NCommon::ParseCounterName(&prefix, &labels, &name, k)) { - if (prefix == "TaskRunner") { - auto maybeTask = labels.find("Task"); - if (maybeTask == labels.end()) { - continue; - } - ui64 taskId; - if (!TryFromString(maybeTask->second, taskId)) { - continue; - } - if (name == "TotalTime") { - BusyTime[taskId] += v.Sum; - } else if (name == "WaitTime") { - BusyTime[taskId] -= v.Sum; - } - } - } - } - */ - /* - for (const auto& [taskId, value] : BusyTime) { - TCounters::TEntry entry = {value, value, value, value, 1}; - queryStat.AddCounter(queryStat.GetCounterName("TaskRunner", {{"Task", ToString(taskId)}}, "BusyTime"), entry); - } - */ - for (const auto& [k, v] : queryStat.Get()) { - std::map<TString, TString> labels; - TString prefix, name; - if (k.StartsWith("TaskRunner") && NCommon::ParseCounterName(&prefix, &labels, &name, k)) { - auto maybeTask = labels.find("Task"); - if (maybeTask == labels.end()) { - aggregatedQueryStat.AddCounter(k, v); - continue; - } - ui64 taskId; - if (!TryFromString(maybeTask->second, taskId)) { - continue; - } - auto maybeStage = task2Stage.find(taskId); - TString stageId = maybeStage == task2Stage.end() - ? "0" - : ToString(maybeStage->second); - ui64 channelId; - bool input = false; - bool output = false; - auto maybeInputChannel = labels.find("InputChannel"); - if (maybeInputChannel != labels.end()) { - if (!TryFromString(maybeInputChannel->second, channelId)) { - continue; - } - ui32 stage = 0; - auto maybeSrcStageId = labels.find("SrcStageId"); - if (maybeSrcStageId != labels.end()) { - TryFromString(maybeSrcStageId->second, stage); - labels.erase(maybeSrcStageId); - } - stage2Input[stageId].insert(channelId); - stage2Input["Total"].insert(channelId); - labels.erase(maybeInputChannel); - labels["Input"] = ToString(stage); - input = true; - } - auto maybeOutputChannel = labels.find("OutputChannel"); - if (maybeOutputChannel != labels.end()) { - if (!TryFromString(maybeOutputChannel->second, channelId)) { - continue; - } - ui32 stage = 0; - auto maybeDstStageId = labels.find("DstStageId"); - if (maybeDstStageId != labels.end()) { - TryFromString(maybeDstStageId->second, stage); - labels.erase(maybeDstStageId); - } - stage2Output[stageId].insert(channelId); - stage2Output["Total"].insert(channelId); - labels.erase(maybeOutputChannel); - labels["Output"] = ToString(stage); - output = true; - } - labels.erase(maybeTask); - labels["Stage"] = ToString(stageId); - stage2Tasks[stageId].insert(taskId); - stage2Tasks["Total"].insert(taskId); - aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", labels, name), v); - if (input || output) { - if (input) { - labels["Input"] = "Total"; - } - if (output) { - labels["Output"] = "Total"; - } - aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", labels, name), v); - } - labels["Stage"] = "Total"; - aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", labels, name), v); - } else { - aggregatedQueryStat.AddCounter(k, v); - } - } - for (const auto& [k, v] : queryStat.GetHistograms()) { - std::map<TString, TString> labels; - TString prefix, name; - if (k.StartsWith("TaskRunner") && NCommon::ParseCounterName(&prefix, &labels, &name, k)) { - auto maybeTask = labels.find("Task"); - if (maybeTask == labels.end()) { - AddHistogram(aggregatedQueryStat.GetHistogram(k), v); - continue; - } - ui64 taskId; - if (!TryFromString(maybeTask->second, taskId)) { - continue; - } - auto maybeStage = task2Stage.find(taskId); - TString stageId = maybeStage == task2Stage.end() - ? "0" - : ToString(maybeStage->second); - labels.erase(maybeTask); - labels["Stage"] = ToString(stageId); - AddHistogram(aggregatedQueryStat.GetHistogram(queryStat.GetCounterName("TaskRunner", labels, name)), v); - } else { - AddHistogram(aggregatedQueryStat.GetHistogram(k), v); - } - } - for (const auto& [stageId, v] : stage2Tasks) { - aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", - {{"Stage", stageId}}, "TasksCount"), static_cast<ui64>(v.size())); - } - for (const auto& [stageId, v] : stage2Input) { - aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", - {{"Stage", stageId},{"Input", "Total"}}, "ChannelsCount"), static_cast<ui64>(v.size())); - } - for (const auto& [stageId, v] : stage2Output) { - aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", - {{"Stage", stageId},{"Output", "Total"}}, "ChannelsCount"), static_cast<ui64>(v.size())); - } - aggregatedQueryStat.AddCounter("StagesCount", static_cast<ui64>(stage2Tasks.size())); - - return aggregatedQueryStat; -} - -} // namespace NYql diff --git a/ydb/library/yql/providers/dq/counters/counters.h b/ydb/library/yql/providers/dq/counters/counters.h index de37fc7a31..d99722fae1 100644 --- a/ydb/library/yql/providers/dq/counters/counters.h +++ b/ydb/library/yql/providers/dq/counters/counters.h @@ -1,9 +1,6 @@ #pragma once #include <ydb/library/yql/core/yql_execution.h> -#include <ydb/library/yql/dq/runtime/dq_input_channel.h> -#include <ydb/library/yql/dq/runtime/dq_output_channel.h> -#include <ydb/library/yql/dq/runtime/dq_tasks_runner.h> #include <util/string/split.h> @@ -73,7 +70,7 @@ struct TCounters { } void AddCounter(const TString& name, TDuration value) const { - auto val = value.MilliSeconds(); + auto val = value.MicroSeconds(); auto& counter = Counters[name]; counter.Sum += val; counter.Min = counter.Count == 0 @@ -196,98 +193,10 @@ struct TCounters { Start.clear(); } -#define ADD_COUNTER(name) \ - do { \ - auto value = currentStats.name - oldStats.name; \ - if (value) { \ - AddCounter(GetCounterName("TaskRunner", labels, #name), value); \ - } \ - oldStats.name = currentStats.name; \ - } while (0); - - void AddInputChannelStats( - const NDq::TDqInputChannelStats& currentStats, - NDq::TDqInputChannelStats& oldStats, - ui64 taskId, - ui64 channelId) - { - std::map<TString, TString> labels = { - {"Task", ToString(taskId)}, - {"InputChannel", ToString(channelId)} - }; - - ADD_COUNTER(Chunks); - ADD_COUNTER(Bytes); - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(RowsInMemory); - ADD_COUNTER(MaxMemoryUsage); - ADD_COUNTER(DeserializationTime); - } - - void AddSourceStats( - const NDq::TDqAsyncInputBufferStats& currentStats, - NDq::TDqAsyncInputBufferStats& oldStats, - ui64 taskId, ui64 inputIndex) - { - std::map<TString, TString> labels = { - {"Task", ToString(taskId)}, - {"SourceIndex", ToString(inputIndex)} - }; - - ADD_COUNTER(Chunks); - ADD_COUNTER(Bytes); - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(RowsInMemory); - ADD_COUNTER(MaxMemoryUsage); - ADD_COUNTER(InputIndex); - } - - void AddOutputChannelStats( - const NDq::TDqOutputChannelStats& currentStats, - NDq::TDqOutputChannelStats& oldStats, - ui64 taskId, ui64 channelId) - { - std::map<TString, TString> labels = { - {"Task", ToString(taskId)}, - {"OutputChannel", ToString(channelId)} - }; - - ADD_COUNTER(Chunks) - ADD_COUNTER(Bytes); - ADD_COUNTER(RowsIn); - ADD_COUNTER(RowsOut); - ADD_COUNTER(MaxMemoryUsage); - ADD_COUNTER(MaxRowsInMemory); - - ADD_COUNTER(SerializationTime); - - ADD_COUNTER(SpilledBytes); - ADD_COUNTER(SpilledRows); - ADD_COUNTER(SpilledBlobs); - } - - void AddTaskRunnerStats( - const NDq::TDqTaskRunnerStats& currentStats, - NDq::TDqTaskRunnerStats& oldStats, - ui64 taskId) - { - std::map<TString, TString> labels = { - {"Task", ToString(taskId)} - }; - - // basic stats - ADD_COUNTER(ComputeCpuTime) - ADD_COUNTER(BuildCpuTime) - - // profile stats - ADD_COUNTER(WaitTime) - ADD_COUNTER(WaitOutputTime) + void ClearCounters() { + Counters.clear(); } -#undef ADD_COUNTER - protected: mutable THashMap<TString, TEntry> Counters; @@ -295,6 +204,4 @@ protected: mutable THashMap<TString, TInstant> Start; }; -TCounters AggregateQueryStatsByStage(TCounters& queryStat, const THashMap<ui64, ui64>& task2Stage); - } // namespace NYql diff --git a/ydb/library/yql/providers/dq/counters/task_counters.cpp b/ydb/library/yql/providers/dq/counters/task_counters.cpp new file mode 100644 index 0000000000..cd56913042 --- /dev/null +++ b/ydb/library/yql/providers/dq/counters/task_counters.cpp @@ -0,0 +1,173 @@ +#include "task_counters.h" +#include <ydb/library/yql/providers/dq/common/yql_dq_common.h> + +namespace NYql { + +void AddHistogram(THashMap<i64, ui64>& aggregatedHist, const THashMap<i64, ui64>& hist) { + for (const auto& [k, v] : hist) { + aggregatedHist[k] += v; + } +} + +TTaskCounters AggregateQueryStatsByStage(TTaskCounters& queryStat, const THashMap<ui64, ui64>& task2Stage, bool collectFull) { + TTaskCounters aggregatedQueryStat; + THashMap<TString, THashSet<ui64>> stage2Tasks; + THashMap<TString, THashSet<ui64>> stage2Output; + THashMap<TString, THashSet<ui64>> stage2Input; + + /* Depends on missing TotalTime statistics + THashMap<ui64, i64> BusyTime; + for (const auto& [k, v] : queryStat.Get()) { + std::map<TString, TString> labels; + TString prefix, name; + if (NCommon::ParseCounterName(&prefix, &labels, &name, k)) { + if (prefix == "TaskRunner") { + auto maybeTask = labels.find("Task"); + if (maybeTask == labels.end()) { + continue; + } + ui64 taskId; + if (!TryFromString(maybeTask->second, taskId)) { + continue; + } + if (name == "TotalTime") { + BusyTime[taskId] += v.Sum; + } else if (name == "WaitTime") { + BusyTime[taskId] -= v.Sum; + } + } + } + } + */ + /* + for (const auto& [taskId, value] : BusyTime) { + TCounters::TEntry entry = {value, value, value, value, 1}; + queryStat.AddCounter(queryStat.GetCounterName("TaskRunner", {{"Task", ToString(taskId)}}, "BusyTime"), entry); + } + */ + for (const auto& [k, v] : queryStat.Get()) { + std::map<TString, TString> labels; + TString prefix, name; + if (k.StartsWith("TaskRunner") && NCommon::ParseCounterName(&prefix, &labels, &name, k)) { + auto maybeTask = labels.find("Task"); + if (maybeTask == labels.end()) { + aggregatedQueryStat.AddCounter(k, v); + continue; + } + ui64 taskId; + if (!TryFromString(maybeTask->second, taskId)) { + continue; + } + auto maybeStage = task2Stage.find(taskId); + TString stageId = maybeStage == task2Stage.end() + ? "0" + : ToString(maybeStage->second); + ui64 channelId; + bool input = false; + bool output = false; + auto maybeInputChannel = labels.find("InputChannel"); + if (maybeInputChannel != labels.end()) { + if (!TryFromString(maybeInputChannel->second, channelId)) { + continue; + } + ui32 stage = 0; + auto maybeSrcStageId = labels.find("SrcStageId"); + if (maybeSrcStageId != labels.end()) { + TryFromString(maybeSrcStageId->second, stage); + labels.erase(maybeSrcStageId); + } + stage2Input[stageId].insert(channelId); + stage2Input["Total"].insert(channelId); + labels.erase(maybeInputChannel); + labels["Input"] = ToString(stage); + input = true; + } + auto maybeOutputChannel = labels.find("OutputChannel"); + if (maybeOutputChannel != labels.end()) { + if (!TryFromString(maybeOutputChannel->second, channelId)) { + continue; + } + ui32 stage = 0; + auto maybeDstStageId = labels.find("DstStageId"); + if (maybeDstStageId != labels.end()) { + TryFromString(maybeDstStageId->second, stage); + labels.erase(maybeDstStageId); + } + stage2Output[stageId].insert(channelId); + stage2Output["Total"].insert(channelId); + labels.erase(maybeOutputChannel); + labels["Output"] = ToString(stage); + output = true; + } + labels.erase(maybeTask); + labels["Stage"] = ToString(stageId); + stage2Tasks[stageId].insert(taskId); + stage2Tasks["Total"].insert(taskId); + if (collectFull) { + aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", labels, name), v); + } + if (input || output) { + if (input) { + labels["Input"] = "Total"; + } + if (output) { + labels["Output"] = "Total"; + } + if (collectFull) { + aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", labels, name), v); + } + } + labels["Stage"] = "Total"; + aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", labels, name), v); + } else { + aggregatedQueryStat.AddCounter(k, v); + } + } + for (const auto& [k, v] : queryStat.GetHistograms()) { + std::map<TString, TString> labels; + TString prefix, name; + if (k.StartsWith("TaskRunner") && NCommon::ParseCounterName(&prefix, &labels, &name, k)) { + auto maybeTask = labels.find("Task"); + if (maybeTask == labels.end()) { + AddHistogram(aggregatedQueryStat.GetHistogram(k), v); + continue; + } + ui64 taskId; + if (!TryFromString(maybeTask->second, taskId)) { + continue; + } + auto maybeStage = task2Stage.find(taskId); + TString stageId = maybeStage == task2Stage.end() + ? "0" + : ToString(maybeStage->second); + labels.erase(maybeTask); + labels["Stage"] = ToString(stageId); + AddHistogram(aggregatedQueryStat.GetHistogram(queryStat.GetCounterName("TaskRunner", labels, name)), v); + } else { + AddHistogram(aggregatedQueryStat.GetHistogram(k), v); + } + } + + for (const auto& [stageId, v] : stage2Tasks) { + if (collectFull || stageId == "Total") { + aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", + {{"Stage", stageId}}, "TasksCount"), static_cast<ui64>(v.size())); + } + } + for (const auto& [stageId, v] : stage2Input) { + if (collectFull || stageId == "Total") { + aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", + {{"Stage", stageId},{"Input", "Total"}}, "ChannelsCount"), static_cast<ui64>(v.size())); + } + } + for (const auto& [stageId, v] : stage2Output) { + if (collectFull || stageId == "Total") { + aggregatedQueryStat.AddCounter(queryStat.GetCounterName("TaskRunner", + {{"Stage", stageId},{"Output", "Total"}}, "ChannelsCount"), static_cast<ui64>(v.size())); + } + } + + return aggregatedQueryStat; +} + +} // namespace NYql diff --git a/ydb/library/yql/providers/dq/counters/task_counters.h b/ydb/library/yql/providers/dq/counters/task_counters.h new file mode 100644 index 0000000000..aee8bad563 --- /dev/null +++ b/ydb/library/yql/providers/dq/counters/task_counters.h @@ -0,0 +1,145 @@ +#pragma once + +#include "counters.h" +#include <ydb/library/yql/dq/runtime/dq_input_channel.h> +#include <ydb/library/yql/dq/runtime/dq_output_channel.h> +#include <ydb/library/yql/dq/runtime/dq_tasks_runner.h> +#include <util/string/split.h> + +namespace NYql { + +struct TTaskCounters : public TCounters { + + void AddAsyncStats(const NDqProto::TDqAsyncBufferStats stats, const std::map<TString, TString>& l, const TString& p) { + if (auto v = stats.GetBytes(); v) SetCounter(GetCounterName("TaskRunner", l, p + "Bytes"), v); + if (auto v = stats.GetRows(); v) SetCounter(GetCounterName("TaskRunner", l, p + "Rows"), v); + if (auto v = stats.GetChunks(); v) SetCounter(GetCounterName("TaskRunner", l, p + "Chunks"), v); + if (auto v = stats.GetSplits(); v) SetCounter(GetCounterName("TaskRunner", l, p + "Splits"), v); + + auto firstMessageMs = stats.GetFirstMessageMs(); + auto lastMessageMs = stats.GetLastMessageMs(); + + if (firstMessageMs) SetCounter(GetCounterName("TaskRunner", l, p + "FirstMessageMs"), firstMessageMs); + if (auto v = stats.GetPauseMessageMs(); v) SetCounter(GetCounterName("TaskRunner", l, p + "PauseMessageMs"), v); + if (auto v = stats.GetResumeMessageMs(); v) SetCounter(GetCounterName("TaskRunner", l, p + "ResumeMessageMs"), v); + if (lastMessageMs) SetCounter(GetCounterName("TaskRunner", l, p + "LastMessageMs"), lastMessageMs); + + if (auto v = stats.GetWaitTimeUs(); v) SetCounter(GetCounterName("TaskRunner", l, p + "WaitTimeUs"), v); + if (auto v = stats.GetWaitPeriods(); v) SetCounter(GetCounterName("TaskRunner", l, p + "WaitPeriods"), v); + + if (firstMessageMs && lastMessageMs && (firstMessageMs < lastMessageMs)) { + SetCounter(GetCounterName("TaskRunner", l, p + "ActiveTimeUs"), + (TInstant::MilliSeconds(lastMessageMs) - TInstant::MilliSeconds(firstMessageMs)).MicroSeconds() + ); + } + } + + void AddAsyncStats(const NDq::TDqAsyncStats stats, const std::map<TString, TString>& l, const TString& p) { + if (stats.Bytes) SetCounter(GetCounterName("TaskRunner", l, p + "Bytes"), stats.Bytes); + if (stats.Rows) SetCounter(GetCounterName("TaskRunner", l, p + "Rows"), stats.Rows); + if (stats.Chunks) SetCounter(GetCounterName("TaskRunner", l, p + "Chunks"), stats.Chunks); + if (stats.Splits) SetCounter(GetCounterName("TaskRunner", l, p + "Splits"), stats.Splits); + + if (stats.FirstMessageTs) SetCounter(GetCounterName("TaskRunner", l, p + "FirstMessageMs"), stats.FirstMessageTs.MilliSeconds()); + if (stats.PauseMessageTs) SetCounter(GetCounterName("TaskRunner", l, p + "PauseMessageMs"), stats.PauseMessageTs.MilliSeconds()); + if (stats.ResumeMessageTs) SetCounter(GetCounterName("TaskRunner", l, p + "ResumeMessageMs"), stats.ResumeMessageTs.MilliSeconds()); + if (stats.LastMessageTs) SetCounter(GetCounterName("TaskRunner", l, p + "LastMessageMs"), stats.LastMessageTs.MilliSeconds()); + + if (stats.WaitTime) SetCounter(GetCounterName("TaskRunner", l, p + "WaitTimeUs"), stats.WaitTime.MicroSeconds()); + if (stats.WaitPeriods) SetCounter(GetCounterName("TaskRunner", l, p + "WaitPeriods"), stats.WaitPeriods); + + auto activeTime = stats.LastMessageTs - stats.FirstMessageTs; + if (activeTime) { + SetCounter(GetCounterName("TaskRunner", l, p + "ActiveTimeUs"), activeTime.MicroSeconds()); + } + } + + void AddInputChannelStats( + const NDq::TDqInputChannelStats& pushStats, + const NDq::TDqInputStats& popStats, + ui64 taskId, ui32 stageId) + { + std::map<TString, TString> labels = { + {"Task", ToString(taskId)}, + {"Stage", ToString(stageId)}, + {"InputChannel", ToString(pushStats.ChannelId)}, + {"SrcStageId", ToString(pushStats.SrcStageId)} + }; + AddAsyncStats(pushStats, labels, "Push"); + AddAsyncStats(popStats, labels, "Pop"); + if (pushStats.DeserializationTime) SetCounter(GetCounterName("TaskRunner", labels, "DeserializationTime"), pushStats.DeserializationTime.MicroSeconds()); + if (pushStats.MaxMemoryUsage) SetCounter(GetCounterName("TaskRunner", labels, "MaxMemoryUsage"), pushStats.MaxMemoryUsage); + if (pushStats.RowsInMemory) SetCounter(GetCounterName("TaskRunner", labels, "RowsInMemory"), pushStats.RowsInMemory); + } + + void AddSourceStats( + const NDq::TDqAsyncInputBufferStats& pushStats, + const NDq::TDqInputStats& popStats, + ui64 taskId, ui32 stageId) + { + std::map<TString, TString> labels = { + {"Task", ToString(taskId)}, + {"Stage", ToString(stageId)}, + {"SourceIndex", ToString(pushStats.InputIndex)} + }; + AddAsyncStats(pushStats, labels, "Push"); + AddAsyncStats(popStats, labels, "Pop"); + if (pushStats.MaxMemoryUsage) SetCounter(GetCounterName("TaskRunner", labels, "MaxMemoryUsage"), pushStats.MaxMemoryUsage); + if (pushStats.RowsInMemory) SetCounter(GetCounterName("TaskRunner", labels, "RowsInMemory"), pushStats.RowsInMemory); + } + + void AddOutputChannelStats( + const NDq::TDqOutputStats& pushStats, + const NDq::TDqOutputChannelStats& popStats, + ui64 taskId, ui32 stageId) + { + std::map<TString, TString> labels = { + {"Task", ToString(taskId)}, + {"Stage", ToString(stageId)}, + {"OutputChannel", ToString(popStats.ChannelId)}, + {"DstStageId", ToString(popStats.DstStageId)} + }; + AddAsyncStats(pushStats, labels, "Push"); + AddAsyncStats(popStats, labels, "Pop"); + if (popStats.MaxMemoryUsage) SetCounter(GetCounterName("TaskRunner", labels, "MaxMemoryUsage"), popStats.MaxMemoryUsage); + if (popStats.MaxRowsInMemory) SetCounter(GetCounterName("TaskRunner", labels, "MaxRowsInMemory"), popStats.MaxRowsInMemory); + if (popStats.SerializationTime) SetCounter(GetCounterName("TaskRunner", labels, "SerializationTime"), popStats.SerializationTime.MicroSeconds()); + if (popStats.SpilledBlobs) SetCounter(GetCounterName("TaskRunner", labels, "SpilledBlobs"), popStats.SpilledBlobs); + if (popStats.SpilledBytes) SetCounter(GetCounterName("TaskRunner", labels, "SpilledBytes"), popStats.SpilledBytes); + if (popStats.SpilledRows) SetCounter(GetCounterName("TaskRunner", labels, "SpilledRows"), popStats.SpilledRows); + } + + void AddTaskRunnerStats( + const NDq::TDqTaskRunnerStats& stats, + ui64 taskId, ui32 stageId) + { + std::map<TString, TString> labels = { + {"Task", ToString(taskId)}, + {"Stage", ToString(stageId)} + }; + if (stats.ComputeCpuTime) SetCounter(GetCounterName("TaskRunner", labels, "ComputeCpuTime"), stats.ComputeCpuTime.MicroSeconds()); + if (stats.BuildCpuTime) SetCounter(GetCounterName("TaskRunner", labels, "BuildCpuTime"), stats.BuildCpuTime.MicroSeconds()); + if (stats.WaitTime) SetCounter(GetCounterName("TaskRunner", labels, "WaitTime"), stats.WaitTime.MicroSeconds()); + if (stats.WaitOutputTime) SetCounter(GetCounterName("TaskRunner", labels, "WaitOutputTime"), stats.WaitOutputTime.MicroSeconds()); + } + + void TakeDeltaCounters(TTaskCounters& CurrentMetrics, TTaskCounters& PrevMetrics) { + for (const auto& [name, entry] : CurrentMetrics.Counters) { + auto it = PrevMetrics.Counters.find(name); + if (it == PrevMetrics.Counters.end()) { + AddCounter(name, entry.Sum); + PrevMetrics.AddCounter(name, entry.Sum); + } else { + auto delta = entry.Sum - it->second.Sum; + if (delta) { + it->second.Sum = entry.Sum; + AddCounter(name, delta); + } + } + } + } +}; + +TTaskCounters AggregateQueryStatsByStage(TTaskCounters& queryStat, const THashMap<ui64, ui64>& task2Stage, bool collectFull = true); + +} // namespace NYql diff --git a/ydb/library/yql/providers/dq/counters/ya.make b/ydb/library/yql/providers/dq/counters/ya.make index acd095ffc5..e4cf363e9e 100644 --- a/ydb/library/yql/providers/dq/counters/ya.make +++ b/ydb/library/yql/providers/dq/counters/ya.make @@ -6,6 +6,7 @@ PEERDIR( SRCS( counters.cpp + task_counters.cpp ) diff --git a/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp b/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp index 3c7904f4dc..ff9be8a613 100644 --- a/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp +++ b/ydb/library/yql/providers/dq/provider/exec/yql_dq_exectransformer.cpp @@ -137,8 +137,7 @@ public: NDq::TDqTaskRunnerSettings settings; settings.OptLLVM = "OFF"; // Don't use LLVM for local execution settings.SecureParams = secureParams; - settings.CollectBasicStats = true; - settings.CollectProfileStats = true; + settings.StatsMode = NDqProto::DQ_STATS_MODE_BASIC; auto runner = NDq::MakeDqTaskRunner(executionContext, settings, {}); auto runnerSettings = NDq::TDqTaskSettings(&task); @@ -165,11 +164,11 @@ public: break; } if (!fillSettings.Discard) { - if (fillSettings.AllResultsBytesLimit && runner->GetOutputChannel(0)->GetStats()->Bytes >= *fillSettings.AllResultsBytesLimit) { + if (fillSettings.AllResultsBytesLimit && runner->GetOutputChannel(0)->GetPopStats().Bytes >= *fillSettings.AllResultsBytesLimit) { result.Truncated = true; break; } - if (fillSettings.RowsLimitPerWrite && runner->GetOutputChannel(0)->GetStats()->RowsOut >= *fillSettings.RowsLimitPerWrite) { + if (fillSettings.RowsLimitPerWrite && runner->GetOutputChannel(0)->GetPopStats().Rows >= *fillSettings.RowsLimitPerWrite) { result.Truncated = true; break; } diff --git a/ydb/library/yql/providers/dq/runtime/task_command_executor.cpp b/ydb/library/yql/providers/dq/runtime/task_command_executor.cpp index 0c37dbcbe1..bf68169309 100644 --- a/ydb/library/yql/providers/dq/runtime/task_command_executor.cpp +++ b/ydb/library/yql/providers/dq/runtime/task_command_executor.cpp @@ -1,7 +1,7 @@ #include "task_command_executor.h" #include <ydb/library/yql/providers/dq/task_runner/tasks_runner_proxy.h> -#include <ydb/library/yql/providers/dq/counters/counters.h> +#include <ydb/library/yql/providers/dq/counters/task_counters.h> #include <ydb/library/yql/providers/dq/common/yql_dq_settings.h> #include <ydb/library/yql/providers/dq/api/protos/dqs.pb.h> #include <ydb/library/yql/providers/dq/api/protos/task_command_executor.pb.h> @@ -42,47 +42,60 @@ namespace NTaskRunnerProxy { static const int CurrentProtocolVersion = 5; // Calls for Sinks template<typename T> -void ToProto(T* s1, const NDq::TDqInputChannelStats* ss) +void ToProto(T& proto, const NDq::TDqAsyncStats& stats) { - s1->SetChannelId(ss->ChannelId); - s1->SetChunks(ss->Chunks); - s1->SetBytes(ss->Bytes); - s1->SetRowsIn(ss->RowsIn); - s1->SetRowsOut(ss->RowsOut); - s1->SetMaxMemoryUsage(ss->MaxMemoryUsage); - s1->SetDeserializationTimeUs(ss->DeserializationTime.MicroSeconds()); + proto.SetBytes(stats.Bytes); + proto.SetRows(stats.Rows); + proto.SetChunks(stats.Chunks); + proto.SetSplits(stats.Splits); + + proto.SetFirstMessageMs(stats.FirstMessageTs.MilliSeconds()); + proto.SetPauseMessageMs(stats.PauseMessageTs.MilliSeconds()); + proto.SetResumeMessageMs(stats.ResumeMessageTs.MilliSeconds()); + proto.SetLastMessageMs(stats.LastMessageTs.MilliSeconds()); + proto.SetWaitTimeUs(stats.WaitTime.MicroSeconds()); + proto.SetWaitPeriods(stats.WaitPeriods); } template<typename T> -void ToProto(T* s1, const NDq::TDqAsyncInputBufferStats* ss) +void ToProto(T& proto, const NDq::IDqInputChannel& channel) { - s1->SetChunks(ss->Chunks); - s1->SetBytes(ss->Bytes); - s1->SetRowsIn(ss->RowsIn); - s1->SetRowsOut(ss->RowsOut); - s1->SetMaxMemoryUsage(ss->MaxMemoryUsage); - s1->SetInputIndex(ss->InputIndex); + proto.SetChannelId(channel.GetPushStats().ChannelId); + proto.SetSrcStageId(channel.GetPushStats().SrcStageId); + ToProto(*proto.MutablePush(), channel.GetPushStats()); + ToProto(*proto.MutablePop(), channel.GetPopStats()); + proto.SetMaxMemoryUsage(channel.GetPushStats().MaxMemoryUsage); + proto.SetDeserializationTimeUs(channel.GetPushStats().DeserializationTime.MicroSeconds()); } template<typename T> -void ToProto(T* s1, const NDq::TDqOutputChannelStats* ss) +void ToProto(T& proto, const NDq::IDqAsyncInputBuffer& asyncBuffer) { - s1->SetChannelId(ss->ChannelId); - s1->SetChunks(ss->Chunks); - s1->SetBytes(ss->Bytes); - s1->SetRowsIn(ss->RowsIn); - s1->SetRowsOut(ss->RowsOut); - s1->SetMaxMemoryUsage(ss->MaxMemoryUsage); + proto.SetInputIndex(asyncBuffer.GetPushStats().InputIndex); + proto.SetIngressName(asyncBuffer.GetPushStats().Type); + ToProto(*proto.MutablePush(), asyncBuffer.GetPushStats()); + ToProto(*proto.MutablePop(), asyncBuffer.GetPopStats()); + proto.SetMaxMemoryUsage(asyncBuffer.GetPushStats().MaxMemoryUsage); } template<typename T> -void ToProto(T* s1, const NDq::TDqAsyncOutputBufferStats* ss) +void ToProto(T& proto, const NDq::IDqOutputChannel& channel) { - s1->SetChunks(ss->Chunks); - s1->SetBytes(ss->Bytes); - s1->SetRowsIn(ss->RowsIn); - s1->SetRowsOut(ss->RowsOut); - s1->SetMaxMemoryUsage(ss->MaxMemoryUsage); + proto.SetChannelId(channel.GetPopStats().ChannelId); + proto.SetDstStageId(channel.GetPopStats().DstStageId); + ToProto(*proto.MutablePush(), channel.GetPushStats()); + ToProto(*proto.MutablePop(), channel.GetPopStats()); + proto.SetMaxMemoryUsage(channel.GetPopStats().MaxMemoryUsage); +} + +template<typename T> +void ToProto(T& proto, const NDq::IDqAsyncOutputBuffer& asyncBuffer) +{ + proto.SetOutputIndex(asyncBuffer.GetPopStats().OutputIndex); + proto.SetEgressName(asyncBuffer.GetPopStats().Type); + ToProto(*proto.MutablePush(), asyncBuffer.GetPushStats()); + ToProto(*proto.MutablePop(), asyncBuffer.GetPopStats()); + proto.SetMaxMemoryUsage(asyncBuffer.GetPopStats().MaxMemoryUsage); } class TTaskCommandExecutor { @@ -126,83 +139,52 @@ public: QueryStat.AddTaskRunnerStats( *Runner->GetStats(), - CurrentStats, - Runner->GetTaskId()); - for (const auto& [inputId, _]: CurrentInputChannelsStats) { - UpdateInputChannelStats(inputId); + Runner->GetTaskId(), StageId); + + auto stats = Runner->GetStats(); + for (const auto& [stageId, stageChannels] : stats->OutputChannels) { + for (const auto& [id, channel] : stageChannels) { + UpdateOutputChannelStats(channel); + } } - for (const auto& [outputId, _]: CurrentOutputChannelsStats) { - UpdateOutputChannelStats(outputId); + for (const auto& [stageId, stageChannels] : stats->InputChannels) { + for (const auto& [id, channel] : stageChannels) { + UpdateInputChannelStats(channel); + } } } - void UpdateOutputChannelStats(ui64 channelId) - { - if (!Runner) { - return; - } - auto s = Runner->GetStats(); - auto maybeOutputChannelStats = s->OutputChannels.find(channelId); - if (maybeOutputChannelStats == s->OutputChannels.end() || !maybeOutputChannelStats->second) { - return; - } - auto maybeOutputChannelOldStats = CurrentOutputChannelsStats.find(channelId); - if (maybeOutputChannelOldStats == CurrentOutputChannelsStats.end()) { - maybeOutputChannelOldStats = CurrentOutputChannelsStats.emplace_hint( - maybeOutputChannelOldStats, channelId, NDq::TDqOutputChannelStats(channelId)); - } + void UpdateOutputChannelStats(NDq::IDqOutputChannel::TPtr channel) { QueryStat.AddOutputChannelStats( - *maybeOutputChannelStats->second, - maybeOutputChannelOldStats->second, - Runner->GetTaskId(), channelId); + channel->GetPushStats(), + channel->GetPopStats(), + Runner->GetTaskId(), StageId); } - void UpdateInputChannelStats(ui64 channelId) + void UpdateInputChannelStats(NDq::IDqInputChannel::TPtr channel) { - if (!Runner) { - return; - } - auto s = Runner->GetStats(); - auto maybeInputChannelStats = s->InputChannels.find(channelId); - if (maybeInputChannelStats == s->InputChannels.end() || !maybeInputChannelStats->second) { - return; - } - auto maybeInputChannelOldStats = CurrentInputChannelsStats.find(channelId); - if (maybeInputChannelOldStats == CurrentInputChannelsStats.end()) { - maybeInputChannelOldStats = CurrentInputChannelsStats.emplace_hint( - maybeInputChannelOldStats, channelId, NDq::TDqInputChannelStats(channelId)); - } QueryStat.AddInputChannelStats( - *maybeInputChannelStats->second, - maybeInputChannelOldStats->second, - Runner->GetTaskId(), channelId); + channel->GetPushStats(), + channel->GetPopStats(), + Runner->GetTaskId(), StageId); } - void UpdateSourceStats(ui64 inputIndex) + void UpdateSourceStats(NDq::IDqAsyncInputBuffer::TPtr source) { - if (!Runner) { - return; - } - auto s = Runner->GetStats(); - auto maybeSourceStats = s->Sources.find(inputIndex); - if (maybeSourceStats == s->Sources.end() || !maybeSourceStats->second) { - return; - } - auto maybeSourceOldStats = CurrentSourcesStats.find(inputIndex); - if (maybeSourceOldStats == CurrentSourcesStats.end()) { - maybeSourceOldStats = CurrentSourcesStats.emplace_hint( - maybeSourceOldStats, inputIndex, NDq::TDqAsyncInputBufferStats(inputIndex)); - } QueryStat.AddSourceStats( - *maybeSourceStats->second, - maybeSourceOldStats->second, - Runner->GetTaskId(), inputIndex); + source->GetPushStats(), + source->GetPopStats(), + Runner->GetTaskId(), StageId); } template<typename T> void UpdateStats(T& t) { UpdateStats(); - QueryStat.FlushCounters(t); + + TTaskCounters deltaStat; + deltaStat.TakeDeltaCounters(QueryStat, PrevStat); + deltaStat.FlushCounters(t); + QueryStat.ClearCounters(); auto currentRusage = TRusage::Get(); TRusage delta; @@ -268,14 +250,14 @@ public: s->SetComputeCpuTimeUs(stats->ComputeCpuTime.MicroSeconds()); // All run statuses metrics - s->SetPendingInputTimeUs(stats->RunStatusTimeMetrics[NDq::ERunStatus::PendingInput].MicroSeconds()); - s->SetPendingOutputTimeUs(stats->RunStatusTimeMetrics[NDq::ERunStatus::PendingOutput].MicroSeconds()); - s->SetFinishTimeUs(stats->RunStatusTimeMetrics[NDq::ERunStatus::Finished].MicroSeconds()); - static_assert(NDq::TRunStatusTimeMetrics::StatusesCount == 3); // Add all statuses here + // s->SetPendingInputTimeUs(stats->RunStatusTimeMetrics[NDq::ERunStatus::PendingInput].MicroSeconds()); + // s->SetPendingOutputTimeUs(stats->RunStatusTimeMetrics[NDq::ERunStatus::PendingOutput].MicroSeconds()); + // s->SetFinishTimeUs(stats->RunStatusTimeMetrics[NDq::ERunStatus::Finished].MicroSeconds()); + // static_assert(NDq::TRunStatusTimeMetrics::StatusesCount == 3); // Add all statuses here - //s->SetTotalTime(stats->TotalTime.MilliSeconds()); - s->SetWaitTimeUs(stats->WaitTime.MicroSeconds()); - s->SetWaitOutputTimeUs(stats->WaitOutputTime.MicroSeconds()); + // s->SetTotalTime(stats->TotalTime.MilliSeconds()); + // s->SetWaitTimeUs(stats->WaitTime.MicroSeconds()); + // s->SetWaitOutputTimeUs(stats->WaitOutputTime.MicroSeconds()); //s->SetMkqlTotalNodes(stats->MkqlTotalNodes); //s->SetMkqlCodegenFunctions(stats->MkqlCodegenFunctions); @@ -285,17 +267,22 @@ public: //s->SetCodeGenFinalizeTime(stats->CodeGenFinalizeTime); //s->SetCodeGenModulePassTime(stats->CodeGenModulePassTime); - for (const auto& [id, ss] : stats->OutputChannels) { - ToProto(s->AddOutputChannels(), ss); + for (const auto& [stageId, stageChannels] : stats->OutputChannels) { + for (const auto& [id, channel] : stageChannels) { + ToProto(*s->AddOutputChannels(), *channel); + } } - for (const auto& [id, ss] : stats->InputChannels) { - ToProto(s->AddInputChannels(), ss); + for (const auto& [stageId, stageChannels] : stats->InputChannels) { + for (const auto& [id, channel] : stageChannels) { + ToProto(*s->AddInputChannels(), *channel); + } } - for (const auto& [id, ss] : stats->Sources) { - ToProto(s->AddSources(), ss); + for (const auto& [id, source] : stats->Sources) { + ToProto(*s->AddSources(), *source); } + return response; } @@ -355,7 +342,7 @@ public: auto guard = Runner->BindAllocator(0); // Explicitly reset memory limit channel->Push(std::move(data)); - UpdateInputChannelStats(channelId); + UpdateInputChannelStats(channel); break; } case NDqProto::TCommandHeader::PUSH_SOURCE: { @@ -400,28 +387,31 @@ public: } source->Push(std::move(buffer), request.GetSpace()); - UpdateSourceStats(channelId); + UpdateSourceStats(source); break; } case NDqProto::TCommandHeader::FINISH: { Y_ENSURE(header.GetVersion() <= CurrentProtocolVersion); Y_ENSURE(taskId == Runner->GetTaskId()); - Runner->GetInputChannel(channelId)->Finish(); - UpdateInputChannelStats(channelId); + auto channel = Runner->GetInputChannel(channelId); + channel->Finish(); + UpdateInputChannelStats(channel); break; } case NDqProto::TCommandHeader::FINISH_OUTPUT: { Y_ENSURE(header.GetVersion() <= CurrentProtocolVersion); Y_ENSURE(taskId == Runner->GetTaskId()); - Runner->GetOutputChannel(channelId)->Finish(); - UpdateOutputChannelStats(channelId); + auto channel = Runner->GetOutputChannel(channelId); + channel->Finish(); + UpdateOutputChannelStats(channel); break; } case NDqProto::TCommandHeader::FINISH_SOURCE: { Y_ENSURE(header.GetVersion() <= CurrentProtocolVersion); Y_ENSURE(taskId == Runner->GetTaskId()); - Runner->GetSource(channelId)->Finish(); - UpdateSourceStats(channelId); + auto source = Runner->GetSource(channelId); + source->Finish(); + UpdateSourceStats(source); break; } case NDqProto::TCommandHeader::DROP_OUTPUT: { @@ -437,8 +427,9 @@ public: case NDqProto::TCommandHeader::TERMINATE_OUTPUT: { Y_ENSURE(header.GetVersion() <= CurrentProtocolVersion); Y_ENSURE(taskId == Runner->GetTaskId()); - Runner->GetOutputChannel(channelId)->Terminate(); - UpdateOutputChannelStats(channelId); + auto channel = Runner->GetOutputChannel(channelId); + channel->Terminate(); + UpdateOutputChannelStats(channel); break; } case NDqProto::TCommandHeader::GET_STORED_BYTES: { @@ -475,8 +466,13 @@ public: response.SetResult(channel->Pop(batch)); YQL_ENSURE(!batch.IsOOB()); *response.MutableData() = std::move(batch.Proto); - UpdateOutputChannelStats(channelId); - QueryStat.FlushCounters(response); + UpdateOutputChannelStats(channel); + + TTaskCounters deltaStat; + deltaStat.TakeDeltaCounters(QueryStat, PrevStat); + deltaStat.FlushCounters(response); + QueryStat.ClearCounters(); + response.MutableStats()->PackFrom(GetStats(taskId)); response.Save(&output); @@ -594,27 +590,24 @@ public: case NDqProto::TCommandHeader::GET_STATS_INPUT: { Y_ENSURE(header.GetVersion() >= 3); Y_ENSURE(taskId == Runner->GetTaskId()); - auto ss = Runner->GetInputChannel(channelId)->GetStats(); NDqProto::TGetStatsInputResponse response; - ToProto(response.MutableStats(), ss); + ToProto(*response.MutableStats(), *Runner->GetInputChannel(channelId)); response.Save(&output); break; } case NDqProto::TCommandHeader::GET_STATS_SOURCE: { Y_ENSURE(header.GetVersion() >= 4); Y_ENSURE(taskId == Runner->GetTaskId()); - auto ss = Runner->GetSource(channelId)->GetStats(); NDqProto::TGetStatsSourceResponse response; - ToProto(response.MutableStats(), ss); + ToProto(*response.MutableStats(), *Runner->GetSource(channelId)); response.Save(&output); break; } case NDqProto::TCommandHeader::GET_STATS_OUTPUT: { Y_ENSURE(header.GetVersion() >= 3); Y_ENSURE(taskId == Runner->GetTaskId()); - auto ss = Runner->GetOutputChannel(channelId)->GetStats(); NDqProto::TGetStatsOutputResponse response; - ToProto(response.MutableStats(), ss); + ToProto(*response.MutableStats(), *Runner->GetOutputChannel(channelId)); response.Save(&output); break; } @@ -677,9 +670,8 @@ public: Y_ENSURE(header.GetVersion() <= CurrentProtocolVersion); Y_ENSURE(taskId == Runner->GetTaskId()); - auto* stats = Runner->GetSink(channelId)->GetStats(); NDqProto::TSinkStatsResponse response; - ToProto(response.MutableStats(), stats); + ToProto(*response.MutableStats(), *Runner->GetSink(channelId)); response.Save(&output); break; } @@ -705,6 +697,7 @@ public: template<typename T> void Prepare(const NDq::TDqTaskSettings& task, const T& taskMeta, TPipedOutput& output) { + this->StageId = task.GetStageId(); NYql::NDqProto::TPrepareResponse result; result.SetResult(true); // COMPAT(aozeritsky) YQL-14268 @@ -754,8 +747,7 @@ public: QueryStat.Measure<void>("MakeDqTaskRunner", [&]() { NDq::TDqTaskRunnerSettings settings; - settings.CollectBasicStats = true; - settings.CollectProfileStats = true; + settings.StatsMode = NYql::NDqProto::DQ_STATS_MODE_PROFILE; settings.TerminateOnError = TerminateOnError; for (const auto& x: taskMeta.GetSecureParams()) { settings.SecureParams[x.first] = x.second; @@ -787,7 +779,9 @@ public: NKikimr::NMiniKQL::IStatsRegistry* JobStats; bool TerminateOnError; TIntrusivePtr<NDq::IDqTaskRunner> Runner; - TCounters QueryStat; + ui32 StageId = 0; + TTaskCounters QueryStat; + TTaskCounters PrevStat; TDqConfiguration::TPtr DqConfiguration = MakeIntrusive<TDqConfiguration>(); TIntrusivePtr<NKikimr::NMiniKQL::IMutableFunctionRegistry> FunctionRegistry; NDq::TDqTaskRunnerContext Ctx; @@ -795,11 +789,6 @@ public: TRusage Rusage; - NDq::TDqTaskRunnerStats CurrentStats; - std::unordered_map<ui64, NDq::TDqInputChannelStats> CurrentInputChannelsStats; - std::unordered_map<ui64, NDq::TDqAsyncInputBufferStats> CurrentSourcesStats; - std::unordered_map<ui64, NDq::TDqOutputChannelStats> CurrentOutputChannelsStats; - i64 LastCommand = -1; i64 LastVersion = -1; ui64 LastTaskId = -1; diff --git a/ydb/library/yql/providers/dq/service/grpc_service.cpp b/ydb/library/yql/providers/dq/service/grpc_service.cpp index 410c354a9e..f70cca9486 100644 --- a/ydb/library/yql/providers/dq/service/grpc_service.cpp +++ b/ydb/library/yql/providers/dq/service/grpc_service.cpp @@ -11,7 +11,7 @@ #include <ydb/library/yql/providers/dq/actors/task_controller.h> #include <ydb/library/yql/providers/dq/actors/graph_execution_events_actor.h> -#include <ydb/library/yql/providers/dq/counters/counters.h> +#include <ydb/library/yql/providers/dq/counters/task_counters.h> #include <ydb/library/yql/providers/dq/common/yql_dq_settings.h> #include <ydb/library/yql/providers/dq/common/yql_dq_common.h> @@ -258,7 +258,7 @@ namespace NYql::NDqs { int MaxRetries = 10; TDuration RetryBackoff = TDuration::MilliSeconds(1000); - NYql::TCounters QueryStat; + NYql::TTaskCounters QueryStat; THashMap<ui64, ui64> Task2Stage; void RestoreRequest() { diff --git a/ydb/library/yql/providers/dq/task_runner/tasks_runner_local.cpp b/ydb/library/yql/providers/dq/task_runner/tasks_runner_local.cpp index 52e4ea5ee8..a842a362ed 100644 --- a/ydb/library/yql/providers/dq/task_runner/tasks_runner_local.cpp +++ b/ydb/library/yql/providers/dq/task_runner/tasks_runner_local.cpp @@ -1,7 +1,7 @@ #include "tasks_runner_local.h" #include "file_cache.h" -#include <ydb/library/yql/providers/dq/counters/counters.h> +#include <ydb/library/yql/providers/dq/counters/task_counters.h> #include <ydb/library/yql/dq/runtime/dq_input_channel.h> #include <ydb/library/yql/dq/runtime/dq_output_channel.h> #include <ydb/library/yql/minikql/invoke_builtins/mkql_builtins.h> @@ -26,25 +26,14 @@ namespace NYql::NTaskRunnerProxy { using namespace NKikimr; using namespace NDq; -#define ADD_COUNTER(name) \ - if (stats->name) { \ - QueryStat.AddCounter(QueryStat.GetCounterName("TaskRunner", labels, #name), stats->name); \ - } - -#define ADD_TIME_COUNTER(name) \ - if (stats->name) { \ - QueryStat.AddTimeCounter(QueryStat.GetCounterName("TaskRunner", labels, #name), stats->name); \ - } - class TLocalInputChannel: public IInputChannel { public: - TLocalInputChannel(const IDqInputChannel::TPtr& channel, ui64 taskId, ui64 channelId, TCounters* queryStat) + TLocalInputChannel(const IDqInputChannel::TPtr& channel, ui64 taskId, ui32 stageId, TTaskCounters* queryStat) : TaskId(taskId) - , ChannelId(channelId) + , StageId(stageId) , Channel(channel) , QueryStat(*queryStat) - , Stats(channelId) - { } + {} void Push(TDqSerializedBatch&& data) override { Channel->Push(std::move(data)); @@ -62,25 +51,23 @@ public: private: void UpdateInputChannelStats() { - QueryStat.AddInputChannelStats(*Channel->GetStats(), Stats, TaskId, ChannelId); + QueryStat.AddInputChannelStats(Channel->GetPushStats(), Channel->GetPopStats(), TaskId, StageId); } ui64 TaskId; - ui64 ChannelId; + ui32 StageId; IDqInputChannel::TPtr Channel; - TCounters& QueryStat; - TDqInputChannelStats Stats; + TTaskCounters& QueryStat; }; class TLocalOutputChannel : public IOutputChannel { public: - TLocalOutputChannel(const IDqOutputChannel::TPtr channel, ui64 taskId, ui64 channelId, TCounters* queryStat) + TLocalOutputChannel(const IDqOutputChannel::TPtr channel, ui64 taskId, ui32 stageId, TTaskCounters* queryStat) : TaskId(taskId) - , ChannelId(channelId) + , StageId(stageId) , Channel(channel) , QueryStat(*queryStat) - , Stats(channelId) - { } + {} [[nodiscard]] NDqProto::TPopResponse Pop(TDqSerializedBatch& data) override { @@ -100,14 +87,13 @@ public: private: void UpdateOutputChannelStats() { - QueryStat.AddOutputChannelStats(*Channel->GetStats(), Stats, TaskId, ChannelId); + QueryStat.AddOutputChannelStats(Channel->GetPushStats(), Channel->GetPopStats(), TaskId, StageId); } ui64 TaskId; - ui64 ChannelId; + ui32 StageId; IDqOutputChannel::TPtr Channel; - TCounters& QueryStat; - TDqOutputChannelStats Stats; + TTaskCounters& QueryStat; }; class TLocalTaskRunner: public ITaskRunner { @@ -147,11 +133,11 @@ public: } IInputChannel::TPtr GetInputChannel(ui64 channelId) override { - return new TLocalInputChannel(Runner->GetInputChannel(channelId), Task.GetId(), channelId, &QueryStat); + return new TLocalInputChannel(Runner->GetInputChannel(channelId), Task.GetId(), Task.GetStageId(), &QueryStat); } IOutputChannel::TPtr GetOutputChannel(ui64 channelId) override { - return new TLocalOutputChannel(Runner->GetOutputChannel(channelId), Task.GetId(), channelId, &QueryStat); + return new TLocalOutputChannel(Runner->GetOutputChannel(channelId), Task.GetId(), Task.GetStageId(), &QueryStat); } IDqAsyncInputBuffer::TPtr GetSource(ui64 index) override { @@ -196,13 +182,12 @@ public: private: void UpdateStats() { - QueryStat.AddTaskRunnerStats(*Runner->GetStats(), Stats, Task.GetId()); + QueryStat.AddTaskRunnerStats(*Runner->GetStats(), Task.GetId(), Task.GetStageId()); } NDq::TDqTaskSettings Task; TIntrusivePtr<IDqTaskRunner> Runner; - TCounters QueryStat; - TDqTaskRunnerStats Stats; + TTaskCounters QueryStat; }; /*______________________________________________________________________________________________*/ @@ -263,8 +248,7 @@ public: Y_UNUSED(traceId); NDq::TDqTaskRunnerSettings settings; settings.TerminateOnError = TerminateOnError; - settings.CollectBasicStats = true; - settings.CollectProfileStats = false; + settings.StatsMode = NDqProto::DQ_STATS_MODE_BASIC; Yql::DqsProto::TTaskMeta taskMeta; task.GetMeta().UnpackTo(&taskMeta); @@ -275,14 +259,13 @@ public: } if ("TaskRunnerStats" == s.GetName()) { if ("Disable" == s.GetValue()) { - settings.CollectBasicStats = false; - settings.CollectProfileStats = false; - } else if ("Profile" == s.GetValue()) { - settings.CollectBasicStats = true; - settings.CollectProfileStats = true; + settings.StatsMode = NDqProto::DQ_STATS_MODE_NONE; } else if ("Basic" == s.GetValue()) { - settings.CollectBasicStats = true; - settings.CollectProfileStats = false; + settings.StatsMode = NDqProto::DQ_STATS_MODE_BASIC; + } else if ("Full" == s.GetValue()) { + settings.StatsMode = NDqProto::DQ_STATS_MODE_FULL; + } else if ("Profile" == s.GetValue()) { + settings.StatsMode = NDqProto::DQ_STATS_MODE_PROFILE; } } } diff --git a/ydb/library/yql/providers/dq/task_runner/tasks_runner_pipe.cpp b/ydb/library/yql/providers/dq/task_runner/tasks_runner_pipe.cpp index 1910a860d0..b9a6491362 100644 --- a/ydb/library/yql/providers/dq/task_runner/tasks_runner_pipe.cpp +++ b/ydb/library/yql/providers/dq/task_runner/tasks_runner_pipe.cpp @@ -460,6 +460,21 @@ private: TVector<TString> ArgsElems; }; +void LoadFromProto(TDqAsyncStats& stats, const NYql::NDqProto::TDqAsyncBufferStats& f) +{ + stats.Bytes = f.GetBytes(); + stats.Rows = f.GetRows(); + stats.Chunks = f.GetChunks(); + stats.Splits = f.GetSplits(); + + stats.FirstMessageTs = TInstant::MilliSeconds(f.GetFirstMessageMs()); + stats.PauseMessageTs = TInstant::MilliSeconds(f.GetPauseMessageMs()); + stats.ResumeMessageTs = TInstant::MilliSeconds(f.GetResumeMessageMs()); + stats.LastMessageTs = TInstant::MilliSeconds(f.GetLastMessageMs()); + stats.WaitTime = TDuration::MicroSeconds(f.GetWaitTimeUs()); + stats.WaitPeriods = f.GetWaitPeriods(); +} + /*______________________________________________________________________________________________*/ @@ -535,20 +550,33 @@ private: class TDqInputChannel: public IDqInputChannel { public: - TDqInputChannel(const IInputChannel::TPtr& channel, ui64 taskId, ui64 channelId, IPipeTaskRunner* taskRunner) + TDqInputChannelStats PushStats; + TDqInputStats PopStats; + + TDqInputChannel(const IInputChannel::TPtr& channel, ui64 taskId, ui64 channelId, ui32 srcStageId, IPipeTaskRunner* taskRunner) : Delegate(channel) , TaskId(taskId) , ChannelId(channelId) - , Stats(ChannelId) , TaskRunner(taskRunner) , Input(TaskRunner->GetInput()) , Output(TaskRunner->GetOutput()) - { } + { + PushStats.ChannelId = channelId; + PushStats.SrcStageId = srcStageId; + } ui64 GetChannelId() const override { return ChannelId; } + const TDqInputChannelStats& GetPushStats() const override { + return PushStats; + } + + const TDqInputStats& GetPopStats() const override { + return PopStats; + } + i64 GetFreeSpace() const override { try { return Delegate->GetFreeSpace(); @@ -605,25 +633,6 @@ public: ythrow yexception() << "unimplemented"; } - const TDqInputChannelStats* GetStats() const override { - try { - NDqProto::TCommandHeader header; - header.SetVersion(3); - header.SetCommand(NDqProto::TCommandHeader::GET_STATS_INPUT); - header.SetTaskId(TaskId); - header.SetChannelId(ChannelId); - header.Save(&Output); - - NDqProto::TGetStatsInputResponse response; - response.Load(&Input); - - Stats.FromProto(response.GetStats()); - return &Stats; - } catch (...) { - TaskRunner->RaiseException(); - } - } - NKikimr::NMiniKQL::TType* GetInputType() const override { ythrow yexception() << "unimplemented"; } @@ -640,11 +649,20 @@ public: return false; } + template<typename T> + void FromProto(const T& f) + { + YQL_ENSURE(PushStats.ChannelId == f.GetChannelId()); + LoadFromProto(PushStats, f.GetPush()); + PushStats.MaxMemoryUsage = f.GetMaxMemoryUsage(); + PushStats.DeserializationTime = TDuration::MicroSeconds(f.GetDeserializationTimeUs()); + LoadFromProto(PopStats, f.GetPop()); + } + private: IInputChannel::TPtr Delegate; ui64 TaskId; ui64 ChannelId; - mutable TDqInputChannelStats Stats; IPipeTaskRunner* TaskRunner; IInputStream& Input; @@ -653,21 +671,24 @@ private: class TDqSource: public IStringSource { public: + TDqAsyncInputBufferStats PushStats; + TDqInputStats PopStats; + TDqSource(ui64 taskId, ui64 inputIndex, IPipeTaskRunner* taskRunner) : TaskId(taskId) - , InputIndex(inputIndex) - , Stats(inputIndex) , TaskRunner(taskRunner) , Input(TaskRunner->GetInput()) , Output(TaskRunner->GetOutput()) - { } + { + PushStats.InputIndex = inputIndex; + } i64 GetFreeSpace() const override { NDqProto::TCommandHeader header; header.SetVersion(4); header.SetCommand(NDqProto::TCommandHeader::GET_FREE_SPACE_SOURCE); header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); + header.SetChannelId(PushStats.InputIndex); header.Save(&Output); NDqProto::TGetFreeSpaceResponse response; @@ -680,7 +701,7 @@ public: header.SetVersion(4); header.SetCommand(NDqProto::TCommandHeader::GET_STORED_BYTES_SOURCE); header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); + header.SetChannelId(PushStats.InputIndex); header.Save(&Output); NDqProto::TGetStoredBytesResponse response; @@ -689,6 +710,14 @@ public: return response.GetResult(); } + const TDqAsyncInputBufferStats& GetPushStats() const override { + return PushStats; + } + + const TDqInputStats& GetPopStats() const override { + return PopStats; + } + bool Empty() const override { ythrow yexception() << "unimplemented"; } @@ -705,7 +734,7 @@ public: header.SetVersion(4); header.SetCommand(NDqProto::TCommandHeader::PUSH_SOURCE); header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); + header.SetChannelId(PushStats.InputIndex); header.Save(&Output); data.Save(&Output); @@ -741,7 +770,7 @@ public: header.SetVersion(4); header.SetCommand(NDqProto::TCommandHeader::PUSH_SOURCE); header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); + header.SetChannelId(PushStats.InputIndex); header.Save(&Output); data.Save(&Output); @@ -758,7 +787,7 @@ public: header.SetVersion(4); header.SetCommand(NDqProto::TCommandHeader::FINISH_SOURCE); header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); + header.SetChannelId(PushStats.InputIndex); header.Save(&Output); } @@ -767,26 +796,7 @@ public: } ui64 GetInputIndex() const override { - return InputIndex; - } - - const TDqAsyncInputBufferStats* GetStats() const override { - try { - NDqProto::TCommandHeader header; - header.SetVersion(4); - header.SetCommand(NDqProto::TCommandHeader::GET_STATS_SOURCE); - header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); - header.Save(&Output); - - NDqProto::TGetStatsSourceResponse response; - response.Load(&Input); - - Stats.FromProto(response.GetStats()); - return &Stats; - } catch (...) { - TaskRunner->RaiseException(); - } + return PushStats.InputIndex; } NKikimr::NMiniKQL::TType* GetInputType() const override { @@ -798,7 +808,7 @@ public: header.SetVersion(4); header.SetCommand(NDqProto::TCommandHeader::GET_SOURCE_TYPE); header.SetTaskId(TaskId); - header.SetChannelId(InputIndex); + header.SetChannelId(PushStats.InputIndex); header.Save(&Output); NDqProto::TGetTypeResponse response; @@ -820,11 +830,18 @@ public: return false; } + template<typename T> + void FromProto(const T& f) + { + YQL_ENSURE(PushStats.InputIndex == f.GetInputIndex()); + LoadFromProto(PushStats, f.GetPush()); + PushStats.RowsInMemory = f.GetRowsInMemory(); + PushStats.MaxMemoryUsage = f.GetMaxMemoryUsage(); + LoadFromProto(PopStats, f.GetPop()); + } + private: ui64 TaskId; - ui64 InputIndex; - mutable TDqAsyncInputBufferStats Stats; - IPipeTaskRunner* TaskRunner; IInputStream& Input; IOutputStream& Output; @@ -887,26 +904,37 @@ private: class TDqOutputChannel: public IDqOutputChannel { public: - TDqOutputChannel(const IOutputChannel::TPtr& channel, ui64 taskId, ui64 channelId, IPipeTaskRunner* taskRunner) + TDqOutputStats PushStats; + TDqOutputChannelStats PopStats; + + TDqOutputChannel(const IOutputChannel::TPtr& channel, ui64 taskId, ui64 channelId, ui32 dstStageId, IPipeTaskRunner* taskRunner) : Delegate(channel) , TaskId(taskId) - , ChannelId(channelId) - , Stats(ChannelId) , TaskRunner(taskRunner) , Input(TaskRunner->GetInput()) , Output(TaskRunner->GetOutput()) { Y_UNUSED(Input); + PopStats.ChannelId = channelId; + PopStats.DstStageId = dstStageId; } ui64 GetChannelId() const override { - return ChannelId; + return PopStats.ChannelId; } ui64 GetValuesCount() const override { ythrow yexception() << "unimplemented"; } + const TDqOutputStats& GetPushStats() const override { + return PushStats; + } + + const TDqOutputChannelStats& GetPopStats() const override { + return PopStats; + } + // <| producer methods [[nodiscard]] bool IsFull() const override { @@ -941,7 +969,7 @@ public: header.SetVersion(3); header.SetCommand(NDqProto::TCommandHeader::FINISH_OUTPUT); header.SetTaskId(TaskId); - header.SetChannelId(ChannelId); + header.SetChannelId(PopStats.ChannelId); header.Save(&Output); } catch (...) { TaskRunner->RaiseException(); @@ -999,7 +1027,7 @@ public: header.SetVersion(3); header.SetCommand(NDqProto::TCommandHeader::DROP_OUTPUT); header.SetTaskId(TaskId); - header.SetChannelId(ChannelId); + header.SetChannelId(PopStats.ChannelId); header.Save(&Output); NDqProto::TDropOutputResponse response; @@ -1016,43 +1044,31 @@ public: return nullptr; } - const TDqOutputChannelStats* GetStats() const override { - try { - NDqProto::TCommandHeader header; - header.SetVersion(3); - header.SetCommand(NDqProto::TCommandHeader::GET_STATS_OUTPUT); - header.SetTaskId(TaskId); - header.SetChannelId(ChannelId); - header.Save(&Output); - - NDqProto::TGetStatsOutputResponse response; - response.Load(&Input); - - Stats.FromProto(response.GetStats()); - return &Stats; - } catch (...) { - TaskRunner->RaiseException(); - } - } - void Terminate() override { try { NDqProto::TCommandHeader header; header.SetVersion(3); header.SetCommand(NDqProto::TCommandHeader::TERMINATE_OUTPUT); header.SetTaskId(TaskId); - header.SetChannelId(ChannelId); + header.SetChannelId(PopStats.ChannelId); header.Save(&Output); } catch (...) { TaskRunner->RaiseException(); } } + template<typename T> + void FromProto(const T& f) + { + YQL_ENSURE(PopStats.ChannelId == f.GetChannelId()); + LoadFromProto(PushStats, f.GetPush()); + LoadFromProto(PopStats, f.GetPop()); + PopStats.MaxMemoryUsage = f.GetMaxMemoryUsage(); + } + private: IOutputChannel::TPtr Delegate; ui64 TaskId; - ui64 ChannelId; - mutable TDqOutputChannelStats Stats; IPipeTaskRunner* TaskRunner; IInputStream& Input; IOutputStream& Output; @@ -1060,17 +1076,28 @@ private: class TDqSink : public IStringSink { public: - TDqSink(ui64 taskId, ui64 index, IPipeTaskRunner* taskRunner) + TDqOutputStats PushStats; + TDqAsyncOutputBufferStats PopStats; + + TDqSink(ui64 taskId, ui64 outputIndex, IPipeTaskRunner* taskRunner) : TaskId(taskId) - , Index(index) , TaskRunner(taskRunner) , Input(TaskRunner->GetInput()) , Output(TaskRunner->GetOutput()) - , Stats(Index) - { } + { + PopStats.OutputIndex = outputIndex; + } ui64 GetOutputIndex() const override { - return Index; + return PopStats.OutputIndex; + } + + const TDqOutputStats& GetPushStats() const override { + return PushStats; + } + + const TDqAsyncOutputBufferStats& GetPopStats() const override { + return PopStats; } ui64 PopString(TVector<TString>& batch, ui64 bytes) override { @@ -1079,7 +1106,7 @@ public: header.SetVersion(5); header.SetCommand(NDqProto::TCommandHeader::SINK_POP); header.SetTaskId(TaskId); - header.SetChannelId(Index); + header.SetChannelId(PopStats.OutputIndex); header.Save(&Output); NDqProto::TSinkPopRequest request; @@ -1106,7 +1133,7 @@ public: header.SetVersion(5); header.SetCommand(NDqProto::TCommandHeader::SINK_POP); header.SetTaskId(TaskId); - header.SetChannelId(Index); + header.SetChannelId(PopStats.OutputIndex); header.Save(&Output); NDqProto::TSinkPopRequest request; @@ -1138,7 +1165,7 @@ public: header.SetVersion(5); header.SetCommand(NDqProto::TCommandHeader::SINK_IS_FINISHED); header.SetTaskId(TaskId); - header.SetChannelId(Index); + header.SetChannelId(PopStats.OutputIndex); header.Save(&Output); NDqProto::TIsFinishedResponse response; @@ -1159,7 +1186,7 @@ public: header.SetVersion(5); header.SetCommand(NDqProto::TCommandHeader::SINK_OUTPUT_TYPE); header.SetTaskId(TaskId); - header.SetChannelId(Index); + header.SetChannelId(PopStats.OutputIndex); header.Save(&Output); NDqProto::TGetTypeResponse response; @@ -1172,25 +1199,6 @@ public: return OutputType; } - const TDqAsyncOutputBufferStats* GetStats() const override { - try { - NDqProto::TCommandHeader header; - header.SetVersion(5); - header.SetCommand(NDqProto::TCommandHeader::SINK_STATS); - header.SetTaskId(TaskId); - header.SetChannelId(Index); - header.Save(&Output); - - NDqProto::TSinkStatsResponse response; - response.Load(&Input); - - Stats.FromProto(response.GetStats()); - return &Stats; - } catch (...) { - TaskRunner->RaiseException(); - } - } - void Finish() override { Y_ABORT("Unimplemented"); } @@ -1234,15 +1242,23 @@ public: Y_ABORT("Unimplemented"); } + template<typename T> + void FromProto(const T& f) + { + YQL_ENSURE(PopStats.OutputIndex == f.GetOutputIndex()); + LoadFromProto(PopStats, f.GetPop()); + PopStats.MaxRowsInMemory = f.GetMaxRowsInMemory(); + PopStats.MaxMemoryUsage = f.GetMaxMemoryUsage(); + LoadFromProto(PushStats, f.GetPush()); + } + private: ui64 TaskId; - ui64 Index; IPipeTaskRunner* TaskRunner; IInputStream& Input; IOutputStream& Output; - mutable TDqAsyncOutputBufferStats Stats; mutable NKikimr::NMiniKQL::TType* OutputType = nullptr; }; @@ -1543,8 +1559,9 @@ public: Delegate->GetInputChannel(channelId), Task.GetId(), channelId, + Task.GetStageId(), Delegate.Get()); - Stats.InputChannels[channelId] = channel->GetStats(); + Stats.InputChannels[Task.GetStageId()][channelId] = channel; } return channel; } @@ -1556,7 +1573,7 @@ public: Task.GetId(), inputIndex, Delegate.Get()); - Stats.Sources[inputIndex] = source->GetStats(); + Stats.Sources[inputIndex] = source; } return source; } @@ -1569,8 +1586,9 @@ public: Delegate->GetOutputChannel(channelId), Task.GetId(), channelId, + Task.GetStageId(), Delegate.Get()); - Stats.OutputChannels[channelId] = channel->GetStats(); + Stats.OutputChannels[Task.GetStageId()][channelId] = channel; } return channel; } @@ -1582,7 +1600,6 @@ public: Task.GetId(), outputIndex, Delegate.Get()); - // Stats.Sinks[outputIndex] = sink->GetStats(); } return sink; } @@ -1669,7 +1686,34 @@ public: NDqProto::TGetStatsResponse response; response.Load(&Delegate->GetInput()); - Stats.FromProto(response.GetStats()); + auto& protoStats = response.GetStats(); + + Stats.BuildCpuTime = TDuration::MicroSeconds(protoStats.GetBuildCpuTimeUs()); + Stats.ComputeCpuTime = TDuration::MicroSeconds(protoStats.GetComputeCpuTimeUs()); + // Stats.RunStatusTimeMetrics.Load(ERunStatus::PendingInput, TDuration::MicroSeconds(f.GetPendingInputTimeUs())); + // Stats.RunStatusTimeMetrics.Load(ERunStatus::PendingOutput, TDuration::MicroSeconds(f.GetPendingOutputTimeUs())); + // Stats.RunStatusTimeMetrics.Load(ERunStatus::Finished, TDuration::MicroSeconds(f.GetFinishTimeUs())); + // Stats.TotalTime = TDuration::MilliSeconds(f.GetTotalTime()); + // Stats.WaitTime = TDuration::MicroSeconds(f.GetWaitTimeUs()); + // Stats.WaitOutputTime = TDuration::MicroSeconds(f.GetWaitOutputTimeUs()); + + // Stats.MkqlTotalNodes = f.GetMkqlTotalNodes(); + // Stats.MkqlCodegenFunctions = f.GetMkqlCodegenFunctions(); + // Stats.CodeGenTotalInstructions = f.GetCodeGenTotalInstructions(); + // Stats.CodeGenTotalFunctions = f.GetCodeGenTotalFunctions(); + // Stats.CodeGenFullTime = f.GetCodeGenFullTime(); + // Stats.CodeGenFinalizeTime = f.GetCodeGenFinalizeTime(); + // Stats.CodeGenModulePassTime = f.GetCodeGenModulePassTime(); + + for (const auto& input : protoStats.GetInputChannels()) { + InputChannels[input.GetChannelId()]->FromProto(input); + } + + for (const auto& output : protoStats.GetOutputChannels()) { + OutputChannels[output.GetChannelId()]->FromProto(output); + } + + // todo: (whcrc) fill sources and ComputeCpuTimeByRun? return &Stats; } catch (...) { @@ -1700,10 +1744,10 @@ private: mutable TDqTaskRunnerStats Stats; mutable TDqMeteringStats MeteringStats; - THashMap<ui64, IDqInputChannel::TPtr> InputChannels; - THashMap<ui64, IDqAsyncInputBuffer::TPtr> Sources; - THashMap<ui64, IDqOutputChannel::TPtr> OutputChannels; - THashMap<ui64, IDqAsyncOutputBuffer::TPtr> Sinks; + mutable THashMap<ui64, TIntrusivePtr<TDqInputChannel>> InputChannels; + THashMap<ui64, TIntrusivePtr<TDqSource>> Sources; + mutable THashMap<ui64, TIntrusivePtr<TDqOutputChannel>> OutputChannels; + THashMap<ui64, TIntrusivePtr<TDqSink>> Sinks; }; /*______________________________________________________________________________________________*/ diff --git a/ydb/library/yql/providers/dq/worker_manager/local_worker_manager.cpp b/ydb/library/yql/providers/dq/worker_manager/local_worker_manager.cpp index 21fd56fdd5..872bc8258c 100644 --- a/ydb/library/yql/providers/dq/worker_manager/local_worker_manager.cpp +++ b/ydb/library/yql/providers/dq/worker_manager/local_worker_manager.cpp @@ -295,7 +295,8 @@ private: taskPtr, computeActorType, Options.TaskRunnerActorFactory, - taskCounters)); + taskCounters, + ev->Get()->Record.GetStatsMode())); } else { actor.Reset(CreateWorkerActor( Options.RuntimeData, diff --git a/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.cpp b/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.cpp index 00a0b89936..1683028b39 100644 --- a/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.cpp +++ b/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.cpp @@ -100,6 +100,7 @@ namespace NYql::NDq { public: TGenericReadActor( ui64 inputIndex, + TCollectStatsLevel statsLevel, NConnector::IClient::TPtr client, const NConnector::NApi::TSelect& select, const NConnector::NApi::TDataSourceInstance& dataSourceInstance, @@ -112,6 +113,7 @@ namespace NYql::NDq { , Select_(select) , DataSourceInstance_(dataSourceInstance) { + IngressStats_.Level = statsLevel; } void Bootstrap() { @@ -445,8 +447,13 @@ namespace NYql::NDq { return InputIndex_; } + const TDqAsyncStats& GetIngressStats() const override { + return IngressStats_; + } + private: const ui64 InputIndex_; + TDqAsyncStats IngressStats_; const NActors::TActorId ComputeActorId_; NConnector::IClient::TPtr Client_; @@ -466,6 +473,7 @@ namespace NYql::NDq { CreateGenericReadActor(NConnector::IClient::TPtr genericClient, Generic::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const THashMap<TString, TString>& /*secureParams*/, const THashMap<TString, TString>& /*taskParams*/, const NActors::TActorId& computeActorId, @@ -506,6 +514,7 @@ namespace NYql::NDq { const auto actor = new TGenericReadActor( inputIndex, + statsLevel, genericClient, params.select(), dsi, diff --git a/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.h b/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.h index a65ecb3340..7ef6bf60a8 100644 --- a/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.h +++ b/ydb/library/yql/providers/generic/actors/yql_generic_read_actor.h @@ -10,8 +10,8 @@ namespace NYql::NDq { std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateGenericReadActor(NConnector::IClient::TPtr genericClient, Generic::TSource&& params, ui64 inputIndex, - const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, - const NActors::TActorId& computeActorId, + TCollectStatsLevel statsLevel, const THashMap<TString, TString>& secureParams, + const THashMap<TString, TString>& taskParams, const NActors::TActorId& computeActorId, ISecuredServiceAccountCredentialsFactory::TPtr credentialsFactory, const NKikimr::NMiniKQL::THolderFactory& holderFactory); diff --git a/ydb/library/yql/providers/generic/actors/yql_generic_source_factory.cpp b/ydb/library/yql/providers/generic/actors/yql_generic_source_factory.cpp index 84d90c7880..6299e1c277 100644 --- a/ydb/library/yql/providers/generic/actors/yql_generic_source_factory.cpp +++ b/ydb/library/yql/providers/generic/actors/yql_generic_source_factory.cpp @@ -12,8 +12,8 @@ namespace NYql::NDq { factory.RegisterSource<Generic::TSource>("GenericSource", [credentialsFactory, genericClient]( Generic::TSource&& settings, IDqAsyncIoFactory::TSourceArguments&& args) { - return CreateGenericReadActor(genericClient, std::move(settings), args.InputIndex, args.SecureParams, - args.TaskParams, args.ComputeActorId, credentialsFactory, args.HolderFactory); + return CreateGenericReadActor(genericClient, std::move(settings), args.InputIndex, args.StatsLevel, + args.SecureParams, args.TaskParams, args.ComputeActorId, credentialsFactory, args.HolderFactory); }); } diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.cpp b/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.cpp index 5785aa909a..794e8b877e 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.cpp +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.cpp @@ -94,6 +94,7 @@ public: TDqPqReadActor( ui64 inputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, ui64 taskId, const THolderFactory& holderFactory, @@ -125,6 +126,7 @@ public: } InitWatermarkTracker(); + IngressStats.Level = statsLevel; } NYdb::NPersQueue::TPersQueueClientSettings GetPersQueueClientSettings() const { @@ -159,7 +161,7 @@ public: } stateProto.SetStartingMessageTimestampMs(StartingMessageTimestamp.MilliSeconds()); - stateProto.SetIngressBytes(IngressBytes); + stateProto.SetIngressBytes(IngressStats.Bytes); TString stateBlob; YQL_ENSURE(stateProto.SerializeToString(&stateBlob)); @@ -197,7 +199,8 @@ public: } } StartingMessageTimestamp = minStartingMessageTs; - IngressBytes = ingressBytes; + IngressStats.Bytes += ingressBytes; + IngressStats.Chunks++; InitWatermarkTracker(); if (ReadSession) { @@ -214,13 +217,13 @@ public: } } - ui64 GetIngressBytes() override { - return IngressBytes; - } - ui64 GetInputIndex() const override { return InputIndex; - }; + } + + const TDqAsyncStats& GetIngressStats() const override { + return IngressStats; + } NYdb::NPersQueue::TPersQueueClient& GetPersQueueClient() { if (!PersQueueClient) { @@ -451,7 +454,7 @@ private: const auto partitionKey = MakePartitionKey(event.GetPartitionStream()); for (const auto& message : event.GetMessages()) { const TString& data = message.GetData(); - Self.IngressBytes += data.size(); + Self.IngressStats.Bytes += data.size(); LWPROBE(PqReadDataReceived, TString(TStringBuilder() << Self.TxId), Self.SourceParams.GetTopicPath(), data); SRC_LOG_T("Data received: " << message.DebugString(true)); @@ -555,6 +558,7 @@ private: private: const ui64 InputIndex; + TDqAsyncStats IngressStats; const TTxId TxId; const i64 BufferSize; const bool RangesMode; @@ -569,7 +573,6 @@ private: NThreading::TFuture<void> EventFuture; THashMap<TPartitionKey, ui64> PartitionToOffset; // {cluster, partition} -> offset of next event. TInstant StartingMessageTimestamp; - ui64 IngressBytes = 0; const NActors::TActorId ComputeActorId; std::queue<std::pair<ui64, NYdb::NPersQueue::TDeferredCommit>> DeferredCommits; NYdb::NPersQueue::TDeferredCommit CurrentDeferredCommit; @@ -583,6 +586,7 @@ private: std::pair<IDqComputeActorAsyncInput*, NActors::IActor*> CreateDqPqReadActor( NPq::NProto::TDqPqTopicSource&& settings, ui64 inputIndex, + TCollectStatsLevel statsLevel, TTxId txId, ui64 taskId, const THashMap<TString, TString>& secureParams, @@ -607,6 +611,7 @@ std::pair<IDqComputeActorAsyncInput*, NActors::IActor*> CreateDqPqReadActor( TDqPqReadActor* actor = new TDqPqReadActor( inputIndex, + statsLevel, txId, taskId, holderFactory, @@ -632,6 +637,7 @@ void RegisterDqPqReadActorFactory(TDqAsyncIoFactory& factory, NYdb::TDriver driv return CreateDqPqReadActor( std::move(settings), args.InputIndex, + args.StatsLevel, args.TxId, args.TaskId, args.SecureParams, diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.h b/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.h index d8411fb04a..b97d5f2442 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.h +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor.h @@ -25,6 +25,7 @@ const i64 PQReadDefaultFreeSpace = 16_MB; std::pair<IDqComputeActorAsyncInput*, NActors::IActor*> CreateDqPqReadActor( NPq::NProto::TDqPqTopicSource&& settings, ui64 inputIndex, + TCollectStatsLevel statsLevel, TTxId txId, ui64 taskId, const THashMap<TString, TString>& secureParams, diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.cpp b/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.cpp index 2c36f71ccc..83181c56a9 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.cpp +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.cpp @@ -100,6 +100,7 @@ class TDqPqWriteActor : public NActors::TActor<TDqPqWriteActor>, public IDqCompu public: TDqPqWriteActor( ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, NPq::NProto::TDqPqTopicSink&& sinkParams, NYdb::TDriver driver, @@ -116,7 +117,9 @@ public: , LogPrefix(TStringBuilder() << "TxId: " << TxId << ", PQ sink. ") , FreeSpace(freeSpace) , PersQueueClient(Driver, GetPersQueueClientSettings()) - { } + { + EgressStats.Level = statsLevel; + } static constexpr char ActorName[] = "DQ_PQ_WRITE_ACTOR"; @@ -201,27 +204,27 @@ public: SourceId = stateProto.GetSourceId(); ConfirmedSeqNo = stateProto.GetConfirmedSeqNo(); NextSeqNo = ConfirmedSeqNo + 1; - EgressBytes = stateProto.GetEgressBytes(); + EgressStats.Bytes = stateProto.GetEgressBytes(); return; } ythrow yexception() << "Invalid state version " << data.GetVersion(); } - ui64 GetEgressBytes() override { - return EgressBytes; - } - void CommitState(const NDqProto::TCheckpoint& checkpoint) override { Y_UNUSED(checkpoint); } i64 GetFreeSpace() const override { return FreeSpace; - }; + } ui64 GetOutputIndex() const override { return OutputIndex; - }; + } + + const TDqAsyncStats& GetEgressStats() const override { + return EgressStats; + } private: STRICT_STFUNC(StateFunc, @@ -316,7 +319,7 @@ private: NPq::NProto::TDqPqTopicSinkState stateProto; stateProto.SetSourceId(GetSourceId()); stateProto.SetConfirmedSeqNo(ConfirmedSeqNo); - stateProto.SetEgressBytes(EgressBytes); + stateProto.SetEgressBytes(EgressStats.Bytes); TString serializedState; YQL_ENSURE(stateProto.SerializeToString(&serializedState)); @@ -333,7 +336,7 @@ private: WriteSession->Write(std::move(token), Buffer.front(), NextSeqNo++); auto itemSize = GetItemSize(Buffer.front()); WaitingAcks.push(itemSize); - EgressBytes += itemSize; + EgressStats.Bytes += itemSize; Buffer.pop(); } @@ -404,6 +407,7 @@ private: private: const ui64 OutputIndex; + TDqAsyncStats EgressStats; const TTxId TxId; const NPq::NProto::TDqPqTopicSink SinkParams; NYdb::TDriver Driver; @@ -412,7 +416,6 @@ private: const TString LogPrefix; i64 FreeSpace = 0; bool Finished = false; - ui64 EgressBytes = 0; NYdb::NPersQueue::TPersQueueClient PersQueueClient; std::shared_ptr<NYdb::NPersQueue::IWriteSession> WriteSession; @@ -430,6 +433,7 @@ private: std::pair<IDqComputeActorAsyncOutput*, NActors::IActor*> CreateDqPqWriteActor( NPq::NProto::TDqPqTopicSink&& settings, ui64 outputIndex, + TCollectStatsLevel statsLevel, TTxId txId, const THashMap<TString, TString>& secureParams, NYdb::TDriver driver, @@ -443,6 +447,7 @@ std::pair<IDqComputeActorAsyncOutput*, NActors::IActor*> CreateDqPqWriteActor( TDqPqWriteActor* actor = new TDqPqWriteActor( outputIndex, + statsLevel, txId, std::move(settings), std::move(driver), @@ -462,6 +467,7 @@ void RegisterDqPqWriteActorFactory(TDqAsyncIoFactory& factory, NYdb::TDriver dri return CreateDqPqWriteActor( std::move(settings), args.OutputIndex, + args.StatsLevel, args.TxId, args.SecureParams, driver, diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.h b/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.h index 88ac95f466..8b0a1785ce 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.h +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_write_actor.h @@ -23,6 +23,7 @@ constexpr i64 DqPqDefaultFreeSpace = 16_MB; std::pair<IDqComputeActorAsyncOutput*, NActors::IActor*> CreateDqPqWriteActor( NPq::NProto::TDqPqTopicSink&& settings, ui64 outputIndex, + TCollectStatsLevel statsLevel, TTxId txId, const THashMap<TString, TString>& secureParams, NYdb::TDriver driver, diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp index ef2d73e35d..5166f89e58 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp +++ b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp @@ -668,6 +668,7 @@ private: class TS3ReadActor : public TActorBootstrapped<TS3ReadActor>, public IDqComputeActorAsyncInput { public: TS3ReadActor(ui64 inputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, IHTTPGateway::TPtr gateway, const THolderFactory& holderFactory, @@ -713,6 +714,7 @@ public: TaskQueueDataLimit = TaskCounters->GetCounter("QueueDataLimit"); TaskQueueDataLimit->Add(ReadActorFactoryCfg.DataInflight); } + IngressStats.Level = statsLevel; } void Bootstrap() { @@ -791,10 +793,13 @@ private: void SaveState(const NDqProto::TCheckpoint&, NDqProto::TSourceState&) final {} void LoadState(const NDqProto::TSourceState&) final {} void CommitState(const NDqProto::TCheckpoint&) final {} - ui64 GetInputIndex() const final { return InputIndex; } - ui64 GetIngressBytes() override { - return IngressBytes; + ui64 GetInputIndex() const final { + return InputIndex; + } + + const TDqAsyncStats& GetIngressStats() const override { + return IngressStats; } TDuration GetCpuTime() override { @@ -875,6 +880,10 @@ private: ContainerCache.Clear(); } + if (!total) { + IngressStats.TryPause(); + } + return total; } bool LastFileWasProcessed() const { @@ -887,10 +896,16 @@ private: const auto path = result->Get()->Path; const auto httpCode = result->Get()->Result.HttpResponseCode; const auto requestId = result->Get()->RequestId; - IngressBytes += result->Get()->Result.size(); LOG_D("TS3ReadActor", "ID: " << id << ", Path: " << path << ", read size: " << result->Get()->Result.size() << ", HTTP response code: " << httpCode << ", request id: [" << requestId << "]"); if (200 == httpCode || 206 == httpCode) { auto size = result->Get()->Result.size(); + + // in TS3ReadActor all files (aka Splits) are loaded in single Chunks + IngressStats.Bytes += size; + IngressStats.Chunks++; + IngressStats.Splits++; + IngressStats.Resume(); + QueueTotalDataSize += size; if (Counters) { QueueBlockCount->Inc(); @@ -952,6 +967,7 @@ private: TPlainContainerCache ContainerCache; const ui64 InputIndex; + TDqAsyncStats IngressStats; const TTxId TxId; const NActors::TActorId ComputeActorId; const IHTTPGateway::TRetryPolicy::TPtr RetryPolicy; @@ -971,7 +987,6 @@ private: NActors::TActorId FileQueueActor; const bool AddPathIndex; const ui64 SizeLimit; - ui64 IngressBytes = 0; TDuration CpuTime; std::queue<std::tuple<IHTTPGateway::TContent, ui64>> Blocks; @@ -2219,6 +2234,7 @@ class TS3StreamReadActor : public TActorBootstrapped<TS3StreamReadActor>, public public: TS3StreamReadActor( ui64 inputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, IHTTPGateway::TPtr gateway, const THolderFactory& holderFactory, @@ -2276,6 +2292,7 @@ public: TaskQueueDataLimit->Add(ReadActorFactoryCfg.DataInflight); RawInflightSize = TaskCounters->GetCounter("RawInflightSize"); } + IngressStats.Level = statsLevel; } void Bootstrap() { @@ -2443,10 +2460,13 @@ private: void SaveState(const NDqProto::TCheckpoint&, NDqProto::TSourceState&) final {} void LoadState(const NDqProto::TSourceState&) final {} void CommitState(const NDqProto::TCheckpoint&) final {} - ui64 GetInputIndex() const final { return InputIndex; } - ui64 GetIngressBytes() override { - return IngressBytes; + ui64 GetInputIndex() const final { + return InputIndex; + } + + const TDqAsyncStats& GetIngressStats() const final { + return IngressStats; } TDuration GetCpuTime() override { @@ -2502,6 +2522,8 @@ private: ContainerCache.Clear(); ArrowTupleContainerCache.Clear(); ArrowRowContainerCache.Clear(); + } else if(!total) { + IngressStats.TryPause(); } return total; } @@ -2581,7 +2603,9 @@ private: void HandleNextBlock(TEvPrivate::TEvNextBlock::TPtr& next) { YQL_ENSURE(!ReadSpec->Arrow); - IngressBytes += next->Get()->IngressDelta; + IngressStats.Bytes += next->Get()->IngressDelta; + IngressStats.Chunks++; + IngressStats.Resume(); CpuTime += next->Get()->CpuTimeDelta; if (Counters) { QueueBlockCount->Inc(); @@ -2592,7 +2616,9 @@ private: void HandleNextRecordBatch(TEvPrivate::TEvNextRecordBatch::TPtr& next) { YQL_ENSURE(ReadSpec->Arrow); - IngressBytes += next->Get()->IngressDelta; + IngressStats.Bytes += next->Get()->IngressDelta; + IngressStats.Chunks++; + IngressStats.Resume(); CpuTime += next->Get()->CpuTimeDelta; if (Counters) { QueueBlockCount->Inc(); @@ -2603,7 +2629,11 @@ private: void HandleFileFinished(TEvPrivate::TEvFileFinished::TPtr& ev) { CoroActors.erase(ev->Sender); - IngressBytes += ev->Get()->IngressDelta; + if (ev->Get()->IngressDelta) { + IngressStats.Bytes += ev->Get()->IngressDelta; + IngressStats.Chunks++; + IngressStats.Resume(); + } CpuTime += ev->Get()->CpuTimeDelta; auto it = RetryStuffForFile.find(ev->Sender); @@ -2629,6 +2659,7 @@ private: TaskDownloadCount->Dec(); } CompletedFiles++; + IngressStats.Splits++; if (!ObjectPathCache.empty()) { TryRegisterCoro(); } else { @@ -2656,6 +2687,7 @@ private: TPlainContainerCache ArrowRowContainerCache; const ui64 InputIndex; + TDqAsyncStats IngressStats; const TTxId TxId; const NActors::TActorId ComputeActorId; const IHTTPGateway::TRetryPolicy::TPtr RetryPolicy; @@ -2673,7 +2705,6 @@ private: size_t CompletedFiles = 0; const TReadSpec::TPtr ReadSpec; std::deque<TReadyBlock> Blocks; - ui64 IngressBytes = 0; TDuration CpuTime; mutable TInstant LastMemoryReport = TInstant::Now(); TReadBufferCounter::TPtr QueueBufferCounter; @@ -2847,6 +2878,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateS3ReadActor( IHTTPGateway::TPtr gateway, NS3::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, @@ -3001,7 +3033,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateS3ReadActor( #undef SET_FLAG #undef SUPPORTED_FLAGS - const auto actor = new TS3StreamReadActor(inputIndex, txId, std::move(gateway), holderFactory, params.GetUrl(), authToken, pathPattern, pathPatternVariant, + const auto actor = new TS3StreamReadActor(inputIndex, statsLevel, txId, std::move(gateway), holderFactory, params.GetUrl(), authToken, pathPattern, pathPatternVariant, std::move(paths), addPathIndex, readSpec, computeActorId, retryPolicy, cfg, counters, taskCounters, fileSizeLimit, memoryQuotaManager); @@ -3011,7 +3043,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateS3ReadActor( if (const auto it = settings.find("sizeLimit"); settings.cend() != it) sizeLimit = FromString<ui64>(it->second); - const auto actor = new TS3ReadActor(inputIndex, txId, std::move(gateway), holderFactory, params.GetUrl(), authToken, pathPattern, pathPatternVariant, + const auto actor = new TS3ReadActor(inputIndex, statsLevel, txId, std::move(gateway), holderFactory, params.GetUrl(), authToken, pathPattern, pathPatternVariant, std::move(paths), addPathIndex, computeActorId, sizeLimit, retryPolicy, cfg, counters, taskCounters, fileSizeLimit); return {actor, actor}; diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h index 330c340f6f..af8e44dcb0 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h +++ b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h @@ -17,6 +17,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateS3ReadA IHTTPGateway::TPtr gateway, NS3::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_sink_factory.cpp b/ydb/library/yql/providers/s3/actors/yql_s3_sink_factory.cpp index d42a800278..595dc46a83 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_sink_factory.cpp +++ b/ydb/library/yql/providers/s3/actors/yql_s3_sink_factory.cpp @@ -21,7 +21,9 @@ void RegisterS3WriteActorFactory(TDqAsyncIoFactory& factory, ISecuredServiceAcco prefixBuilder << restartCount << "_"; } - return CreateS3WriteActor(args.TypeEnv, *args.HolderFactory.GetFunctionRegistry(), args.RandomProvider, gateway, std::move(settings), args.OutputIndex, args.TxId, prefixBuilder, args.SecureParams, args.Callback, credentialsFactory, retryPolicy); + return CreateS3WriteActor(args.TypeEnv, *args.HolderFactory.GetFunctionRegistry(), args.RandomProvider, + gateway, std::move(settings), args.OutputIndex, args.StatsLevel, args.TxId, prefixBuilder, + args.SecureParams, args.Callback, credentialsFactory, retryPolicy); }); } diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_source_factory.cpp b/ydb/library/yql/providers/s3/actors/yql_s3_source_factory.cpp index a152467e39..03938548a6 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_source_factory.cpp +++ b/ydb/library/yql/providers/s3/actors/yql_s3_source_factory.cpp @@ -22,7 +22,7 @@ void RegisterS3ReadActorFactory( factory.RegisterSource<NS3::TSource>("S3Source", [credentialsFactory, gateway, retryPolicy, cfg, counters](NS3::TSource&& settings, IDqAsyncIoFactory::TSourceArguments&& args) { return CreateS3ReadActor(args.TypeEnv, args.HolderFactory, gateway, - std::move(settings), args.InputIndex, args.TxId, args.SecureParams, + std::move(settings), args.InputIndex, args.StatsLevel, args.TxId, args.SecureParams, args.TaskParams, args.ReadRanges, args.ComputeActorId, credentialsFactory, retryPolicy, cfg, counters, args.TaskCounters, args.MemoryQuotaManager); }); diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.cpp b/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.cpp index 1478d11015..26e0efbea0 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.cpp +++ b/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.cpp @@ -417,6 +417,7 @@ private: class TS3WriteActor : public TActorBootstrapped<TS3WriteActor>, public IDqComputeActorAsyncOutput { public: TS3WriteActor(ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const TString& prefix, IHTTPGateway::TPtr gateway, @@ -455,6 +456,7 @@ public: DefaultRandomProvider = CreateDefaultRandomProvider(); RandomProvider = DefaultRandomProvider.Get(); } + EgressStats.Level = statsLevel; } void Bootstrap() { @@ -466,7 +468,15 @@ public: private: void CommitState(const NDqProto::TCheckpoint&) final {}; void LoadState(const NDqProto::TSinkState&) final {}; - ui64 GetOutputIndex() const final { return OutputIndex; } + + ui64 GetOutputIndex() const final { + return OutputIndex; + } + + const TDqAsyncStats& GetEgressStats() const final { + return EgressStats; + } + i64 GetFreeSpace() const final { return std::accumulate(FileWriteActors.cbegin(), FileWriteActors.cend(), i64(MemoryLimit), [](i64 free, const std::pair<const TString, std::vector<TS3FileWriteActor*>>& item) { @@ -501,6 +511,7 @@ private: void SendData(TUnboxedValueBatch&& data, i64, const TMaybe<NDqProto::TCheckpoint>&, bool finished) final { std::unordered_set<TS3FileWriteActor*> processedActors; YQL_ENSURE(!data.IsWide(), "Wide stream is not supported yet"); + EgressStats.Resume(); data.ForEachRow([&](const auto& row) { const auto& key = MakePartitionKey(row); const auto [keyIt, insertedNew] = FileWriteActors.emplace(key, std::vector<TS3FileWriteActor*>()); @@ -545,10 +556,6 @@ private: data.clear(); } - ui64 GetEgressBytes() override { - return EgressBytes; - } - void Handle(TEvPrivate::TEvUploadError::TPtr& result) { LOG_W("TS3WriteActor", "TEvUploadError " << result->Get()->Issues.ToOneLineString()); @@ -569,7 +576,10 @@ private: void Handle(TEvPrivate::TEvUploadFinished::TPtr& result) { if (const auto it = FileWriteActors.find(result->Get()->Key); FileWriteActors.cend() != it) { - EgressBytes += result->Get()->UploadSize; + EgressStats.Bytes += result->Get()->UploadSize; + EgressStats.Chunks++; + EgressStats.Splits++; + EgressStats.Resume(); if (const auto ft = std::find_if(it->second.cbegin(), it->second.cend(), [&](TS3FileWriteActor* actor){ return result->Get()->Url == actor->GetUrl(); }); it->second.cend() != ft) { (*ft)->PassAway(); it->second.erase(ft); @@ -607,6 +617,7 @@ private: const IHTTPGateway::TRetryPolicy::TPtr RetryPolicy; const ui64 OutputIndex; + TDqAsyncStats EgressStats; const TTxId TxId; const TString Prefix; IDqComputeActorAsyncOutput::ICallbacks *const Callbacks; @@ -620,7 +631,6 @@ private: const TString Compression; const bool Multipart; bool Finished = false; - ui64 EgressBytes = 0; std::unordered_map<TString, std::vector<TS3FileWriteActor*>> FileWriteActors; bool DirtyWrite; @@ -636,6 +646,7 @@ std::pair<IDqComputeActorAsyncOutput*, NActors::IActor*> CreateS3WriteActor( IHTTPGateway::TPtr gateway, NS3::TSink&& params, ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const TString& prefix, const THashMap<TString, TString>& secureParams, @@ -647,6 +658,7 @@ std::pair<IDqComputeActorAsyncOutput*, NActors::IActor*> CreateS3WriteActor( const auto credentialsProviderFactory = CreateCredentialsProviderFactoryForStructuredToken(credentialsFactory, token); const auto actor = new TS3WriteActor( outputIndex, + statsLevel, txId, prefix, std::move(gateway), diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.h b/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.h index 3787a9f849..848845e045 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.h +++ b/ydb/library/yql/providers/s3/actors/yql_s3_write_actor.h @@ -16,6 +16,7 @@ std::pair<IDqComputeActorAsyncOutput*, NActors::IActor*> CreateS3WriteActor( IHTTPGateway::TPtr gateway, NS3::TSink&& params, ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const TString& prefix, const THashMap<TString, TString>& secureParams, diff --git a/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.cpp b/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.cpp index 8ff898d5c9..c26cd4cbe6 100644 --- a/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.cpp +++ b/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.cpp @@ -122,6 +122,7 @@ public: TDqSolomonWriteActor( ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, TDqSolomonWriteParams&& writeParams, NYql::NDq::IDqComputeActorAsyncOutput::ICallbacks* callbacks, @@ -143,6 +144,7 @@ public: , CredentialsProvider(credentialsProvider) { SINK_LOG_D("Init"); + EgressStats.Level = statsLevel; } STRICT_STFUNC(StateFunc, @@ -172,6 +174,7 @@ public: } metricsCount += UserMetricsEncoder.Append(value); + EgressStats.Rows++; }); if (metricsCount != 0) { @@ -203,6 +206,10 @@ public: return OutputIndex; }; + const TDqAsyncStats& GetEgressStats() const override { + return EgressStats; + } + private: struct TDqSolomonWriteActorMetrics { explicit TDqSolomonWriteActorMetrics(const ::NMonitoring::TDynamicCounterPtr& counters) { @@ -385,6 +392,8 @@ private: *Metrics.SentMetrics += metricsToSend.MetricsCount; InflightBuffer.emplace(Cookie++, TMetricsInflight { httpSenderId, metricsToSend.MetricsCount, bodySize }); + EgressStats.Bytes += bodySize; + EgressStats.Chunks++; return true; } @@ -470,6 +479,7 @@ private: private: const ui64 OutputIndex; + TDqAsyncStats EgressStats; const TTxId TxId; const TString LogPrefix; const TDqSolomonWriteParams WriteParams; @@ -494,6 +504,7 @@ private: std::pair<NYql::NDq::IDqComputeActorAsyncOutput*, NActors::IActor*> CreateDqSolomonWriteActor( NYql::NSo::NProto::TDqSolomonShard&& settings, ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const THashMap<TString, TString>& secureParams, NYql::NDq::IDqComputeActorAsyncOutput::ICallbacks* callbacks, @@ -513,6 +524,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncOutput*, NActors::IActor*> CreateDqSolo TDqSolomonWriteActor* actor = new TDqSolomonWriteActor( outputIndex, + statsLevel, txId, std::move(params), callbacks, @@ -533,6 +545,7 @@ void RegisterDQSolomonWriteActorFactory(TDqAsyncIoFactory& factory, ISecuredServ return CreateDqSolomonWriteActor( std::move(settings), args.OutputIndex, + args.StatsLevel, args.TxId, args.SecureParams, args.Callback, diff --git a/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.h b/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.h index 3d422e333c..20dbbf829b 100644 --- a/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.h +++ b/ydb/library/yql/providers/solomon/async_io/dq_solomon_write_actor.h @@ -21,6 +21,7 @@ constexpr i64 DqSolomonDefaultFreeSpace = 16_MB; std::pair<NYql::NDq::IDqComputeActorAsyncOutput*, NActors::IActor*> CreateDqSolomonWriteActor( NYql::NSo::NProto::TDqSolomonShard&& settings, ui64 outputIndex, + TCollectStatsLevel statsLevel, const TTxId& txId, const THashMap<TString, TString>& secureParams, NYql::NDq::IDqComputeActorAsyncOutput::ICallbacks* callbacks, diff --git a/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.cpp b/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.cpp index 745d4a4a41..5ec9dd26ff 100644 --- a/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.cpp +++ b/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.cpp @@ -72,6 +72,7 @@ class TYdbReadActor : public TActorBootstrapped<TYdbReadActor>, public IDqComput public: TYdbReadActor( ui64 inputIndex, + TCollectStatsLevel statsLevel, const TString& database, const TString& endpoint, std::shared_ptr<::NYdb::ICredentialsProviderFactory> credentialsProviderFactory, @@ -96,7 +97,9 @@ public: , RequestSent(false) , RequestsDone(!EndKey.empty() && RangeFinished(LastReadKey, EndKey, KeyColumnTypes)) , MemoryUsed(0U) - {} + { + IngressStats.Level = statsLevel; + } void Bootstrap() { Become(&TYdbReadActor::StateFunc); @@ -109,7 +112,14 @@ private: void SaveState(const NDqProto::TCheckpoint&, NDqProto::TSourceState&) final {} void LoadState(const NDqProto::TSourceState&) final {} void CommitState(const NDqProto::TCheckpoint&) final {} - ui64 GetInputIndex() const final { return InputIndex; } + + ui64 GetInputIndex() const final { + return InputIndex; + } + + const TDqAsyncStats& GetIngressStats() const final { + return IngressStats; + } STRICT_STFUNC(StateFunc, hFunc(TEvPrivate::TEvScanResult, Handle); @@ -202,6 +212,7 @@ private: static constexpr auto MaxQueueVolume = 4_MB; const ui64 InputIndex; + TDqAsyncStats IngressStats; const NActors::TActorId ComputeActorId; TActorSystem* const ActorSystem; @@ -231,6 +242,7 @@ private: std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateYdbReadActor( NYql::NYdb::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, const NActors::TActorId& computeActorId, @@ -261,7 +273,7 @@ std::pair<NYql::NDq::IDqComputeActorAsyncInput*, IActor*> CreateYdbReadActor( ui64 maxRowsInRequest = 0ULL; ui64 maxBytesInRequest = 0ULL; - const auto actor = new TYdbReadActor(inputIndex, params.GetDatabase(), params.GetEndpoint(), credentialsProviderFactory, params.GetSecure(), params.GetTable(), std::move(driver), computeActorId, columns, keyColumnTypes, maxRowsInRequest, maxBytesInRequest, keyFrom, keyTo); + const auto actor = new TYdbReadActor(inputIndex, statsLevel, params.GetDatabase(), params.GetEndpoint(), credentialsProviderFactory, params.GetSecure(), params.GetTable(), std::move(driver), computeActorId, columns, keyColumnTypes, maxRowsInRequest, maxBytesInRequest, keyFrom, keyTo); return {actor, actor}; } diff --git a/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.h b/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.h index 1a67a7fb3f..d5fd497803 100644 --- a/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.h +++ b/ydb/library/yql/providers/ydb/actors/yql_ydb_read_actor.h @@ -12,6 +12,7 @@ namespace NYql::NDq { std::pair<NYql::NDq::IDqComputeActorAsyncInput*, NActors::IActor*> CreateYdbReadActor( NYql::NYdb::TSource&& params, ui64 inputIndex, + TCollectStatsLevel statsLevel, const THashMap<TString, TString>& secureParams, const THashMap<TString, TString>& taskParams, const NActors::TActorId& computeActorId, diff --git a/ydb/library/yql/providers/ydb/actors/yql_ydb_source_factory.cpp b/ydb/library/yql/providers/ydb/actors/yql_ydb_source_factory.cpp index d9f8ae9daf..f8a554c765 100644 --- a/ydb/library/yql/providers/ydb/actors/yql_ydb_source_factory.cpp +++ b/ydb/library/yql/providers/ydb/actors/yql_ydb_source_factory.cpp @@ -8,7 +8,8 @@ namespace NYql::NDq { void RegisterYdbReadActorFactory(NYql::NDq::TDqAsyncIoFactory& factory, ::NYdb::TDriver driver, ISecuredServiceAccountCredentialsFactory::TPtr credentialsFactory) { factory.RegisterSource<NYql::NYdb::TSource>("YdbSource", [driver, credentialsFactory](NYql::NYdb::TSource&& settings, IDqAsyncIoFactory::TSourceArguments&& args) { - return CreateYdbReadActor(std::move(settings), args.InputIndex, args.SecureParams, args.TaskParams, args.ComputeActorId, driver, credentialsFactory); + return CreateYdbReadActor(std::move(settings), args.InputIndex, args.StatsLevel, args.SecureParams, + args.TaskParams, args.ComputeActorId, driver, credentialsFactory); }); } |