diff --git a/ydb/core/fq/libs/row_dispatcher/events/data_plane.h b/ydb/core/fq/libs/row_dispatcher/events/data_plane.h index 88953d424432..891c729cf0f9 100644 --- a/ydb/core/fq/libs/row_dispatcher/events/data_plane.h +++ b/ydb/core/fq/libs/row_dispatcher/events/data_plane.h @@ -8,6 +8,7 @@ #include #include +#include #include namespace NFq { @@ -148,9 +149,9 @@ struct TEvRowDispatcher { }; struct TEvSessionStatistic : public NActors::TEventLocal { - TEvSessionStatistic(const TopicSessionStatistic& stat) + TEvSessionStatistic(const TTopicSessionStatistic& stat) : Stat(stat) {} - TopicSessionStatistic Stat; + TTopicSessionStatistic Stat; }; struct TEvHeartbeat : public NActors::TEventPB { @@ -182,16 +183,19 @@ struct TEvRowDispatcher { }; struct TEvPurecalcCompileResponse : public NActors::TEventLocal { - explicit TEvPurecalcCompileResponse(const TString& error) - : Error(error) + TEvPurecalcCompileResponse(NYql::NDqProto::StatusIds::StatusCode status, NYql::TIssues issues) + : Status(status) + , Issues(std::move(issues)) {} explicit TEvPurecalcCompileResponse(IProgramHolder::TPtr programHolder) : ProgramHolder(std::move(programHolder)) + , Status(NYql::NDqProto::StatusIds::SUCCESS) {} IProgramHolder::TPtr ProgramHolder; // Same holder that passed into TEvPurecalcCompileRequest - TString Error; + NYql::NDqProto::StatusIds::StatusCode Status; + NYql::TIssues Issues; }; }; diff --git a/ydb/core/fq/libs/row_dispatcher/events/topic_session_stats.h b/ydb/core/fq/libs/row_dispatcher/events/topic_session_stats.h index 6c7f254a74ae..707c014618dc 100644 --- a/ydb/core/fq/libs/row_dispatcher/events/topic_session_stats.h +++ b/ydb/core/fq/libs/row_dispatcher/events/topic_session_stats.h @@ -5,7 +5,7 @@ namespace NFq { -struct TopicSessionClientStatistic { +struct TTopicSessionClientStatistic { NActors::TActorId ReadActorId; ui32 PartitionId = 0; i64 UnreadRows = 0; // Current value @@ -15,7 +15,7 @@ struct TopicSessionClientStatistic { bool IsWaiting = false; // Current value i64 ReadLagMessages = 0; // Current value ui64 InitialOffset = 0; - void Add(const TopicSessionClientStatistic& stat) { + void Add(const TTopicSessionClientStatistic& stat) { UnreadRows = stat.UnreadRows; UnreadBytes = stat.UnreadBytes; Offset = stat.Offset; @@ -29,28 +29,64 @@ struct TopicSessionClientStatistic { } }; -struct TopicSessionCommonStatistic { +struct TParserStatistic { + TDuration ParserLatency; + + void Add(const TParserStatistic& stat) { + ParserLatency = stat.ParserLatency != TDuration::Zero() ? stat.ParserLatency : ParserLatency; + } +}; + +struct TFiltersStatistic { + TDuration FilterLatency; + + void Add(const TFiltersStatistic& stat) { + FilterLatency = stat.FilterLatency != TDuration::Zero() ? stat.FilterLatency : FilterLatency; + } +}; + +struct TFormatHandlerStatistic { + TDuration ParseAndFilterLatency; + + TParserStatistic ParserStats; + TFiltersStatistic FilterStats; + + void Add(const TFormatHandlerStatistic& stat) { + ParseAndFilterLatency = stat.ParseAndFilterLatency != TDuration::Zero() ? stat.ParseAndFilterLatency : ParseAndFilterLatency; + + ParserStats.Add(stat.ParserStats); + FilterStats.Add(stat.FilterStats); + } +}; + +struct TTopicSessionCommonStatistic { ui64 UnreadBytes = 0; // Current value ui64 RestartSessionByOffsets = 0; ui64 ReadBytes = 0; // Increment ui64 ReadEvents = 0; // Increment ui64 LastReadedOffset = 0; - TDuration ParseAndFilterLatency; - void Add(const TopicSessionCommonStatistic& stat) { + + std::unordered_map FormatHandlers; + + void Add(const TTopicSessionCommonStatistic& stat) { UnreadBytes = stat.UnreadBytes; RestartSessionByOffsets = stat.RestartSessionByOffsets; ReadBytes += stat.ReadBytes; ReadEvents += stat.ReadEvents; LastReadedOffset = stat.LastReadedOffset; - ParseAndFilterLatency = stat.ParseAndFilterLatency != TDuration::Zero() ? stat.ParseAndFilterLatency : ParseAndFilterLatency; + + for (const auto& [formatName, foramtStats] : stat.FormatHandlers) { + FormatHandlers[formatName].Add(foramtStats); + } } + void Clear() { ReadBytes = 0; ReadEvents = 0; } }; -struct TopicSessionParams { +struct TTopicSessionParams { TString ReadGroup; TString Endpoint; TString Database; @@ -58,10 +94,10 @@ struct TopicSessionParams { ui64 PartitionId = 0; }; -struct TopicSessionStatistic { - TopicSessionParams SessionKey; - TVector Clients; - TopicSessionCommonStatistic Common; +struct TTopicSessionStatistic { + TTopicSessionParams SessionKey; + std::vector Clients; + TTopicSessionCommonStatistic Common; }; } // namespace NFq diff --git a/ydb/core/fq/libs/row_dispatcher/events/ya.make b/ydb/core/fq/libs/row_dispatcher/events/ya.make index 60f0b00e7e90..2968bf590ebe 100644 --- a/ydb/core/fq/libs/row_dispatcher/events/ya.make +++ b/ydb/core/fq/libs/row_dispatcher/events/ya.make @@ -7,8 +7,11 @@ SRCS( PEERDIR( ydb/core/fq/libs/events ydb/core/fq/libs/row_dispatcher/protos + ydb/library/actors/core ydb/library/yql/providers/pq/provider + + yql/essentials/public/issue ) END() diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/common/common.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/common/common.cpp new file mode 100644 index 000000000000..8340bbf62f19 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/common/common.cpp @@ -0,0 +1,13 @@ +#include "common.h" + +#include + +namespace NFq::NRowDispatcher { + +//// TSchemaColumn + +TString TSchemaColumn::ToString() const { + return TStringBuilder() << "'" << Name << "' : " << TypeYson; +} + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/common/common.h b/ydb/core/fq/libs/row_dispatcher/format_handler/common/common.h new file mode 100644 index 000000000000..10c5701b5d4e --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/common/common.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include +#include + +namespace NFq::NRowDispatcher { + +using EStatusId = NYql::NDqProto::StatusIds; +using TStatusCode = EStatusId::StatusCode; +using TStatus = NKikimr::TYQLConclusionSpecialStatus; + +template +using TValueStatus = NKikimr::TConclusionImpl; + +struct TSchemaColumn { + TString Name; + TString TypeYson; + + bool operator==(const TSchemaColumn& other) const = default; + + TString ToString() const; +}; + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/common/ya.make b/ydb/core/fq/libs/row_dispatcher/format_handler/common/ya.make new file mode 100644 index 000000000000..e4d689c0a1aa --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/common/ya.make @@ -0,0 +1,16 @@ +LIBRARY() + +SRCS( + common.cpp +) + +PEERDIR( + ydb/library/conclusion + ydb/library/yql/dq/actors/protos + + yql/essentials/public/issue +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/filters/filters_set.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/filters_set.cpp new file mode 100644 index 000000000000..606557918503 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/filters_set.cpp @@ -0,0 +1,273 @@ +#include "filters_set.h" + +#include + +namespace NFq::NRowDispatcher { + +namespace { + +class TTopicFilters : public ITopicFilters { + struct TCounters { + const NMonitoring::TDynamicCounterPtr Counters; + + NMonitoring::TDynamicCounters::TCounterPtr ActiveFilters; + NMonitoring::TDynamicCounters::TCounterPtr InFlightCompileRequests; + NMonitoring::TDynamicCounters::TCounterPtr CompileErrors; + + TCounters(NMonitoring::TDynamicCounterPtr counters) + : Counters(counters) + { + Register(); + } + + private: + void Register() { + ActiveFilters = Counters->GetCounter("ActiveFilters", false); + InFlightCompileRequests = Counters->GetCounter("InFlightCompileRequests", false); + CompileErrors = Counters->GetCounter("CompileErrors", true); + } + }; + + struct TStats { + void AddFilterLatency(TDuration filterLatency) { + FilterLatency = std::max(FilterLatency, filterLatency); + } + + void Clear() { + FilterLatency = TDuration::Zero(); + } + + TDuration FilterLatency; + }; + + class TFilterHandler { + public: + TFilterHandler(TTopicFilters& self, IFilteredDataConsumer::TPtr consumer, IPurecalcFilter::TPtr purecalcFilter) + : Self(self) + , Consumer(std::move(consumer)) + , PurecalcFilter(std::move(purecalcFilter)) + , LogPrefix(TStringBuilder() << Self.LogPrefix << "TFilterHandler " << Consumer->GetFilterId() << " : ") + {} + + ~TFilterHandler() { + if (InFlightCompilationId) { + Self.Counters.InFlightCompileRequests->Dec(); + } else if (FilterStarted) { + Self.Counters.ActiveFilters->Dec(); + } + } + + IFilteredDataConsumer::TPtr GetConsumer() const { + return Consumer; + } + + IPurecalcFilter::TPtr GetPurecalcFilter() const { + return PurecalcFilter; + } + + bool IsStarted() const { + return FilterStarted; + } + + void CompileFilter() { + if (!PurecalcFilter) { + StartFilter(); + return; + } + + InFlightCompilationId = Self.FreeCompileId++; + Self.Counters.InFlightCompileRequests->Inc(); + Y_ENSURE(Self.InFlightCompilations.emplace(InFlightCompilationId, Consumer->GetFilterId()).second, "Got duplicated compilation event id"); + + LOG_ROW_DISPATCHER_TRACE("Send compile request with id " << InFlightCompilationId); + NActors::TActivationContext::ActorSystem()->Send(new NActors::IEventHandle(Self.Config.CompileServiceId, Self.Owner, PurecalcFilter->GetCompileRequest().release(), 0, InFlightCompilationId)); + } + + void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) { + if (ev->Cookie != InFlightCompilationId) { + LOG_ROW_DISPATCHER_DEBUG("Outdated compiler response ignored for id " << ev->Cookie << ", current compile id " << InFlightCompilationId); + return; + } + + Y_ENSURE(InFlightCompilationId, "Unexpected compilation response"); + InFlightCompilationId = 0; + Self.Counters.InFlightCompileRequests->Dec(); + + if (!ev->Get()->ProgramHolder) { + auto status = TStatus::Fail(ev->Get()->Status, std::move(ev->Get()->Issues)); + LOG_ROW_DISPATCHER_ERROR("Filter compilation error: " << status.GetErrorMessage()); + + Self.Counters.CompileErrors->Inc(); + Consumer->OnFilteringError(status.AddParentIssue("Failed to compile client filter")); + return; + } + + Y_ENSURE(PurecalcFilter, "Unexpected compilation response for client without filter"); + PurecalcFilter->OnCompileResponse(std::move(ev)); + + LOG_ROW_DISPATCHER_TRACE("Filter compilation finished"); + StartFilter(); + } + + private: + void StartFilter() { + FilterStarted = true; + Self.Counters.ActiveFilters->Inc(); + Consumer->OnFilterStarted(); + } + + private: + TTopicFilters& Self; + const IFilteredDataConsumer::TPtr Consumer; + const IPurecalcFilter::TPtr PurecalcFilter; + const TString LogPrefix; + + ui64 InFlightCompilationId = 0; + bool FilterStarted = false; + }; + +public: + explicit TTopicFilters(NActors::TActorId owner, const TTopicFiltersConfig& config, NMonitoring::TDynamicCounterPtr counters) + : Config(config) + , Owner(owner) + , LogPrefix("TTopicFilters: ") + , Counters(counters) + {} + + ~TTopicFilters() { + Filters.clear(); + } + +public: + void FilterData(const TVector& columnIndex, const TVector& offsets, const TVector*>& values, ui64 numberRows) override { + LOG_ROW_DISPATCHER_TRACE("FilterData for " << Filters.size() << " clients, number rows: " << numberRows); + + const TInstant startFilter = TInstant::Now(); + for (const auto& [_, filterHandler] : Filters) { + const auto consumer = filterHandler.GetConsumer(); + if (const auto nextOffset = consumer->GetNextMessageOffset(); !numberRows || (nextOffset && offsets[numberRows - 1] < *nextOffset)) { + LOG_ROW_DISPATCHER_TRACE("Ignore filtering for " << consumer->GetFilterId() << ", historical offset"); + continue; + } + if (!filterHandler.IsStarted()) { + LOG_ROW_DISPATCHER_TRACE("Ignore filtering for " << consumer->GetFilterId() << ", client filter is not compiled"); + continue; + } + + if (filterHandler.GetPurecalcFilter()) { + PushToFilter(filterHandler, offsets, columnIndex, values, numberRows); + continue; + } + + // Clients without filters + LOG_ROW_DISPATCHER_TRACE("Add " << numberRows << " rows to client " << consumer->GetFilterId() << " without filtering"); + for (ui64 rowId = 0; rowId < numberRows; ++rowId) { + consumer->OnFilteredData(rowId); + } + } + Stats.AddFilterLatency(TInstant::Now() - startFilter); + } + + void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) override { + LOG_ROW_DISPATCHER_TRACE("Got compile response for request with id " << ev->Cookie); + + const auto requestIt = InFlightCompilations.find(ev->Cookie); + if (requestIt == InFlightCompilations.end()) { + LOG_ROW_DISPATCHER_DEBUG("Compile response ignored for id " << ev->Cookie); + return; + } + + const auto filterId = requestIt->second; + InFlightCompilations.erase(requestIt); + + const auto filterIt = Filters.find(filterId); + if (filterIt == Filters.end()) { + LOG_ROW_DISPATCHER_DEBUG("Compile response ignored for id " << ev->Cookie << ", filter with id " << filterId << " not found"); + return; + } + + filterIt->second.OnCompileResponse(std::move(ev)); + } + + TStatus AddFilter(IFilteredDataConsumer::TPtr filter) override { + LOG_ROW_DISPATCHER_TRACE("Create filter with id " << filter->GetFilterId()); + + IPurecalcFilter::TPtr purecalcFilter; + if (filter->GetWhereFilter()) { + auto filterStatus = CreatePurecalcFilter(filter); + if (filterStatus.IsFail()) { + return filterStatus; + } + purecalcFilter = filterStatus.DetachResult(); + } + + const auto [it, inserted] = Filters.insert({filter->GetFilterId(), TFilterHandler(*this, filter, std::move(purecalcFilter))}); + if (!inserted) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to create new filter, filter with id " << filter->GetFilterId() << " already exists"); + } + + it->second.CompileFilter(); + return TStatus::Success(); + } + + void RemoveFilter(NActors::TActorId filterId) override { + LOG_ROW_DISPATCHER_TRACE("Remove filter with id " << filterId); + Filters.erase(filterId); + } + + TFiltersStatistic GetStatistics() override { + TFiltersStatistic statistics; + statistics.FilterLatency = Stats.FilterLatency; + Stats.Clear(); + + return statistics; + } + +private: + void PushToFilter(const TFilterHandler& filterHandler, const TVector& offsets, const TVector& columnIndex, const TVector*>& values, ui64 numberRows) { + const auto filter = filterHandler.GetPurecalcFilter(); + Y_ENSURE(filter, "Expected initialized filter"); + + const auto consumer = filterHandler.GetConsumer(); + const auto& columnIds = consumer->GetColumnIds(); + + TVector*> result; + result.reserve(columnIds.size()); + for (ui64 columnId : columnIds) { + Y_ENSURE(columnId < columnIndex.size(), "Unexpected column id " << columnId << ", it is larger than index array size " << columnIndex.size()); + const ui64 index = columnIndex[columnId]; + + Y_ENSURE(index < values.size(), "Unexpected column index " << index << ", it is larger than values array size " << values.size()); + if (const auto value = values[index]) { + result.emplace_back(value); + } else { + LOG_ROW_DISPATCHER_TRACE("Ignore filtering for " << consumer->GetFilterId() << ", client got parsing error for column " << columnId); + return; + } + } + + LOG_ROW_DISPATCHER_TRACE("Pass " << numberRows << " rows to purecalc filter (client id: " << consumer->GetFilterId() << ")"); + filter->FilterData(result, numberRows); + } + +private: + const TTopicFiltersConfig Config; + const NActors::TActorId Owner; + const TString LogPrefix; + + ui64 FreeCompileId = 1; // 0 <=> compilation is not started + std::unordered_map InFlightCompilations; + std::unordered_map Filters; + + // Metrics + const TCounters Counters; + TStats Stats; +}; + +} // anonymous namespace + +ITopicFilters::TPtr CreateTopicFilters(NActors::TActorId owner, const TTopicFiltersConfig& config, NMonitoring::TDynamicCounterPtr counters) { + return MakeIntrusive(owner, config, counters); +} + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/filters/filters_set.h b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/filters_set.h new file mode 100644 index 000000000000..427e73fa53fa --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/filters_set.h @@ -0,0 +1,45 @@ +#pragma once + +#include "purecalc_filter.h" + +#include + +#include + +namespace NFq::NRowDispatcher { + +class IFilteredDataConsumer : public IPurecalcFilterConsumer { +public: + using TPtr = TIntrusivePtr; + +public: + virtual NActors::TActorId GetFilterId() const = 0; + virtual const TVector& GetColumnIds() const = 0; + virtual TMaybe GetNextMessageOffset() const = 0; + + virtual void OnFilterStarted() = 0; + virtual void OnFilteringError(TStatus status) = 0; +}; + +class ITopicFilters : public TThrRefBase, public TNonCopyable { +public: + using TPtr = TIntrusivePtr; + +public: + // columnIndex - mapping from stable column id to index in values array + virtual void FilterData(const TVector& columnIndex, const TVector& offsets, const TVector*>& values, ui64 numberRows) = 0; + virtual void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) = 0; + + virtual TStatus AddFilter(IFilteredDataConsumer::TPtr filter) = 0; + virtual void RemoveFilter(NActors::TActorId filterId) = 0; + + virtual TFiltersStatistic GetStatistics() = 0; +}; + +struct TTopicFiltersConfig { + NActors::TActorId CompileServiceId; +}; + +ITopicFilters::TPtr CreateTopicFilters(NActors::TActorId owner, const TTopicFiltersConfig& config, NMonitoring::TDynamicCounterPtr counters); + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/filters/purecalc_filter.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/purecalc_filter.cpp new file mode 100644 index 000000000000..be4ce2a97d50 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/purecalc_filter.cpp @@ -0,0 +1,346 @@ +#include "purecalc_filter.h" + +#include + +#include +#include +#include + +namespace NFq::NRowDispatcher { + +namespace { + +constexpr const char* OFFSET_FIELD_NAME = "_offset"; + +NYT::TNode CreateTypeNode(const TString& fieldType) { + return NYT::TNode::CreateList() + .Add("DataType") + .Add(fieldType); +} + +void AddField(NYT::TNode& node, const TString& fieldName, const TString& fieldType) { + node.Add( + NYT::TNode::CreateList() + .Add(fieldName) + .Add(CreateTypeNode(fieldType)) + ); +} + +void AddColumn(NYT::TNode& node, const TSchemaColumn& column) { + TString parseTypeError; + TStringOutput errorStream(parseTypeError); + NYT::TNode parsedType; + if (!NYql::NCommon::ParseYson(parsedType, column.TypeYson, errorStream)) { + throw yexception() << "Failed to parse column '" << column.Name << "' type yson " << column.TypeYson << ", error: " << parseTypeError; + } + + node.Add( + NYT::TNode::CreateList() + .Add(column.Name) + .Add(parsedType) + ); +} + +NYT::TNode MakeInputSchema(const TVector& columns) { + auto structMembers = NYT::TNode::CreateList(); + AddField(structMembers, OFFSET_FIELD_NAME, "Uint64"); + for (const auto& column : columns) { + AddColumn(structMembers, column); + } + return NYT::TNode::CreateList().Add("StructType").Add(std::move(structMembers)); +} + +NYT::TNode MakeOutputSchema() { + auto structMembers = NYT::TNode::CreateList(); + AddField(structMembers, OFFSET_FIELD_NAME, "Uint64"); + return NYT::TNode::CreateList().Add("StructType").Add(std::move(structMembers)); +} + +struct TInputType { + const TVector*>& Values; + const ui64 NumberRows; +}; + +class TFilterInputSpec : public NYql::NPureCalc::TInputSpecBase { +public: + explicit TFilterInputSpec(const NYT::TNode& schema) + : Schemas({schema}) + {} + +public: + const TVector& GetSchemas() const override { + return Schemas; + } + +private: + const TVector Schemas; +}; + +class TFilterInputConsumer : public NYql::NPureCalc::IConsumer { +public: + TFilterInputConsumer(const TFilterInputSpec& spec, NYql::NPureCalc::TWorkerHolder worker) + : Worker(std::move(worker)) + { + const NKikimr::NMiniKQL::TStructType* structType = Worker->GetInputType(); + const ui64 count = structType->GetMembersCount(); + + THashMap schemaPositions; + for (ui64 i = 0; i < count; ++i) { + const auto name = structType->GetMemberName(i); + if (name == OFFSET_FIELD_NAME) { + OffsetPosition = i; + } else { + schemaPositions[name] = i; + } + } + + const auto& fields = spec.GetSchemas()[0][1]; + Y_ENSURE(fields.IsList(), "Unexpected input spec type"); + Y_ENSURE(count == fields.Size(), "Unexpected purecalc schema size"); + + FieldsPositions.reserve(count); + for (const auto& field : fields.AsList()) { + const auto& name = field[0].AsString(); + if (name != OFFSET_FIELD_NAME) { + FieldsPositions.emplace_back(schemaPositions[name]); + } + } + } + + ~TFilterInputConsumer() override { + with_lock(Worker->GetScopedAlloc()) { + Cache.Clear(); + } + } + +public: + void OnObject(TInputType input) override { + Y_ENSURE(FieldsPositions.size() == input.Values.size(), "Unexpected input scheme size"); + + NKikimr::NMiniKQL::TThrowingBindTerminator bind; + with_lock (Worker->GetScopedAlloc()) { + Y_DEFER { + // Clear cache after each object because + // values allocated on another allocator and should be released + Cache.Clear(); + Worker->GetGraph().Invalidate(); + }; + + auto& holderFactory = Worker->GetGraph().GetHolderFactory(); + + for (ui64 rowId = 0; rowId < input.NumberRows; ++rowId) { + NYql::NUdf::TUnboxedValue* items = nullptr; + NYql::NUdf::TUnboxedValue result = Cache.NewArray(holderFactory, static_cast(input.Values.size() + 1), items); + + items[OffsetPosition] = NYql::NUdf::TUnboxedValuePod(rowId); + + for (ui64 fieldId = 0; const auto column : input.Values) { + items[FieldsPositions[fieldId++]] = column->at(rowId); + } + + Worker->Push(std::move(result)); + } + } + } + + void OnFinish() override { + NKikimr::NMiniKQL::TBindTerminator bind(Worker->GetGraph().GetTerminator()); + with_lock(Worker->GetScopedAlloc()) { + Worker->OnFinish(); + } + } + +private: + const NYql::NPureCalc::TWorkerHolder Worker; + NKikimr::NMiniKQL::TPlainContainerCache Cache; + + ui64 OffsetPosition = 0; + TVector FieldsPositions; +}; + +class TFilterOutputSpec : public NYql::NPureCalc::TOutputSpecBase { +public: + explicit TFilterOutputSpec(const NYT::TNode& schema) + : Schema(schema) + {} + +public: + const NYT::TNode& GetSchema() const override { + return Schema; + } + +private: + const NYT::TNode Schema; +}; + +class TFilterOutputConsumer : public NYql::NPureCalc::IConsumer { +public: + explicit TFilterOutputConsumer(IPurecalcFilterConsumer::TPtr consumer) + : Consumer(std::move(consumer)) + {} + +public: + void OnObject(ui64 value) override { + Consumer->OnFilteredData(value); + } + + void OnFinish() override { + } + +private: + const IPurecalcFilterConsumer::TPtr Consumer; +}; + +class TFilterPushRelayImpl : public NYql::NPureCalc::IConsumer { +public: + TFilterPushRelayImpl(const TFilterOutputSpec& outputSpec, NYql::NPureCalc::IPushStreamWorker* worker, THolder> underlying) + : Underlying(std::move(underlying)) + , Worker(worker) + { + Y_UNUSED(outputSpec); + } + +public: + void OnObject(const NYql::NUdf::TUnboxedValue* value) override { + Y_ENSURE(value->GetListLength() == 1, "Unexpected output schema size"); + + auto unguard = Unguard(Worker->GetScopedAlloc()); + Underlying->OnObject(value->GetElement(0).Get()); + } + + void OnFinish() override { + auto unguard = Unguard(Worker->GetScopedAlloc()); + Underlying->OnFinish(); + } + +private: + const THolder> Underlying; + NYql::NPureCalc::IWorker* Worker; +}; + +} // anonymous namespace + +} // namespace NFq::NRowDispatcher + +template <> +struct NYql::NPureCalc::TInputSpecTraits { + static constexpr bool IsPartial = false; + static constexpr bool SupportPushStreamMode = true; + + using TConsumerType = THolder>; + + static TConsumerType MakeConsumer(const NFq::NRowDispatcher::TFilterInputSpec& spec, NYql::NPureCalc::TWorkerHolder worker) { + return MakeHolder(spec, std::move(worker)); + } +}; + +template <> +struct NYql::NPureCalc::TOutputSpecTraits { + static const constexpr bool IsPartial = false; + static const constexpr bool SupportPushStreamMode = true; + + static void SetConsumerToWorker(const NFq::NRowDispatcher::TFilterOutputSpec& outputSpec, NYql::NPureCalc::IPushStreamWorker* worker, THolder> consumer) { + worker->SetConsumer(MakeHolder(outputSpec, worker, std::move(consumer))); + } +}; + +namespace NFq::NRowDispatcher { + +namespace { + +class TProgramHolder : public IProgramHolder { +public: + using TPtr = TIntrusivePtr; + +public: + TProgramHolder(IPurecalcFilterConsumer::TPtr consumer, const TString& sql) + : Consumer(consumer) + , Sql(sql) + {} + + NYql::NPureCalc::IConsumer& GetConsumer() { + Y_ENSURE(InputConsumer, "Program is not compiled"); + return *InputConsumer; + } + +public: + void CreateProgram(NYql::NPureCalc::IProgramFactoryPtr programFactory) override { + // Program should be stateless because input values + // allocated on another allocator and should be released + Program = programFactory->MakePushStreamProgram( + TFilterInputSpec(MakeInputSchema(Consumer->GetColumns())), + TFilterOutputSpec(MakeOutputSchema()), + Sql, + NYql::NPureCalc::ETranslationMode::SQL + ); + InputConsumer = Program->Apply(MakeHolder(Consumer)); + } + +private: + const IPurecalcFilterConsumer::TPtr Consumer; + const TString Sql; + + THolder> Program; + THolder> InputConsumer; +}; + +class TPurecalcFilter : public IPurecalcFilter { +public: + TPurecalcFilter(IPurecalcFilterConsumer::TPtr consumer) + : Consumer(consumer) + , PurecalcSettings(consumer->GetPurecalcSettings()) + , LogPrefix("TPurecalcFilter: ") + , ProgramHolder(MakeIntrusive(consumer, GenerateSql())) + {} + +public: + void FilterData(const TVector*>& values, ui64 numberRows) override { + LOG_ROW_DISPATCHER_TRACE("Do filtering for " << numberRows << " rows"); + ProgramHolder->GetConsumer().OnObject({.Values = values, .NumberRows = numberRows}); + } + + std::unique_ptr GetCompileRequest() override { + Y_ENSURE(ProgramHolder, "Can not create compile request twice"); + auto result = std::make_unique(std::move(ProgramHolder), PurecalcSettings); + ProgramHolder = nullptr; + return result; + } + + void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) override { + Y_ENSURE(!ProgramHolder, "Can not handle compile response twice"); + + auto result = static_cast(ev->Get()->ProgramHolder.Release()); + Y_ENSURE(result, "Unexpected compile response"); + + ProgramHolder = TIntrusivePtr(result); + } + +private: + TString GenerateSql() { + TStringStream str; + str << "PRAGMA config.flags(\"LLVM\", \"" << (PurecalcSettings.EnabledLLVM ? "ON" : "OFF") << "\");\n"; + str << "SELECT " << OFFSET_FIELD_NAME << " FROM Input " << Consumer->GetWhereFilter() << ";\n"; + + LOG_ROW_DISPATCHER_DEBUG("Generated sql:\n" << str.Str()); + return str.Str(); + } + +private: + const IPurecalcFilterConsumer::TPtr Consumer; + const TPurecalcCompileSettings PurecalcSettings; + const TString LogPrefix; + + TProgramHolder::TPtr ProgramHolder; +}; + +} // anonymous namespace + +TValueStatus CreatePurecalcFilter(IPurecalcFilterConsumer::TPtr consumer) { + try { + return IPurecalcFilter::TPtr(MakeIntrusive(consumer)); + } catch (...) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to create purecalc filter with predicate '" << consumer->GetWhereFilter() << "', got unexpected exception: " << CurrentExceptionMessage()); + } +} + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/filters/purecalc_filter.h b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/purecalc_filter.h new file mode 100644 index 000000000000..62ad0b7899e4 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/purecalc_filter.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + +#include + +namespace NFq::NRowDispatcher { + +class IPurecalcFilterConsumer : public TThrRefBase { +public: + using TPtr = TIntrusivePtr; + +public: + virtual const TVector& GetColumns() const = 0; + virtual const TString& GetWhereFilter() const = 0; + virtual TPurecalcCompileSettings GetPurecalcSettings() const = 0; + + virtual void OnFilteredData(ui64 rowId) = 0; +}; + +class IPurecalcFilter : public TThrRefBase, public TNonCopyable { +public: + using TPtr = TIntrusivePtr; + +public: + virtual void FilterData(const TVector*>& values, ui64 numberRows) = 0; + + virtual std::unique_ptr GetCompileRequest() = 0; // Should be called exactly once + virtual void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) = 0; +}; + +TValueStatus CreatePurecalcFilter(IPurecalcFilterConsumer::TPtr consumer); + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/filters/ya.make b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/ya.make new file mode 100644 index 000000000000..629c468519f3 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/filters/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +SRCS( + purecalc_filter.cpp + filters_set.cpp +) + +PEERDIR( + ydb/core/fq/libs/actors/logging + ydb/core/fq/libs/row_dispatcher/events + ydb/core/fq/libs/row_dispatcher/format_handler/common + ydb/core/fq/libs/row_dispatcher/purecalc_no_pg_wrapper + + ydb/library/actors/core + + yql/essentials/minikql + yql/essentials/minikql/computation + yql/essentials/minikql/comp_nodes + yql/essentials/minikql/invoke_builtins + yql/essentials/providers/common/schema/parser + yql/essentials/public/udf +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/format_handler.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/format_handler.cpp new file mode 100644 index 000000000000..1a8bdedf4bf1 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/format_handler.cpp @@ -0,0 +1,595 @@ +#include "format_handler.h" + +#include + +#include +#include +#include + +#include + +#include + +namespace NFq::NRowDispatcher { + +namespace { + +class TTopicFormatHandler : public NActors::TActor, public ITopicFormatHandler, public TTypeParser { + using TBase = NActors::TActor; + + struct TCounters { + const NMonitoring::TDynamicCounterPtr CountersRoot; + const NMonitoring::TDynamicCounterPtr CountersSubgroup; + + NMonitoring::TDynamicCounters::TCounterPtr ActiveFormatHandlers; + NMonitoring::TDynamicCounters::TCounterPtr ActiveClients; + + TCounters(NMonitoring::TDynamicCounterPtr counters, const TSettings& settings) + : CountersRoot(counters) + , CountersSubgroup(counters->GetSubgroup("format", settings.ParsingFormat)) + { + Register(); + } + + private: + void Register() { + ActiveFormatHandlers = CountersRoot->GetCounter("ActiveFormatHandlers", false); + + ActiveClients = CountersSubgroup->GetCounter("ActiveClients", false); + } + }; + + struct TColumnDesc { + ui64 ColumnId; // Stable column id, used in TClientHandler + TString TypeYson; + std::unordered_set Clients; // Set of clients with this column + }; + + class TParserHandler : public IParsedDataConsumer { + public: + using TPtr = TIntrusivePtr; + + public: + TParserHandler(TTopicFormatHandler& self, TVector parerSchema) + : Self(self) + , ParerSchema(parerSchema) + , LogPrefix(TStringBuilder() << Self.LogPrefix << "TParserHandler: ") + {} + + public: + const TVector& GetColumns() const override { + return ParerSchema; + } + + void OnParsingError(TStatus status) override { + LOG_ROW_DISPATCHER_ERROR("Got parsing error: " << status.GetErrorMessage()); + Self.FatalError(status); + } + + void OnParsedData(ui64 numberRows) override { + LOG_ROW_DISPATCHER_TRACE("Got parsed data, number rows: " << numberRows); + + Self.ParsedData.assign(ParerSchema.size(), nullptr); + for (size_t i = 0; i < ParerSchema.size(); ++i) { + auto columnStatus = Self.Parser->GetParsedColumn(i); + if (Y_LIKELY(columnStatus.IsSuccess())) { + Self.ParsedData[i] = columnStatus.DetachResult(); + } else { + OnColumnError(i, columnStatus); + } + } + + Self.Offsets = &Self.Parser->GetOffsets(); + Self.FilterData(numberRows); + } + + private: + void OnColumnError(ui64 columnIndex, TStatus status) { + const auto& column = ParerSchema[columnIndex]; + LOG_ROW_DISPATCHER_WARN("Failed to parse column " << column.ToString() << ", " << status.GetErrorMessage()); + + const auto columnIt = Self.ColumnsDesc.find(column.Name); + if (columnIt == Self.ColumnsDesc.end()) { + return; + } + + for (const auto clientId : columnIt->second.Clients) { + const auto clientIt = Self.Clients.find(clientId); + if (clientIt != Self.Clients.end()) { + clientIt->second->OnClientError(status); + } + } + } + + private: + TTopicFormatHandler& Self; + const TVector ParerSchema; + const TString LogPrefix; + }; + + class TClientHandler : public IFilteredDataConsumer { + public: + using TPtr = TIntrusivePtr; + + public: + TClientHandler(TTopicFormatHandler& self, IClientDataConsumer::TPtr client) + : Self(self) + , Client(client) + , Columns(Client->GetColumns()) + , LogPrefix(TStringBuilder() << Self.LogPrefix << "TClientHandler " << Client->GetClientId() << ": ") + , FilteredRow(Columns.size()) + { + ColumnsIds.reserve(Columns.size()); + } + + IClientDataConsumer::TPtr GetClient() const { + return Client; + } + + bool IsClientStarted() const { + return ClientStarted; + } + + TStatus SetupColumns() { + if (Columns.empty()) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, "Client should have at least one column in schema"); + } + + for (const auto& column : Columns) { + const auto it = Self.ColumnsDesc.find(column.Name); + if (it != Self.ColumnsDesc.end()) { + if (it->second.TypeYson != column.TypeYson) { + return TStatus::Fail(EStatusId::SCHEME_ERROR, TStringBuilder() << "Use the same column type in all queries via RD, current type for column `" << column.Name << "` is " << it->second.TypeYson << " (requested type is " << column.TypeYson <<")"); + } + + it->second.Clients.emplace(Client->GetClientId()); + ColumnsIds.emplace_back(it->second.ColumnId); + } else { + ui64 columnId = Self.MaxColumnId; + if (Self.FreeColumnIds) { + columnId = Self.FreeColumnIds.back(); + Self.FreeColumnIds.pop_back(); + } + + Self.ColumnsDesc[column.Name] = TColumnDesc{.ColumnId = columnId, .TypeYson = column.TypeYson, .Clients = {Client->GetClientId()}}; + Self.MaxColumnId = std::max(Self.MaxColumnId, columnId + 1); + ColumnsIds.emplace_back(columnId); + } + } + + return SetupPacker(); + } + + TQueue>> ExtractClientData() { + FinishPacking(); + TQueue>> result; + result.swap(ClientData); + LOG_ROW_DISPATCHER_TRACE("ExtractClientData, number batches: " << result.size()); + return result; + } + + void OnClientError(TStatus status) { + LOG_ROW_DISPATCHER_WARN("OnClientError, " << status.GetErrorMessage()); + Client->OnClientError(std::move(status)); + } + + public: + NActors::TActorId GetFilterId() const override { + return Client->GetClientId(); + } + + const TVector& GetColumns() const override { + return Columns; + } + + const TVector& GetColumnIds() const override { + return ColumnsIds; + } + + TMaybe GetNextMessageOffset() const override { + return Client->GetNextMessageOffset(); + } + + const TString& GetWhereFilter() const override { + return Client->GetWhereFilter(); + } + + TPurecalcCompileSettings GetPurecalcSettings() const override { + return Client->GetPurecalcSettings(); + } + + void OnFilteringError(TStatus status) override { + Client->OnClientError(status); + } + + void OnFilterStarted() override { + ClientStarted = true; + Client->StartClientSession(); + } + + void OnFilteredData(ui64 rowId) override { + const ui64 offset = Self.Offsets->at(rowId); + if (const auto nextOffset = Client->GetNextMessageOffset(); nextOffset && offset < *nextOffset) { + LOG_ROW_DISPATCHER_TRACE("OnFilteredData, skip historical offset: " << offset << ", next message offset: " << *nextOffset); + return; + } + + Y_DEFER { + // Values allocated on parser allocator and should be released + FilteredRow.assign(Columns.size(), NYql::NUdf::TUnboxedValue()); + }; + + for (size_t i = 0; const ui64 columnId : ColumnsIds) { + // All data was locked in parser, so copy is safe + FilteredRow[i++] = Self.ParsedData[Self.ParserSchemaIndex[columnId]]->at(rowId); + } + DataPacker->AddWideItem(FilteredRow.data(), FilteredRow.size()); + FilteredOffsets.emplace_back(offset); + + const ui64 newPackerSize = DataPacker->PackedSizeEstimate(); + LOG_ROW_DISPATCHER_TRACE("OnFilteredData, row id: " << rowId << ", offset: " << offset << ", new packer size: " << newPackerSize); + Client->AddDataToClient(offset, newPackerSize - DataPackerSize); + + DataPackerSize = newPackerSize; + if (DataPackerSize > MAX_BATCH_SIZE) { + FinishPacking(); + } + } + + private: + TStatus SetupPacker() { + TVector columnTypes; + columnTypes.reserve(Columns.size()); + for (const auto& column : Columns) { + auto status = Self.ParseTypeYson(column.TypeYson); + if (status.IsFail()) { + return status; + } + columnTypes.emplace_back(status.DetachResult()); + } + + with_lock(Self.Alloc) { + const auto rowType = Self.ProgramBuilder->NewMultiType(columnTypes); + DataPacker = std::make_unique>(rowType); + } + return TStatus::Success(); + } + + void FinishPacking() { + if (!DataPacker->IsEmpty()) { + LOG_ROW_DISPATCHER_TRACE("FinishPacking, batch size: " << DataPackerSize << ", number rows: " << FilteredOffsets.size()); + ClientData.emplace(NYql::MakeReadOnlyRope(DataPacker->Finish()), FilteredOffsets); + DataPackerSize = 0; + FilteredOffsets.clear(); + } + } + + private: + TTopicFormatHandler& Self; + const IClientDataConsumer::TPtr Client; + const TVector Columns; + const TString LogPrefix; + + TVector ColumnsIds; + bool ClientStarted = false; + + // Filtered data + ui64 DataPackerSize = 0; + TVector FilteredRow; // Temporary value holder for DataPacket + std::unique_ptr> DataPacker; + TVector FilteredOffsets; // Offsets of current batch in DataPacker + TQueue>> ClientData; // vector of (messages batch, [offsets]) + }; + +public: + TTopicFormatHandler(const TFormatHandlerConfig& config, const TSettings& settings, NMonitoring::TDynamicCounterPtr counters) + : TBase(&TTopicFormatHandler::StateFunc) + , TTypeParser(__LOCATION__) + , Config(config) + , Settings(settings) + , LogPrefix(TStringBuilder() << "TTopicFormatHandler [" << Settings.ParsingFormat << "]: ") + , Counters(counters, settings) + { + Counters.ActiveFormatHandlers->Inc(); + } + + ~TTopicFormatHandler() { + Counters.ActiveFormatHandlers->Dec(); + Counters.ActiveClients->Set(0); + + with_lock(Alloc) { + Clients.clear(); + } + } + + STRICT_STFUNC_EXC(StateFunc, + hFunc(TEvRowDispatcher::TEvPurecalcCompileResponse, Handle); + hFunc(NActors::TEvents::TEvWakeup, Handle); + hFunc(NActors::TEvents::TEvPoison, Handle);, + ExceptionFunc(std::exception, HandleException) + ) + + void Handle(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr& ev) { + if (Filters) { + Filters->OnCompileResponse(std::move(ev)); + } + } + + void Handle(NActors::TEvents::TEvWakeup::TPtr&) { + RefreshScheduled = false; + + if (Parser) { + LOG_ROW_DISPATCHER_TRACE("Refresh parser"); + Parser->Refresh(); + ScheduleRefresh(); + } + } + + void Handle(NActors::TEvents::TEvPoison::TPtr&) { + with_lock(Alloc) { + Clients.clear(); + } + PassAway(); + } + + void HandleException(const std::exception& error) { + LOG_ROW_DISPATCHER_ERROR("Got unexpected exception: " << error.what()); + FatalError(TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Format handler error, got unexpected exception: " << error.what())); + } + +public: + void ParseMessages(const TVector& messages) override { + LOG_ROW_DISPATCHER_TRACE("Send " << messages.size() << " messages to parser"); + + if (Parser) { + Parser->ParseMessages(messages); + ScheduleRefresh(); + } else if (!Clients.empty()) { + FatalError(TStatus::Fail(EStatusId::INTERNAL_ERROR, "Failed to parse messages, expected empty clients set without parser")); + } + } + + TQueue>> ExtractClientData(NActors::TActorId clientId) override { + const auto it = Clients.find(clientId); + if (it == Clients.end()) { + return {}; + } + return it->second->ExtractClientData(); + } + + TStatus AddClient(IClientDataConsumer::TPtr client) override { + LOG_ROW_DISPATCHER_DEBUG("Add client with id " << client->GetClientId()); + + auto clientHandler = MakeIntrusive(*this, client); + if (!Clients.emplace(client->GetClientId(), clientHandler).second) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to create new client, client with id " << client->GetClientId() << " already exists"); + } + Counters.ActiveClients->Inc(); + + if (auto status = clientHandler->SetupColumns(); status.IsFail()) { + RemoveClient(client->GetClientId()); + return status.AddParentIssue("Failed to modify common parsing schema"); + } + + if (auto status = UpdateParser(); status.IsFail()) { + RemoveClient(client->GetClientId()); + return status.AddParentIssue("Failed to update parser with new client"); + } + + CreateFilters(); + if (auto status = Filters->AddFilter(clientHandler); status.IsFail()) { + RemoveClient(client->GetClientId()); + return status.AddParentIssue("Failed to create filter for new client"); + } + + return TStatus::Success(); + } + + void RemoveClient(NActors::TActorId clientId) override { + LOG_ROW_DISPATCHER_DEBUG("Remove client with id " << clientId); + + if (Filters) { + Filters->RemoveFilter(clientId); + } + + const auto it = Clients.find(clientId); + if (it == Clients.end()) { + return; + } + + const auto client = it->second->GetClient(); + Counters.ActiveClients->Dec(); + Clients.erase(it); + + for (const auto& column : client->GetColumns()) { + const auto columnIt = ColumnsDesc.find(column.Name); + if (columnIt == ColumnsDesc.end()) { + continue; + } + + columnIt->second.Clients.erase(client->GetClientId()); + if (columnIt->second.Clients.empty()) { + FreeColumnIds.emplace_back(columnIt->second.ColumnId); + ColumnsDesc.erase(columnIt); + } + } + + if (auto status = UpdateParser(); status.IsFail()) { + FatalError(status.AddParentIssue("Failed to update parser after removing client")); + } + } + + bool HasClients() const override { + return !Clients.empty(); + } + + TFormatHandlerStatistic GetStatistics() override { + TFormatHandlerStatistic statistics; + if (Parser) { + Parser->FillStatistics(statistics); + } + if (Filters) { + statistics.FilterStats = Filters->GetStatistics(); + } + return statistics; + } + +protected: + NActors::TActorId GetSelfId() const override { + return SelfId(); + } + +private: + void ScheduleRefresh() { + if (const auto refreshPeriod = Config.JsonParserConfig.LatencyLimit; !RefreshScheduled && refreshPeriod) { + RefreshScheduled = true; + Schedule(refreshPeriod, new NActors::TEvents::TEvWakeup()); + } + } + + TStatus UpdateParser() { + TVector parerSchema; + parerSchema.reserve(ColumnsDesc.size()); + for (const auto& [columnName, columnDesc] : ColumnsDesc) { + parerSchema.emplace_back(TSchemaColumn{.Name = columnName, .TypeYson = columnDesc.TypeYson}); + } + + if (ParserHandler && parerSchema == ParserHandler->GetColumns()) { + return TStatus::Success(); + } + + if (Parser) { + Parser->Refresh(true); + Parser.Reset(); + } + + LOG_ROW_DISPATCHER_DEBUG("UpdateParser to new schema with size " << parerSchema.size()); + ParserHandler = MakeIntrusive(*this, std::move(parerSchema)); + + if (const ui64 schemaSize = ParserHandler->GetColumns().size()) { + auto newParser = CreateParserForFormat(); + if (newParser.IsFail()) { + return newParser; + } + + LOG_ROW_DISPATCHER_DEBUG("Parser was updated on new schema with " << schemaSize << " columns"); + + Parser = newParser.DetachResult(); + ParserSchemaIndex.resize(MaxColumnId, std::numeric_limits::max()); + for (ui64 i = 0; const auto& [_, columnDesc] : ColumnsDesc) { + ParserSchemaIndex[columnDesc.ColumnId] = i++; + } + } else { + LOG_ROW_DISPATCHER_INFO("No columns to parse, reset parser"); + } + + return TStatus::Success(); + } + + TValueStatus CreateParserForFormat() const { + if (Settings.ParsingFormat == "raw") { + return CreateRawParser(ParserHandler); + } + if (Settings.ParsingFormat == "json_each_row") { + return CreateJsonParser(ParserHandler, Config.JsonParserConfig); + } + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Unsupported parsing format: " << Settings.ParsingFormat); + } + + void CreateFilters() { + if (!Filters) { + Filters = CreateTopicFilters(SelfId(), Config.FiltersConfig, Counters.CountersSubgroup); + } + } + + void FilterData(ui64 numberRows) { + if (!numberRows) { + return; + } + + const ui64 lastOffset = Offsets->at(numberRows - 1); + LOG_ROW_DISPATCHER_TRACE("Send " << numberRows << " messages to filters, first offset: " << Offsets->front() << ", last offset: " << lastOffset); + + if (Filters) { + Filters->FilterData(ParserSchemaIndex, *Offsets, ParsedData, numberRows); + } + + for (const auto& [_, client] : Clients) { + if (client->IsClientStarted()) { + LOG_ROW_DISPATCHER_TRACE("Commit client " << client->GetClient()->GetClientId() << " offset " << lastOffset); + client->GetClient()->UpdateClientOffset(lastOffset); + } + } + } + + void FatalError(TStatus status) const { + LOG_ROW_DISPATCHER_ERROR("Got fatal error: " << status.GetErrorMessage()); + for (const auto& [_, client] : Clients) { + client->OnClientError(status); + } + } + +private: + const TFormatHandlerConfig Config; + const TSettings Settings; + const TString LogPrefix; + + // Columns indexes + ui64 MaxColumnId = 0; + TVector FreeColumnIds; + TVector ParserSchemaIndex; // Column id to index in parser schema + std::map ColumnsDesc; + std::unordered_map Clients; + + // Perser and filters + ITopicParser::TPtr Parser; + TParserHandler::TPtr ParserHandler; + ITopicFilters::TPtr Filters; + + // Parsed data + const TVector* Offsets; + TVector*> ParsedData; + bool RefreshScheduled = false; + + // Metrics + const TCounters Counters; +}; + +} // anonymous namespace + +//// ITopicFormatHandler::TDestroy + +void ITopicFormatHandler::TDestroy::Destroy(ITopicFormatHandler* handler) { + if (NActors::TlsActivationContext) { + NActors::TActivationContext::ActorSystem()->Send(handler->GetSelfId(), new NActors::TEvents::TEvPoison()); + } else { + // Destroy from not AS thread my be caused only in case AS destruction (so handler will be deleted) + } +} + +ITopicFormatHandler::TPtr CreateTopicFormatHandler(const NActors::TActorContext& owner, const TFormatHandlerConfig& config, const ITopicFormatHandler::TSettings& settings, NMonitoring::TDynamicCounterPtr counters) { + const auto handler = new TTopicFormatHandler(config, settings, counters); + owner.RegisterWithSameMailbox(handler); + return ITopicFormatHandler::TPtr(handler); +} + +TFormatHandlerConfig CreateFormatHandlerConfig(const NConfig::TRowDispatcherConfig& rowDispatcherConfig, NActors::TActorId compileServiceId) { + return { + .JsonParserConfig = CreateJsonParserConfig(rowDispatcherConfig.GetJsonParser()), + .FiltersConfig = { + .CompileServiceId = compileServiceId + } + }; +} + +namespace NTests { + +ITopicFormatHandler::TPtr CreateTestFormatHandler(const TFormatHandlerConfig& config, const ITopicFormatHandler::TSettings& settings) { + const auto handler = new TTopicFormatHandler(config, settings, MakeIntrusive()); + NActors::TActivationContext::ActorSystem()->Register(handler); + return ITopicFormatHandler::TPtr(handler); +} + +} // namespace NTests + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/format_handler.h b/ydb/core/fq/libs/row_dispatcher/format_handler/format_handler.h new file mode 100644 index 000000000000..809d59b42e07 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/format_handler.h @@ -0,0 +1,78 @@ +#pragma once + +#include +#include + +#include +#include + +namespace NFq::NRowDispatcher { + +static constexpr ui64 MAX_BATCH_SIZE = 10_MB; + +class IClientDataConsumer : public TThrRefBase { +public: + using TPtr = TIntrusivePtr; + +public: + virtual TVector GetColumns() const = 0; + virtual const TString& GetWhereFilter() const = 0; + virtual TPurecalcCompileSettings GetPurecalcSettings() const = 0; + virtual NActors::TActorId GetClientId() const = 0; + virtual TMaybe GetNextMessageOffset() const = 0; + + virtual void OnClientError(TStatus status) = 0; + + virtual void StartClientSession() = 0; + virtual void AddDataToClient(ui64 offset, ui64 rowSize) = 0; + virtual void UpdateClientOffset(ui64 offset) = 0; +}; + +class ITopicFormatHandler : public TNonCopyable { +private: + class TDestroy { + public: + static void Destroy(ITopicFormatHandler* handler); + }; + +public: + using TPtr = THolder; + + struct TSettings { + TString ParsingFormat = "raw"; + + std::strong_ordering operator<=>(const TSettings& other) const = default; + }; + +public: + virtual void ParseMessages(const TVector& messages) = 0; + + // vector of (messages batch, [offsets]) + virtual TQueue>> ExtractClientData(NActors::TActorId clientId) = 0; + + virtual TStatus AddClient(IClientDataConsumer::TPtr client) = 0; + virtual void RemoveClient(NActors::TActorId clientId) = 0; + virtual bool HasClients() const = 0; + + virtual TFormatHandlerStatistic GetStatistics() = 0; + +protected: + virtual NActors::TActorId GetSelfId() const = 0; +}; + +// Static properties for all format handlers +struct TFormatHandlerConfig { + TJsonParserConfig JsonParserConfig; + TTopicFiltersConfig FiltersConfig; +}; + +ITopicFormatHandler::TPtr CreateTopicFormatHandler(const NActors::TActorContext& owner, const TFormatHandlerConfig& config, const ITopicFormatHandler::TSettings& settings, NMonitoring::TDynamicCounterPtr counters); +TFormatHandlerConfig CreateFormatHandlerConfig(const NConfig::TRowDispatcherConfig& rowDispatcherConfig, NActors::TActorId compileServiceId); + +namespace NTests { + +ITopicFormatHandler::TPtr CreateTestFormatHandler(const TFormatHandlerConfig& config, const ITopicFormatHandler::TSettings& settings); + +} // namespace NTests + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/json_parser.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/json_parser.cpp new file mode 100644 index 000000000000..9b241c9635d9 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/json_parser.cpp @@ -0,0 +1,507 @@ +#include "json_parser.h" + +#include "parser_base.h" + +#include + +#include + +#include + +#include +#include +#include +#include + +namespace NFq::NRowDispatcher { + +namespace { + +#define CHECK_JSON_ERROR(value) \ + const simdjson::error_code error = value; \ + if (Y_UNLIKELY(error)) \ + +struct TJsonParserBuffer { + size_t NumberValues = 0; + bool Finished = false; + TInstant CreationStartTime = TInstant::Now(); + TVector Offsets = {}; + + bool IsReady() const { + return !Finished && NumberValues > 0; + } + + size_t GetSize() const { + return Values.size(); + } + + void Reserve(size_t size, size_t numberValues) { + Values.reserve(size + simdjson::SIMDJSON_PADDING); + Offsets.reserve(numberValues); + } + + void AddMessage(const NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage& message) { + Y_ENSURE(!Finished, "Cannot add messages into finished buffer"); + NumberValues++; + Values << message.GetData(); + Offsets.emplace_back(message.GetOffset()); + } + + std::pair Finish() { + Y_ENSURE(!Finished, "Cannot finish buffer twice"); + Finished = true; + Values << TString(simdjson::SIMDJSON_PADDING, ' '); + return {Values.data(), Values.size()}; + } + + void Clear() { + Y_ENSURE(Finished, "Cannot clear not finished buffer"); + NumberValues = 0; + Finished = false; + CreationStartTime = TInstant::Now(); + Values.clear(); + Offsets.clear(); + } + +private: + TStringBuilder Values = {}; +}; + +class TColumnParser { +public: + const std::string Name; // Used for column index by std::string_view + const TString TypeYson; + +public: + TColumnParser(const TString& name, const TString& typeYson, ui64 maxNumberRows) + : Name(name) + , TypeYson(typeYson) + , Status(TStatus::Success()) + { + ParsedRows.reserve(maxNumberRows); + } + + TStatus InitParser(const NKikimr::NMiniKQL::TType* typeMkql) { + return Status = ExtractDataSlot(typeMkql); + } + + const TVector& GetParsedRows() const { + return ParsedRows; + } + + TStatus GetStatus() const { + return Status; + } + + void ParseJsonValue(ui64 offset, ui64 rowId, simdjson::builtin::ondemand::value jsonValue, NYql::NUdf::TUnboxedValue& resultValue) { + if (Y_UNLIKELY(Status.IsFail())) { + return; + } + ParsedRows.emplace_back(rowId); + + if (DataSlot != NYql::NUdf::EDataSlot::Json) { + ParseDataType(std::move(jsonValue), resultValue, Status); + } else { + ParseJsonType(std::move(jsonValue), resultValue, Status); + } + + if (IsOptional && resultValue) { + resultValue = resultValue.MakeOptional(); + } + + if (Y_UNLIKELY(Status.IsFail())) { + Status.AddParentIssue(TStringBuilder() << "Failed to parse json string at offset " << offset << ", got parsing error for column '" << Name << "' with type " << TypeYson); + } + } + + void ValidateNumberValues(size_t expectedNumberValues, ui64 firstOffset) { + if (Status.IsFail()) { + return; + } + if (Y_UNLIKELY(!IsOptional && ParsedRows.size() < expectedNumberValues)) { + Status = TStatus::Fail(EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Failed to parse json messages, found " << expectedNumberValues - ParsedRows.size() << " missing values from offset " << firstOffset << " in non optional column '" << Name << "' with type " << TypeYson); + } + } + + void ClearParsedRows() { + ParsedRows.clear(); + Status = TStatus::Success(); + } + +private: + TStatus ExtractDataSlot(const NKikimr::NMiniKQL::TType* type) { + switch (type->GetKind()) { + case NKikimr::NMiniKQL::TTypeBase::EKind::Data: { + auto slotStatus = GetDataSlot(type); + if (slotStatus.IsFail()) { + return slotStatus; + } + DataSlot = slotStatus.DetachResult(); + DataTypeName = NYql::NUdf::GetDataTypeInfo(DataSlot).Name; + return TStatus::Success(); + } + + case NKikimr::NMiniKQL::TTypeBase::EKind::Optional: { + if (IsOptional) { + return TStatus::Fail(EStatusId::UNSUPPORTED, TStringBuilder() << "Nested optionals is not supported as input type"); + } + IsOptional = true; + return ExtractDataSlot(AS_TYPE(NKikimr::NMiniKQL::TOptionalType, type)->GetItemType()); + } + + default: { + return TStatus::Fail(EStatusId::UNSUPPORTED, TStringBuilder() << "Unsupported type kind: " << type->GetKindAsStr()); + } + } + } + + Y_FORCE_INLINE void ParseDataType(simdjson::builtin::ondemand::value jsonValue, NYql::NUdf::TUnboxedValue& resultValue, TStatus& status) const { + simdjson::builtin::ondemand::json_type cellType; + CHECK_JSON_ERROR(jsonValue.type().get(cellType)) { + return GetParsingError(error, jsonValue, "determine json value type", status); + } + + switch (cellType) { + case simdjson::builtin::ondemand::json_type::number: { + switch (DataSlot) { + case NYql::NUdf::EDataSlot::Int8: + ParseJsonNumber(jsonValue.get_int64(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Int16: + ParseJsonNumber(jsonValue.get_int64(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Int32: + ParseJsonNumber(jsonValue.get_int64(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Int64: + ParseJsonNumber(jsonValue.get_int64(), resultValue, status); + break; + + case NYql::NUdf::EDataSlot::Uint8: + ParseJsonNumber(jsonValue.get_uint64(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Uint16: + ParseJsonNumber(jsonValue.get_uint64(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Uint32: + ParseJsonNumber(jsonValue.get_uint64(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Uint64: + ParseJsonNumber(jsonValue.get_uint64(), resultValue, status); + break; + + case NYql::NUdf::EDataSlot::Double: + ParseJsonDouble(jsonValue.get_double(), resultValue, status); + break; + case NYql::NUdf::EDataSlot::Float: + ParseJsonDouble(jsonValue.get_double(), resultValue, status); + break; + + default: + status = TStatus::Fail(EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Number value is not expected for data type " << DataTypeName); + break; + } + if (Y_UNLIKELY(status.IsFail())) { + status.AddParentIssue(TStringBuilder() << "Failed to parse data type " << DataTypeName << " from json number (raw: '" << TruncateString(jsonValue.raw_json_token()) << "')"); + } + return; + } + + case simdjson::builtin::ondemand::json_type::string: { + std::string_view rawString; + CHECK_JSON_ERROR(jsonValue.get_string(rawString)) { + return GetParsingError(error, jsonValue, "extract json string", status); + } + + resultValue = LockObject(NKikimr::NMiniKQL::ValueFromString(DataSlot, rawString)); + if (Y_UNLIKELY(!resultValue)) { + status = TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse data type " << DataTypeName << " from json string: '" << TruncateString(rawString) << "'"); + } + return; + } + + case simdjson::builtin::ondemand::json_type::array: + case simdjson::builtin::ondemand::json_type::object: { + std::string_view rawJson; + CHECK_JSON_ERROR(jsonValue.raw_json().get(rawJson)) { + return GetParsingError(error, jsonValue, "extract json value", status); + } + status = TStatus::Fail(EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Found unexpected nested value (raw: '" << TruncateString(rawJson) << "'), expected data type " < + Y_FORCE_INLINE static void ParseJsonNumber(simdjson::simdjson_result jsonNumber, NYql::NUdf::TUnboxedValue& resultValue, TStatus& status) { + CHECK_JSON_ERROR(jsonNumber.error()) { + status = TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to extract json integer number, error: " << simdjson::error_message(error)); + return; + } + + TJsonNumber number = jsonNumber.value(); + if (Y_UNLIKELY(number < std::numeric_limits::min() || std::numeric_limits::max() < number)) { + status = TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Number is out of range [" << ToString(std::numeric_limits::min()) << ", " << ToString(std::numeric_limits::max()) << "]"); + return; + } + + resultValue = NYql::NUdf::TUnboxedValuePod(static_cast(number)); + } + + template + Y_FORCE_INLINE static void ParseJsonDouble(simdjson::simdjson_result jsonNumber, NYql::NUdf::TUnboxedValue& resultValue, TStatus& status) { + CHECK_JSON_ERROR(jsonNumber.error()) { + status = TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to extract json float number, error: " << simdjson::error_message(error)); + return; + } + + resultValue = NYql::NUdf::TUnboxedValuePod(static_cast(jsonNumber.value())); + } + + static void GetParsingError(simdjson::error_code error, simdjson::builtin::ondemand::value jsonValue, const TString& description, TStatus& status) { + status = TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to " << description << ", current token: '" << TruncateString(jsonValue.raw_json_token()) << "', error: " << simdjson::error_message(error)); + } + +private: + NYql::NUdf::EDataSlot DataSlot; + TString DataTypeName; + bool IsOptional = false; + + TVector ParsedRows; + TStatus Status; +}; + +class TJsonParser : public TTopicParserBase { +public: + using TBase = TTopicParserBase; + using TPtr = TIntrusivePtr; + +public: + TJsonParser(IParsedDataConsumer::TPtr consumer, const TJsonParserConfig& config) + : TBase(std::move(consumer), __LOCATION__) + , Config(config) + , NumberColumns(Consumer->GetColumns().size()) + , MaxNumberRows((config.BufferCellCount - 1) / NumberColumns + 1) + , LogPrefix("TJsonParser: ") + , ParsedValues(NumberColumns) + { + Columns.reserve(NumberColumns); + for (const auto& column : Consumer->GetColumns()) { + Columns.emplace_back(column.Name, column.TypeYson, MaxNumberRows); + } + + ColumnsIndex.reserve(NumberColumns); + for (size_t i = 0; i < NumberColumns; i++) { + ColumnsIndex.emplace(std::string_view(Columns[i].Name), i); + } + + for (size_t i = 0; i < NumberColumns; i++) { + ParsedValues[i].resize(MaxNumberRows); + } + + Buffer.Reserve(Config.BatchSize, MaxNumberRows); + + LOG_ROW_DISPATCHER_INFO("Simdjson active implementation " << simdjson::get_active_implementation()->name()); + Parser.threaded = false; + } + + TStatus InitColumnsParsers() { + for (auto& column : Columns) { + auto typeStatus = ParseTypeYson(column.TypeYson); + if (typeStatus.IsFail()) { + return TStatus(typeStatus).AddParentIssue(TStringBuilder() << "Failed to parse column '" << column.Name << "' type " << column.TypeYson); + } + if (auto status = column.InitParser(typeStatus.DetachResult()); status.IsFail()) { + return status.AddParentIssue(TStringBuilder() << "Failed to create parser for column '" << column.Name << "' with type " << column.TypeYson); + } + } + return TStatus::Success(); + } + +public: + void ParseMessages(const TVector& messages) override { + LOG_ROW_DISPATCHER_TRACE("Add " << messages.size() << " messages to parse"); + + Y_ENSURE(!Buffer.Finished, "Cannot parse messages with finished buffer"); + for (const auto& message : messages) { + Buffer.AddMessage(message); + if (Buffer.IsReady() && (Buffer.NumberValues >= MaxNumberRows || Buffer.GetSize() >= Config.BatchSize)) { + ParseBuffer(); + } + } + + if (Buffer.IsReady()) { + if (!Config.LatencyLimit) { + ParseBuffer(); + } else { + LOG_ROW_DISPATCHER_TRACE("Collecting data to parse, skip parsing, current buffer size: " << Buffer.GetSize()); + } + } + } + + void Refresh(bool force) override { + TBase::Refresh(force); + + if (!Buffer.IsReady()) { + return; + } + + const auto creationDuration = TInstant::Now() - Buffer.CreationStartTime; + if (force || creationDuration > Config.LatencyLimit) { + ParseBuffer(); + } else { + LOG_ROW_DISPATCHER_TRACE("Refresh, skip parsing, buffer creation duration: " << creationDuration); + } + } + + const TVector& GetOffsets() const override { + return Buffer.Offsets; + } + + TValueStatus*> GetParsedColumn(ui64 columnId) const override { + if (auto status = Columns[columnId].GetStatus(); status.IsFail()) { + return status; + } + return &ParsedValues[columnId]; + } + +protected: + TStatus DoParsing() override { + Y_ENSURE(Buffer.IsReady(), "Nothing to parse"); + Y_ENSURE(Buffer.NumberValues <= MaxNumberRows, "Too many values to parse"); + + const auto [values, size] = Buffer.Finish(); + LOG_ROW_DISPATCHER_TRACE("Do parsing, first offset: " << Buffer.Offsets.front() << ", values:\n" << values); + + simdjson::ondemand::document_stream documents; + CHECK_JSON_ERROR(Parser.iterate_many(values, size, simdjson::ondemand::DEFAULT_BATCH_SIZE).get(documents)) { + return TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse message batch from offset " << Buffer.Offsets.front() << ", json documents was corrupted: " << simdjson::error_message(error)); + } + + size_t rowId = 0; + for (auto document : documents) { + if (Y_UNLIKELY(rowId >= Buffer.NumberValues)) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to parse json messages, expected " << Buffer.NumberValues << " json rows from offset " << Buffer.Offsets.front() << " but got " << rowId + 1); + } + + const ui64 offset = Buffer.Offsets[rowId]; + CHECK_JSON_ERROR(document.error()) { + return TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json message for offset " << offset << ", json document was corrupted: " << simdjson::error_message(error)); + } + + for (auto item : document.get_object()) { + CHECK_JSON_ERROR(item.error()) { + return TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json message for offset " << offset << ", json item was corrupted: " << simdjson::error_message(error)); + } + + const auto it = ColumnsIndex.find(item.escaped_key().value()); + if (it == ColumnsIndex.end()) { + continue; + } + + const size_t columnId = it->second; + Columns[columnId].ParseJsonValue(offset, rowId, item.value(), ParsedValues[columnId][rowId]); + } + rowId++; + } + + if (Y_UNLIKELY(rowId != Buffer.NumberValues)) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to parse json messages, expected " << Buffer.NumberValues << " json rows from offset " << Buffer.Offsets.front() << " but got " << rowId); + } + + const ui64 firstOffset = Buffer.Offsets.front(); + for (auto& column : Columns) { + column.ValidateNumberValues(rowId, firstOffset); + } + + return TStatus::Success(); + } + + void ClearBuffer() override { + for (size_t i = 0; i < Columns.size(); ++i) { + auto& parsedColumn = ParsedValues[i]; + for (size_t rowId : Columns[i].GetParsedRows()) { + ClearObject(parsedColumn[rowId]); + } + Columns[i].ClearParsedRows(); + } + Buffer.Clear(); + } + +private: + const TJsonParserConfig Config; + const ui64 NumberColumns; + const ui64 MaxNumberRows; + const TString LogPrefix; + + TVector Columns; + absl::flat_hash_map ColumnsIndex; + + TJsonParserBuffer Buffer; + simdjson::ondemand::parser Parser; + TVector> ParsedValues; +}; + +} // anonymous namespace + +TValueStatus CreateJsonParser(IParsedDataConsumer::TPtr consumer, const TJsonParserConfig& config) { + TJsonParser::TPtr parser = MakeIntrusive(consumer, config); + if (auto status = parser->InitColumnsParsers(); status.IsFail()) { + return status; + } + + return ITopicParser::TPtr(parser); +} + +TJsonParserConfig CreateJsonParserConfig(const NConfig::TJsonParserConfig& parserConfig) { + TJsonParserConfig result; + if (const auto batchSize = parserConfig.GetBatchSizeBytes()) { + result.BatchSize = batchSize; + } + if (const auto bufferCellCount = parserConfig.GetBufferCellCount()) { + result.BufferCellCount = bufferCellCount; + } + result.LatencyLimit = TDuration::MilliSeconds(parserConfig.GetBatchCreationTimeoutMs()); + return result; +} + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/json_parser.h b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/json_parser.h new file mode 100644 index 000000000000..fe34abb3cd43 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/json_parser.h @@ -0,0 +1,20 @@ +#pragma once + +#include "parser_abstract.h" + +#include + +#include + +namespace NFq::NRowDispatcher { + +struct TJsonParserConfig { + ui64 BatchSize = 1_MB; + TDuration LatencyLimit; + ui64 BufferCellCount = 1000000; // (number rows) * (number columns) limit +}; + +TValueStatus CreateJsonParser(IParsedDataConsumer::TPtr consumer, const TJsonParserConfig& config); +TJsonParserConfig CreateJsonParserConfig(const NConfig::TJsonParserConfig& parserConfig); + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_abstract.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_abstract.cpp new file mode 100644 index 000000000000..696eab2c06fb --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_abstract.cpp @@ -0,0 +1 @@ +#include "parser_abstract.h" diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_abstract.h b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_abstract.h new file mode 100644 index 000000000000..c952ba8e4b56 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_abstract.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include + +#include + +#include + +namespace NFq::NRowDispatcher { + +class IParsedDataConsumer : public TThrRefBase { +public: + using TPtr = TIntrusivePtr; + +public: + virtual const TVector& GetColumns() const = 0; + + virtual void OnParsingError(TStatus status) = 0; + virtual void OnParsedData(ui64 numberRows) = 0; +}; + +class ITopicParser : public TThrRefBase, public TNonCopyable { +public: + using TPtr = TIntrusivePtr; + +public: + virtual void ParseMessages(const TVector& messages) = 0; + virtual void Refresh(bool force = false) = 0; + + virtual const TVector& GetOffsets() const = 0; + virtual TValueStatus*> GetParsedColumn(ui64 columnId) const = 0; + + virtual void FillStatistics(TFormatHandlerStatistic& statistic) = 0; +}; + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_base.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_base.cpp new file mode 100644 index 000000000000..663ece63a433 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_base.cpp @@ -0,0 +1,140 @@ +#include "parser_base.h" + +#include + +#include +#include +#include + +namespace NFq::NRowDispatcher { + +//// TTypeParser + +TTypeParser::TTypeParser(const TSourceLocation& location) + : Alloc(location, NKikimr::TAlignedPagePoolCounters(), true, false) + , FunctionRegistry(NKikimr::NMiniKQL::CreateFunctionRegistry(&PrintBackTrace, NKikimr::NMiniKQL::CreateBuiltinRegistry(), false, {})) + , TypeEnv(std::make_unique(Alloc)) + , ProgramBuilder(std::make_unique(*TypeEnv, *FunctionRegistry)) +{} + +TTypeParser::~TTypeParser() { + with_lock (Alloc) { + FunctionRegistry.Reset(); + TypeEnv.reset(); + ProgramBuilder.reset(); + } +} + +TValueStatus TTypeParser::ParseTypeYson(const TString& typeYson) const { + TString parseTypeError; + TStringOutput errorStream(parseTypeError); + NKikimr::NMiniKQL::TType* typeMkql = nullptr; + + with_lock (Alloc) { + typeMkql = NYql::NCommon::ParseTypeFromYson(TStringBuf(typeYson), *ProgramBuilder, errorStream); + } + + if (!typeMkql) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to parse type from yson: " << parseTypeError); + } + return typeMkql; +} + +//// TTopicParserBase::TStats + +void TTopicParserBase::TStats::AddParserLatency(TDuration parserLatency) { + ParserLatency = std::max(ParserLatency, parserLatency); +} + +void TTopicParserBase::TStats::AddParseAndFilterLatency(TDuration parseAndFilterLatency) { + ParseAndFilterLatency = std::max(ParseAndFilterLatency, parseAndFilterLatency); +} + +void TTopicParserBase::TStats::Clear() { + ParserLatency = TDuration::Zero(); + ParseAndFilterLatency = TDuration::Zero(); +} + +//// TTopicParserBase + +TTopicParserBase::TTopicParserBase(IParsedDataConsumer::TPtr consumer, const TSourceLocation& location) + : TTypeParser(location) + , Consumer(std::move(consumer)) +{} + +void TTopicParserBase::Refresh(bool force) { + Y_UNUSED(force); +} + +void TTopicParserBase::FillStatistics(TFormatHandlerStatistic& statistic) { + statistic.ParseAndFilterLatency = Stats.ParseAndFilterLatency; + statistic.ParserStats.ParserLatency = Stats.ParserLatency; + + Stats.Clear(); +} + +void TTopicParserBase::ParseBuffer() { + const TInstant startParseAndFilter = TInstant::Now(); + try { + Y_DEFER { + with_lock(Alloc) { + ClearBuffer(); + } + }; + + TStatus status = TStatus::Success(); + const TInstant startParse = TInstant::Now(); + with_lock(Alloc) { + status = DoParsing(); + } + Stats.AddParserLatency(TInstant::Now() - startParse); + + if (status.IsSuccess()) { + Consumer->OnParsedData(GetOffsets().size()); + } else { + Consumer->OnParsingError(status); + } + } catch (...) { + Consumer->OnParsingError(TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to parse messages from offset " << GetOffsets().front() << ", got unexpected exception: " << CurrentExceptionMessage())); + } + Stats.AddParseAndFilterLatency(TInstant::Now() - startParseAndFilter); +} + +//// Functions + +NYql::NUdf::TUnboxedValue LockObject(NYql::NUdf::TUnboxedValue&& value) { + // All UnboxedValue's with type Boxed or String should be locked + // because after parsing they will be used under another MKQL allocator in purecalc filters + + const i32 numberRefs = value.LockRef(); + + // -1 - value is embbeded or empty, otherwise value should have exactly one ref + Y_ENSURE(numberRefs == -1 || numberRefs == 1); + + return value; +} + +void ClearObject(NYql::NUdf::TUnboxedValue& value) { + // Value should be unlocked with same number of refs + value.UnlockRef(1); + value.Clear(); +} + +TValueStatus GetDataSlot(const NKikimr::NMiniKQL::TType* type) { + Y_ENSURE(type->GetKind() == NKikimr::NMiniKQL::TTypeBase::EKind::Data, "Expected data type"); + + const auto* dataType = AS_TYPE(NKikimr::NMiniKQL::TDataType, type); + if (const auto dataSlot = dataType->GetDataSlot()) { + return *dataSlot; + } + return TStatus::Fail(EStatusId::UNSUPPORTED, TStringBuilder() << "Unsupported data type with id: " << dataType->GetSchemeType()); +} + +TString TruncateString(std::string_view rawString, size_t maxSize) { + if (rawString.size() <= maxSize) { + return TString(rawString); + } + return TStringBuilder() << rawString.substr(0, maxSize) << " truncated..."; +} + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_base.h b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_base.h new file mode 100644 index 000000000000..f6d47268e508 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/parser_base.h @@ -0,0 +1,68 @@ +#pragma once + +#include "parser_abstract.h" + +#include +#include + +namespace NFq::NRowDispatcher { + +class TTypeParser { +public: + explicit TTypeParser(const TSourceLocation& location); + virtual ~TTypeParser(); + + TValueStatus ParseTypeYson(const TString& typeYson) const; + +protected: + NKikimr::NMiniKQL::TScopedAlloc Alloc; + NKikimr::NMiniKQL::IFunctionRegistry::TPtr FunctionRegistry; + std::unique_ptr TypeEnv; + std::unique_ptr ProgramBuilder; +}; + +class TTopicParserBase : public ITopicParser, public TTypeParser { +private: + struct TStats { + void AddParserLatency(TDuration parserLatency); + void AddParseAndFilterLatency(TDuration parseAndFilterLatency); + void Clear(); + + TDuration ParserLatency; + TDuration ParseAndFilterLatency; + }; + +public: + using TPtr = TIntrusivePtr; + +public: + TTopicParserBase(IParsedDataConsumer::TPtr consumer, const TSourceLocation& location); + virtual ~TTopicParserBase() = default; + +public: + virtual void Refresh(bool force = false) override; + virtual void FillStatistics(TFormatHandlerStatistic& statistic) override; + +protected: + // Called with binded alloc + virtual TStatus DoParsing() = 0; + virtual void ClearBuffer() = 0; + +protected: + void ParseBuffer(); + +protected: + const IParsedDataConsumer::TPtr Consumer; + +private: + TStats Stats; +}; + +NYql::NUdf::TUnboxedValue LockObject(NYql::NUdf::TUnboxedValue&& value); +void ClearObject(NYql::NUdf::TUnboxedValue& value); + +TValueStatus GetDataSlot(const NKikimr::NMiniKQL::TType* type); + +TString TruncateString(std::string_view rawString, size_t maxSize = 1_KB); + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/raw_parser.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/raw_parser.cpp new file mode 100644 index 000000000000..b230b8dd0a50 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/raw_parser.cpp @@ -0,0 +1,125 @@ +#include "raw_parser.h" + +#include "parser_base.h" + +#include + +#include +#include + +namespace NFq::NRowDispatcher { + +namespace { + +class TRawParser : public TTopicParserBase { +public: + using TBase = TTopicParserBase; + using TPtr = TIntrusivePtr; + +public: + TRawParser(IParsedDataConsumer::TPtr consumer, const TSchemaColumn& schema) + : TBase(std::move(consumer), __LOCATION__) + , Schema(schema) + , LogPrefix("TRawParser: ") + {} + + TStatus InitColumnParser() { + auto typeStatus = ParseTypeYson(Schema.TypeYson); + if (typeStatus.IsFail()) { + return typeStatus; + } + + for (NKikimr::NMiniKQL::TType* type = typeStatus.DetachResult(); true; type = AS_TYPE(NKikimr::NMiniKQL::TOptionalType, type)->GetItemType()) { + if (type->GetKind() == NKikimr::NMiniKQL::TTypeBase::EKind::Data) { + auto slotStatus = GetDataSlot(type); + if (slotStatus.IsFail()) { + return slotStatus; + } + DataSlot = slotStatus.DetachResult(); + return TStatus::Success(); + } + + if (type->GetKind() != NKikimr::NMiniKQL::TTypeBase::EKind::Optional) { + return TStatus::Fail(EStatusId::UNSUPPORTED, TStringBuilder() << "Unsupported type kind for raw format: " << type->GetKindAsStr()); + } + + NumberOptionals++; + } + + return TStatus::Success(); + } + +public: + void ParseMessages(const TVector& messages) override { + LOG_ROW_DISPATCHER_TRACE("Add " << messages.size() << " messages to parse"); + + for (const auto& message : messages) { + CurrentMessage = message.GetData(); + Offsets.emplace_back(message.GetOffset()); + ParseBuffer(); + } + } + + const TVector& GetOffsets() const override { + return Offsets; + } + + TValueStatus*> GetParsedColumn(ui64 columnId) const override { + Y_ENSURE(columnId == 0, "Invalid column id for raw parser"); + return &ParsedColumn; + } + +protected: + TStatus DoParsing() override { + LOG_ROW_DISPATCHER_TRACE("Do parsing, first offset: " << Offsets.front() << ", value: " << CurrentMessage); + + NYql::NUdf::TUnboxedValue value = LockObject(NKikimr::NMiniKQL::ValueFromString(DataSlot, CurrentMessage)); + if (value) { + for (size_t i = 0; i < NumberOptionals; ++i) { + value = value.MakeOptional(); + } + } else if (!NumberOptionals) { + return TStatus::Fail(EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse massege at offset " << Offsets.back() << ", can't parse data type " << NYql::NUdf::GetDataTypeInfo(DataSlot).Name << " from string: '" << TruncateString(CurrentMessage) << "'"); + } + + ParsedColumn.emplace_back(std::move(value)); + return TStatus::Success(); + } + + void ClearBuffer() override { + for (auto& parsedValue : ParsedColumn) { + ClearObject(parsedValue); + } + ParsedColumn.clear(); + Offsets.clear(); + } + +private: + const TSchemaColumn Schema; + const TString LogPrefix; + + NYql::NUdf::EDataSlot DataSlot; + ui64 NumberOptionals = 0; + + TString CurrentMessage; + TVector Offsets; + TVector ParsedColumn; +}; + +} // anonymous namespace + +TValueStatus CreateRawParser(IParsedDataConsumer::TPtr consumer) { + const auto& columns = consumer->GetColumns(); + if (columns.size() != 1) { + return TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Expected only one column for raw format, but got " << columns.size()); + } + + TRawParser::TPtr parser = MakeIntrusive(consumer, columns[0]); + if (auto status = parser->InitColumnParser(); status.IsFail()) { + return status.AddParentIssue(TStringBuilder() << "Failed to create raw parser for column " << columns[0].ToString()); + } + + return ITopicParser::TPtr(parser); +} + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/raw_parser.h b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/raw_parser.h new file mode 100644 index 000000000000..f920543a13bc --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/raw_parser.h @@ -0,0 +1,9 @@ +#pragma once + +#include "parser_abstract.h" + +namespace NFq::NRowDispatcher { + +TValueStatus CreateRawParser(IParsedDataConsumer::TPtr consumer); + +} // namespace NFq::NRowDispatcher diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/ya.make b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/ya.make new file mode 100644 index 000000000000..ec51348a980c --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/parsers/ya.make @@ -0,0 +1,33 @@ +LIBRARY() + +SRCS( + parser_abstract.cpp + parser_base.cpp + json_parser.cpp + raw_parser.cpp +) + +PEERDIR( + contrib/libs/simdjson + + library/cpp/containers/absl_flat_hash + + ydb/core/fq/libs/actors/logging + ydb/core/fq/libs/row_dispatcher/events + ydb/core/fq/libs/row_dispatcher/format_handler/common + + ydb/public/sdk/cpp/client/ydb_topic/include + + yql/essentials/minikql + yql/essentials/minikql/dom + yql/essentials/minikql/invoke_builtins + yql/essentials/providers/common/schema +) + +CFLAGS( + -Wno-assume +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ut_common.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ut_common.cpp new file mode 100644 index 000000000000..d7b3812f87c6 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ut_common.cpp @@ -0,0 +1,251 @@ +#include "ut_common.h" + +#include + +#include + +#include + +#include + +#include +#include + +namespace NFq::NRowDispatcher::NTests { + +namespace { + +class TPurecalcCompileServiceMock : public NActors::TActor { + using TBase = NActors::TActor; + +public: + TPurecalcCompileServiceMock(NActors::TActorId owner) + : TBase(&TPurecalcCompileServiceMock::StateFunc) + , Owner(owner) + , ProgramFactory(NYql::NPureCalc::MakeProgramFactory()) + {} + + STRICT_STFUNC(StateFunc, + hFunc(TEvRowDispatcher::TEvPurecalcCompileRequest, Handle); + ) + + void Handle(TEvRowDispatcher::TEvPurecalcCompileRequest::TPtr& ev) { + IProgramHolder::TPtr programHolder = std::move(ev->Get()->ProgramHolder); + + try { + programHolder->CreateProgram(ProgramFactory); + } catch (const NYql::NPureCalc::TCompileError& e) { + UNIT_FAIL("Failed to compile purecalc filter: sql: " << e.GetYql() << ", error: " << e.GetIssues()); + } + + Send(ev->Sender, new TEvRowDispatcher::TEvPurecalcCompileResponse(std::move(programHolder)), 0, ev->Cookie); + Send(Owner, new NActors::TEvents::TEvPing()); + } + +private: + const NActors::TActorId Owner; + const NYql::NPureCalc::IProgramFactoryPtr ProgramFactory; +}; + +void SegmentationFaultHandler(int) { + Cerr << "segmentation fault call stack:" << Endl; + FormatBackTrace(&Cerr); + abort(); +} + +//// TBaseFixture::ICell + +class TOptionalCell : public TBaseFixture::ICell { +public: + TOptionalCell(ICell::TPtr value) + : Value(value) + {} + +public: + TString GetType() const override { + return TStringBuilder() << "[OptionalType; " << Value->GetType() << "]"; + } + + void Validate(const NYql::NUdf::TUnboxedValue& parsedValue) const override { + if (!parsedValue) { + return; + } + Value->Validate(parsedValue.GetOptionalValue()); + } + +private: + const ICell::TPtr Value; +}; + +class TStringSell : public TBaseFixture::ICell { +public: + TStringSell(const TString& value) + : Value(value) + {} + +public: + TString GetType() const override { + return "[DataType; String]"; + } + + void Validate(const NYql::NUdf::TUnboxedValue& parsedValue) const override { + UNIT_ASSERT_VALUES_EQUAL(Value, TString(parsedValue.AsStringRef())); + } + +private: + const TString Value; +}; + +class TUint64Sell : public TBaseFixture::ICell { +public: + TUint64Sell(ui64 value) + : Value(value) + {} + +public: + TString GetType() const override { + return "[DataType; Uint64]"; + } + + void Validate(const NYql::NUdf::TUnboxedValue& parsedValue) const override { + UNIT_ASSERT_VALUES_EQUAL(Value, parsedValue.Get()); + } + +private: + const ui64 Value; +}; + +} // anonymous namespace + +//// TBaseFixture::TRow + +TBaseFixture::TRow& TBaseFixture::TRow::AddCell(ICell::TPtr cell, bool optional) { + if (optional) { + cell = MakeIntrusive(cell); + } + + Cells.emplace_back(cell); + return *this; +} + +TBaseFixture::TRow& TBaseFixture::TRow::AddString(const TString& value, bool optional) { + return AddCell(MakeIntrusive(value), optional); +} + +TBaseFixture::TRow& TBaseFixture::TRow::AddUint64(ui64 value, bool optional) { + return AddCell(MakeIntrusive(value), optional); +} + +//// TBaseFixture::TBatch + +TBaseFixture::TBatch::TBatch(std::initializer_list rows) + : Rows(rows) +{} + +TBaseFixture::TBatch& TBaseFixture::TBatch::AddRow(TRow row) { + Rows.emplace_back(row); + return *this; +} + +//// TBaseFixture + +TBaseFixture::TBaseFixture() + : TTypeParser(__LOCATION__) + , MemoryInfo("TBaseFixture alloc") + , HolderFactory(std::make_unique(Alloc.Ref(), MemoryInfo)) + , Runtime(1) +{ + NKikimr::EnableYDBBacktraceFormat(); + signal(SIGSEGV, &SegmentationFaultHandler); + + Alloc.Ref().UseRefLocking = true; +} + +void TBaseFixture::SetUp(NUnitTest::TTestContext&) { + // Init runtime + TAutoPtr app = new NKikimr::TAppPrepare(); + Runtime.SetLogBackend(NActors::CreateStderrBackend()); + Runtime.SetLogPriority(NKikimrServices::FQ_ROW_DISPATCHER, NActors::NLog::PRI_TRACE); + Runtime.SetDispatchTimeout(TDuration::Seconds(5)); + Runtime.Initialize(app->Unwrap()); + + // Init tls context + auto* actorSystem = Runtime.GetActorSystem(0); + Mailbox = std::make_unique(); + ExecutorThread = std::make_unique(0, actorSystem, nullptr, nullptr, "test thread"); + ActorCtx = std::make_unique(*Mailbox, *ExecutorThread, GetCycleCountFast(), NActors::TActorId()); + PrevActorCtx = NActors::TlsActivationContext; + NActors::TlsActivationContext = ActorCtx.get(); +} + +void TBaseFixture::TearDown(NUnitTest::TTestContext&) { + with_lock(Alloc) { + ProgramBuilder.reset(); + TypeEnv.reset(); + FunctionRegistry.Reset(); + HolderFactory.reset(); + } + + // Release tls context + NActors::TlsActivationContext = PrevActorCtx; + PrevActorCtx = nullptr; +} + +void TBaseFixture::CheckMessageBatch(TRope serializedBatch, const TBatch& expectedBatch) const { + const auto& expectedRows = expectedBatch.Rows; + UNIT_ASSERT_C(!expectedRows.empty(), "Expected batch should not be empty"); + + // Parse row type (take first row for infer) + const auto& expectedFirstRow = expectedRows.front().Cells; + TVector columnTypes; + columnTypes.reserve(expectedFirstRow.size()); + for (const auto& cell : expectedFirstRow) { + columnTypes.emplace_back(CheckSuccess(ParseTypeYson(cell->GetType()))); + } + + with_lock(Alloc) { + // Parse messages + const auto rowType = ProgramBuilder->NewMultiType(columnTypes); + const auto dataUnpacker = std::make_unique>(rowType); + + NKikimr::NMiniKQL::TUnboxedValueBatch parsedData(rowType); + dataUnpacker->UnpackBatch(NYql::MakeChunkedBuffer(std::move(serializedBatch)), *HolderFactory, parsedData); + + // Validate data + UNIT_ASSERT_VALUES_EQUAL(parsedData.RowCount(), expectedRows.size()); + UNIT_ASSERT_VALUES_EQUAL(parsedData.Width(), expectedFirstRow.size()); + + ui64 rowIndex = 0; + parsedData.ForEachRowWide([&](NYql::NUdf::TUnboxedValue* values, ui32 width){ + UNIT_ASSERT_GE(expectedRows.size(), rowIndex + 1); + const auto& expectedRow = expectedRows[rowIndex++].Cells; + + UNIT_ASSERT_VALUES_EQUAL(width, expectedRow.size()); + for (ui32 i = 0; i < width; ++i) { + expectedRow[i]->Validate(values[i]); + } + }); + } +} + +NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage TBaseFixture::GetMessage(ui64 offset, const TString& data) { + NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessageInformation info(offset, "", 0, TInstant::Zero(), TInstant::Zero(), nullptr, nullptr, 0, ""); + return NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage(data, nullptr, info, nullptr); +} + +//// Functions + +NActors::IActor* CreatePurecalcCompileServiceMock(NActors::TActorId owner) { + return new TPurecalcCompileServiceMock(owner); +} + +void CheckSuccess(const TStatus& status) { + UNIT_ASSERT_C(status.IsSuccess(), "Status is not success, " << status.GetErrorMessage()); +} + +void CheckError(const TStatus& status, TStatusCode expectedStatusCode, const TString& expectedMessage) { + UNIT_ASSERT_C(status.GetStatus() == expectedStatusCode, "Expected error status " << NYql::NDqProto::StatusIds_StatusCode_Name(expectedStatusCode) << ", but got: " << status.GetErrorMessage()); + UNIT_ASSERT_STRING_CONTAINS_C(status.GetErrorMessage(), expectedMessage, "Unexpected error message, Status: " << NYql::NDqProto::StatusIds_StatusCode_Name(status.GetStatus())); +} + +} // namespace NFq::NRowDispatcher::NTests diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ut_common.h b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ut_common.h new file mode 100644 index 000000000000..dd1c3d1c9fe9 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ut_common.h @@ -0,0 +1,87 @@ +#pragma once + +#include + +#include + +#include + +#include + +namespace NFq::NRowDispatcher::NTests { + +class TBaseFixture : public NUnitTest::TBaseFixture, public TTypeParser { +public: + // Helper classes for checking serialized rows in multi type format + class ICell : public TThrRefBase { + public: + using TPtr = TIntrusivePtr; + + public: + virtual TString GetType() const = 0; + virtual void Validate(const NYql::NUdf::TUnboxedValue& parsedValue) const = 0; + }; + + class TRow { + public: + TRow() = default; + + TRow& AddCell(ICell::TPtr cell, bool optional); + + TRow& AddString(const TString& value, bool optional = false); + TRow& AddUint64(ui64 value, bool optional = false); + + public: + TVector Cells; + }; + + class TBatch { + public: + TBatch() = default; + TBatch(std::initializer_list rows); + + TBatch& AddRow(TRow row); + + public: + TVector Rows; + }; + +public: + TBaseFixture(); + +public: + virtual void SetUp(NUnitTest::TTestContext& ctx) override; + virtual void TearDown(NUnitTest::TTestContext& ctx) override; + +public: + void CheckMessageBatch(TRope serializedBatch, const TBatch& expectedBatch) const; + + static NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage GetMessage(ui64 offset, const TString& data); + +public: + NKikimr::NMiniKQL::TMemoryUsageInfo MemoryInfo; + std::unique_ptr HolderFactory; + NActors::TTestActorRuntime Runtime; + +private: + // Like NKikimr::TActorSystemStub but with Runtime as actor system in tls context + // it enables logging in unit test thread + // and using NActors::TActivationContext::ActorSystem() method + std::unique_ptr Mailbox; + std::unique_ptr ExecutorThread; + std::unique_ptr ActorCtx; + NActors::TActivationContext* PrevActorCtx; +}; + +NActors::IActor* CreatePurecalcCompileServiceMock(NActors::TActorId owner); + +void CheckSuccess(const TStatus& status); +void CheckError(const TStatus& status, TStatusCode expectedStatusCode, const TString& expectedMessage); + +template +TValue CheckSuccess(TValueStatus valueStatus) { + UNIT_ASSERT_C(valueStatus.IsSuccess(), "Value status is not success, " << valueStatus.GetErrorMessage()); + return valueStatus.DetachResult(); +} + +} // namespace NFq::NRowDispatcher::NTests diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ya.make b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ya.make new file mode 100644 index 000000000000..11a0802e50b6 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/common/ya.make @@ -0,0 +1,29 @@ +LIBRARY() + +SRCS( + ut_common.cpp +) + +PEERDIR( + library/cpp/testing/unittest + + ydb/core/base + + ydb/core/fq/libs/row_dispatcher/format_handler/common + + ydb/core/testlib + ydb/core/testlib/actors + ydb/core/testlib/basics + + ydb/library/yql/dq/actors + ydb/library/yql/dq/common + + yql/essentials/minikql + yql/essentials/minikql/computation + yql/essentials/minikql/invoke_builtins + yql/essentials/providers/common/schema/mkql +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/format_handler_ut.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/format_handler_ut.cpp new file mode 100644 index 000000000000..c643dba27026 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/format_handler_ut.cpp @@ -0,0 +1,384 @@ +#include +#include + +namespace NFq::NRowDispatcher::NTests { + +namespace { + +class TFormatHadlerFixture : public TBaseFixture { +public: + using TBase = TBaseFixture; + using TCallback = std::function>>&& data)>; + + class TClientDataConsumer : public IClientDataConsumer { + public: + using TPtr = TIntrusivePtr; + + public: + TClientDataConsumer(NActors::TActorId clientId, const TVector& columns, const TString& whereFilter, TCallback callback, ui64 expectedFilteredRows = 0) + : Callback(callback) + , ClientId(clientId) + , Columns(columns) + , WhereFilter(whereFilter) + , ExpectedFilteredRows(expectedFilteredRows) + {} + + void Freeze() { + Frozen = true; + } + + void ExpectOffsets(const TVector expectedOffsets) { + if (Frozen) { + return; + } + for (const ui64 offset : expectedOffsets) { + Offsets.emplace(offset); + } + HasData = false; + } + + void ExpectError(TStatusCode statusCode, const TString& message) { + UNIT_ASSERT_C(!ExpectedError, "Can not add existing error, client id: " << ClientId); + ExpectedError = {statusCode, message}; + } + + void Validate() const { + UNIT_ASSERT_C(Offsets.empty(), "Found " << Offsets.size() << " missing batches, client id: " << ClientId); + UNIT_ASSERT_VALUES_EQUAL_C(ExpectedFilteredRows, 0, "Found " << ExpectedFilteredRows << " not filtered rows, client id: " << ClientId); + UNIT_ASSERT_C(!ExpectedError, "Expected error: " << ExpectedError->second << ", client id: " << ClientId); + } + + bool IsStarted() const { + return Started; + } + + bool IsFinished() const { + return Frozen || !HasData; + } + + public: + TVector GetColumns() const override { + return Columns; + } + + const TString& GetWhereFilter() const override { + return WhereFilter; + } + + TPurecalcCompileSettings GetPurecalcSettings() const override { + return {.EnabledLLVM = false}; + } + + virtual NActors::TActorId GetClientId() const override { + return ClientId; + } + + TMaybe GetNextMessageOffset() const override { + return Nothing(); + } + + void OnClientError(TStatus status) override { + UNIT_ASSERT_C(!Offsets.empty(), "Unexpected message batch, status: " << status.GetErrorMessage() << ", client id: " << ClientId); + + if (ExpectedError) { + CheckError(status, ExpectedError->first, ExpectedError->second); + ExpectedError = std::nullopt; + } else { + CheckSuccess(status); + } + } + + void StartClientSession() override { + Started = true; + } + + void AddDataToClient(ui64 offset, ui64 rowSize) override { + UNIT_ASSERT_C(Started, "Unexpected data for not started session"); + UNIT_ASSERT_C(rowSize >= 0, "Expected non zero row size"); + UNIT_ASSERT_C(!ExpectedError, "Expected error: " << ExpectedError->second << ", client id: " << ClientId); + UNIT_ASSERT_C(ExpectedFilteredRows > 0, "Too many rows filtered, client id: " << ClientId); + UNIT_ASSERT_C(!Offsets.empty(), "Unexpected message batch, offset: " << offset << ", client id: " << ClientId); + ExpectedFilteredRows--; + HasData = true; + } + + void UpdateClientOffset(ui64 offset) override { + UNIT_ASSERT_C(Started, "Unexpected offset for not started session"); + UNIT_ASSERT_C(!ExpectedError, "Error is not handled: " << ExpectedError->second << ", client id: " << ClientId); + UNIT_ASSERT_C(!Offsets.empty(), "Unexpected message batch, offset: " << offset << ", client id: " << ClientId); + UNIT_ASSERT_VALUES_EQUAL_C(Offsets.front(), offset, "Unexpected commit offset, client id: " << ClientId); + Offsets.pop(); + } + + public: + const TCallback Callback; + + private: + const NActors::TActorId ClientId; + const TVector Columns; + const TString WhereFilter; + + bool Started = false; + bool HasData = false; + bool Frozen = false; + ui64 ExpectedFilteredRows = 0; + std::queue Offsets; + std::optional> ExpectedError; + }; + +public: + void SetUp(NUnitTest::TTestContext& ctx) override { + TBase::SetUp(ctx); + + CompileNotifier = Runtime.AllocateEdgeActor(); + CompileService = Runtime.Register(CreatePurecalcCompileServiceMock(CompileNotifier)); + + CreateFormatHandler({ + .JsonParserConfig = {}, + .FiltersConfig = {.CompileServiceId = CompileService} + }); + } + + void TearDown(NUnitTest::TTestContext& ctx) override { + for (const auto& client : Clients) { + client->Validate(); + } + ClientIds.clear(); + Clients.clear(); + FormatHandler.Reset(); + + TBase::TearDown(ctx); + } + +public: + void CreateFormatHandler(const TFormatHandlerConfig& config, ITopicFormatHandler::TSettings settings = {.ParsingFormat = "json_each_row"}) { + FormatHandler = CreateTestFormatHandler(config, settings); + } + + TStatus MakeClient(const TVector& columns, const TString& whereFilter, TCallback callback, ui64 expectedFilteredRows = 1) { + ClientIds.emplace_back(ClientIds.size(), 0, 0, 0); + + auto client = MakeIntrusive(ClientIds.back(), columns, whereFilter, callback, expectedFilteredRows); + auto status = FormatHandler->AddClient(client); + if (status.IsFail()) { + return status; + } + + Clients.emplace_back(client); + if (!client->IsStarted()) { + // Wait filter compilation + const auto response = Runtime.GrabEdgeEvent(CompileNotifier, TDuration::Seconds(5)); + UNIT_ASSERT_C(response, "Compilation is not performed for filter: " << whereFilter); + } + + return TStatus::Success(); + } + + void ParseMessages(const TVector& messages, TVector expectedOffsets = {}) { + for (auto& client : Clients) { + client->ExpectOffsets(expectedOffsets ? expectedOffsets : TVector{messages.back().GetOffset()}); + } + FormatHandler->ParseMessages(messages); + ExtractClientsData(); + } + + void CheckClientError(const TVector& messages, NActors::TActorId clientId, TStatusCode statusCode, const TString& message) { + for (auto& client : Clients) { + client->ExpectOffsets({messages.back().GetOffset()}); + if (client->GetClientId() == clientId) { + client->ExpectError(statusCode, message); + } + } + FormatHandler->ParseMessages(messages); + ExtractClientsData(); + } + + void RemoveClient(NActors::TActorId clientId) { + for (auto& client : Clients) { + if (client->GetClientId() == clientId) { + client->Freeze(); + } + } + FormatHandler->RemoveClient(clientId); + } + +private: + void ExtractClientsData() { + for (auto& client : Clients) { + auto data = FormatHandler->ExtractClientData(client->GetClientId()); + if (client->IsFinished()) { + UNIT_ASSERT_VALUES_EQUAL_C(data.size(), 0, "Expected empty data for finished clients"); + } else { + client->Callback(std::move(data)); + } + } + } + +public: + NActors::TActorId CompileNotifier; + NActors::TActorId CompileService; + TVector ClientIds; + TVector Clients; + ITopicFormatHandler::TPtr FormatHandler; +}; + +} // anonymous namespace + + +Y_UNIT_TEST_SUITE(TestFormatHandler) { + Y_UNIT_TEST_F(ManyJsonClients, TFormatHadlerFixture) { + const ui64 firstOffset = 42; + const TSchemaColumn commonColumn = {"com_col", "[DataType; String]"}; + + CheckSuccess(MakeClient( + {commonColumn, {"col_first", "[DataType; String]"}}, + "WHERE col_first = \"str_first__large__\"", + [&](TQueue>>&& data) { + UNIT_ASSERT_VALUES_EQUAL(data.size(), 1); + + auto [messages, offsets] = data.front(); + UNIT_ASSERT_VALUES_EQUAL(offsets.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(offsets.front(), firstOffset + 1); + + CheckMessageBatch(messages, TBatch().AddRow( + TRow().AddString("event2").AddString("str_first__large__") + )); + } + )); + + CheckSuccess(MakeClient( + {commonColumn, {"col_second", "[DataType; String]"}}, + "WHERE col_second = \"str_second\"", + [&](TQueue>>&& data) { + UNIT_ASSERT_VALUES_EQUAL(data.size(), 1); + + auto [messages, offsets] = data.front(); + UNIT_ASSERT_VALUES_EQUAL(offsets.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(offsets.front(), firstOffset); + + CheckMessageBatch(messages, TBatch().AddRow( + TRow().AddString("event1").AddString("str_second") + )); + } + )); + + ParseMessages({ + GetMessage(firstOffset, R"({"com_col": "event1", "col_first": "some_str", "col_second": "str_second"})"), + GetMessage(firstOffset + 1, R"({"com_col": "event2", "col_second": "some_str", "col_first": "str_first__large__"})") + }); + + RemoveClient(ClientIds.back()); + + ParseMessages({ + GetMessage(firstOffset + 2, R"({"com_col": "event1", "col_first": "some_str", "col_second": "str_second"})") + }); + } + + Y_UNIT_TEST_F(ManyRawClients, TFormatHadlerFixture) { + CreateFormatHandler( + {.JsonParserConfig = {}, .FiltersConfig = {.CompileServiceId = CompileService}}, + {.ParsingFormat = "raw"} + ); + + const ui64 firstOffset = 42; + const TVector schema = {{"data", "[DataType; String]"}}; + const TVector testData = { + R"({"col_a": "str1__++___str2", "col_b": 12345})", + R"({"col_a": "str13__++___str23", "col_b": ["A", "B", "C"]})", + R"({"col_a": false, "col_b": {"X": "Y"}})" + }; + + CheckSuccess(MakeClient(schema, "WHERE FALSE", [&](TQueue>>&&) {}, 0)); + + auto trueChacker = [&](TQueue>>&& data) { + UNIT_ASSERT_VALUES_EQUAL(data.size(), 1); + auto [messages, offsets] = data.front(); + + TBatch expectedBatch; + UNIT_ASSERT(!offsets.empty()); + for (ui64 offset : offsets) { + UNIT_ASSERT(offset - firstOffset < testData.size()); + expectedBatch.AddRow( + TRow().AddString(testData[offset - firstOffset]) + ); + } + + CheckMessageBatch(messages, expectedBatch); + }; + CheckSuccess(MakeClient(schema, "WHERE TRUE", trueChacker, 3)); + CheckSuccess(MakeClient(schema, "", trueChacker, 2)); + + ParseMessages({ + GetMessage(firstOffset, testData[0]), + GetMessage(firstOffset + 1, testData[1]) + }, {firstOffset, firstOffset + 1}); + + RemoveClient(ClientIds.back()); + + ParseMessages({ + GetMessage(firstOffset + 2, testData[2]) + }); + } + + Y_UNIT_TEST_F(ClientValidation, TFormatHadlerFixture) { + const TVector schema = {{"data", "[DataType; String]"}}; + const TString filter = "WHERE FALSE"; + const auto callback = [&](TQueue>>&&) {}; + CheckSuccess(MakeClient(schema, filter, callback, 0)); + + CheckError( + FormatHandler->AddClient(MakeIntrusive(ClientIds.back(), schema, filter, callback)), + EStatusId::INTERNAL_ERROR, + "Failed to create new client, client with id [0:0:0] already exists" + ); + + CheckError( + MakeClient({{"data", "[OptionalType; [DataType; String]]"}}, filter, callback, 0), + EStatusId::SCHEME_ERROR, + "Failed to modify common parsing schema" + ); + + CheckError( + MakeClient({{"data_2", "[ListType; [DataType; String]]"}}, filter, callback, 0), + EStatusId::UNSUPPORTED, + "Failed to update parser with new client" + ); + } + + Y_UNIT_TEST_F(ClientError, TFormatHadlerFixture) { + const ui64 firstOffset = 42; + const TSchemaColumn commonColumn = {"com_col", "[DataType; String]"}; + + CheckSuccess(MakeClient( + {commonColumn, {"col_first", "[OptionalType; [DataType; Uint8]]"}}, + "WHERE TRUE", + [&](TQueue>>&& data) {}, + 0 + )); + + CheckSuccess(MakeClient( + {commonColumn, {"col_second", "[DataType; String]"}}, + "WHERE col_second = \"str_second\"", + [&](TQueue>>&& data) { + UNIT_ASSERT_VALUES_EQUAL(data.size(), 1); + + auto [messages, offsets] = data.front(); + UNIT_ASSERT_VALUES_EQUAL(offsets.size(), 1); + UNIT_ASSERT_VALUES_EQUAL(offsets.front(), firstOffset); + + CheckMessageBatch(messages, TBatch().AddRow( + TRow().AddString("event1").AddString("str_second") + )); + } + )); + + CheckClientError( + {GetMessage(firstOffset, R"({"com_col": "event1", "col_first": "some_str", "col_second": "str_second"})")}, + ClientIds[0], + EStatusId::BAD_REQUEST, + TStringBuilder() << "Failed to parse json string at offset " << firstOffset << ", got parsing error for column 'col_first' with type [OptionalType; [DataType; Uint8]]" + ); + } +} + +} // namespace NFq::NRowDispatcher::NTests diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/topic_filter_ut.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/topic_filter_ut.cpp new file mode 100644 index 000000000000..7f0297219fa4 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/topic_filter_ut.cpp @@ -0,0 +1,427 @@ +#include +#include +#include + +#include + +namespace NFq::NRowDispatcher::NTests { + +namespace { + +class TFiterFixture : public TBaseFixture { +public: + using TBase = TBaseFixture; + using TCallback = std::function; + + class TFilterConsumer : public IFilteredDataConsumer { + public: + using TPtr = TIntrusivePtr; + + public: + TFilterConsumer(const TVector& columns, const TString& whereFilter, TCallback callback, std::optional> compileError) + : Columns(columns) + , WhereFilter(whereFilter) + , Callback(callback) + , CompileError(compileError) + {} + + public: + const TVector& GetColumns() const override { + return Columns; + } + + const TString& GetWhereFilter() const override { + return WhereFilter; + } + + TPurecalcCompileSettings GetPurecalcSettings() const override { + return {.EnabledLLVM = false}; + } + + NActors::TActorId GetFilterId() const override { + return FilterId; + } + + const TVector& GetColumnIds() const override { + return ColumnIds; + } + + TMaybe GetNextMessageOffset() const override { + return Nothing(); + } + + void OnFilterStarted() override { + Started = true; + UNIT_ASSERT_C(!CompileError, "Expected compile error: " << CompileError->second); + } + + void OnFilteringError(TStatus status) override { + if (CompileError) { + Started = true; + CheckError(status, CompileError->first, CompileError->second); + } else { + UNIT_FAIL("Filtering failed: " << status.GetErrorMessage()); + } + } + + void OnFilteredData(ui64 rowId) override { + UNIT_ASSERT_C(Started, "Unexpected data for not started filter"); + Callback(rowId); + } + + protected: + NActors::TActorId FilterId; + TVector ColumnIds; + bool Started = false; + + private: + const TVector Columns; + const TString WhereFilter; + const TCallback Callback; + const std::optional> CompileError; + }; + +public: + virtual void SetUp(NUnitTest::TTestContext& ctx) override { + TBase::SetUp(ctx); + + CompileServiceActorId = Runtime.Register(CreatePurecalcCompileService()); + } + + virtual void TearDown(NUnitTest::TTestContext& ctx) override { + with_lock (Alloc) { + for (auto& holder : Holders) { + for (auto& value : holder) { + ClearObject(value); + } + } + Holders.clear(); + } + Filter.Reset(); + FilterHandler.Reset(); + + TBase::TearDown(ctx); + } + +public: + virtual TStatus MakeFilter(const TVector& columns, const TString& whereFilter, TCallback callback) { + FilterHandler = MakeIntrusive(columns, whereFilter, callback, CompileError); + + auto filterStatus = CreatePurecalcFilter(FilterHandler); + if (filterStatus.IsFail()) { + return filterStatus; + } + + Filter = filterStatus.DetachResult(); + CompileFilter(); + return TStatus::Success(); + } + + void Push(const TVector*>& values, ui64 numberRows = 0) { + Filter->FilterData(values, numberRows ? numberRows : values.front()->size()); + } + + const TVector* MakeVector(size_t size, std::function valueCreator) { + with_lock (Alloc) { + auto& holder = Holders.emplace_front(); + for (size_t i = 0; i < size; ++i) { + holder.emplace_back(LockObject(valueCreator(i))); + } + return &holder; + } + } + + template + const TVector* MakeVector(const TVector& values, bool optional = false) { + return MakeVector(values.size(), [&](size_t i) { + NYql::NUdf::TUnboxedValuePod unboxedValue = NYql::NUdf::TUnboxedValuePod(values[i]); + return optional ? unboxedValue.MakeOptional() : unboxedValue; + }); + } + + const TVector* MakeStringVector(const TVector& values, bool optional = false) { + return MakeVector(values.size(), [&](size_t i) { + NYql::NUdf::TUnboxedValuePod stringValue = NKikimr::NMiniKQL::MakeString(values[i]); + return optional ? stringValue.MakeOptional() : stringValue; + }); + } + + const TVector* MakeEmptyVector(size_t size) { + return MakeVector(size, [&](size_t) { + return NYql::NUdf::TUnboxedValuePod(); + }); + } + +private: + void CompileFilter() { + const auto edgeActor = Runtime.AllocateEdgeActor(); + Runtime.Send(CompileServiceActorId, edgeActor, Filter->GetCompileRequest().release()); + auto response = Runtime.GrabEdgeEvent(edgeActor, TDuration::Seconds(5)); + + UNIT_ASSERT_C(response, "Failed to get compile response"); + if (!CompileError) { + UNIT_ASSERT_C(response->Get()->ProgramHolder, "Failed to compile program, error: " << response->Get()->Issues.ToOneLineString()); + Filter->OnCompileResponse(std::move(response)); + FilterHandler->OnFilterStarted(); + } else { + CheckError(TStatus::Fail(response->Get()->Status, response->Get()->Issues), CompileError->first, CompileError->second); + } + } + +public: + NActors::TActorId CompileServiceActorId; + TFilterConsumer::TPtr FilterHandler; + IPurecalcFilter::TPtr Filter; + TList> Holders; + + std::optional> CompileError; +}; + +class TFilterSetFixture : public TFiterFixture { +public: + using TBase = TFiterFixture; + + class TFilterSetConsumer : public TFilterConsumer { + public: + using TBase = TFilterConsumer; + using TPtr = TIntrusivePtr; + + public: + TFilterSetConsumer(NActors::TActorId filterId, const TVector& columnIds, const TVector& columns, const TString& whereFilter, TCallback callback, std::optional> compileError) + : TBase(columns, whereFilter, callback, compileError) + { + FilterId = filterId; + ColumnIds = columnIds; + } + + bool IsStarted() const { + return Started; + } + }; + +public: + void SetUp(NUnitTest::TTestContext& ctx) override { + TBase::SetUp(ctx); + + CompileNotifier = Runtime.AllocateEdgeActor(); + FiltersSet = CreateTopicFilters(CompileNotifier, {.CompileServiceId = CompileServiceActorId}, MakeIntrusive()); + } + + void TearDown(NUnitTest::TTestContext& ctx) override { + FilterIds.clear(); + FiltersSet.Reset(); + + TBase::TearDown(ctx); + } + +public: + TStatus MakeFilter(const TVector& columns, const TString& whereFilter, TCallback callback) override { + TVector columnIds; + columnIds.reserve(columns.size()); + for (const auto& column : columns) { + if (const auto it = ColumnIndex.find(column.Name); it != ColumnIndex.end()) { + columnIds.emplace_back(it->second); + } else { + columnIds.emplace_back(ColumnIndex.size()); + ColumnIndex.insert({column.Name, ColumnIndex.size()}); + } + } + FilterIds.emplace_back(FilterIds.size(), 0, 0, 0); + + auto filterSetHandler = MakeIntrusive(FilterIds.back(), columnIds, columns, whereFilter, callback, CompileError); + if (auto status = FiltersSet->AddFilter(filterSetHandler); status.IsFail()) { + return status; + } + + if (!filterSetHandler->IsStarted()) { + // Wait filter compilation + auto response = Runtime.GrabEdgeEvent(CompileNotifier, TDuration::Seconds(5)); + UNIT_ASSERT_C(response, "Compilation is not performed for filter: " << whereFilter); + FiltersSet->OnCompileResponse(std::move(response)); + } + + return TStatus::Success(); + } + + void FilterData(const TVector& columnIndex, const TVector*>& values, ui64 numberRows = 0) { + numberRows = numberRows ? numberRows : values.front()->size(); + FiltersSet->FilterData(columnIndex, TVector(numberRows, std::numeric_limits::max()), values, numberRows); + } + +public: + TVector FilterIds; + std::unordered_map ColumnIndex; + + NActors::TActorId CompileNotifier; + ITopicFilters::TPtr FiltersSet; +}; + +} // anonymous namespace + +Y_UNIT_TEST_SUITE(TestPurecalcFilter) { + Y_UNIT_TEST_F(Simple1, TFiterFixture) { + std::unordered_set result; + CheckSuccess(MakeFilter( + {{"a1", "[DataType; String]"}, {"a2", "[DataType; Uint64]"}, {"a@3", "[OptionalType; [DataType; String]]"}}, + "where a2 > 100", + [&](ui64 offset) { + result.insert(offset); + } + )); + + Push({MakeStringVector({"hello1"}), MakeVector({99}), MakeStringVector({"zapuskaem"}, true)}); + UNIT_ASSERT_VALUES_EQUAL(0, result.size()); + + Push({MakeStringVector({"hello2"}), MakeVector({101}), MakeStringVector({"gusya"}, true)}); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(*result.begin(), 0); + } + + Y_UNIT_TEST_F(Simple2, TFiterFixture) { + std::unordered_set result; + CheckSuccess(MakeFilter( + {{"a2", "[DataType; Uint64]"}, {"a1", "[DataType; String]"}}, + "where a2 > 100", + [&](ui64 offset) { + result.insert(offset); + } + )); + + Push({MakeVector({99}), MakeStringVector({"hello1"})}); + UNIT_ASSERT_VALUES_EQUAL(0, result.size()); + + Push({MakeVector({101}), MakeStringVector({"hello2"})}); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(*result.begin(), 0); + } + + Y_UNIT_TEST_F(ManyValues, TFiterFixture) { + std::unordered_set result; + CheckSuccess(MakeFilter( + {{"a1", "[DataType; String]"}, {"a2", "[DataType; Uint64]"}, {"a3", "[DataType; String]"}}, + "where a2 > 100", + [&](ui64 offset) { + result.insert(offset); + } + )); + + const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; + for (ui64 i = 0; i < 5; ++i) { + result.clear(); + Push({MakeStringVector({"hello1", "hello2"}), MakeVector({99, 101}), MakeStringVector({largeString, largeString})}); + UNIT_ASSERT_VALUES_EQUAL_C(1, result.size(), i); + UNIT_ASSERT_VALUES_EQUAL_C(*result.begin(), 1, i); + } + } + + Y_UNIT_TEST_F(NullValues, TFiterFixture) { + std::unordered_set result; + CheckSuccess(MakeFilter( + {{"a1", "[OptionalType; [DataType; Uint64]]"}, {"a2", "[DataType; String]"}}, + "where a1 is null", + [&](ui64 offset) { + result.insert(offset); + } + )); + + Push({MakeEmptyVector(1), MakeStringVector({"str"})}); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(*result.begin(), 0); + } + + Y_UNIT_TEST_F(PartialPush, TFiterFixture) { + std::unordered_set result; + CheckSuccess(MakeFilter( + {{"a1", "[DataType; String]"}, {"a2", "[DataType; Uint64]"}, {"a@3", "[OptionalType; [DataType; String]]"}}, + "where a2 > 50", + [&](ui64 offset) { + result.insert(offset); + } + )); + + Push({MakeStringVector({"hello1", "hello2"}), MakeVector({99, 101}), MakeStringVector({"zapuskaem", "gusya"}, true)}, 1); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(*result.begin(), 0); + } + + Y_UNIT_TEST_F(CompilationValidation, TFiterFixture) { + CompileError = {EStatusId::INTERNAL_ERROR, "Failed to compile purecalc program subissue: {
: Error: Compile issues: generated.sql:2:36: Error: Unexpected token '.' : cannot match to any predicted input... } subissue: {
: Error: Final yql:"}; + MakeFilter( + {{"a1", "[DataType; String]"}}, + "where a2 ... 50", + [&](ui64 offset) {} + ); + } +} + +Y_UNIT_TEST_SUITE(TestFilterSet) { + Y_UNIT_TEST_F(FilterGroup, TFilterSetFixture) { + const TSchemaColumn commonColumn = {"common_col", "[DataType; String]"}; + const TVector whereFilters = { + "where col_0 == \"str1\"", + "where col_1 == \"str2\"", + "" // Empty filter <=> where true + }; + + TVector> results(whereFilters.size()); + for (size_t i = 0; i < results.size(); ++i) { + CheckSuccess(MakeFilter( + {commonColumn, {TStringBuilder() << "col_" << i, "[DataType; String]"}}, + whereFilters[i], + [&, index = i](ui64 offset) { + results[index].push_back(offset); + } + )); + } + + FilterData({0, 1, 2, 3}, { + MakeStringVector({"common_1", "common_2", "common_3"}), + MakeStringVector({"str1", "str2", "str3"}), + MakeStringVector({"str1", "str3", "str2"}), + MakeStringVector({"str2", "str3", "str1"}) + }); + + FiltersSet->RemoveFilter(FilterIds.back()); + + FilterData({0, 3, 1, 2}, { + MakeStringVector({"common_3"}), + MakeStringVector({"str2"}), + MakeStringVector({"str3"}), + MakeStringVector({"str1"}) + }); + + TVector> expectedResults = { + {0, 0}, + {2, 0}, + {0, 1, 2} + }; + for (size_t i = 0; i < results.size(); ++i) { + UNIT_ASSERT_VALUES_EQUAL_C(results[i], expectedResults[i], i); + } + } + + Y_UNIT_TEST_F(DuplicationValidation, TFilterSetFixture) { + CheckSuccess(MakeFilter( + {{"a1", "[DataType; String]"}}, + "where a1 = \"str1\"", + [&](ui64 offset) {} + )); + + CheckError( + FiltersSet->AddFilter(MakeIntrusive(FilterIds.back(), TVector(), TVector(), TString(), [&](ui64 offset) {}, CompileError)), + EStatusId::INTERNAL_ERROR, + "Failed to create new filter, filter with id [0:0:0] already exists" + ); + } + + Y_UNIT_TEST_F(CompilationValidation, TFilterSetFixture) { + CompileError = {EStatusId::INTERNAL_ERROR, "Failed to compile client filter subissue: {
: Error: Failed to compile purecalc program subissue: {
: Error: Compile issues: generated.sql:2:36: Error: Unexpected token '.' : cannot match to any predicted input... } subissue: {
: Error: Final yql:"}; + MakeFilter( + {{"a1", "[DataType; String]"}}, + "where a2 ... 50", + [&](ui64 offset) {} + ); + } +} + +} // namespace NFq::NRowDispatcher::NTests diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/topic_parser_ut.cpp b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/topic_parser_ut.cpp new file mode 100644 index 000000000000..e2d46d53679d --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/topic_parser_ut.cpp @@ -0,0 +1,477 @@ +#include +#include +#include + +namespace NFq::NRowDispatcher::NTests { + +namespace { + +class TBaseParserFixture : public TBaseFixture { +public: + static constexpr ui64 FIRST_OFFSET = 42; + + using TBase = TBaseFixture; + using TCallback = std::function*> result)>; + + class TParsedDataConsumer : public IParsedDataConsumer { + public: + using TPtr = TIntrusivePtr; + + public: + TParsedDataConsumer(const TBaseParserFixture& self, const TVector& columns, TCallback callback) + : Self(self) + , Columns(columns) + , Callback(callback) + {} + + void ExpectColumnError(ui64 columnId, TStatusCode statusCode, const TString& message) { + UNIT_ASSERT_C(ExpectedErrors.insert({columnId, {statusCode, message}}).second, "Can not add existing column error"); + } + + void ExpectCommonError(TStatusCode statusCode, const TString& message) { + UNIT_ASSERT_C(!ExpectedCommonError, "Can not add existing common error"); + ExpectedCommonError = {statusCode, message}; + } + + public: + const TVector& GetColumns() const override { + return Columns; + } + + void OnParsingError(TStatus status) override { + NumberBatches++; + CurrentOffset++; + if (ExpectedCommonError) { + CheckError(status, ExpectedCommonError->first, ExpectedCommonError->second); + ExpectedCommonError = std::nullopt; + } else { + CheckSuccess(status); + } + } + + void OnParsedData(ui64 numberRows) override { + NumberBatches++; + UNIT_ASSERT_C(!ExpectedCommonError, "Expected common error: " << ExpectedCommonError->second); + + const auto& offsets = Self.Parser->GetOffsets(); + UNIT_ASSERT_VALUES_EQUAL_C(offsets.size(), numberRows, "Unexpected offsets size"); + for (const ui64 offset : offsets) { + UNIT_ASSERT_VALUES_EQUAL_C(offset, CurrentOffset, "Unexpected offset"); + CurrentOffset++; + } + + TVector*> result(Columns.size(), nullptr); + for (ui64 i = 0; i < Columns.size(); ++i) { + if (const auto it = ExpectedErrors.find(i); it != ExpectedErrors.end()) { + CheckError(Self.Parser->GetParsedColumn(i), it->second.first, it->second.second); + ExpectedErrors.erase(i); + } else { + result[i] = CheckSuccess(Self.Parser->GetParsedColumn(i)); + } + } + Callback(numberRows, std::move(result)); + } + + public: + ui64 CurrentOffset = FIRST_OFFSET; + ui64 NumberBatches = 0; + + private: + const TBaseParserFixture& Self; + const TVector Columns; + const TCallback Callback; + + std::optional> ExpectedCommonError; + std::unordered_map> ExpectedErrors; + }; + +public: + void TearDown(NUnitTest::TTestContext& ctx) override { + if (ParserHandler) { + UNIT_ASSERT_VALUES_EQUAL_C(ExpectedBatches, ParserHandler->NumberBatches, "Unexpected number of batches"); + } + Parser.Reset(); + ParserHandler.Reset(); + + TBase::TearDown(ctx); + } + +public: + TStatus MakeParser(TVector columns, TCallback callback) { + ParserHandler = MakeIntrusive(*this, columns, callback); + + auto parserStatus = CreateParser(); + if (parserStatus.IsFail()) { + return parserStatus; + } + + Parser = parserStatus.DetachResult(); + return TStatus::Success(); + } + + TStatus MakeParser(TVector columnNames, TString columnType, TCallback callback) { + TVector columns; + for (const auto& columnName : columnNames) { + columns.push_back({.Name = columnName, .TypeYson = columnType}); + } + return MakeParser(columns, callback); + } + + TStatus MakeParser(TVector columnNames, TString columnType) { + return MakeParser(columnNames, columnType, [](ui64, TVector*>) {}); + } + + TStatus MakeParser(TVector columns) { + return MakeParser(columns, [](ui64, TVector*>) {}); + } + + void PushToParser(ui64 offset, const TString& data) { + ExpectedBatches++; + Parser->ParseMessages({GetMessage(offset, data)}); + } + + void CheckColumnError(const TString& data, ui64 columnId, TStatusCode statusCode, const TString& message) { + ExpectedBatches++; + ParserHandler->ExpectColumnError(columnId, statusCode, message); + Parser->ParseMessages({GetMessage(ParserHandler->CurrentOffset, data)}); + } + + void CheckBatchError(const TString& data, TStatusCode statusCode, const TString& message) { + ExpectedBatches++; + ParserHandler->ExpectCommonError(statusCode, message); + Parser->ParseMessages({GetMessage(ParserHandler->CurrentOffset, data)}); + } + +protected: + virtual TValueStatus CreateParser() = 0; + +public: + TParsedDataConsumer::TPtr ParserHandler; + ITopicParser::TPtr Parser; + ui64 ExpectedBatches = 0; +}; + +class TJsonParserFixture : public TBaseParserFixture { + using TBase = TBaseParserFixture; + +public: + TJsonParserFixture() + : TBase() + , Config({.BatchSize = 1_MB, .LatencyLimit = TDuration::Zero(), .BufferCellCount = 1000}) + {} + +protected: + TValueStatus CreateParser() override { + return CreateJsonParser(ParserHandler, Config); + } + +public: + TJsonParserConfig Config; +}; + +class TRawParserFixture : public TBaseParserFixture { +protected: + TValueStatus CreateParser() override { + return CreateRawParser(ParserHandler); + } +}; + +} // anonymous namespace + +Y_UNIT_TEST_SUITE(TestJsonParser) { + Y_UNIT_TEST_F(Simple1, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[DataType; String]"}, {"a2", "[OptionalType; [DataType; Uint64]]"}}, [](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[0]->at(0).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL(101, result[1]->at(0).GetOptionalValue().Get()); + })); + PushToParser(FIRST_OFFSET, R"({"a1": "hello1", "a2": 101, "event": "event1"})"); + } + + Y_UNIT_TEST_F(Simple2, TJsonParserFixture) { + CheckSuccess(MakeParser({"a2", "a1"}, "[DataType; String]", [](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + UNIT_ASSERT_VALUES_EQUAL("101", TString(result[0]->at(0).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[1]->at(0).AsStringRef())); + })); + PushToParser(FIRST_OFFSET, R"({"a1": "hello1", "a2": "101", "event": "event1"})"); + } + + Y_UNIT_TEST_F(Simple3, TJsonParserFixture) { + CheckSuccess(MakeParser({"a1", "a2"}, "[DataType; String]", [](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + UNIT_ASSERT_VALUES_EQUAL("101", TString(result[0]->at(0).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[1]->at(0).AsStringRef())); + })); + PushToParser(FIRST_OFFSET,R"({"a2": "hello1", "a1": "101", "event": "event1"})"); + } + + Y_UNIT_TEST_F(Simple4, TJsonParserFixture) { + CheckSuccess(MakeParser({"a2", "a1"}, "[DataType; String]", [](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[0]->at(0).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("101", TString(result[1]->at(0).AsStringRef())); + })); + PushToParser(FIRST_OFFSET, R"({"a2": "hello1", "a1": "101", "event": "event1"})"); + } + + Y_UNIT_TEST_F(LargeStrings, TJsonParserFixture) { + ExpectedBatches = 1; + + const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; + + CheckSuccess(MakeParser({"col"}, "[DataType; String]", [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(2, numberRows); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0]->at(0).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0]->at(1).AsStringRef())); + })); + + const TString jsonString = TStringBuilder() << "{\"col\": \"" << largeString << "\"}"; + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, jsonString), + GetMessage(FIRST_OFFSET + 1, jsonString) + }); + } + + Y_UNIT_TEST_F(ManyValues, TJsonParserFixture) { + ExpectedBatches = 1; + + CheckSuccess(MakeParser({"a1", "a2"}, "[DataType; String]", [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(3, numberRows); + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + for (size_t i = 0; i < numberRows; ++i) { + UNIT_ASSERT_VALUES_EQUAL_C("hello1", TString(result[0]->at(i).AsStringRef()), i); + UNIT_ASSERT_VALUES_EQUAL_C("101", TString(result[1]->at(i).AsStringRef()), i); + } + })); + + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, R"({"a1": "hello1", "a2": "101", "event": "event1"})"), + GetMessage(FIRST_OFFSET + 1, R"({"a1": "hello1", "a2": "101", "event": "event2"})"), + GetMessage(FIRST_OFFSET + 2, R"({"a2": "101", "a1": "hello1", "event": "event3"})") + }); + } + + Y_UNIT_TEST_F(MissingFields, TJsonParserFixture) { + ExpectedBatches = 1; + + CheckSuccess(MakeParser({{"a1", "[OptionalType; [DataType; String]]"}, {"a2", "[OptionalType; [DataType; Uint64]]"}}, [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(3, numberRows); + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + for (size_t i = 0; i < numberRows; ++i) { + if (i == 2) { + UNIT_ASSERT_C(!result[0]->at(i), i); + } else { + NYql::NUdf::TUnboxedValue value = result[0]->at(i).GetOptionalValue(); + UNIT_ASSERT_VALUES_EQUAL_C("hello1", TString(value.AsStringRef()), i); + } + if (i == 1) { + UNIT_ASSERT_C(!result[1]->at(i), i); + } else { + UNIT_ASSERT_VALUES_EQUAL_C(101, result[1]->at(i).GetOptionalValue().Get(), i); + } + } + })); + + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, R"({"a1": "hello1", "a2": 101 , "event": "event1"})"), + GetMessage(FIRST_OFFSET + 1, R"({"a1": "hello1", "event": "event2"})"), + GetMessage(FIRST_OFFSET + 2, R"({"a2": "101", "a1": null, "event": "event3"})") + }); + } + + Y_UNIT_TEST_F(NestedTypes, TJsonParserFixture) { + ExpectedBatches = 1; + + CheckSuccess(MakeParser({{"nested", "[OptionalType; [DataType; Json]]"}, {"a1", "[DataType; String]"}}, [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(4, numberRows); + + UNIT_ASSERT_VALUES_EQUAL(2, result.size()); + UNIT_ASSERT_VALUES_EQUAL("{\"key\": \"value\"}", TString(result[0]->at(0).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[1]->at(0).AsStringRef())); + + UNIT_ASSERT_VALUES_EQUAL("[\"key1\", \"key2\"]", TString(result[0]->at(1).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("hello2", TString(result[1]->at(1).AsStringRef())); + + UNIT_ASSERT_VALUES_EQUAL("\"some string\"", TString(result[0]->at(2).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("hello3", TString(result[1]->at(2).AsStringRef())); + + UNIT_ASSERT_VALUES_EQUAL("123456", TString(result[0]->at(3).AsStringRef())); + UNIT_ASSERT_VALUES_EQUAL("hello4", TString(result[1]->at(3).AsStringRef())); + })); + + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, R"({"a1": "hello1", "nested": {"key": "value"}})"), + GetMessage(FIRST_OFFSET + 1, R"({"a1": "hello2", "nested": ["key1", "key2"]})"), + GetMessage(FIRST_OFFSET + 2, R"({"a1": "hello3", "nested": "some string"})"), + GetMessage(FIRST_OFFSET + 3, R"({"a1": "hello4", "nested": 123456})") + }); + } + + Y_UNIT_TEST_F(SimpleBooleans, TJsonParserFixture) { + ExpectedBatches = 1; + + CheckSuccess(MakeParser({{"a", "[DataType; Bool]"}}, [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(2, numberRows); + + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(true, result[0]->at(0).Get()); + UNIT_ASSERT_VALUES_EQUAL(false, result[0]->at(1).Get()); + })); + + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, R"({"a": true})"), + GetMessage(FIRST_OFFSET + 1, R"({"a": false})") + }); + } + + Y_UNIT_TEST_F(ManyBatches, TJsonParserFixture) { + ExpectedBatches = 2; + Config.BufferCellCount = 1; + + const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; + CheckSuccess(MakeParser({"col"}, "[DataType; String]", [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0]->at(0).AsStringRef())); + })); + + const TString jsonString = TStringBuilder() << "{\"col\": \"" << largeString << "\"}"; + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, jsonString), + GetMessage(FIRST_OFFSET + 1, jsonString) + }); + } + + Y_UNIT_TEST_F(LittleBatches, TJsonParserFixture) { + ExpectedBatches = 2; + Config.BatchSize = 10; + + const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; + CheckSuccess(MakeParser({"col"}, "[DataType; String]", [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0]->at(0).AsStringRef())); + })); + + const TString jsonString = TStringBuilder() << "{\"col\": \"" << largeString << "\"}"; + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, jsonString), + GetMessage(FIRST_OFFSET + 1, jsonString) + }); + } + + Y_UNIT_TEST_F(MissingFieldsValidation, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[DataType; String]"}, {"a2", "[DataType; Uint64]"}})); + CheckColumnError(R"({"a2": 105, "event": "event1"})", 0, EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Failed to parse json messages, found 1 missing values from offset " << FIRST_OFFSET << " in non optional column 'a1' with type [DataType; String]"); + CheckColumnError(R"({"a1": "hello1", "a2": null, "event": "event1"})", 1, EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 1 << ", got parsing error for column 'a2' with type [DataType; Uint64] subissue: {
: Error: Found unexpected null value, expected non optional data type Uint64 }"); + } + + Y_UNIT_TEST_F(TypeKindsValidation, TJsonParserFixture) { + CheckError( + MakeParser({{"a1", "[[BAD TYPE]]"}}), + EStatusId::INTERNAL_ERROR, + "Failed to parse column 'a1' type [[BAD TYPE]] subissue: {
: Error: Failed to parse type from yson: Failed to parse scheme from YSON:" + ); + CheckError( + MakeParser({{"a2", "[OptionalType; [DataType; String]]"}, {"a1", "[ListType; [DataType; String]]"}}), + EStatusId::UNSUPPORTED, + "Failed to create parser for column 'a1' with type [ListType; [DataType; String]] subissue: {
: Error: Unsupported type kind: List }" + ); + } + + Y_UNIT_TEST_F(NumbersValidation, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[OptionalType; [DataType; String]]"}, {"a2", "[DataType; Uint8]"}, {"a3", "[OptionalType; [DataType; Float]]"}})); + CheckColumnError(R"({"a1": 456, "a2": 42})", 0, EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET << ", got parsing error for column 'a1' with type [OptionalType; [DataType; String]] subissue: {
: Error: Failed to parse data type String from json number (raw: '456') subissue: {
: Error: Number value is not expected for data type String } }"); + CheckColumnError(R"({"a1": "456", "a2": -42})", 1, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 1 << ", got parsing error for column 'a2' with type [DataType; Uint8] subissue: {
: Error: Failed to parse data type Uint8 from json number (raw: '-42') subissue: {
: Error: Failed to extract json integer number, error: INCORRECT_TYPE: The JSON element does not have the requested type. } }"); + CheckColumnError(R"({"a1": "str", "a2": 99999})", 1, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 2 << ", got parsing error for column 'a2' with type [DataType; Uint8] subissue: {
: Error: Failed to parse data type Uint8 from json number (raw: '99999') subissue: {
: Error: Number is out of range [0, 255] } }"); + CheckColumnError(R"({"a1": "456", "a2": 42, "a3": 1.11.1})", 2, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 3 << ", got parsing error for column 'a3' with type [OptionalType; [DataType; Float]] subissue: {
: Error: Failed to parse data type Float from json number (raw: '1.11.1') subissue: {
: Error: Failed to extract json float number, error: NUMBER_ERROR: Problem while parsing a number } }"); + } + + Y_UNIT_TEST_F(StringsValidation, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[OptionalType; [DataType; Uint8]]"}})); + CheckColumnError(R"({"a1": "-456"})", 0, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET << ", got parsing error for column 'a1' with type [OptionalType; [DataType; Uint8]] subissue: {
: Error: Failed to parse data type Uint8 from json string: '-456' }"); + } + + Y_UNIT_TEST_F(NestedJsonValidation, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[OptionalType; [DataType; Json]]"}, {"a2", "[OptionalType; [DataType; String]]"}})); + CheckColumnError(R"({"a1": {"key": "value"}, "a2": {"key2": "value2"}})", 1, EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET << ", got parsing error for column 'a2' with type [OptionalType; [DataType; String]] subissue: {
: Error: Found unexpected nested value (raw: '{\"key2\": \"value2\"}'), expected data type String, please use Json type for nested values }"); + CheckColumnError(R"({"a1": {"key": "value", "nested": {"a": "b", "c":}}, "a2": "str"})", 0, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 1 << ", got parsing error for column 'a1' with type [OptionalType; [DataType; Json]] subissue: {
: Error: Found bad json value: '{\"key\": \"value\", \"nested\": {\"a\": \"b\", \"c\":}}' }"); + CheckColumnError(R"({"a1": {"key" "value"}, "a2": "str"})", 0, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 2 << ", got parsing error for column 'a1' with type [OptionalType; [DataType; Json]] subissue: {
: Error: Failed to extract json value, current token: '{', error: TAPE_ERROR: The JSON document has an improper structure: missing or superfluous commas, braces, missing keys, etc. }"); + } + + Y_UNIT_TEST_F(BoolsValidation, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[OptionalType; [DataType; String]]"}, {"a2", "[DataType; Bool]"}})); + CheckColumnError(R"({"a1": true, "a2": false})", 0, EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET << ", got parsing error for column 'a1' with type [OptionalType; [DataType; String]] subissue: {
: Error: Found unexpected bool value, expected data type String }"); + CheckColumnError(R"({"a1": "true", "a2": falce})", 1, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET + 1 << ", got parsing error for column 'a2' with type [DataType; Bool] subissue: {
: Error: Failed to extract json bool, current token: 'falce', error: INCORRECT_TYPE: The JSON element does not have the requested type. }"); + } + + Y_UNIT_TEST_F(JsonStructureValidation, TJsonParserFixture) { + CheckSuccess(MakeParser({{"a1", "[OptionalType; [DataType; String]]"}})); + CheckColumnError(R"({"a1": Yelse})", 0, EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json string at offset " << FIRST_OFFSET << ", got parsing error for column 'a1' with type [OptionalType; [DataType; String]] subissue: {
: Error: Failed to determine json value type, current token: 'Yelse', error: TAPE_ERROR: The JSON document has an improper structure: missing or superfluous commas, braces, missing keys, etc. }"); + CheckBatchError(R"({"a1": "st""r"})", EStatusId::BAD_REQUEST, TStringBuilder() << "Failed to parse json message for offset " << FIRST_OFFSET + 1 << ", json item was corrupted: TAPE_ERROR: The JSON document has an improper structure: missing or superfluous commas, braces, missing keys, etc."); + CheckBatchError(R"({"a1": "x"} {"a1": "y"})", EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to parse json messages, expected 1 json rows from offset " << FIRST_OFFSET + 2 << " but got 2"); + CheckBatchError(R"({)", EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to parse json messages, expected 1 json rows from offset " << FIRST_OFFSET + 3 << " but got 0"); + } +} + +Y_UNIT_TEST_SUITE(TestRawParser) { + Y_UNIT_TEST_F(Simple, TRawParserFixture) { + CheckSuccess(MakeParser({{"data", "[OptionalType; [DataType; String]]"}}, [](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + + NYql::NUdf::TUnboxedValue value = result[0]->at(0).GetOptionalValue(); + UNIT_ASSERT_VALUES_EQUAL(R"({"a1": "hello1__large_str", "a2": 101, "event": "event1"})", TString(value.AsStringRef())); + })); + PushToParser(FIRST_OFFSET, R"({"a1": "hello1__large_str", "a2": 101, "event": "event1"})"); + } + + Y_UNIT_TEST_F(ManyValues, TRawParserFixture) { + TVector data = { + R"({"a1": "hello1", "a2": "101", "event": "event1"})", + R"({"a1": "hello1", "a2": "101", "event": "event2"})", + R"({"a2": "101", "a1": "hello1", "event": "event3"})" + }; + ExpectedBatches = data.size(); + + int i = 0; + CheckSuccess(MakeParser({"a1"}, "[DataType; String]", [&](ui64 numberRows, TVector*> result) { + UNIT_ASSERT_VALUES_EQUAL(1, numberRows); + UNIT_ASSERT_VALUES_EQUAL(1, result.size()); + UNIT_ASSERT_VALUES_EQUAL(data[i], TString(result[0]->at(0).AsStringRef())); + i++; + })); + + Parser->ParseMessages({ + GetMessage(FIRST_OFFSET, data[0]), + GetMessage(FIRST_OFFSET + 1, data[1]), + GetMessage(FIRST_OFFSET + 2, data[2]) + }); + } + + Y_UNIT_TEST_F(TypeKindsValidation, TRawParserFixture) { + CheckError( + MakeParser({{"a1", "[DataType; String]"}, {"a2", "[DataType; String]"}}), + EStatusId::INTERNAL_ERROR, + "Expected only one column for raw format, but got 2" + ); + CheckError( + MakeParser({{"a1", "[[BAD TYPE]]"}}), + EStatusId::INTERNAL_ERROR, + "Failed to create raw parser for column 'a1' : [[BAD TYPE]] subissue: {
: Error: Failed to parse type from yson: Failed to parse scheme from YSON:" + ); + CheckError( + MakeParser({{"a1", "[ListType; [DataType; String]]"}}), + EStatusId::UNSUPPORTED, + "Failed to create raw parser for column 'a1' : [ListType; [DataType; String]] subissue: {
: Error: Unsupported type kind for raw format: List }" + ); + } +} + +} // namespace NFq::NRowDispatcher::NTests diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ut/ya.make b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/ya.make new file mode 100644 index 000000000000..54b30a2c7d48 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ut/ya.make @@ -0,0 +1,22 @@ +UNITTEST_FOR(ydb/core/fq/libs/row_dispatcher/format_handler) + +SRCS( + format_handler_ut.cpp + topic_filter_ut.cpp + topic_parser_ut.cpp +) + +PEERDIR( + ydb/core/fq/libs/row_dispatcher/format_handler + ydb/core/fq/libs/row_dispatcher/format_handler/filters + ydb/core/fq/libs/row_dispatcher/format_handler/parsers + ydb/core/fq/libs/row_dispatcher/format_handler/ut/common + + yql/essentials/sql/pg_dummy +) + +SIZE(MEDIUM) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/ydb/core/fq/libs/row_dispatcher/format_handler/ya.make b/ydb/core/fq/libs/row_dispatcher/format_handler/ya.make new file mode 100644 index 000000000000..cb14937b89c2 --- /dev/null +++ b/ydb/core/fq/libs/row_dispatcher/format_handler/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +SRCS( + format_handler.cpp +) + +PEERDIR( + ydb/core/fq/libs/actors/logging + ydb/core/fq/libs/row_dispatcher/events + ydb/core/fq/libs/row_dispatcher/format_handler/common + ydb/core/fq/libs/row_dispatcher/format_handler/filters + ydb/core/fq/libs/row_dispatcher/format_handler/parsers + + ydb/library/actors/core + ydb/library/actors/util + + ydb/library/yql/dq/common +) + +YQL_LAST_ABI_VERSION() + +END() + +RECURSE_FOR_TESTS( + ut +) diff --git a/ydb/core/fq/libs/row_dispatcher/json_filter.cpp b/ydb/core/fq/libs/row_dispatcher/json_filter.cpp deleted file mode 100644 index 0938e74da67c..000000000000 --- a/ydb/core/fq/libs/row_dispatcher/json_filter.cpp +++ /dev/null @@ -1,392 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include - -#include -#include -#include - -namespace { - -using TCallback = NFq::TJsonFilter::TCallback; -const char* OffsetFieldName = "_offset"; -TString LogPrefix = "JsonFilter: "; - -NYT::TNode CreateTypeNode(const TString& fieldType) { - return NYT::TNode::CreateList() - .Add("DataType") - .Add(fieldType); -} - -NYT::TNode CreateOptionalTypeNode(const TString& fieldType) { - return NYT::TNode::CreateList() - .Add("OptionalType") - .Add(CreateTypeNode(fieldType)); -} - -void AddField(NYT::TNode& node, const TString& fieldName, const TString& fieldType) { - node.Add( - NYT::TNode::CreateList() - .Add(fieldName) - .Add(CreateTypeNode(fieldType)) - ); -} - -void AddTypedField(NYT::TNode& node, const TString& fieldName, const TString& fieldTypeYson) { - NYT::TNode parsedType; - Y_ENSURE(NYql::NCommon::ParseYson(parsedType, fieldTypeYson, Cerr), "Invalid field type"); - - // TODO: remove this when the re-parsing is removed from pq read actor - if (parsedType == CreateTypeNode("Json")) { - parsedType = CreateTypeNode("String"); - } else if (parsedType == CreateOptionalTypeNode("Json")) { - parsedType = CreateOptionalTypeNode("String"); - } - - node.Add( - NYT::TNode::CreateList() - .Add(fieldName) - .Add(parsedType) - ); -} - -NYT::TNode MakeInputSchema(const TVector& columns, const TVector& types) { - auto structMembers = NYT::TNode::CreateList(); - AddField(structMembers, OffsetFieldName, "Uint64"); - for (size_t i = 0; i < columns.size(); ++i) { - AddTypedField(structMembers, columns[i], types[i]); - } - return NYT::TNode::CreateList().Add("StructType").Add(std::move(structMembers)); -} - -NYT::TNode MakeOutputSchema() { - auto structMembers = NYT::TNode::CreateList(); - AddField(structMembers, OffsetFieldName, "Uint64"); - AddField(structMembers, "data", "String"); - return NYT::TNode::CreateList().Add("StructType").Add(std::move(structMembers)); -} - -struct TInputType { - const TVector& Offsets; - const TVector*>& Values; - const ui64 RowsOffset; // offset of first value - const ui64 NumberRows; - - ui64 GetOffset(ui64 rowId) const { - return Offsets[rowId + RowsOffset]; - } -}; - -class TFilterInputSpec : public NYql::NPureCalc::TInputSpecBase { -public: - TFilterInputSpec(const NYT::TNode& schema) - : Schemas({schema}) { - } - - const TVector& GetSchemas() const override { - return Schemas; - } - -private: - TVector Schemas; -}; - -class TFilterInputConsumer : public NYql::NPureCalc::IConsumer { -public: - TFilterInputConsumer( - const TFilterInputSpec& spec, - NYql::NPureCalc::TWorkerHolder worker) - : Worker(std::move(worker)) { - const NKikimr::NMiniKQL::TStructType* structType = Worker->GetInputType(); - const auto count = structType->GetMembersCount(); - - THashMap schemaPositions; - for (ui32 i = 0; i < count; ++i) { - const auto name = structType->GetMemberName(i); - if (name == OffsetFieldName) { - OffsetPosition = i; - continue; - } - schemaPositions[name] = i; - } - - const NYT::TNode& schema = spec.GetSchemas()[0]; - const auto& fields = schema[1]; - Y_ENSURE(count == fields.Size()); - Y_ENSURE(fields.IsList()); - for (size_t i = 0; i < fields.Size(); ++i) { - auto name = fields[i][0].AsString(); - if (name == OffsetFieldName) { - continue; - } - FieldsPositions.push_back(schemaPositions[name]); - } - } - - ~TFilterInputConsumer() override { - with_lock(Worker->GetScopedAlloc()) { - Cache.Clear(); - } - } - - void OnObject(TInputType input) override { - Y_ENSURE(FieldsPositions.size() == input.Values.size()); - - NKikimr::NMiniKQL::TThrowingBindTerminator bind; - with_lock (Worker->GetScopedAlloc()) { - Y_DEFER { - // Clear cache after each object because - // values allocated on another allocator and should be released - Cache.Clear(); - Worker->GetGraph().Invalidate(); - }; - - auto& holderFactory = Worker->GetGraph().GetHolderFactory(); - - // TODO: use blocks here - for (size_t rowId = 0; rowId < input.NumberRows; ++rowId) { - NYql::NUdf::TUnboxedValue* items = nullptr; - - NYql::NUdf::TUnboxedValue result = Cache.NewArray( - holderFactory, - static_cast(input.Values.size() + 1), - items); - - items[OffsetPosition] = NYql::NUdf::TUnboxedValuePod(input.GetOffset(rowId)); - - size_t fieldId = 0; - for (const auto column : input.Values) { - items[FieldsPositions[fieldId++]] = column->at(rowId); - } - - Worker->Push(std::move(result)); - } - } - } - - void OnFinish() override { - NKikimr::NMiniKQL::TBindTerminator bind(Worker->GetGraph().GetTerminator()); - with_lock(Worker->GetScopedAlloc()) { - Worker->OnFinish(); - } - } - -private: - NYql::NPureCalc::TWorkerHolder Worker; - NKikimr::NMiniKQL::TPlainContainerCache Cache; - size_t OffsetPosition = 0; - TVector FieldsPositions; -}; - -class TFilterOutputConsumer: public NYql::NPureCalc::IConsumer> { -public: - TFilterOutputConsumer(TCallback callback) - : Callback(callback) { - } - - void OnObject(std::pair value) override { - Callback(value.first, value.second); - } - - void OnFinish() override { - Y_UNREACHABLE(); - } -private: - TCallback Callback; -}; - -class TFilterOutputSpec: public NYql::NPureCalc::TOutputSpecBase { -public: - explicit TFilterOutputSpec(const NYT::TNode& schema) - : Schema(schema) - {} - -public: - const NYT::TNode& GetSchema() const override { - return Schema; - } - -private: - NYT::TNode Schema; -}; - -class TFilterPushRelayImpl: public NYql::NPureCalc::IConsumer { -public: - TFilterPushRelayImpl(const TFilterOutputSpec& /*outputSpec*/, NYql::NPureCalc::IPushStreamWorker* worker, THolder>> underlying) - : Underlying(std::move(underlying)) - , Worker(worker) - {} -public: - void OnObject(const NYql::NUdf::TUnboxedValue* value) override { - auto unguard = Unguard(Worker->GetScopedAlloc()); - Y_ENSURE(value->GetListLength() == 2); - ui64 offset = value->GetElement(0).Get(); - const auto& cell = value->GetElement(1); - Y_ENSURE(cell); - TString str(cell.AsStringRef()); - Underlying->OnObject(std::make_pair(offset, str)); - } - - void OnFinish() override { - auto unguard = Unguard(Worker->GetScopedAlloc()); - Underlying->OnFinish(); - } - -private: - THolder>> Underlying; - NYql::NPureCalc::IWorker* Worker; -}; - -} - -template <> -struct NYql::NPureCalc::TInputSpecTraits { - static constexpr bool IsPartial = false; - static constexpr bool SupportPushStreamMode = true; - - using TConsumerType = THolder>; - - static TConsumerType MakeConsumer( - const TFilterInputSpec& spec, - NYql::NPureCalc::TWorkerHolder worker) - { - return MakeHolder(spec, std::move(worker)); - } -}; - -template <> -struct NYql::NPureCalc::TOutputSpecTraits { - static const constexpr bool IsPartial = false; - static const constexpr bool SupportPushStreamMode = true; - - static void SetConsumerToWorker(const TFilterOutputSpec& outputSpec, NYql::NPureCalc::IPushStreamWorker* worker, THolder>> consumer) { - worker->SetConsumer(MakeHolder(outputSpec, worker, std::move(consumer))); - } -}; - -namespace NFq { - -class TProgramHolder : public IProgramHolder { -public: - using TPtr = TIntrusivePtr; - -public: - TProgramHolder(const TVector& columns, const TVector& types, const TString& sql, TCallback callback) - : Columns(columns) - , Types(types) - , Sql(sql) - , Callback(callback) - { - Y_ENSURE(columns.size() == types.size(), "Number of columns and types should by equal"); - } - - NYql::NPureCalc::IConsumer& GetConsumer() { - Y_ENSURE(InputConsumer, "Program is not compiled"); - return *InputConsumer; - } - -public: - void CreateProgram(NYql::NPureCalc::IProgramFactoryPtr programFactory) override { - Program = programFactory->MakePushStreamProgram( - TFilterInputSpec(MakeInputSchema(Columns, Types)), - TFilterOutputSpec(MakeOutputSchema()), - Sql, - NYql::NPureCalc::ETranslationMode::SQL - ); - InputConsumer = Program->Apply(MakeHolder(Callback)); - } - -private: - const TVector Columns; - const TVector Types; - const TString Sql; - const TCallback Callback; - - THolder> Program; - THolder> InputConsumer; -}; - -class TJsonFilter::TImpl { -public: - TImpl(const TVector& columns, const TVector& types, const TString& whereFilter, TCallback callback, const TPurecalcCompileSettings& purecalcSettings) - : PurecalcSettings(purecalcSettings) - , Sql(GenerateSql(whereFilter)) - , ProgramHolder(MakeIntrusive(columns, types, Sql, callback)) - {} - - void Push(const TVector& offsets, const TVector*>& values, ui64 rowsOffset, ui64 numberRows) { - Y_ENSURE(ProgramHolder, "Program is not compiled"); - Y_ENSURE(values, "Expected non empty schema"); - ProgramHolder->GetConsumer().OnObject({.Offsets = offsets, .Values = values, .RowsOffset = rowsOffset, .NumberRows = numberRows}); - } - - TString GetSql() const { - return Sql; - } - - std::unique_ptr GetCompileRequest() { - Y_ENSURE(ProgramHolder, "Can not create compile request twice"); - auto result = std::make_unique(std::move(ProgramHolder), PurecalcSettings); - ProgramHolder = nullptr; - return result; - } - - void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) { - Y_ENSURE(!ProgramHolder, "Can not handle compile response twice"); - - auto result = static_cast(ev->Get()->ProgramHolder.Release()); - Y_ENSURE(result, "Unexpected compile response"); - - ProgramHolder = TIntrusivePtr(result); - } - -private: - TString GenerateSql(const TString& whereFilter) { - TStringStream str; - str << "PRAGMA config.flags(\"LLVM\", \"" << (PurecalcSettings.EnabledLLVM ? "ON" : "OFF") << "\");\n"; - str << "$filtered = SELECT * FROM Input " << whereFilter << ";\n"; - - str << "SELECT " << OffsetFieldName << ", Unwrap(Json::SerializeJson(Yson::From(RemoveMembers(TableRow(), [\"" << OffsetFieldName; - str << "\"])))) as data FROM $filtered"; - LOG_ROW_DISPATCHER_DEBUG("Generated sql: " << str.Str()); - return str.Str(); - } - -private: - const TPurecalcCompileSettings PurecalcSettings; - const TString Sql; - TProgramHolder::TPtr ProgramHolder; -}; - -TJsonFilter::TJsonFilter(const TVector& columns, const TVector& types, const TString& whereFilter, TCallback callback, const TPurecalcCompileSettings& purecalcSettings) - : Impl(std::make_unique(columns, types, whereFilter, callback, purecalcSettings)) -{} - -TJsonFilter::~TJsonFilter() { -} - -void TJsonFilter::Push(const TVector& offsets, const TVector*>& values, ui64 rowsOffset, ui64 numberRows) { - Impl->Push(offsets, values, rowsOffset, numberRows); -} - -TString TJsonFilter::GetSql() { - return Impl->GetSql(); -} - -std::unique_ptr TJsonFilter::GetCompileRequest() { - return Impl->GetCompileRequest(); -} - -void TJsonFilter::OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev) { - Impl->OnCompileResponse(std::move(ev)); -} - -std::unique_ptr NewJsonFilter(const TVector& columns, const TVector& types, const TString& whereFilter, TCallback callback, const TPurecalcCompileSettings& purecalcSettings) { - return std::unique_ptr(new TJsonFilter(columns, types, whereFilter, callback, purecalcSettings)); -} - -} // namespace NFq diff --git a/ydb/core/fq/libs/row_dispatcher/json_filter.h b/ydb/core/fq/libs/row_dispatcher/json_filter.h deleted file mode 100644 index dc3d51f5ecc8..000000000000 --- a/ydb/core/fq/libs/row_dispatcher/json_filter.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include - -#include - -namespace NFq { - -class TJsonFilter { -public: - using TCallback = std::function; - -public: - TJsonFilter( - const TVector& columns, - const TVector& types, - const TString& whereFilter, - TCallback callback, - const TPurecalcCompileSettings& purecalcSettings); - - ~TJsonFilter(); - - void Push(const TVector& offsets, const TVector*>& values, ui64 rowsOffset, ui64 numberRows); - TString GetSql(); - - std::unique_ptr GetCompileRequest(); // Should be called exactly once - void OnCompileResponse(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr ev); - -private: - class TImpl; - const std::unique_ptr Impl; -}; - -std::unique_ptr NewJsonFilter( - const TVector& columns, - const TVector& types, - const TString& whereFilter, - TJsonFilter::TCallback callback, - const TPurecalcCompileSettings& purecalcSettings); - -} // namespace NFq diff --git a/ydb/core/fq/libs/row_dispatcher/json_parser.cpp b/ydb/core/fq/libs/row_dispatcher/json_parser.cpp deleted file mode 100644 index f5064ca0f07d..000000000000 --- a/ydb/core/fq/libs/row_dispatcher/json_parser.cpp +++ /dev/null @@ -1,486 +0,0 @@ -#include "json_parser.h" - -#include - -#include -#include -#include -#include -#include -#include -#include -#include - -#include - -#include - -namespace { - -TString LogPrefix = "JsonParser: "; - -constexpr ui64 DEFAULT_BATCH_SIZE = 1_MB; -constexpr ui64 DEFAULT_BUFFER_CELL_COUNT = 1000000; - -struct TJsonParserBuffer { - size_t NumberValues = 0; - bool Finished = false; - TInstant CreationStartTime = TInstant::Now(); - TVector Offsets = {}; - - bool IsReady() const { - return !Finished && NumberValues > 0; - } - - size_t GetSize() const { - return Values.size(); - } - - void Reserve(size_t size, size_t numberValues) { - Values.reserve(size + simdjson::SIMDJSON_PADDING); - Offsets.reserve(numberValues); - } - - void AddMessage(const NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage& message) { - Y_ENSURE(!Finished, "Cannot add messages into finished buffer"); - NumberValues++; - Values << message.GetData(); - Offsets.emplace_back(message.GetOffset()); - } - - std::pair Finish() { - Y_ENSURE(!Finished, "Cannot finish buffer twice"); - Finished = true; - Values << TString(simdjson::SIMDJSON_PADDING, ' '); - return {Values.data(), Values.size()}; - } - - void Clear() { - Y_ENSURE(Finished, "Cannot clear not finished buffer"); - NumberValues = 0; - Finished = false; - CreationStartTime = TInstant::Now(); - Values.clear(); - Offsets.clear(); - } - -private: - TStringBuilder Values = {}; -}; - -class TColumnParser { - using TParser = std::function; - -public: - const std::string Name; - const TString TypeYson; - const NKikimr::NMiniKQL::TType* TypeMkql; - const bool IsOptional = false; - TVector ParsedRows; - -public: - TColumnParser(const TString& name, const TString& typeYson, ui64 maxNumberRows, NKikimr::NMiniKQL::TProgramBuilder& programBuilder) - : Name(name) - , TypeYson(typeYson) - , TypeMkql(NYql::NCommon::ParseTypeFromYson(TStringBuf(typeYson), programBuilder, Cerr)) - , IsOptional(TypeMkql->IsOptional()) - { - ParsedRows.reserve(maxNumberRows); - try { - Parser = CreateParser(TypeMkql); - } catch (...) { - throw NFq::TJsonParserError(Name) << "Failed to create parser for column '" << Name << "' with type " << TypeYson << ", description: " << CurrentExceptionMessage(); - } - } - - void ParseJsonValue(ui64 rowId, simdjson::builtin::ondemand::value jsonValue, NYql::NUdf::TUnboxedValue& resultValue) { - ParsedRows.emplace_back(rowId); - Parser(jsonValue, resultValue); - } - - void ValidateNumberValues(size_t expectedNumberValues, ui64 firstOffset) const { - if (Y_UNLIKELY(!IsOptional && ParsedRows.size() < expectedNumberValues)) { - throw NFq::TJsonParserError(Name) << "Failed to parse json messages, found " << expectedNumberValues - ParsedRows.size() << " missing values from offset " << firstOffset << " in non optional column '" << Name << "' with type " << TypeYson; - } - } - -private: - TParser CreateParser(const NKikimr::NMiniKQL::TType* type, bool optional = false) const { - switch (type->GetKind()) { - case NKikimr::NMiniKQL::TTypeBase::EKind::Data: { - const auto* dataType = AS_TYPE(NKikimr::NMiniKQL::TDataType, type); - if (const auto dataSlot = dataType->GetDataSlot()) { - return GetJsonValueParser(*dataSlot, optional); - } - throw NFq::TJsonParserError() << "unsupported data type with id " << dataType->GetSchemeType(); - } - - case NKikimr::NMiniKQL::TTypeBase::EKind::Optional: { - return AddOptional(CreateParser(AS_TYPE(NKikimr::NMiniKQL::TOptionalType, type)->GetItemType(), true)); - } - - default: { - throw NFq::TJsonParserError() << "unsupported type kind " << type->GetKindAsStr(); - } - } - } - - static TParser AddOptional(TParser parser) { - return [parser](simdjson::builtin::ondemand::value jsonValue, NYql::NUdf::TUnboxedValue& resultValue) { - parser(std::move(jsonValue), resultValue); - if (resultValue) { - resultValue = resultValue.MakeOptional(); - } - }; - } - - static TParser GetJsonValueParser(NYql::NUdf::EDataSlot dataSlot, bool optional) { - if (dataSlot == NYql::NUdf::EDataSlot::Json) { - return GetJsonValueExtractor(); - } - - const auto& typeInfo = NYql::NUdf::GetDataTypeInfo(dataSlot); - return [dataSlot, optional, &typeInfo](simdjson::builtin::ondemand::value jsonValue, NYql::NUdf::TUnboxedValue& resultValue) { - switch (jsonValue.type()) { - case simdjson::builtin::ondemand::json_type::number: { - try { - switch (dataSlot) { - case NYql::NUdf::EDataSlot::Int8: - resultValue = ParseJsonNumber(jsonValue.get_int64().value()); - break; - case NYql::NUdf::EDataSlot::Int16: - resultValue = ParseJsonNumber(jsonValue.get_int64().value()); - break; - case NYql::NUdf::EDataSlot::Int32: - resultValue = ParseJsonNumber(jsonValue.get_int64().value()); - break; - case NYql::NUdf::EDataSlot::Int64: - resultValue = ParseJsonNumber(jsonValue.get_int64().value()); - break; - - case NYql::NUdf::EDataSlot::Uint8: - resultValue = ParseJsonNumber(jsonValue.get_uint64().value()); - break; - case NYql::NUdf::EDataSlot::Uint16: - resultValue = ParseJsonNumber(jsonValue.get_uint64().value()); - break; - case NYql::NUdf::EDataSlot::Uint32: - resultValue = ParseJsonNumber(jsonValue.get_uint64().value()); - break; - case NYql::NUdf::EDataSlot::Uint64: - resultValue = ParseJsonNumber(jsonValue.get_uint64().value()); - break; - - case NYql::NUdf::EDataSlot::Double: - resultValue = NYql::NUdf::TUnboxedValuePod(jsonValue.get_double().value()); - break; - case NYql::NUdf::EDataSlot::Float: - resultValue = NYql::NUdf::TUnboxedValuePod(static_cast(jsonValue.get_double().value())); - break; - - default: - throw NFq::TJsonParserError() << "number value is not expected for data type " << typeInfo.Name; - } - } catch (...) { - throw NFq::TJsonParserError() << "failed to parse data type " << typeInfo.Name << " from json number (raw: '" << TruncateString(jsonValue.raw_json_token()) << "'), error: " << CurrentExceptionMessage(); - } - break; - } - - case simdjson::builtin::ondemand::json_type::string: { - const auto rawString = jsonValue.get_string().value(); - resultValue = NKikimr::NMiniKQL::ValueFromString(dataSlot, rawString); - if (Y_UNLIKELY(!resultValue)) { - throw NFq::TJsonParserError() << "failed to parse data type " << typeInfo.Name << " from json string: '" << TruncateString(rawString) << "'"; - } - LockObject(resultValue); - break; - } - - case simdjson::builtin::ondemand::json_type::array: - case simdjson::builtin::ondemand::json_type::object: { - throw NFq::TJsonParserError() << "found unexpected nested value (raw: '" << TruncateString(jsonValue.raw_json().value()) << "'), expected data type " < - static NYql::NUdf::TUnboxedValuePod ParseJsonNumber(TJsonNumber number) { - if (number < std::numeric_limits::min() || std::numeric_limits::max() < number) { - throw NFq::TJsonParserError() << "number is out of range"; - } - return NYql::NUdf::TUnboxedValuePod(static_cast(number)); - } - - static void LockObject(NYql::NUdf::TUnboxedValue& value) { - // All UnboxedValue's with type Boxed or String should be locked - // because after parsing they will be used under another MKQL allocator in purecalc filters - - const i32 numberRefs = value.LockRef(); - - // -1 - value is embbeded or empty, otherwise value should have exactly one ref - Y_ENSURE(numberRefs == -1 || numberRefs == 1); - } - - static TString TruncateString(std::string_view rawString, size_t maxSize = 1_KB) { - if (rawString.size() <= maxSize) { - return TString(rawString); - } - return TStringBuilder() << rawString.substr(0, maxSize) << " truncated..."; - } - -private: - TParser Parser; -}; - -} // anonymous namespace - -namespace NFq { - -//// TJsonParser - -class TJsonParser::TImpl { -public: - TImpl(const TVector& columns, const TVector& types, TCallback parseCallback, ui64 batchSize, TDuration batchCreationTimeout, ui64 bufferCellCount) - : Alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), true, false) - , TypeEnv(std::make_unique(Alloc)) - , BatchSize(batchSize ? batchSize : DEFAULT_BATCH_SIZE) - , MaxNumberRows(((bufferCellCount ? bufferCellCount : DEFAULT_BUFFER_CELL_COUNT) - 1) / columns.size() + 1) - , BatchCreationTimeout(batchCreationTimeout) - , ParseCallback(parseCallback) - , ParsedValues(columns.size()) - { - Y_ENSURE(columns.size() == types.size(), "Number of columns and types should by equal"); - - with_lock (Alloc) { - auto functonRegistry = NKikimr::NMiniKQL::CreateFunctionRegistry(&PrintBackTrace, NKikimr::NMiniKQL::CreateBuiltinRegistry(), false, {}); - NKikimr::NMiniKQL::TProgramBuilder programBuilder(*TypeEnv, *functonRegistry); - - Columns.reserve(columns.size()); - for (size_t i = 0; i < columns.size(); i++) { - Columns.emplace_back(columns[i], types[i], MaxNumberRows, programBuilder); - } - } - - ColumnsIndex.reserve(columns.size()); - for (size_t i = 0; i < columns.size(); i++) { - ColumnsIndex.emplace(std::string_view(Columns[i].Name), i); - } - - for (size_t i = 0; i < columns.size(); i++) { - ParsedValues[i].resize(MaxNumberRows); - } - - Buffer.Reserve(BatchSize, MaxNumberRows); - - LOG_ROW_DISPATCHER_INFO("Simdjson active implementation " << simdjson::get_active_implementation()->name()); - Parser.threaded = false; - } - - bool IsReady() const { - return Buffer.IsReady() && (Buffer.GetSize() >= BatchSize || TInstant::Now() - Buffer.CreationStartTime >= BatchCreationTimeout); - } - - TInstant GetCreationDeadline() const { - return Buffer.IsReady() ? Buffer.CreationStartTime + BatchCreationTimeout : TInstant::Zero(); - } - - size_t GetNumberValues() const { - return Buffer.IsReady() ? Buffer.NumberValues : 0; - } - - const TVector& GetOffsets() { - return Buffer.Offsets; - } - - void AddMessages(const TVector& messages) { - Y_ENSURE(!Buffer.Finished, "Cannot add messages into finished buffer"); - for (const auto& message : messages) { - Buffer.AddMessage(message); - if (Buffer.IsReady() && Buffer.GetSize() >= BatchSize) { - Parse(); - } - } - } - - void Parse() { - Y_ENSURE(Buffer.IsReady(), "Nothing to parse"); - - const auto [values, size] = Buffer.Finish(); - LOG_ROW_DISPATCHER_TRACE("Parse values:\n" << values); - - with_lock (Alloc) { - Y_DEFER { - // Clear all UV in case of exception - ClearColumns(); - Buffer.Clear(); - }; - - size_t rowId = 0; - size_t parsedRows = 0; - simdjson::ondemand::document_stream documents = Parser.iterate_many(values, size, simdjson::ondemand::DEFAULT_BATCH_SIZE); - for (auto document : documents) { - if (Y_UNLIKELY(parsedRows >= Buffer.NumberValues)) { - throw NFq::TJsonParserError() << "Failed to parse json messages, expected " << Buffer.NumberValues << " json rows from offset " << Buffer.Offsets.front() << " but got " << parsedRows + 1; - } - for (auto item : document.get_object()) { - const auto it = ColumnsIndex.find(item.escaped_key().value()); - if (it == ColumnsIndex.end()) { - continue; - } - - const size_t columnId = it->second; - auto& columnParser = Columns[columnId]; - try { - columnParser.ParseJsonValue(rowId, item.value(), ParsedValues[columnId][rowId]); - } catch (...) { - throw NFq::TJsonParserError(columnParser.Name) << "Failed to parse json string at offset " << Buffer.Offsets[rowId] << ", got parsing error for column '" << columnParser.Name << "' with type " << columnParser.TypeYson << ", description: " << CurrentExceptionMessage(); - } - } - - rowId++; - parsedRows++; - if (rowId == MaxNumberRows) { - FlushColumns(parsedRows, MaxNumberRows); - rowId = 0; - } - } - - if (Y_UNLIKELY(parsedRows != Buffer.NumberValues)) { - throw NFq::TJsonParserError() << "Failed to parse json messages, expected " << Buffer.NumberValues << " json rows from offset " << Buffer.Offsets.front() << " but got " << rowId; - } - if (rowId) { - FlushColumns(parsedRows, rowId); - } - } - } - - TString GetDescription() const { - TStringBuilder description = TStringBuilder() << "Columns: "; - for (const auto& column : Columns) { - description << "'" << column.Name << "':" << column.TypeYson << " "; - } - description << "\nNumber values in buffer: " << Buffer.NumberValues << ", buffer size: " << Buffer.GetSize() << ", finished: " << Buffer.Finished; - return description; - } - - ~TImpl() { - with_lock (Alloc) { - ParsedValues.clear(); - Columns.clear(); - TypeEnv.reset(); - } - } - -private: - void FlushColumns(size_t parsedRows, size_t savedRows) { - const ui64 firstOffset = Buffer.Offsets.front(); - for (const auto& column : Columns) { - column.ValidateNumberValues(savedRows, firstOffset); - } - - { - auto unguard = Unguard(Alloc); - ParseCallback(parsedRows - savedRows, savedRows, ParsedValues); - } - - ClearColumns(); - } - - void ClearColumns() { - for (size_t i = 0; i < Columns.size(); ++i) { - auto& parsedColumn = ParsedValues[i]; - for (size_t rowId : Columns[i].ParsedRows) { - auto& parsedRow = parsedColumn[rowId]; - parsedRow.UnlockRef(1); - parsedRow.Clear(); - } - Columns[i].ParsedRows.clear(); - } - } - -private: - NKikimr::NMiniKQL::TScopedAlloc Alloc; - std::unique_ptr TypeEnv; - - const ui64 BatchSize; - const ui64 MaxNumberRows; - const TDuration BatchCreationTimeout; - const TCallback ParseCallback; - TVector Columns; - absl::flat_hash_map ColumnsIndex; - - TJsonParserBuffer Buffer; - simdjson::ondemand::parser Parser; - - TVector> ParsedValues; -}; - -TJsonParser::TJsonParser(const TVector& columns, const TVector& types, TCallback parseCallback, ui64 batchSize, TDuration batchCreationTimeout, ui64 bufferCellCount) - : Impl(std::make_unique(columns, types, parseCallback, batchSize, batchCreationTimeout, bufferCellCount)) -{} - -TJsonParser::~TJsonParser() { -} - -void TJsonParser::AddMessages(const TVector& messages) { - Impl->AddMessages(messages); -} - -bool TJsonParser::IsReady() const { - return Impl->IsReady(); -} - -TInstant TJsonParser::GetCreationDeadline() const { - return Impl->GetCreationDeadline(); -} - -size_t TJsonParser::GetNumberValues() const { - return Impl->GetNumberValues(); -} - -const TVector& TJsonParser::GetOffsets() const { - return Impl->GetOffsets(); -} - -void TJsonParser::Parse() { - Impl->Parse(); -} - -TString TJsonParser::GetDescription() const { - return Impl->GetDescription(); -} - -std::unique_ptr NewJsonParser(const TVector& columns, const TVector& types, TJsonParser::TCallback parseCallback, ui64 batchSize, TDuration batchCreationTimeout, ui64 bufferCellCount) { - return std::unique_ptr(new TJsonParser(columns, types, parseCallback, batchSize, batchCreationTimeout, bufferCellCount)); -} - -} // namespace NFq diff --git a/ydb/core/fq/libs/row_dispatcher/json_parser.h b/ydb/core/fq/libs/row_dispatcher/json_parser.h deleted file mode 100644 index e96a8f752599..000000000000 --- a/ydb/core/fq/libs/row_dispatcher/json_parser.h +++ /dev/null @@ -1,50 +0,0 @@ -#pragma once - -#include - -#include - -namespace NFq { - -class TJsonParserError: public yexception { -public: - TJsonParserError() = default; - TJsonParserError(const std::string& fieldName) - : FieldName(fieldName) - {} - - TMaybe GetField() const noexcept { - return FieldName; - } - -private: - TMaybe FieldName; -}; - - -class TJsonParser { -public: - using TCallback = std::function>& parsedValues)>; - -public: - TJsonParser(const TVector& columns, const TVector& types, TCallback parseCallback, ui64 batchSize, TDuration batchCreationTimeout, ui64 bufferCellCount); - ~TJsonParser(); - - bool IsReady() const; - TInstant GetCreationDeadline() const; - size_t GetNumberValues() const; - const TVector& GetOffsets() const; - - void AddMessages(const TVector& messages); - void Parse(); - - TString GetDescription() const; - -private: - class TImpl; - const std::unique_ptr Impl; -}; - -std::unique_ptr NewJsonParser(const TVector& columns, const TVector& types, TJsonParser::TCallback parseCallback, ui64 batchSize, TDuration batchCreationTimeout, ui64 bufferCellCount); - -} // namespace NFq diff --git a/ydb/core/fq/libs/row_dispatcher/protos/events.proto b/ydb/core/fq/libs/row_dispatcher/protos/events.proto index 8a981fd7c691..819c0f9264df 100644 --- a/ydb/core/fq/libs/row_dispatcher/protos/events.proto +++ b/ydb/core/fq/libs/row_dispatcher/protos/events.proto @@ -6,6 +6,8 @@ option cc_enable_arenas = true; import "ydb/library/actors/protos/actors.proto"; import "ydb/library/yql/providers/pq/proto/dq_io.proto"; import "ydb/library/yql/dq/actors/protos/dq_events.proto"; +import "ydb/library/yql/dq/actors/protos/dq_status_codes.proto"; +import "ydb/public/api/protos/ydb_issue_message.proto"; message TEvGetAddressRequest { NYql.NPq.NProto.TDqPqTopicSource Source = 1; @@ -53,8 +55,10 @@ message TEvStopSession { } message TEvMessage { - string Json = 1; - uint64 Offset = 2; + reserved 1; + reserved 2; + uint32 PayloadId = 3; + repeated uint64 Offsets = 4; } message TEvMessageBatch { @@ -72,8 +76,10 @@ message TEvStatistics { } message TEvSessionError { - string Message = 1; + reserved 1; uint32 PartitionId = 2; + NYql.NDqProto.StatusIds.StatusCode StatusCode = 3; + repeated Ydb.Issue.IssueMessage Issues = 4; optional NYql.NDqProto.TMessageTransportMeta TransportMeta = 100; } diff --git a/ydb/core/fq/libs/row_dispatcher/protos/ya.make b/ydb/core/fq/libs/row_dispatcher/protos/ya.make index c2d06e232661..71a7967d22e8 100644 --- a/ydb/core/fq/libs/row_dispatcher/protos/ya.make +++ b/ydb/core/fq/libs/row_dispatcher/protos/ya.make @@ -8,6 +8,7 @@ PEERDIR( ydb/library/actors/protos ydb/library/yql/dq/actors/protos ydb/library/yql/providers/pq/proto + ydb/public/api/protos ) EXCLUDE_TAGS(GO_PROTO) diff --git a/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/compile_service.cpp b/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/compile_service.cpp index 02b29c005883..57e410f5ebb9 100644 --- a/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/compile_service.cpp +++ b/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/compile_service.cpp @@ -1,6 +1,8 @@ #include "compile_service.h" +#include #include +#include #include @@ -16,6 +18,7 @@ class TPurecalcCompileService : public NActors::TActor public: TPurecalcCompileService() : TBase(&TPurecalcCompileService::StateFunc) + , LogPrefix("TPurecalcCompileService: ") {} STRICT_STFUNC(StateFunc, @@ -23,20 +26,25 @@ class TPurecalcCompileService : public NActors::TActor ) void Handle(TEvRowDispatcher::TEvPurecalcCompileRequest::TPtr& ev) { + LOG_ROW_DISPATCHER_TRACE("Got compile request with id: " << ev->Cookie); IProgramHolder::TPtr programHolder = std::move(ev->Get()->ProgramHolder); - TString error; + TStatus status = TStatus::Success(); try { programHolder->CreateProgram(GetOrCreateFactory(ev->Get()->Settings)); - } catch (const NYql::NPureCalc::TCompileError& e) { - error = TStringBuilder() << "Failed to compile purecalc filter: sql: " << e.GetYql() << ", error: " << e.GetIssues(); + } catch (const NYql::NPureCalc::TCompileError& error) { + status = TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Compile issues: " << error.GetIssues()) + .AddIssue(TStringBuilder() << "Final yql: " << error.GetYql()) + .AddParentIssue(TStringBuilder() << "Failed to compile purecalc program"); } catch (...) { - error = TStringBuilder() << "Failed to compile purecalc filter, unexpected exception: " << CurrentExceptionMessage(); + status = TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Failed to compile purecalc program, got unexpected exception: " << CurrentExceptionMessage()); } - if (error) { - Send(ev->Sender, new TEvRowDispatcher::TEvPurecalcCompileResponse(error), 0, ev->Cookie); + if (status.IsFail()) { + LOG_ROW_DISPATCHER_ERROR("Compilation failed for request with id: " << ev->Cookie); + Send(ev->Sender, new TEvRowDispatcher::TEvPurecalcCompileResponse(status.GetStatus(), status.GetErrorDescription()), 0, ev->Cookie); } else { + LOG_ROW_DISPATCHER_TRACE("Compilation completed for request with id: " << ev->Cookie); Send(ev->Sender, new TEvRowDispatcher::TEvPurecalcCompileResponse(std::move(programHolder)), 0, ev->Cookie); } } @@ -54,6 +62,8 @@ class TPurecalcCompileService : public NActors::TActor } private: + const TString LogPrefix; + std::map ProgramFactories; }; diff --git a/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/ya.make b/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/ya.make index af6747855195..a1508a8591b2 100644 --- a/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/ya.make +++ b/ydb/core/fq/libs/row_dispatcher/purecalc_compilation/ya.make @@ -5,7 +5,9 @@ SRCS( ) PEERDIR( + ydb/core/fq/libs/actors/logging ydb/core/fq/libs/row_dispatcher/events + ydb/core/fq/libs/row_dispatcher/format_handler/common ydb/core/fq/libs/row_dispatcher/purecalc_no_pg_wrapper ydb/library/actors/core diff --git a/ydb/core/fq/libs/row_dispatcher/row_dispatcher.cpp b/ydb/core/fq/libs/row_dispatcher/row_dispatcher.cpp index 03d01620da5b..0293badd5a01 100644 --- a/ydb/core/fq/libs/row_dispatcher/row_dispatcher.cpp +++ b/ydb/core/fq/libs/row_dispatcher/row_dispatcher.cpp @@ -99,7 +99,7 @@ struct TAggQueryStat { NYql::TCounters::TEntry ReadLagMessages; bool IsWaiting = false; - void Add(const TopicSessionClientStatistic& stat) { + void Add(const TTopicSessionClientStatistic& stat) { ReadBytes.Add(NYql::TCounters::TEntry(stat.ReadBytes)); UnreadBytes.Add(NYql::TCounters::TEntry(stat.UnreadBytes)); UnreadRows.Add(NYql::TCounters::TEntry(stat.UnreadRows)); @@ -324,13 +324,13 @@ class TRowDispatcher : public TActorBootstrapped { ConsumerCounters Counters; bool PendingGetNextBatch = false; bool PendingNewDataArrived = false; - TopicSessionClientStatistic Stat; + TTopicSessionClientStatistic Stat; ui64 Generation; }; struct SessionInfo { TMap> Consumers; // key - ReadActor actor id - TopicSessionCommonStatistic Stat; // Increments + TTopicSessionCommonStatistic Stat; // Increments NYql::TCounters::TEntry AggrReadBytes; }; @@ -667,11 +667,16 @@ TString TRowDispatcher::GetInternalState() { for (auto& [actorId, sessionInfo] : sessionsInfo.Sessions) { str << " / " << LeftPad(actorId, 32) << " data rate " << toHumanDR(sessionInfo.AggrReadBytes.Sum) << " unread bytes " << toHuman(sessionInfo.Stat.UnreadBytes) - << " offset " << LeftPad(sessionInfo.Stat.LastReadedOffset, 12) << " restarts by offsets " << sessionInfo.Stat.RestartSessionByOffsets - << " parse and filter lantecy " << sessionInfo.Stat.ParseAndFilterLatency << "\n"; + << " offset " << LeftPad(sessionInfo.Stat.LastReadedOffset, 12) << " restarts by offsets " << sessionInfo.Stat.RestartSessionByOffsets << "\n"; ui64 maxInitialOffset = 0; ui64 minInitialOffset = std::numeric_limits::max(); + for (const auto& [formatName, formatStats] : sessionInfo.Stat.FormatHandlers) { + str << " " << formatName + << " parse and filter lantecy " << formatStats.ParseAndFilterLatency + << " (parse " << formatStats.ParserStats.ParserLatency << ", filter " << formatStats.FilterStats.FilterLatency << ")\n"; + } + for (auto& [readActorId, consumer] : sessionInfo.Consumers) { str << " " << consumer->QueryId << " " << LeftPad(readActorId, 32) << " unread bytes " << toHuman(consumer->Stat.UnreadBytes) << " (" << leftPad(consumer->Stat.UnreadRows) << " rows) " diff --git a/ydb/core/fq/libs/row_dispatcher/topic_session.cpp b/ydb/core/fq/libs/row_dispatcher/topic_session.cpp index 4fec6e7af0d1..a9642647b9dc 100644 --- a/ydb/core/fq/libs/row_dispatcher/topic_session.cpp +++ b/ydb/core/fq/libs/row_dispatcher/topic_session.cpp @@ -3,26 +3,21 @@ #include "common.h" #include - #include -#include +#include + #include #include -#include -#include -#include +#include + #include -#include #include -#include -#include -#include - namespace NFq { using namespace NActors; +using namespace NRowDispatcher; namespace { @@ -30,8 +25,9 @@ namespace { struct TTopicSessionMetrics { void Init(const ::NMonitoring::TDynamicCounterPtr& counters, const TString& topicPath, ui32 partitionId) { - TopicGroup = counters->GetSubgroup("topic", CleanupCounterValueString(topicPath)); + AllSessionsDataRate = counters->GetCounter("AllSessionsDataRate", true); + PartitionGroup = TopicGroup->GetSubgroup("partition", ToString(partitionId)); InFlyAsyncInputData = PartitionGroup->GetCounter("InFlyAsyncInputData"); InFlySubscribe = PartitionGroup->GetCounter("InFlySubscribe"); @@ -39,8 +35,6 @@ struct TTopicSessionMetrics { RestartSessionByOffsets = counters->GetCounter("RestartSessionByOffsets", true); SessionDataRate = PartitionGroup->GetCounter("SessionDataRate", true); WaitEventTimeMs = PartitionGroup->GetHistogram("WaitEventTimeMs", NMonitoring::ExponentialHistogram(13, 2, 1)); // ~ 1ms -> ~ 8s - AllSessionsDataRate = counters->GetCounter("AllSessionsDataRate", true); - InFlightCompileRequests = PartitionGroup->GetCounter("InFlightCompileRequests"); } ::NMonitoring::TDynamicCounterPtr TopicGroup; @@ -50,66 +44,38 @@ struct TTopicSessionMetrics { ::NMonitoring::TDynamicCounters::TCounterPtr ReconnectRate; ::NMonitoring::TDynamicCounters::TCounterPtr RestartSessionByOffsets; ::NMonitoring::TDynamicCounters::TCounterPtr SessionDataRate; - ::NMonitoring::TDynamicCounters::TCounterPtr InFlightCompileRequests; ::NMonitoring::THistogramPtr WaitEventTimeMs; ::NMonitoring::TDynamicCounters::TCounterPtr AllSessionsDataRate; - }; struct TEvPrivate { // Event ids enum EEv : ui32 { - EvBegin = EventSpaceBegin(NActors::TEvents::ES_PRIVATE), + EvBegin = EventSpaceBegin(TEvents::ES_PRIVATE), EvPqEventsReady = EvBegin + 10, EvCreateSession, EvSendStatisticToReadActor, - EvDataAfterFilteration, - EvDataFiltered, EvSendStatisticToRowDispatcher, - EvStartParsing, EvReconnectSession, EvEnd }; - static_assert(EvEnd < EventSpaceEnd(NActors::TEvents::ES_PRIVATE), "expect EvEnd < EventSpaceEnd(NActors::TEvents::ES_PRIVATE)"); + static_assert(EvEnd < EventSpaceEnd(TEvents::ES_PRIVATE), "expect EvEnd < EventSpaceEnd(TEvents::ES_PRIVATE)"); // Events - struct TEvPqEventsReady : public NActors::TEventLocal {}; - struct TEvCreateSession : public NActors::TEventLocal {}; - struct TEvSendStatisticToRowDispatcher : public NActors::TEventLocal {}; - struct TEvSendStatisticToReadActor : public NActors::TEventLocal {}; - struct TEvStartParsing : public NActors::TEventLocal {}; - struct TEvReconnectSession : public NActors::TEventLocal {}; - - struct TEvDataFiltered : public NActors::TEventLocal { - explicit TEvDataFiltered(ui64 offset) - : Offset(offset) - {} - const ui64 Offset; - }; - - struct TEvDataAfterFilteration : public NActors::TEventLocal { - TEvDataAfterFilteration(ui64 offset, const TString& json, TActorId readActorId) - : Offset(offset) - , Json(json) - , ReadActorId(readActorId) { } - ui64 Offset; - TString Json; - TActorId ReadActorId; - }; + struct TEvPqEventsReady : public TEventLocal {}; + struct TEvCreateSession : public TEventLocal {}; + struct TEvSendStatisticToRowDispatcher : public TEventLocal {}; + struct TEvSendStatisticToReadActor : public TEventLocal {}; + struct TEvReconnectSession : public TEventLocal {}; }; -ui64 SendStatisticPeriodSec = 2; -ui64 MaxBatchSizeBytes = 10000000; -ui64 MaxHandledEventsCount = 1000; -ui64 MaxHandledEventsSize = 1000000; - -TVector GetVector(const google::protobuf::RepeatedPtrField& value) { - return {value.begin(), value.end()}; -} +constexpr ui64 SendStatisticPeriodSec = 2; +constexpr ui64 MaxHandledEventsCount = 1000; +constexpr ui64 MaxHandledEventsSize = 1000000; class TTopicSession : public TActorBootstrapped { private: - using TParserInputType = TSet>; + using TBase = TActorBootstrapped; struct TStats { void Add(ui64 dataSize, ui64 events) { @@ -119,18 +85,19 @@ class TTopicSession : public TActorBootstrapped { void Clear() { Bytes = 0; Events = 0; - ParseAndFilterLatency = TDuration::Zero(); } ui64 Bytes = 0; ui64 Events = 0; - TDuration ParseAndFilterLatency; }; - struct TClientsInfo { - TClientsInfo( - const NFq::TEvRowDispatcher::TEvStartSession::TPtr& ev, - NMonitoring::TDynamicCounterPtr& counters) - : Settings(ev->Get()->Record) + struct TClientsInfo : public IClientDataConsumer { + using TPtr = TIntrusivePtr; + + TClientsInfo(TTopicSession& self, const TString& logPrefix, const ITopicFormatHandler::TSettings& handlerSettings, const NFq::TEvRowDispatcher::TEvStartSession::TPtr& ev, NMonitoring::TDynamicCounterPtr& counters) + : Self(self) + , LogPrefix(logPrefix) + , HandlerSettings(handlerSettings) + , Settings(ev->Get()->Record) , ReadActorId(ev->Sender) , FilteredDataRate(counters->GetCounter("FilteredDataRate", true)) , RestartSessionByOffsetsByQuery(counters->GetCounter("RestartSessionByOffsetsByQuery", true)) @@ -141,21 +108,88 @@ class TTopicSession : public TActorBootstrapped { } Y_UNUSED(TDuration::TryParse(Settings.GetSource().GetReconnectPeriod(), ReconnectPeriod)); } - NFq::NRowDispatcherProto::TEvStartSession Settings; - NActors::TActorId ReadActorId; - std::unique_ptr Filter; // empty if no predicate - ui64 InFlightCompilationId = 0; - TQueue> Buffer; + + TActorId GetClientId() const override { + return ReadActorId; + } + + TMaybe GetNextMessageOffset() const override { + return NextMessageOffset; + } + + TVector GetColumns() const override { + const auto& source = Settings.GetSource(); + Y_ENSURE(source.ColumnsSize() == source.ColumnTypesSize(), "Columns size and types size should be equal, but got " << source.ColumnsSize() << " columns and " << source.ColumnTypesSize() << " types"); + + TVector Columns; + Columns.reserve(source.ColumnsSize()); + for (ui64 i = 0; i < source.ColumnsSize(); ++i) { + Columns.emplace_back(TSchemaColumn{.Name = source.GetColumns().Get(i), .TypeYson = source.GetColumnTypes().Get(i)}); + } + + return Columns; + } + + const TString& GetWhereFilter() const override { + return Settings.GetSource().GetPredicate(); + } + + TPurecalcCompileSettings GetPurecalcSettings() const override { + return {.EnabledLLVM = Settings.GetSource().GetEnabledLLVM()}; + } + + void OnClientError(TStatus status) override { + Self.SendSessionError(ReadActorId, status); + } + + void StartClientSession() override { + Self.StartClientSession(*this); + } + + void AddDataToClient(ui64 offset, ui64 rowSize) override { + Y_ENSURE(!NextMessageOffset || offset >= *NextMessageOffset, "Unexpected historical offset"); + + LOG_ROW_DISPATCHER_TRACE("AddDataToClient to " << ReadActorId << ", offset: " << offset << ", serialized size: " << rowSize); + + NextMessageOffset = offset + 1; + UnreadRows++; + UnreadBytes += rowSize; + Self.UnreadBytes += rowSize; + Self.SendDataArrived(*this); + } + + void UpdateClientOffset(ui64 offset) override { + LOG_ROW_DISPATCHER_TRACE("UpdateClientOffset for " << ReadActorId << ", new offset: " << offset); + if (!NextMessageOffset || *NextMessageOffset < offset + 1) { + NextMessageOffset = offset + 1; + } + if (!UnreadRows) { + if (!ProcessedNextMessageOffset || *ProcessedNextMessageOffset < offset + 1) { + ProcessedNextMessageOffset = offset + 1; + } + } + } + + // Settings + TTopicSession& Self; + const TString& LogPrefix; + const ITopicFormatHandler::TSettings HandlerSettings; + const NFq::NRowDispatcherProto::TEvStartSession Settings; + const TActorId ReadActorId; + TDuration ReconnectPeriod; + + // State + ui64 UnreadRows = 0; ui64 UnreadBytes = 0; bool DataArrivedSent = false; TMaybe NextMessageOffset; // offset to restart topic session TMaybe ProcessedNextMessageOffset; // offset of fully processed data (to save to checkpoint) - TVector FieldsIds; - TDuration ReconnectPeriod; + + // Metrics + ui64 InitialOffset = 0; TStats Stat; // Send (filtered) to read_actor NMonitoring::TDynamicCounters::TCounterPtr FilteredDataRate; // filtered NMonitoring::TDynamicCounters::TCounterPtr RestartSessionByOffsetsByQuery; - ui64 InitialOffset = 0; }; struct TTopicEventProcessor { @@ -170,59 +204,46 @@ class TTopicSession : public TActorBootstrapped { TTopicSession& Self; const TString& LogPrefix; - ui64& dataReceivedEventSize; - }; - - struct TParserSchema { - TVector FieldsMap; // index - FieldId (from FieldsIndexes), value - parsing schema offset - TParserInputType InputType; - }; - - struct TFieldDescription { - ui64 IndexInParserSchema = 0; - TString Type; + ui64& DataReceivedEventSize; }; - struct TFiltersCompileState { - ui64 FreeId = 0; - std::unordered_map InFlightCompilations = {}; - }; - - bool InflightReconnect = false; - TDuration ReconnectPeriod; + // Settings const TString ReadGroup; const TString TopicPath; const TString TopicPathPartition; const TString Endpoint; const TString Database; - NActors::TActorId RowDispatcherActorId; - NActors::TActorId CompileServiceActorId; - ui32 PartitionId; - NYdb::TDriver Driver; - std::shared_ptr CredentialsProviderFactory; - NYql::ITopicClient::TPtr TopicClient; - std::shared_ptr ReadSession; + const TActorId RowDispatcherActorId; + const ui32 PartitionId; + const NYdb::TDriver Driver; + const NYql::IPqGateway::TPtr PqGateway; + const std::shared_ptr CredentialsProviderFactory; + const NConfig::TRowDispatcherConfig Config; + const TFormatHandlerConfig FormatHandlerConfig; const i64 BufferSize; TString LogPrefix; - TStats SessionStats; - TStats ClientsStats; + + // State + bool InflightReconnect = false; + TDuration ReconnectPeriod; + + NYql::ITopicClient::TPtr TopicClient; + std::shared_ptr ReadSession; + std::map FormatHandlers; + std::unordered_map Clients; + ui64 LastMessageOffset = 0; bool IsWaitingEvents = false; - bool IsStartParsingScheduled = false; - THashMap Clients; - THashSet ClientsWithoutPredicate; - std::unique_ptr Parser; - TFiltersCompileState FiltersCompilation; - NConfig::TRowDispatcherConfig Config; ui64 UnreadBytes = 0; - const ::NMonitoring::TDynamicCounterPtr Counters; - TTopicSessionMetrics Metrics; - TParserSchema ParserSchema; - THashMap FieldsIndexes; - NYql::IPqGateway::TPtr PqGateway; TMaybe ConsumerName; - ui64 RestartSessionByOffsets = 0; + + // Metrics TInstant WaitEventStartedAt; + ui64 RestartSessionByOffsets = 0; + TStats SessionStats; + TStats ClientsStats; + TTopicSessionMetrics Metrics; + const ::NMonitoring::TDynamicCounterPtr Counters; public: explicit TTopicSession( @@ -231,8 +252,8 @@ class TTopicSession : public TActorBootstrapped { const TString& endpoint, const TString& database, const NConfig::TRowDispatcherConfig& config, - NActors::TActorId rowDispatcherActorId, - NActors::TActorId compileServiceActorId, + TActorId rowDispatcherActorId, + TActorId compileServiceActorId, ui32 partitionId, NYdb::TDriver driver, std::shared_ptr credentialsProviderFactory, @@ -253,73 +274,53 @@ class TTopicSession : public TActorBootstrapped { void CloseTopicSession(); void SubscribeOnNextEvent(); void SendToParsing(const TVector& messages); - void DoParsing(bool force = false); - void DoFiltering(ui64 rowsOffset, ui64 numberRows, const TVector>& parsedValues); void SendData(TClientsInfo& info); - void UpdateParser(); - void FatalError(const TString& message, const std::unique_ptr* filter, bool addParserDescription, const TMaybe& fieldName); + void FatalError(TStatus status); void SendDataArrived(TClientsInfo& client); void StopReadSession(); TString GetSessionId() const; void HandleNewEvents(); TInstant GetMinStartingMessageTimestamp() const; - void AddDataToClient(TClientsInfo& client, ui64 offset, const TString& json); void StartClientSession(TClientsInfo& info); - std::pair CreateItem(const NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage& message); - void Handle(NFq::TEvPrivate::TEvPqEventsReady::TPtr&); void Handle(NFq::TEvPrivate::TEvCreateSession::TPtr&); void Handle(NFq::TEvPrivate::TEvReconnectSession::TPtr&); - void Handle(NFq::TEvPrivate::TEvDataAfterFilteration::TPtr&); void Handle(NFq::TEvPrivate::TEvSendStatisticToReadActor::TPtr&); - void Handle(NFq::TEvPrivate::TEvDataFiltered::TPtr&); void Handle(NFq::TEvPrivate::TEvSendStatisticToRowDispatcher::TPtr&); void Handle(TEvRowDispatcher::TEvGetNextBatch::TPtr&); void Handle(NFq::TEvRowDispatcher::TEvStopSession::TPtr& ev); void Handle(NFq::TEvRowDispatcher::TEvStartSession::TPtr& ev); - void Handle(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr& ev); void HandleException(const std::exception& err); void SendStatisticToRowDispatcher(); - void SendSessionError(NActors::TActorId readActorId, const TString& message); - TVector*> RebuildJson(const TClientsInfo& info, const TVector>& parsedValues); - void UpdateParserSchema(const TParserInputType& inputType); - void UpdateFieldsIds(TClientsInfo& clientInfo); + void SendSessionError(TActorId readActorId, TStatus status); bool CheckNewClient(NFq::TEvRowDispatcher::TEvStartSession::TPtr& ev); - TString GetAnyQueryIdByFieldName(const TString& fieldName); private: STRICT_STFUNC_EXC(StateFunc, hFunc(NFq::TEvPrivate::TEvPqEventsReady, Handle); hFunc(NFq::TEvPrivate::TEvCreateSession, Handle); - hFunc(NFq::TEvPrivate::TEvDataAfterFilteration, Handle); hFunc(NFq::TEvPrivate::TEvSendStatisticToReadActor, Handle); - hFunc(NFq::TEvPrivate::TEvDataFiltered, Handle); hFunc(NFq::TEvPrivate::TEvSendStatisticToRowDispatcher, Handle); hFunc(NFq::TEvPrivate::TEvReconnectSession, Handle); hFunc(TEvRowDispatcher::TEvGetNextBatch, Handle); hFunc(NFq::TEvRowDispatcher::TEvStartSession, Handle); - hFunc(TEvRowDispatcher::TEvPurecalcCompileResponse, Handle); - sFunc(NFq::TEvPrivate::TEvStartParsing, DoParsing); - cFunc(NActors::TEvents::TEvPoisonPill::EventType, PassAway); + cFunc(TEvents::TEvPoisonPill::EventType, PassAway); hFunc(NFq::TEvRowDispatcher::TEvStopSession, Handle);, ExceptionFunc(std::exception, HandleException) ) STRICT_STFUNC(ErrorState, { - cFunc(NActors::TEvents::TEvPoisonPill::EventType, PassAway); + cFunc(TEvents::TEvPoisonPill::EventType, PassAway); IgnoreFunc(NFq::TEvPrivate::TEvPqEventsReady); IgnoreFunc(NFq::TEvPrivate::TEvCreateSession); - IgnoreFunc(NFq::TEvPrivate::TEvDataAfterFilteration); IgnoreFunc(NFq::TEvPrivate::TEvSendStatisticToReadActor); - IgnoreFunc(NFq::TEvPrivate::TEvDataFiltered); IgnoreFunc(TEvRowDispatcher::TEvGetNextBatch); IgnoreFunc(NFq::TEvRowDispatcher::TEvStartSession); IgnoreFunc(NFq::TEvRowDispatcher::TEvStopSession); IgnoreFunc(NFq::TEvPrivate::TEvSendStatisticToRowDispatcher); - IgnoreFunc(TEvRowDispatcher::TEvPurecalcCompileResponse); }) }; @@ -329,8 +330,8 @@ TTopicSession::TTopicSession( const TString& endpoint, const TString& database, const NConfig::TRowDispatcherConfig& config, - NActors::TActorId rowDispatcherActorId, - NActors::TActorId compileServiceActorId, + TActorId rowDispatcherActorId, + TActorId compileServiceActorId, ui32 partitionId, NYdb::TDriver driver, std::shared_ptr credentialsProviderFactory, @@ -343,17 +344,16 @@ TTopicSession::TTopicSession( , Endpoint(endpoint) , Database(database) , RowDispatcherActorId(rowDispatcherActorId) - , CompileServiceActorId(compileServiceActorId) , PartitionId(partitionId) , Driver(std::move(driver)) + , PqGateway(pqGateway) , CredentialsProviderFactory(credentialsProviderFactory) + , Config(config) + , FormatHandlerConfig(CreateFormatHandlerConfig(config, compileServiceActorId)) , BufferSize(maxBufferSize) , LogPrefix("TopicSession") - , Config(config) , Counters(counters) - , PqGateway(pqGateway) -{ -} +{} void TTopicSession::Bootstrap() { Become(&TTopicSession::StateFunc); @@ -369,7 +369,8 @@ void TTopicSession::Bootstrap() { void TTopicSession::PassAway() { LOG_ROW_DISPATCHER_DEBUG("PassAway"); StopReadSession(); - NActors::TActorBootstrapped::PassAway(); + FormatHandlers.clear(); + TBase::PassAway(); } void TTopicSession::SubscribeOnNextEvent() { @@ -385,7 +386,7 @@ void TTopicSession::SubscribeOnNextEvent() { LOG_ROW_DISPATCHER_TRACE("SubscribeOnNextEvent"); IsWaitingEvents = true; Metrics.InFlySubscribe->Inc(); - NActors::TActorSystem* actorSystem = NActors::TActivationContext::ActorSystem(); + TActorSystem* actorSystem = TActivationContext::ActorSystem(); WaitEventStartedAt = TInstant::Now(); ReadSession->WaitEvent().Subscribe([actorSystem, selfId = SelfId()](const auto&){ actorSystem->Send(selfId, new NFq::TEvPrivate::TEvPqEventsReady()); @@ -412,7 +413,7 @@ TInstant TTopicSession::GetMinStartingMessageTimestamp() const { auto result = TInstant::Max(); Y_ENSURE(!Clients.empty()); for (const auto& [actorId, info] : Clients) { - ui64 time = info.Settings.GetStartingMessageTimestampMs(); + ui64 time = info->Settings.GetStartingMessageTimestampMs(); result = std::min(result, TInstant::MilliSeconds(time)); } return result; @@ -446,17 +447,15 @@ void TTopicSession::CreateTopicSession() { } if (!ReadSession) { - UpdateParser(); - // Use any sourceParams. - const NYql::NPq::NProto::TDqPqTopicSource& sourceParams = Clients.begin()->second.Settings.GetSource(); + const NYql::NPq::NProto::TDqPqTopicSource& sourceParams = Clients.begin()->second->Settings.GetSource(); ReadSession = GetTopicClient(sourceParams).CreateReadSession(GetReadSessionSettings(sourceParams)); SubscribeOnNextEvent(); } - if (!InflightReconnect && Clients) { + if (!InflightReconnect && !Clients.empty()) { // Use any sourceParams. - ReconnectPeriod = Clients.begin()->second.ReconnectPeriod; + ReconnectPeriod = Clients.begin()->second->ReconnectPeriod; if (ReconnectPeriod != TDuration::Zero()) { LOG_ROW_DISPATCHER_INFO("ReconnectPeriod " << ReconnectPeriod.ToString()); Metrics.ReconnectRate->Inc(); @@ -480,35 +479,13 @@ void TTopicSession::Handle(NFq::TEvPrivate::TEvCreateSession::TPtr&) { CreateTopicSession(); } -TVector*> TTopicSession::RebuildJson(const TClientsInfo& info, const TVector>& parsedValues) { - TVector*> result; - const auto& offsets = ParserSchema.FieldsMap; - result.reserve(info.FieldsIds.size()); - for (auto fieldId : info.FieldsIds) { - Y_ENSURE(fieldId < offsets.size(), "fieldId " << fieldId << ", offsets.size() " << offsets.size()); - auto offset = offsets[fieldId]; - Y_ENSURE(offset < parsedValues.size(), "offset " << offset << ", jsonBatch.size() " << parsedValues.size()); - result.push_back(&parsedValues[offset]); - } - return result; -} - -void TTopicSession::Handle(NFq::TEvPrivate::TEvDataAfterFilteration::TPtr& ev) { - LOG_ROW_DISPATCHER_TRACE("TEvDataAfterFilteration, read actor id " << ev->Get()->ReadActorId.ToString() << ", " << ev->Get()->Json); - auto it = Clients.find(ev->Get()->ReadActorId); - if (it == Clients.end()) { - LOG_ROW_DISPATCHER_ERROR("Skip DataAfterFilteration, wrong read actor, id " << ev->Get()->ReadActorId.ToString()); - return; - } - AddDataToClient(it->second, ev->Get()->Offset, ev->Get()->Json); -} - void TTopicSession::Handle(NFq::TEvPrivate::TEvSendStatisticToReadActor::TPtr&) { LOG_ROW_DISPATCHER_TRACE("TEvSendStatisticToReadActor"); Schedule(TDuration::Seconds(Config.GetSendStatusPeriodSec()), new NFq::TEvPrivate::TEvSendStatisticToReadActor()); - + auto readBytes = ClientsStats.Bytes; - for (auto& [actorId, info] : Clients) { + for (auto& [actorId, infoPtr] : Clients) { + auto& info = *infoPtr; if (!info.ProcessedNextMessageOffset) { continue; } @@ -534,20 +511,6 @@ void TTopicSession::Handle(NFq::TEvPrivate::TEvReconnectSession::TPtr&) { Schedule(ReconnectPeriod, new NFq::TEvPrivate::TEvReconnectSession()); } -void TTopicSession::Handle(NFq::TEvPrivate::TEvDataFiltered::TPtr& ev) { - LOG_ROW_DISPATCHER_TRACE("TEvDataFiltered, last offset " << ev->Get()->Offset); - for (auto& [actorId, info] : Clients) { - if (!info.NextMessageOffset || *info.NextMessageOffset < ev->Get()->Offset + 1) { - info.NextMessageOffset = ev->Get()->Offset + 1; - } - if (info.Buffer.empty()) { - if (!info.ProcessedNextMessageOffset || *info.ProcessedNextMessageOffset < ev->Get()->Offset + 1) { - info.ProcessedNextMessageOffset = ev->Get()->Offset + 1; - } - } - } -} - void TTopicSession::Handle(TEvRowDispatcher::TEvGetNextBatch::TPtr& ev) { LOG_ROW_DISPATCHER_TRACE("TEvGetNextBatch from " << ev->Sender.ToString()); Metrics.InFlyAsyncInputData->Set(0); @@ -556,7 +519,7 @@ void TTopicSession::Handle(TEvRowDispatcher::TEvGetNextBatch::TPtr& ev) { LOG_ROW_DISPATCHER_ERROR("Wrong client, sender " << ev->Sender); return; } - SendData(it->second); + SendData(*it->second); SubscribeOnNextEvent(); } @@ -604,16 +567,18 @@ void TTopicSession::TTopicEventProcessor::operator()(NYdb::NTopic::TReadSessionE Self.ClientsStats.Add(dataSize, event.GetMessages().size()); Self.Metrics.SessionDataRate->Add(dataSize); Self.Metrics.AllSessionsDataRate->Add(dataSize); - dataReceivedEventSize += dataSize; + DataReceivedEventSize += dataSize; Self.SendToParsing(event.GetMessages()); } void TTopicSession::TTopicEventProcessor::operator()(NYdb::NTopic::TSessionClosedEvent& ev) { - TString message = TStringBuilder() << "Read session to topic \"" << Self.TopicPathPartition << "\" was closed: " << ev.DebugString(); - LOG_ROW_DISPATCHER_DEBUG(message); - NYql::TIssues issues; - issues.AddIssue(message); - Self.FatalError(issues.ToOneLineString(), nullptr, false, Nothing()); + const TString message = TStringBuilder() << "Read session to topic \"" << Self.TopicPathPartition << "\" was closed"; + LOG_ROW_DISPATCHER_DEBUG(message << ": " << ev.DebugString()); + + Self.FatalError(TStatus::Fail( + NYql::NDq::YdbStatusToDqStatus(static_cast(ev.GetStatus())), + ev.GetIssues() + ).AddParentIssue(message)); } void TTopicSession::TTopicEventProcessor::operator()(NYdb::NTopic::TReadSessionEvent::TStartPartitionSessionEvent& event) { @@ -621,10 +586,9 @@ void TTopicSession::TTopicEventProcessor::operator()(NYdb::NTopic::TReadSessionE TMaybe minOffset; for (const auto& [actorId, info] : Self.Clients) { - if (!minOffset - || (info.NextMessageOffset && (info.NextMessageOffset < *minOffset))) { - minOffset = info.NextMessageOffset; - } + if (!minOffset || (info->NextMessageOffset && *info->NextMessageOffset < *minOffset)) { + minOffset = info->NextMessageOffset; + } } LOG_ROW_DISPATCHER_DEBUG("Confirm StartPartitionSession with offset " << minOffset); event.Confirm(minOffset); @@ -643,105 +607,31 @@ void TTopicSession::TTopicEventProcessor::operator()(NYdb::NTopic::TReadSessionE LOG_ROW_DISPATCHER_WARN("TPartitionSessionClosedEvent"); } -std::pair TTopicSession::CreateItem(const NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage& message) { - const TString& data = message.GetData(); - i64 usedSpace = data.size(); - NYql::NUdf::TUnboxedValuePod item = NKikimr::NMiniKQL::MakeString(NYql::NUdf::TStringRef(data.data(), data.size())); - return std::make_pair(item, usedSpace); -} - TString TTopicSession::GetSessionId() const { return ReadSession ? ReadSession->GetSessionId() : TString{"empty"}; } void TTopicSession::SendToParsing(const TVector& messages) { - for (const auto& readActorId : ClientsWithoutPredicate) { - const auto it = Clients.find(readActorId); - Y_ENSURE(it != Clients.end(), "Internal error: unknown client"); - auto& info = it->second; - if (info.Filter) { - continue; - } - - for (const auto& message : messages) { - LOG_ROW_DISPATCHER_TRACE("Send message with offset " << message.GetOffset() << " to client " << info.ReadActorId <<" without parsing/filtering"); - AddDataToClient(info, message.GetOffset(), message.GetData()); + LOG_ROW_DISPATCHER_TRACE("SendToParsing, messages: " << messages.size()); + for (const auto& [_, formatHandler] : FormatHandlers) { + if (formatHandler->HasClients()) { + formatHandler->ParseMessages(messages); } } - - if (ClientsWithoutPredicate.size() == Clients.size()) { - return; - } - - Parser->AddMessages(messages); - DoParsing(); } -void TTopicSession::DoParsing(bool force) { - if (!Parser->IsReady() && !force) { - const TInstant batchCreationDeadline = Parser->GetCreationDeadline(); - LOG_ROW_DISPATCHER_TRACE("Collecting data to parse, skip parsing, creation deadline " << batchCreationDeadline); - if (!IsStartParsingScheduled && batchCreationDeadline) { - IsStartParsingScheduled = true; - Schedule(batchCreationDeadline, new TEvPrivate::TEvStartParsing()); - } - return; - } - - if (!Parser->GetNumberValues()) { - return; - } - - IsStartParsingScheduled = false; - LOG_ROW_DISPATCHER_TRACE("SendToParsing, first offset: " << Parser->GetOffsets().front() << ", number values in buffer " << Parser->GetOffsets().size()); - - TInstant startParseAndFilter = TInstant::Now(); - try { - Parser->Parse(); // this call processes parse and filter steps - } catch (const NFq::TJsonParserError& e) { - FatalError(e.what(), nullptr, true, e.GetField()); - } catch (const std::exception& e) { - FatalError(e.what(), nullptr, true, Nothing()); - } - auto parseAndFilterLatency = TInstant::Now() - startParseAndFilter; - SessionStats.ParseAndFilterLatency = Max(SessionStats.ParseAndFilterLatency, parseAndFilterLatency); - ClientsStats.ParseAndFilterLatency = Max(ClientsStats.ParseAndFilterLatency, parseAndFilterLatency); -} - -void TTopicSession::DoFiltering(ui64 rowsOffset, ui64 numberRows, const TVector>& parsedValues) { - const auto& offsets = Parser->GetOffsets(); - Y_ENSURE(rowsOffset < offsets.size(), "Invalid first row ofset"); - Y_ENSURE(numberRows, "Expected non empty parsed batch"); - Y_ENSURE(parsedValues, "Expected non empty schema"); - auto lastOffset = offsets[rowsOffset + numberRows - 1]; - LOG_ROW_DISPATCHER_TRACE("SendToFiltering, first offset: " << offsets[rowsOffset] << ", last offset: " << lastOffset); - - for (auto& [actorId, info] : Clients) { - if (info.InFlightCompilationId) { // filter compilation in flight - continue; - } - if (info.NextMessageOffset && lastOffset < info.NextMessageOffset) { // the batch has already been processed - continue; - } - try { - if (info.Filter) { - info.Filter->Push(offsets, RebuildJson(info, parsedValues), rowsOffset, numberRows); - } - } catch (const std::exception& e) { - FatalError(e.what(), &info.Filter, false, Nothing()); - } +void TTopicSession::SendData(TClientsInfo& info) { + TQueue>> buffer; + if (const auto formatIt = FormatHandlers.find(info.HandlerSettings); formatIt != FormatHandlers.end()) { + buffer = formatIt->second->ExtractClientData(info.GetClientId()); } - Send(SelfId(), new TEvPrivate::TEvDataFiltered(offsets.back())); -} - -void TTopicSession::SendData(TClientsInfo& info) { info.DataArrivedSent = false; - if (info.Buffer.empty()) { + if (buffer.empty()) { LOG_ROW_DISPATCHER_TRACE("Buffer empty"); } ui64 dataSize = 0; - ui64 eventsSize = info.Buffer.size(); + ui64 eventsSize = info.UnreadRows; if (!info.NextMessageOffset) { LOG_ROW_DISPATCHER_ERROR("Try SendData() without NextMessageOffset, " << info.ReadActorId @@ -755,58 +645,40 @@ void TTopicSession::SendData(TClientsInfo& info) { event->ReadActorId = info.ReadActorId; ui64 batchSize = 0; - while (!info.Buffer.empty()) { - const auto& [offset, json] = info.Buffer.front(); - info.UnreadBytes -= json.size(); - UnreadBytes -= json.size(); - batchSize += json.size(); + while (!buffer.empty()) { + auto [serializedData, offsets] = std::move(buffer.front()); + Y_ENSURE(!offsets.empty(), "Expected non empty message batch"); + buffer.pop(); + + batchSize += serializedData.GetSize(); + NFq::NRowDispatcherProto::TEvMessage message; - message.SetJson(json); - message.SetOffset(offset); - event->Record.AddMessages()->CopyFrom(message); - event->Record.SetNextMessageOffset(offset + 1); - info.Buffer.pop(); + message.SetPayloadId(event->AddPayload(std::move(serializedData))); + message.MutableOffsets()->Assign(offsets.begin(), offsets.end()); + event->Record.AddMessages()->CopyFrom(std::move(message)); + event->Record.SetNextMessageOffset(offsets.back() + 1); - if (batchSize > MaxBatchSizeBytes) { + if (batchSize > MAX_BATCH_SIZE) { break; } } dataSize += batchSize; - if (info.Buffer.empty()) { + if (buffer.empty()) { event->Record.SetNextMessageOffset(*info.NextMessageOffset); } LOG_ROW_DISPATCHER_TRACE("SendData to " << info.ReadActorId << ", batch size " << event->Record.MessagesSize()); Send(RowDispatcherActorId, event.release()); - } while(!info.Buffer.empty()); + } while(!buffer.empty()); + + UnreadBytes -= info.UnreadBytes; + info.UnreadRows = 0; + info.UnreadBytes = 0; + info.Stat.Add(dataSize, eventsSize); info.FilteredDataRate->Add(dataSize); info.ProcessedNextMessageOffset = *info.NextMessageOffset; } -void TTopicSession::UpdateFieldsIds(TClientsInfo& info) { - const auto& source = info.Settings.GetSource(); - for (size_t i = 0; i < source.ColumnsSize(); ++i) { - const auto& name = source.GetColumns().Get(i); - auto it = FieldsIndexes.find(name); - if (it == FieldsIndexes.end()) { - auto nextIndex = FieldsIndexes.size(); - info.FieldsIds.push_back(nextIndex); - FieldsIndexes[name] = {nextIndex, source.GetColumnTypes().Get(i)}; - } else { - info.FieldsIds.push_back(it->second.IndexInParserSchema); - } - } -} - -bool HasJsonColumns(const NYql::NPq::NProto::TDqPqTopicSource& sourceParams) { - for (const auto& type : sourceParams.GetColumnTypes()) { - if (type.Contains("Json")) { - return true; - } - } - return false; -} - void TTopicSession::StartClientSession(TClientsInfo& info) { if (ReadSession) { if (info.Settings.HasOffset() && info.Settings.GetOffset() <= LastMessageOffset) { @@ -818,88 +690,38 @@ void TTopicSession::StartClientSession(TClientsInfo& info) { } } - if (Parser) { - // Parse remains data before changing parsing schema - DoParsing(true); - } - UpdateParser(); - if (!ReadSession) { Schedule(TDuration::Seconds(Config.GetTimeoutBeforeStartSessionSec()), new NFq::TEvPrivate::TEvCreateSession()); } } void TTopicSession::Handle(NFq::TEvRowDispatcher::TEvStartSession::TPtr& ev) { - LOG_ROW_DISPATCHER_INFO("New client: read actor id " << ev->Sender.ToString() << ", predicate: " - << ev->Get()->Record.GetSource().GetPredicate() << ", offset: " << ev->Get()->Record.GetOffset()); + const auto& source = ev->Get()->Record.GetSource(); + LOG_ROW_DISPATCHER_INFO("New client: read actor id " << ev->Sender.ToString() << ", predicate: " << source.GetPredicate() << ", offset: " << ev->Get()->Record.GetOffset()); if (!CheckNewClient(ev)) { return; } - auto columns = GetVector(ev->Get()->Record.GetSource().GetColumns()); - auto types = GetVector(ev->Get()->Record.GetSource().GetColumnTypes()); + const TString& format = source.GetFormat(); + ITopicFormatHandler::TSettings handlerSettings = {.ParsingFormat = format ? format : "raw"}; - try { - if (Parser) { - // Parse remains data before adding new client - DoParsing(true); - } - auto queryGroup = Counters->GetSubgroup("query_id", ev->Get()->Record.GetQueryId()); - auto readGroup = queryGroup->GetSubgroup("read_group", CleanupCounterValueString(ReadGroup)); - auto& clientInfo = Clients.emplace( - std::piecewise_construct, - std::forward_as_tuple(ev->Sender), - std::forward_as_tuple(ev, readGroup)).first->second; - UpdateFieldsIds(clientInfo); - - const auto& source = clientInfo.Settings.GetSource(); - TString predicate = source.GetPredicate(); - - // TODO: remove this when the re-parsing is removed from pq read actor - if (predicate.empty() && HasJsonColumns(source)) { - predicate = "WHERE TRUE"; - } + auto queryGroup = Counters->GetSubgroup("query_id", ev->Get()->Record.GetQueryId()); + auto readGroup = queryGroup->GetSubgroup("read_group", CleanupCounterValueString(ReadGroup)); + auto clientInfo = Clients.insert({ev->Sender, MakeIntrusive(*this, LogPrefix, handlerSettings, ev, readGroup)}).first->second; - if (!predicate.empty()) { - clientInfo.Filter = NewJsonFilter( - columns, - types, - predicate, - [&, actorId = clientInfo.ReadActorId](ui64 offset, const TString& json){ - Send(SelfId(), new NFq::TEvPrivate::TEvDataAfterFilteration(offset, json, actorId)); - }, - {.EnabledLLVM = source.GetEnabledLLVM()} - ); - - clientInfo.InFlightCompilationId = ++FiltersCompilation.FreeId; - Y_ENSURE(FiltersCompilation.InFlightCompilations.emplace(clientInfo.InFlightCompilationId, ev->Sender).second, "Got duplicated compilation event id"); - LOG_ROW_DISPATCHER_TRACE("Send compile request with id " << clientInfo.InFlightCompilationId); - - Send(CompileServiceActorId, clientInfo.Filter->GetCompileRequest().release(), 0, clientInfo.InFlightCompilationId); - Metrics.InFlightCompileRequests->Inc(); - } else { - ClientsWithoutPredicate.insert(ev->Sender); - - // In case of in flight compilation topic session will be checked after getting compile response - StartClientSession(clientInfo); - } - } catch (...) { - FatalError("Adding new client failed, got unexpected exception: " + CurrentExceptionMessage(), nullptr, true, Nothing()); + auto formatIt = FormatHandlers.find(handlerSettings); + if (formatIt == FormatHandlers.end()) { + formatIt = FormatHandlers.insert({handlerSettings, CreateTopicFormatHandler(ActorContext(), FormatHandlerConfig, handlerSettings, Metrics.PartitionGroup)}).first; } - ConsumerName = ev->Get()->Record.GetSource().GetConsumerName(); - SendStatisticToRowDispatcher(); -} -void TTopicSession::AddDataToClient(TClientsInfo& info, ui64 offset, const TString& json) { - if (info.NextMessageOffset && offset < info.NextMessageOffset) { + if (auto status = formatIt->second->AddClient(clientInfo); status.IsFail()) { + SendSessionError(clientInfo->ReadActorId, status); return; } - info.NextMessageOffset = offset + 1; - info.Buffer.push(std::make_pair(offset, json)); - info.UnreadBytes += json.size(); - UnreadBytes += json.size(); - SendDataArrived(info); + + ConsumerName = source.GetConsumerName(); + SendStatisticToRowDispatcher(); } void TTopicSession::Handle(NFq::TEvRowDispatcher::TEvStopSession::TPtr& ev) { @@ -911,106 +733,38 @@ void TTopicSession::Handle(NFq::TEvRowDispatcher::TEvStopSession::TPtr& ev) { LOG_ROW_DISPATCHER_DEBUG("Wrong ClientSettings"); return; } - auto& info = it->second; + auto& info = *it->second; UnreadBytes -= info.UnreadBytes; + if (const auto formatIt = FormatHandlers.find(info.HandlerSettings); formatIt != FormatHandlers.end()) { + formatIt->second->RemoveClient(info.GetClientId()); + if (!formatIt->second->HasClients()) { + FormatHandlers.erase(formatIt); + } + } Clients.erase(it); - ClientsWithoutPredicate.erase(ev->Sender); if (Clients.empty()) { StopReadSession(); } - UpdateParser(); SubscribeOnNextEvent(); } -void CollectColumns(const NYql::NPq::NProto::TDqPqTopicSource& sourceParams, TSet>& columns) { - auto size = sourceParams.GetColumns().size(); - Y_ENSURE(size == sourceParams.GetColumnTypes().size()); - - for (int i = 0; i < size; ++i) { - auto name = sourceParams.GetColumns().Get(i); - auto type = sourceParams.GetColumnTypes().Get(i); - columns.emplace(name, type); - } -} - -void TTopicSession::UpdateParserSchema(const TParserInputType& inputType) { - ParserSchema.FieldsMap.clear(); - ParserSchema.FieldsMap.resize(FieldsIndexes.size()); - ui64 offset = 0; - for (const auto& [name, type]: inputType) { - Y_ENSURE(FieldsIndexes.contains(name)); - ui64 index = FieldsIndexes[name].IndexInParserSchema; - ParserSchema.FieldsMap[index] = offset++; - } - ParserSchema.InputType = inputType; -} - -void TTopicSession::UpdateParser() { - TSet> namesWithTypes; - for (auto& [readActorId, info] : Clients) { - CollectColumns(info.Settings.GetSource(), namesWithTypes); - } - - if (namesWithTypes == ParserSchema.InputType) { - return; - } - if (namesWithTypes.empty()) { - LOG_ROW_DISPATCHER_INFO("No columns to parse, reset parser"); - Parser.reset(); - return; - } - - try { - UpdateParserSchema(namesWithTypes); - - TVector names; - TVector types; - names.reserve(namesWithTypes.size()); - types.reserve(namesWithTypes.size()); - for (const auto& [name, type] : namesWithTypes) { - names.push_back(name); - types.push_back(type); - } - - LOG_ROW_DISPATCHER_TRACE("Init JsonParser with columns: " << JoinSeq(',', names)); - const auto& parserConfig = Config.GetJsonParser(); - Parser = NewJsonParser(names, types, [this](ui64 rowsOffset, ui64 numberRows, const TVector>& parsedValues) { - DoFiltering(rowsOffset, numberRows, parsedValues); - }, parserConfig.GetBatchSizeBytes(), TDuration::MilliSeconds(parserConfig.GetBatchCreationTimeoutMs()), parserConfig.GetBufferCellCount()); - } catch (const NFq::TJsonParserError& e) { - FatalError(e.what(), nullptr, true, e.GetField()); - } catch (const NYql::NPureCalc::TCompileError& e) { - FatalError(e.GetIssues(), nullptr, true, Nothing()); - } -} - -void TTopicSession::FatalError(const TString& message, const std::unique_ptr* filter, bool addParserDescription, const TMaybe& fieldName) { - TStringStream str; - str << message; - if (Parser && addParserDescription) { - str << ", parser description:\n" << Parser->GetDescription(); - } - if (filter) { - str << ", filter sql:\n" << (*filter)->GetSql(); - } - if (fieldName) { - auto queryId = GetAnyQueryIdByFieldName(*fieldName); - str << ", the field (" << *fieldName << ") has been added by query: " + queryId; - } - LOG_ROW_DISPATCHER_ERROR("FatalError: " << str.Str()); +void TTopicSession::FatalError(TStatus status) { + LOG_ROW_DISPATCHER_ERROR("FatalError: " << status.GetErrorMessage()); for (auto& [readActorId, info] : Clients) { LOG_ROW_DISPATCHER_DEBUG("Send TEvSessionError to " << readActorId); - SendSessionError(readActorId, str.Str()); + SendSessionError(readActorId, status); } StopReadSession(); Become(&TTopicSession::ErrorState); - ythrow yexception() << "FatalError: " << str.Str(); // To exit from current stack and call once PassAway() in HandleException(). + ythrow yexception() << "FatalError: " << status.GetErrorMessage(); // To exit from current stack and call once PassAway() in HandleException(). } -void TTopicSession::SendSessionError(NActors::TActorId readActorId, const TString& message) { +void TTopicSession::SendSessionError(TActorId readActorId, TStatus status) { + LOG_ROW_DISPATCHER_WARN("SendSessionError to " << readActorId << ", status: " << status.GetErrorMessage()); auto event = std::make_unique(); - event->Record.SetMessage(message); + event->Record.SetStatusCode(status.GetStatus()); + NYql::IssuesToMessage(status.GetErrorDescription(), event->Record.MutableIssues()); event->Record.SetPartitionId(PartitionId); event->ReadActorId = readActorId; Send(RowDispatcherActorId, event.release()); @@ -1026,7 +780,7 @@ void TTopicSession::StopReadSession() { } void TTopicSession::SendDataArrived(TClientsInfo& info) { - if (info.Buffer.empty() || info.DataArrivedSent) { + if (!info.UnreadBytes || info.DataArrivedSent) { return; } info.DataArrivedSent = true; @@ -1042,27 +796,27 @@ void TTopicSession::HandleException(const std::exception& e) { if (CurrentStateFunc() == &TThis::ErrorState) { return; } - FatalError(TString("Internal error: exception: ") + e.what(), nullptr, false, Nothing()); + FatalError(TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Session error, got unexpected exception: " << e.what())); } void TTopicSession::SendStatisticToRowDispatcher() { - TopicSessionStatistic sessionStatistic; + TTopicSessionStatistic sessionStatistic; auto& commonStatistic = sessionStatistic.Common; commonStatistic.UnreadBytes = UnreadBytes; commonStatistic.RestartSessionByOffsets = RestartSessionByOffsets; commonStatistic.ReadBytes = SessionStats.Bytes; commonStatistic.ReadEvents = SessionStats.Events; - commonStatistic.ParseAndFilterLatency = SessionStats.ParseAndFilterLatency; commonStatistic.LastReadedOffset = LastMessageOffset; SessionStats.Clear(); - sessionStatistic.SessionKey = TopicSessionParams{ReadGroup, Endpoint, Database, TopicPath, PartitionId}; + sessionStatistic.SessionKey = TTopicSessionParams{ReadGroup, Endpoint, Database, TopicPath, PartitionId}; sessionStatistic.Clients.reserve(Clients.size()); - for (auto& [readActorId, info] : Clients) { - TopicSessionClientStatistic clientStatistic; + for (const auto& [readActorId, infoPtr] : Clients) { + auto& info = *infoPtr; + TTopicSessionClientStatistic clientStatistic; clientStatistic.PartitionId = PartitionId; clientStatistic.ReadActorId = readActorId; - clientStatistic.UnreadRows = info.Buffer.size(); + clientStatistic.UnreadRows = info.UnreadRows; clientStatistic.UnreadBytes = info.UnreadBytes; clientStatistic.Offset = info.NextMessageOffset.GetOrElse(0); clientStatistic.ReadBytes = info.Stat.Bytes; @@ -1072,6 +826,12 @@ void TTopicSession::SendStatisticToRowDispatcher() { info.Stat.Clear(); sessionStatistic.Clients.emplace_back(std::move(clientStatistic)); } + + commonStatistic.FormatHandlers.reserve(FormatHandlers.size()); + for (const auto& [settings, handler] : FormatHandlers) { + commonStatistic.FormatHandlers.emplace(settings.ParsingFormat, handler->GetStatistics()); + } + auto event = std::make_unique(sessionStatistic); Send(RowDispatcherActorId, event.release()); } @@ -1085,102 +845,39 @@ bool TTopicSession::CheckNewClient(NFq::TEvRowDispatcher::TEvStartSession::TPtr& auto it = Clients.find(ev->Sender); if (it != Clients.end()) { LOG_ROW_DISPATCHER_ERROR("Such a client already exists"); - SendSessionError(ev->Sender, "Internal error: such a client already exists"); + SendSessionError(ev->Sender, TStatus::Fail(EStatusId::INTERNAL_ERROR, TStringBuilder() << "Client with id " << ev->Sender << " already exists")); return false; } const auto& source = ev->Get()->Record.GetSource(); if (!Config.GetWithoutConsumer() && ConsumerName && ConsumerName != source.GetConsumerName()) { LOG_ROW_DISPATCHER_INFO("Different consumer, expected " << ConsumerName << ", actual " << source.GetConsumerName() << ", send error"); - SendSessionError(ev->Sender, TStringBuilder() << "Use the same consumer in all queries via RD (current consumer " << ConsumerName << ")"); + SendSessionError(ev->Sender, TStatus::Fail(EStatusId::PRECONDITION_FAILED, TStringBuilder() << "Use the same consumer in all queries via RD (current consumer " << ConsumerName << ")")); return false; } - Y_ENSURE(source.ColumnsSize() == source.ColumnTypesSize()); - for (size_t i = 0; i < source.ColumnsSize(); ++i) { - const auto& name = source.GetColumns().Get(i); - const auto& type = source.GetColumnTypes().Get(i); - const auto it = FieldsIndexes.find(name); - if (it != FieldsIndexes.end() && it->second.Type != type) { - LOG_ROW_DISPATCHER_INFO("Different column `" << name << "` type, expected " << it->second.Type << ", actual " << type << ", send error"); - SendSessionError(ev->Sender, TStringBuilder() << "Use the same column type in all queries via RD, current type for column `" << name << "` is " << it->second.Type << " (requested type is " << type <<")"); - return false; - } - } - return true; } -void TTopicSession::Handle(TEvRowDispatcher::TEvPurecalcCompileResponse::TPtr& ev) { - LOG_ROW_DISPATCHER_TRACE("Got compile response for reauest with id " << ev->Cookie); - - const auto requestIt = FiltersCompilation.InFlightCompilations.find(ev->Cookie); - if (requestIt == FiltersCompilation.InFlightCompilations.end()) { - LOG_ROW_DISPATCHER_TRACE("Compile response ignored for id " << ev->Cookie); - return; - } - - const auto clientId = requestIt->second; - FiltersCompilation.InFlightCompilations.erase(requestIt); - Metrics.InFlightCompileRequests->Dec(); - - if (!ev->Get()->ProgramHolder) { - TString message = TStringBuilder() << "Filed to compile purecalc program, error: " << ev->Get()->Error; - LOG_ROW_DISPATCHER_ERROR(message); - FatalError(message, nullptr, false, Nothing()); - return; - } - - const auto clientIt = Clients.find(clientId); - if (clientIt == Clients.end()) { - LOG_ROW_DISPATCHER_TRACE("Compile response ignored for id " << ev->Cookie << ", client with id " << clientId << " not found"); - return; - } - - auto& clientInfo = clientIt->second; - if (ev->Cookie != clientInfo.InFlightCompilationId) { - LOG_ROW_DISPATCHER_TRACE("Outdated compiler response ignored for id " << ev->Cookie << ", client with id " << clientId << " changed"); - return; - } - - Y_ENSURE(clientInfo.Filter, "Unexpected completion response for client without filter"); - clientInfo.Filter->OnCompileResponse(std::move(ev)); - clientInfo.InFlightCompilationId = 0; - StartClientSession(clientInfo); -} - -TString TTopicSession::GetAnyQueryIdByFieldName(const TString& fieldName) { - TSet> namesWithTypes; - for (auto& [readActorId, info] : Clients) { - for (const auto& name : info.Settings.GetSource().GetColumns()) { - if (name != fieldName) { - continue; - } - return info.Settings.GetQueryId(); - } - } - return "Unknown"; -} - -} // namespace +} // anonymous namespace //////////////////////////////////////////////////////////////////////////////// - -std::unique_ptr NewTopicSession( + +std::unique_ptr NewTopicSession( const TString& readGroup, const TString& topicPath, const TString& endpoint, const TString& database, const NConfig::TRowDispatcherConfig& config, - NActors::TActorId rowDispatcherActorId, - NActors::TActorId compileServiceActorId, + TActorId rowDispatcherActorId, + TActorId compileServiceActorId, ui32 partitionId, NYdb::TDriver driver, std::shared_ptr credentialsProviderFactory, const ::NMonitoring::TDynamicCounterPtr& counters, const NYql::IPqGateway::TPtr& pqGateway, ui64 maxBufferSize) { - return std::unique_ptr(new TTopicSession(readGroup, topicPath, endpoint, database, config, rowDispatcherActorId, compileServiceActorId, partitionId, std::move(driver), credentialsProviderFactory, counters, pqGateway, maxBufferSize)); + return std::unique_ptr(new TTopicSession(readGroup, topicPath, endpoint, database, config, rowDispatcherActorId, compileServiceActorId, partitionId, std::move(driver), credentialsProviderFactory, counters, pqGateway, maxBufferSize)); } -} // namespace NFq +} // namespace NFq diff --git a/ydb/core/fq/libs/row_dispatcher/ut/json_filter_ut.cpp b/ydb/core/fq/libs/row_dispatcher/ut/json_filter_ut.cpp deleted file mode 100644 index 7f4e3ed1fc4a..000000000000 --- a/ydb/core/fq/libs/row_dispatcher/ut/json_filter_ut.cpp +++ /dev/null @@ -1,206 +0,0 @@ -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include - -#include - -#include - -namespace { - -using namespace NKikimr; -using namespace NFq; - -class TFixture : public NUnitTest::TBaseFixture { - -public: - TFixture() - : Runtime(true) - , Alloc(__LOCATION__, NKikimr::TAlignedPagePoolCounters(), true, false) - { - Alloc.Ref().UseRefLocking = true; - } - - static void SegmentationFaultHandler(int) { - Cerr << "segmentation fault call stack:" << Endl; - FormatBackTrace(&Cerr); - abort(); - } - - void SetUp(NUnitTest::TTestContext&) override { - NKikimr::EnableYDBBacktraceFormat(); - signal(SIGSEGV, &SegmentationFaultHandler); - - TAutoPtr app = new TAppPrepare(); - Runtime.Initialize(app->Unwrap()); - Runtime.SetLogPriority(NKikimrServices::FQ_ROW_DISPATCHER, NLog::PRI_DEBUG); - Runtime.SetDispatchTimeout(TDuration::Seconds(5)); - - CompileServiceActorId = Runtime.Register(NRowDispatcher::CreatePurecalcCompileService()); - } - - void TearDown(NUnitTest::TTestContext& /* context */) override { - with_lock (Alloc) { - for (const auto& holder : Holders) { - for (const auto& value : holder) { - Alloc.Ref().UnlockObject(value); - } - } - Holders.clear(); - } - Filter.reset(); - } - - void MakeFilter( - const TVector& columns, - const TVector& types, - const TString& whereFilter, - NFq::TJsonFilter::TCallback callback) { - Filter = NFq::NewJsonFilter( - columns, - types, - whereFilter, - callback, - {.EnabledLLVM = false}); - - const auto edgeActor = Runtime.AllocateEdgeActor(); - Runtime.Send(CompileServiceActorId, edgeActor, Filter->GetCompileRequest().release()); - auto response = Runtime.GrabEdgeEvent(edgeActor, TDuration::Seconds(5)); - - UNIT_ASSERT_C(response, "Failed to get compile response"); - UNIT_ASSERT_C(response->Get()->ProgramHolder, "Failed to compile program, error: " << response->Get()->Error); - Filter->OnCompileResponse(std::move(response)); - } - - void Push(const TVector& offsets, const TVector*>& values) { - Filter->Push(offsets, values, 0, values.front()->size()); - } - - const TVector* MakeVector(size_t size, std::function valueCreator) { - with_lock (Alloc) { - Holders.emplace_front(); - for (size_t i = 0; i < size; ++i) { - Holders.front().emplace_back(valueCreator(i)); - Alloc.Ref().LockObject(Holders.front().back()); - } - return &Holders.front(); - } - } - - template - const TVector* MakeVector(const TVector& values, bool optional = false) { - return MakeVector(values.size(), [&](size_t i) { - NYql::NUdf::TUnboxedValuePod unboxedValue = NYql::NUdf::TUnboxedValuePod(values[i]); - return optional ? unboxedValue.MakeOptional() : unboxedValue; - }); - } - - const TVector* MakeStringVector(const TVector& values, bool optional = false) { - return MakeVector(values.size(), [&](size_t i) { - NYql::NUdf::TUnboxedValuePod stringValue = NKikimr::NMiniKQL::MakeString(values[i]); - return optional ? stringValue.MakeOptional() : stringValue; - }); - } - - const TVector* MakeEmptyVector(size_t size) { - return MakeVector(size, [&](size_t) { - return NYql::NUdf::TUnboxedValuePod(); - }); - } - - NActors::TTestActorRuntime Runtime; - TActorSystemStub ActorSystemStub; - TActorId CompileServiceActorId; - std::unique_ptr Filter; - - NKikimr::NMiniKQL::TScopedAlloc Alloc; - TList> Holders; -}; - -Y_UNIT_TEST_SUITE(TJsonFilterTests) { - Y_UNIT_TEST_F(Simple1, TFixture) { - TMap result; - MakeFilter( - {"a1", "a2", "a@3"}, - {"[DataType; String]", "[DataType; Uint64]", "[OptionalType; [DataType; String]]"}, - "where a2 > 100", - [&](ui64 offset, const TString& json) { - result[offset] = json; - }); - Push({5}, {MakeStringVector({"hello1"}), MakeVector({99}), MakeStringVector({"zapuskaem"}, true)}); - Push({6}, {MakeStringVector({"hello2"}), MakeVector({101}), MakeStringVector({"gusya"}, true)}); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(R"({"a1":"hello2","a2":101,"a@3":"gusya"})", result[6]); - } - - Y_UNIT_TEST_F(Simple2, TFixture) { - TMap result; - MakeFilter( - {"a2", "a1"}, - {"[DataType; Uint64]", "[DataType; String]"}, - "where a2 > 100", - [&](ui64 offset, const TString& json) { - result[offset] = json; - }); - Push({5}, {MakeVector({99}), MakeStringVector({"hello1"})}); - Push({6}, {MakeVector({101}), MakeStringVector({"hello2"})}); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(R"({"a1":"hello2","a2":101})", result[6]); - } - - Y_UNIT_TEST_F(ManyValues, TFixture) { - TMap result; - MakeFilter( - {"a1", "a2", "a3"}, - {"[DataType; String]", "[DataType; Uint64]", "[DataType; String]"}, - "where a2 > 100", - [&](ui64 offset, const TString& json) { - result[offset] = json; - }); - const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; - for (ui64 i = 0; i < 5; ++i) { - Push({2 * i, 2 * i + 1}, {MakeStringVector({"hello1", "hello2"}), MakeVector({99, 101}), MakeStringVector({largeString, largeString})}); - UNIT_ASSERT_VALUES_EQUAL_C(i + 1, result.size(), i); - UNIT_ASSERT_VALUES_EQUAL_C(TStringBuilder() << "{\"a1\":\"hello2\",\"a2\":101,\"a3\":\"" << largeString << "\"}", result[2 * i + 1], i); - } - } - - Y_UNIT_TEST_F(NullValues, TFixture) { - TMap result; - MakeFilter( - {"a1", "a2"}, - {"[OptionalType; [DataType; Uint64]]", "[DataType; String]"}, - "where a1 is null", - [&](ui64 offset, const TString& json) { - result[offset] = json; - }); - Push({5}, {MakeEmptyVector(1), MakeStringVector({"str"})}); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(R"({"a1":null,"a2":"str"})", result[5]); - } - - Y_UNIT_TEST_F(PartialPush, TFixture) { - TMap result; - MakeFilter( - {"a1", "a2", "a@3"}, - {"[DataType; String]", "[DataType; Uint64]", "[OptionalType; [DataType; String]]"}, - "where a2 > 50", - [&](ui64 offset, const TString& json) { - result[offset] = json; - }); - Filter->Push({5, 6, 7}, {MakeStringVector({"hello1", "hello2"}), MakeVector({99, 101}), MakeStringVector({"zapuskaem", "gusya"}, true)}, 1, 1); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(R"({"a1":"hello1","a2":99,"a@3":"zapuskaem"})", result[6]); - } -} - -} diff --git a/ydb/core/fq/libs/row_dispatcher/ut/json_parser_ut.cpp b/ydb/core/fq/libs/row_dispatcher/ut/json_parser_ut.cpp deleted file mode 100644 index 80a2b29322c6..000000000000 --- a/ydb/core/fq/libs/row_dispatcher/ut/json_parser_ut.cpp +++ /dev/null @@ -1,319 +0,0 @@ -#include - -#include -#include - -#include - -#include -#include -#include - -#include - -#include - -namespace { - -using namespace NKikimr; -using namespace NFq; - -class TFixture : public NUnitTest::TBaseFixture { - -public: - TFixture() - : Runtime(true) {} - - static void SegmentationFaultHandler(int) { - Cerr << "segmentation fault call stack:" << Endl; - FormatBackTrace(&Cerr); - abort(); - } - - void SetUp(NUnitTest::TTestContext&) override { - NKikimr::EnableYDBBacktraceFormat(); - signal(SIGSEGV, &SegmentationFaultHandler); - - TAutoPtr app = new TAppPrepare(); - Runtime.SetLogBackend(CreateStderrBackend()); - Runtime.SetLogPriority(NKikimrServices::FQ_ROW_DISPATCHER, NLog::PRI_TRACE); - Runtime.Initialize(app->Unwrap()); - } - - void TearDown(NUnitTest::TTestContext& /* context */) override { - if (Parser) { - Parser.reset(); - } - } - - void MakeParser(TVector columns, TVector types, TJsonParser::TCallback callback, ui64 batchSize = 1_MB, ui64 bufferCellCount = 1000) { - Parser = NFq::NewJsonParser(columns, types, callback, batchSize, TDuration::Hours(1), bufferCellCount); - } - - void MakeParser(TVector columns, TJsonParser::TCallback callback) { - MakeParser(columns, TVector(columns.size(), "[DataType; String]"), callback); - } - - void MakeParser(TVector columns, TVector types) { - MakeParser(columns, types, [](ui64, ui64, const TVector>&) {}); - } - - void MakeParser(TVector columns) { - MakeParser(columns, TVector(columns.size(), "[DataType; String]")); - } - - void PushToParser(ui64 offset, const TString& data) { - Parser->AddMessages({GetMessage(offset, data)}); - Parser->Parse(); - } - - static NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage GetMessage(ui64 offset, const TString& data) { - NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessageInformation info(offset, "", 0, TInstant::Zero(), TInstant::Zero(), nullptr, nullptr, 0, ""); - return NYdb::NTopic::TReadSessionEvent::TDataReceivedEvent::TMessage(data, nullptr, info, nullptr); - } - - TActorSystemStub actorSystemStub; - NActors::TTestActorRuntime Runtime; - std::unique_ptr Parser; -}; - -Y_UNIT_TEST_SUITE(TJsonParserTests) { - Y_UNIT_TEST_F(Simple1, TFixture) { - MakeParser({"a1", "a2"}, {"[DataType; String]", "[OptionalType; [DataType; Uint64]]"}, [](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(1, numberRows); - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[0][0].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL(101, result[1][0].GetOptionalValue().Get()); - }); - PushToParser(42,R"({"a1": "hello1", "a2": 101, "event": "event1"})"); - } - - Y_UNIT_TEST_F(Simple2, TFixture) { - MakeParser({"a2", "a1"}, [](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(1, numberRows); - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - UNIT_ASSERT_VALUES_EQUAL("101", TString(result[0][0].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[1][0].AsStringRef())); - }); - PushToParser(42,R"({"a1": "hello1", "a2": "101", "event": "event1"})"); - } - - Y_UNIT_TEST_F(Simple3, TFixture) { - MakeParser({"a1", "a2"}, [](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(1, numberRows); - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - UNIT_ASSERT_VALUES_EQUAL("101", TString(result[0][0].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[1][0].AsStringRef())); - }); - PushToParser(42,R"({"a2": "hello1", "a1": "101", "event": "event1"})"); - } - - Y_UNIT_TEST_F(Simple4, TFixture) { - MakeParser({"a2", "a1"}, [](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(1, numberRows); - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[0][0].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("101", TString(result[1][0].AsStringRef())); - }); - PushToParser(42, R"({"a2": "hello1", "a1": "101", "event": "event1"})"); - } - - Y_UNIT_TEST_F(LargeStrings, TFixture) { - const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; - - MakeParser({"col"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(2, numberRows); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0][0].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0][1].AsStringRef())); - }); - - const TString jsonString = TStringBuilder() << "{\"col\": \"" << largeString << "\"}"; - Parser->AddMessages({ - GetMessage(42, jsonString), - GetMessage(43, jsonString) - }); - Parser->Parse(); - } - - Y_UNIT_TEST_F(ManyValues, TFixture) { - MakeParser({"a1", "a2"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(3, numberRows); - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - for (size_t i = 0; i < numberRows; ++i) { - UNIT_ASSERT_VALUES_EQUAL_C("hello1", TString(result[0][i].AsStringRef()), i); - UNIT_ASSERT_VALUES_EQUAL_C("101", TString(result[1][i].AsStringRef()), i); - } - }); - - Parser->AddMessages({ - GetMessage(42, R"({"a1": "hello1", "a2": "101", "event": "event1"})"), - GetMessage(43, R"({"a1": "hello1", "a2": "101", "event": "event2"})"), - GetMessage(44, R"({"a2": "101", "a1": "hello1", "event": "event3"})") - }); - Parser->Parse(); - } - - Y_UNIT_TEST_F(MissingFields, TFixture) { - MakeParser({"a1", "a2"}, {"[OptionalType; [DataType; String]]", "[OptionalType; [DataType; Uint64]]"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(3, numberRows); - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - for (size_t i = 0; i < numberRows; ++i) { - if (i == 2) { - UNIT_ASSERT_C(!result[0][i], i); - } else { - NYql::NUdf::TUnboxedValue value = result[0][i].GetOptionalValue(); - UNIT_ASSERT_VALUES_EQUAL_C("hello1", TString(value.AsStringRef()), i); - } - if (i == 1) { - UNIT_ASSERT_C(!result[1][i], i); - } else { - UNIT_ASSERT_VALUES_EQUAL_C(101, result[1][i].GetOptionalValue().Get(), i); - } - } - }); - - Parser->AddMessages({ - GetMessage(42, R"({"a1": "hello1", "a2": 101 , "event": "event1"})"), - GetMessage(43, R"({"a1": "hello1", "event": "event2"})"), - GetMessage(44, R"({"a2": "101", "a1": null, "event": "event3"})") - }); - Parser->Parse(); - } - - Y_UNIT_TEST_F(NestedTypes, TFixture) { - MakeParser({"nested", "a1"}, {"[OptionalType; [DataType; Json]]", "[DataType; String]"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(4, numberRows); - - UNIT_ASSERT_VALUES_EQUAL(2, result.size()); - UNIT_ASSERT_VALUES_EQUAL("{\"key\": \"value\"}", TString(result[0][0].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("hello1", TString(result[1][0].AsStringRef())); - - UNIT_ASSERT_VALUES_EQUAL("[\"key1\", \"key2\"]", TString(result[0][1].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("hello2", TString(result[1][1].AsStringRef())); - - UNIT_ASSERT_VALUES_EQUAL("\"some string\"", TString(result[0][2].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("hello3", TString(result[1][2].AsStringRef())); - - UNIT_ASSERT_VALUES_EQUAL("123456", TString(result[0][3].AsStringRef())); - UNIT_ASSERT_VALUES_EQUAL("hello4", TString(result[1][3].AsStringRef())); - }); - - Parser->AddMessages({ - GetMessage(42, R"({"a1": "hello1", "nested": {"key": "value"}})"), - GetMessage(43, R"({"a1": "hello2", "nested": ["key1", "key2"]})"), - GetMessage(43, R"({"a1": "hello3", "nested": "some string"})"), - GetMessage(43, R"({"a1": "hello4", "nested": 123456})") - }); - Parser->Parse(); - } - - Y_UNIT_TEST_F(SimpleBooleans, TFixture) { - MakeParser({"a"}, {"[DataType; Bool]"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(2, numberRows); - - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(true, result[0][0].Get()); - UNIT_ASSERT_VALUES_EQUAL(false, result[0][1].Get()); - }); - - Parser->AddMessages({ - GetMessage(42, R"({"a": true})"), - GetMessage(43, R"({"a": false})") - }); - Parser->Parse(); - } - - Y_UNIT_TEST_F(ManyBatches, TFixture) { - const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; - - ui64 currentOffset = 0; - MakeParser({"col"}, {"[DataType; String]"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(currentOffset, rowsOffset); - currentOffset++; - - UNIT_ASSERT_VALUES_EQUAL(1, numberRows); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0][0].AsStringRef())); - }, 1_MB, 1); - - const TString jsonString = TStringBuilder() << "{\"col\": \"" << largeString << "\"}"; - Parser->AddMessages({ - GetMessage(42, jsonString), - GetMessage(43, jsonString) - }); - Parser->Parse(); - } - - Y_UNIT_TEST_F(LittleBatches, TFixture) { - const TString largeString = "abcdefghjkl1234567890+abcdefghjkl1234567890"; - - ui64 currentOffset = 42; - MakeParser({"col"}, {"[DataType; String]"}, [&](ui64 rowsOffset, ui64 numberRows, const TVector>& result) { - UNIT_ASSERT_VALUES_EQUAL(Parser->GetOffsets().size(), 1); - UNIT_ASSERT_VALUES_EQUAL(Parser->GetOffsets().front(), currentOffset); - currentOffset++; - - UNIT_ASSERT_VALUES_EQUAL(0, rowsOffset); - UNIT_ASSERT_VALUES_EQUAL(1, numberRows); - UNIT_ASSERT_VALUES_EQUAL(1, result.size()); - UNIT_ASSERT_VALUES_EQUAL(largeString, TString(result[0][0].AsStringRef())); - }, 10); - - const TString jsonString = TStringBuilder() << "{\"col\": \"" << largeString << "\"}"; - Parser->AddMessages({ - GetMessage(42, jsonString), - GetMessage(43, jsonString) - }); - UNIT_ASSERT_VALUES_EQUAL(Parser->GetNumberValues(), 0); - } - - Y_UNIT_TEST_F(MissingFieldsValidation, TFixture) { - MakeParser({"a1", "a2"}, {"[DataType; String]", "[DataType; Uint64]"}); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": "hello1", "a2": null, "event": "event1"})"), TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a2' with type [DataType; Uint64], description: (NFq::TJsonParserError) found unexpected null value, expected non optional data type Uint64"); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a2": 105, "event": "event1"})"), TJsonParserError, "Failed to parse json messages, found 1 missing values from offset 42 in non optional column 'a1' with type [DataType; String]"); - } - - Y_UNIT_TEST_F(TypeKindsValidation, TFixture) { - UNIT_ASSERT_EXCEPTION_CONTAINS( - MakeParser({"a2", "a1"}, {"[OptionalType; [DataType; String]]", "[ListType; [DataType; String]]"}), - NFq::TJsonParserError, - "Failed to create parser for column 'a1' with type [ListType; [DataType; String]], description: (NFq::TJsonParserError) unsupported type kind List" - ); - } - - Y_UNIT_TEST_F(NumbersValidation, TFixture) { - MakeParser({"a1", "a2"}, {"[OptionalType; [DataType; String]]", "[DataType; Uint8]"}); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": 456, "a2": 42})"), NFq::TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a1' with type [OptionalType; [DataType; String]], description: (NFq::TJsonParserError) failed to parse data type String from json number (raw: '456'), error: (NFq::TJsonParserError) number value is not expected for data type String"); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": "456", "a2": -42})"), NFq::TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a2' with type [DataType; Uint8], description: (NFq::TJsonParserError) failed to parse data type Uint8 from json number (raw: '-42'), error: (simdjson::simdjson_error) INCORRECT_TYPE: The JSON element does not have the requested type."); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": "str", "a2": 99999})"), NFq::TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a2' with type [DataType; Uint8], description: (NFq::TJsonParserError) failed to parse data type Uint8 from json number (raw: '99999'), error: (NFq::TJsonParserError) number is out of range"); - } - - Y_UNIT_TEST_F(NestedJsonValidation, TFixture) { - MakeParser({"a1", "a2"}, {"[OptionalType; [DataType; Json]]", "[OptionalType; [DataType; String]]"}); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": {"key": "value"}, "a2": {"key2": "value2"}})"), NFq::TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a2' with type [OptionalType; [DataType; String]], description: (NFq::TJsonParserError) found unexpected nested value (raw: '{\"key2\": \"value2\"}'), expected data type String, please use Json type for nested values"); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": {"key" "value"}, "a2": "str"})"), NFq::TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a1' with type [OptionalType; [DataType; Json]], description: (simdjson::simdjson_error) TAPE_ERROR: The JSON document has an improper structure: missing or superfluous commas, braces, missing keys, etc."); - } - - Y_UNIT_TEST_F(BoolsValidation, TFixture) { - MakeParser({"a1", "a2"}, {"[OptionalType; [DataType; String]]", "[DataType; Bool]"}); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a1": true, "a2": false})"), NFq::TJsonParserError, "Failed to parse json string at offset 42, got parsing error for column 'a1' with type [OptionalType; [DataType; String]], description: (NFq::TJsonParserError) found unexpected bool value, expected data type String"); - } - - Y_UNIT_TEST_F(ThrowExceptionByError, TFixture) { - MakeParser({"a"}); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"(ydb)"), simdjson::simdjson_error, "INCORRECT_TYPE: The JSON element does not have the requested type."); - UNIT_ASSERT_EXCEPTION_CONTAINS(PushToParser(42, R"({"a": "value1"} {"a": "value2"})"), NFq::TJsonParserError, "Failed to parse json messages, expected 1 json rows from offset 42 but got 2"); - } -} - -} diff --git a/ydb/core/fq/libs/row_dispatcher/ut/topic_session_ut.cpp b/ydb/core/fq/libs/row_dispatcher/ut/topic_session_ut.cpp index 84c8c425b51c..7016ca9c2b15 100644 --- a/ydb/core/fq/libs/row_dispatcher/ut/topic_session_ut.cpp +++ b/ydb/core/fq/libs/row_dispatcher/ut/topic_session_ut.cpp @@ -1,11 +1,9 @@ -#include - #include #include #include #include -#include +#include #include #include @@ -16,61 +14,25 @@ #include #include +#include + +namespace NFq::NRowDispatcher::NTests { namespace { using namespace NKikimr; -using namespace NFq; using namespace NYql::NDq; const ui64 TimeoutBeforeStartSessionSec = 3; const ui64 GrabTimeoutSec = 4 * TimeoutBeforeStartSessionSec; -class TPurecalcCompileServiceMock : public NActors::TActor { - using TBase = NActors::TActor; - +class TFixture : public NTests::TBaseFixture { public: - TPurecalcCompileServiceMock(TActorId owner) - : TBase(&TPurecalcCompileServiceMock::StateFunc) - , Owner(owner) - , ProgramFactory(NYql::NPureCalc::MakeProgramFactory()) - {} - - STRICT_STFUNC(StateFunc, - hFunc(TEvRowDispatcher::TEvPurecalcCompileRequest, Handle); - ) - - void Handle(TEvRowDispatcher::TEvPurecalcCompileRequest::TPtr& ev) { - IProgramHolder::TPtr programHolder = std::move(ev->Get()->ProgramHolder); - - try { - programHolder->CreateProgram(ProgramFactory); - } catch (const NYql::NPureCalc::TCompileError& e) { - UNIT_ASSERT_C(false, "Failed to compile purecalc filter: sql: " << e.GetYql() << ", error: " << e.GetIssues()); - } - - Send(ev->Sender, new TEvRowDispatcher::TEvPurecalcCompileResponse(std::move(programHolder)), 0, ev->Cookie); - Send(Owner, new NActors::TEvents::TEvPing()); - } - -private: - const TActorId Owner; - const NYql::NPureCalc::IProgramFactoryPtr ProgramFactory; -}; + using TBase = NTests::TBaseFixture; -class TFixture : public NUnitTest::TBaseFixture { public: - TFixture() - : Runtime(true) - {} - - void SetUp(NUnitTest::TTestContext&) override { - TAutoPtr app = new TAppPrepare(); - Runtime.Initialize(app->Unwrap()); - Runtime.SetLogPriority(NKikimrServices::FQ_ROW_DISPATCHER, NLog::PRI_TRACE); - Runtime.SetDispatchTimeout(TDuration::Seconds(5)); - - NKikimr::EnableYDBBacktraceFormat(); + void SetUp(NUnitTest::TTestContext& ctx) override { + TBase::SetUp(ctx); ReadActorId1 = Runtime.AllocateEdgeActor(); ReadActorId2 = Runtime.AllocateEdgeActor(); @@ -95,7 +57,7 @@ class TFixture : public NUnitTest::TBaseFixture { nullptr); CompileNotifier = Runtime.AllocateEdgeActor(); - const auto compileServiceActorId = Runtime.Register(new TPurecalcCompileServiceMock(CompileNotifier)); + const auto compileServiceActorId = Runtime.Register(CreatePurecalcCompileServiceMock(CompileNotifier)); TopicSession = Runtime.Register(NewTopicSession( "read_group", @@ -119,9 +81,6 @@ class TFixture : public NUnitTest::TBaseFixture { UNIT_ASSERT(Runtime.DispatchEvents(options)); } - void TearDown(NUnitTest::TTestContext& /* context */) override { - } - void StartSession(TActorId readActorId, const NYql::NPq::NProto::TDqPqTopicSource& source, TMaybe readOffset = Nothing(), bool expectedError = false) { auto event = new NFq::TEvRowDispatcher::TEvStartSession( source, @@ -145,6 +104,7 @@ class TFixture : public NUnitTest::TBaseFixture { settings.SetEndpoint(GetDefaultPqEndpoint()); settings.SetTopicPath(topic); settings.SetConsumerName(consumer); + settings.SetFormat("json_each_row"); settings.MutableToken()->SetName("token"); settings.SetDatabase(GetDefaultPqDatabase()); settings.AddColumns("dt"); @@ -164,28 +124,28 @@ class TFixture : public NUnitTest::TBaseFixture { Runtime.Send(new IEventHandle(TopicSession, readActorId, event.release())); } - void ExpectMessageBatch(NActors::TActorId readActorId, const std::vector& expected) { + void ExpectMessageBatch(NActors::TActorId readActorId, const TBatch& expected) { Runtime.Send(new IEventHandle(TopicSession, readActorId, new TEvRowDispatcher::TEvGetNextBatch())); auto eventHolder = Runtime.GrabEdgeEvent(RowDispatcherActorId, TDuration::Seconds(GrabTimeoutSec)); UNIT_ASSERT(eventHolder.Get() != nullptr); UNIT_ASSERT_VALUES_EQUAL(eventHolder->Get()->ReadActorId, readActorId); - UNIT_ASSERT_VALUES_EQUAL(expected.size(), eventHolder->Get()->Record.MessagesSize()); - for (size_t i = 0; i < expected.size(); ++i) { - NFq::NRowDispatcherProto::TEvMessage message = eventHolder->Get()->Record.GetMessages(i); - std::cerr << "message.GetJson() " << message.GetJson() << std::endl; - UNIT_ASSERT_VALUES_EQUAL(expected[i], message.GetJson()); - } + UNIT_ASSERT_VALUES_EQUAL(1, eventHolder->Get()->Record.MessagesSize()); + + NFq::NRowDispatcherProto::TEvMessage message = eventHolder->Get()->Record.GetMessages(0); + UNIT_ASSERT_VALUES_EQUAL(message.OffsetsSize(), expected.Rows.size()); + CheckMessageBatch(eventHolder->Get()->GetPayload(message.GetPayloadId()), expected); } - TString ExpectSessionError(NActors::TActorId readActorId, TMaybe message = Nothing()) { + void ExpectSessionError(NActors::TActorId readActorId, TStatusCode statusCode, TString message = "") { auto eventHolder = Runtime.GrabEdgeEvent(RowDispatcherActorId, TDuration::Seconds(GrabTimeoutSec)); UNIT_ASSERT(eventHolder.Get() != nullptr); UNIT_ASSERT_VALUES_EQUAL(eventHolder->Get()->ReadActorId, readActorId); - if (message) { - UNIT_ASSERT_STRING_CONTAINS(TString(eventHolder->Get()->Record.GetMessage()), *message); - } - return eventHolder->Get()->Record.GetMessage(); + + const auto& record = eventHolder->Get()->Record; + NYql::TIssues issues; + NYql::IssuesFromMessage(record.GetIssues(), issues); + NTests::CheckError(TStatus::Fail(record.GetStatusCode(), std::move(issues)), statusCode, message); } void ExpectNewDataArrived(TSet readActorIds) { @@ -203,7 +163,13 @@ class TFixture : public NUnitTest::TBaseFixture { auto eventHolder = Runtime.GrabEdgeEvent(RowDispatcherActorId, TDuration::Seconds(GrabTimeoutSec)); UNIT_ASSERT(eventHolder.Get() != nullptr); UNIT_ASSERT_VALUES_EQUAL(eventHolder->Get()->ReadActorId, readActorId); - return eventHolder->Get()->Record.MessagesSize(); + + size_t numberMessages = 0; + for (const auto& message : eventHolder->Get()->Record.GetMessages()) { + numberMessages += message.OffsetsSize(); + } + + return numberMessages; } void ExpectStatisticToReadActor(TSet readActorIds, ui64 expectedNextMessageOffset) { @@ -217,8 +183,11 @@ class TFixture : public NUnitTest::TBaseFixture { } } - NActors::TTestActorRuntime Runtime; - TActorSystemStub ActorSystemStub; + static TRow JsonMessage(ui64 index) { + return TRow().AddUint64(100 * index).AddString(TStringBuilder() << "value" << index); + } + +public: NActors::TActorId TopicSession; NActors::TActorId RowDispatcherActorId; NActors::TActorId CompileNotifier; @@ -236,6 +205,8 @@ class TFixture : public NUnitTest::TBaseFixture { const TString Json4 = "{\"dt\":400,\"value\":\"value4\"}"; }; +} // anonymous namespace + Y_UNIT_TEST_SUITE(TopicSessionTests) { Y_UNIT_TEST_F(TwoSessionsWithoutOffsets, TFixture) { const TString topicName = "topic1"; @@ -248,21 +219,21 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { std::vector data = { Json1 }; PQWrite(data, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); - ExpectMessageBatch(ReadActorId1, { Json1 }); - ExpectMessageBatch(ReadActorId2, { Json1 }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(1) }); ExpectStatisticToReadActor({ReadActorId1, ReadActorId2}, 1); data = { Json2 }; PQWrite(data, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); ExpectStatisticToReadActor({ReadActorId1, ReadActorId2}, 1); - ExpectMessageBatch(ReadActorId1, data); - ExpectMessageBatch(ReadActorId2, data); + ExpectMessageBatch(ReadActorId1, { JsonMessage(2) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(2) }); ExpectStatisticToReadActor({ReadActorId1, ReadActorId2}, 2); auto source2 = BuildSource(topicName, false, "OtherConsumer"); StartSession(ReadActorId3, source2, Nothing(), true); - ExpectSessionError(ReadActorId3, "Use the same consumer"); + ExpectSessionError(ReadActorId3, EStatusId::PRECONDITION_FAILED, "Use the same consumer"); StopSession(ReadActorId1, source); StopSession(ReadActorId2, source); @@ -282,8 +253,8 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { ExpectNewDataArrived({ReadActorId1, ReadActorId2}); Runtime.Send(new IEventHandle(TopicSession, ReadActorId1, new TEvRowDispatcher::TEvGetNextBatch())); Runtime.Send(new IEventHandle(TopicSession, ReadActorId2, new TEvRowDispatcher::TEvGetNextBatch())); - ExpectMessageBatch(ReadActorId1, { Json1 }); - ExpectMessageBatch(ReadActorId2, { Json1 }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(1) }); StopSession(ReadActorId1, source1); StopSession(ReadActorId2, source2); @@ -301,8 +272,8 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { const std::vector data = { Json1 }; PQWrite(data, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); - ExpectMessageBatch(ReadActorId1, { Json1 }); - ExpectMessageBatch(ReadActorId2, { Json1 }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(1) }); StopSession(ReadActorId1, source1); StopSession(ReadActorId2, source2); @@ -318,16 +289,16 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { const std::vector data = { Json1 }; PQWrite(data, topicName); ExpectNewDataArrived({ReadActorId1}); - ExpectMessageBatch(ReadActorId1, data); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1) }); StartSession(ReadActorId2, source); const std::vector data2 = { Json2 }; PQWrite(data2, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); - - ExpectMessageBatch(ReadActorId1, data2); - ExpectMessageBatch(ReadActorId2, data2); + + ExpectMessageBatch(ReadActorId1, { JsonMessage(2) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(2) }); StopSession(ReadActorId1, source); StopSession(ReadActorId2, source); @@ -345,17 +316,17 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { StartSession(ReadActorId2, source, 2); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); - std::vector expected1 = { Json2, Json3}; + TBatch expected1 = { JsonMessage(2), JsonMessage(3) }; ExpectMessageBatch(ReadActorId1, expected1); - std::vector expected2 = { Json3 }; + TBatch expected2 = { JsonMessage(3) }; ExpectMessageBatch(ReadActorId2, expected2); const std::vector data2 = { Json4 }; PQWrite(data2, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); - ExpectMessageBatch(ReadActorId1, data2); - ExpectMessageBatch(ReadActorId2, data2); + ExpectMessageBatch(ReadActorId1, { JsonMessage(4) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(4) }); StopSession(ReadActorId1, source); StopSession(ReadActorId2, source); @@ -371,7 +342,7 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { const std::vector data = { "not json", "noch einmal / nicht json" }; PQWrite(data, topicName); - ExpectSessionError(ReadActorId1, "INCORRECT_TYPE: The JSON element does not have the requested type."); + ExpectSessionError(ReadActorId1, EStatusId::BAD_REQUEST, "INCORRECT_TYPE: The JSON element does not have the requested type."); StopSession(ReadActorId1, source); } @@ -379,15 +350,26 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { const TString topicName = "wrong_field"; PQCreateStream(topicName); Init(topicName); + auto source = BuildSource(topicName); StartSession(ReadActorId1, source); - const std::vector data = {"{\"dt\":100}"}; - PQWrite(data, topicName); - auto error = ExpectSessionError(ReadActorId1); - UNIT_ASSERT_STRING_CONTAINS(error, "Failed to parse json messages, found 1 missing values"); - UNIT_ASSERT_STRING_CONTAINS(error, "the field (value) has been added by query"); + source.AddColumns("field1"); + source.AddColumnTypes("[DataType; String]"); + StartSession(ReadActorId2, source); + + PQWrite({ Json1 }, topicName); + ExpectNewDataArrived({ReadActorId1}); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1) }); + ExpectSessionError(ReadActorId2, EStatusId::PRECONDITION_FAILED, "Failed to parse json messages, found 1 missing values"); + + PQWrite({ Json2 }, topicName); + ExpectNewDataArrived({ReadActorId1}); + ExpectMessageBatch(ReadActorId1, { JsonMessage(2) }); + ExpectSessionError(ReadActorId2, EStatusId::PRECONDITION_FAILED, "Failed to parse json messages, found 1 missing values"); + StopSession(ReadActorId1, source); + StopSession(ReadActorId2, source); } Y_UNIT_TEST_F(RestartSessionIfNewClientWithOffset, TFixture) { @@ -400,7 +382,7 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { const std::vector data = { Json1, Json2, Json3 }; // offset 0, 1, 2 PQWrite(data, topicName); ExpectNewDataArrived({ReadActorId1}); - ExpectMessageBatch(ReadActorId1, data); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1), JsonMessage(2), JsonMessage(3) }); // Restart topic session. StartSession(ReadActorId2, source, 1); @@ -409,8 +391,8 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { PQWrite({ Json4 }, topicName); ExpectNewDataArrived({ReadActorId1}); - ExpectMessageBatch(ReadActorId1, { Json4 }); - ExpectMessageBatch(ReadActorId2, { Json2, Json3, Json4 }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(4) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(2), JsonMessage(3), JsonMessage(4) }); StopSession(ReadActorId1, source); StopSession(ReadActorId2, source); @@ -421,14 +403,14 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { Init(topicName); auto source = BuildSource(topicName); StartSession(ReadActorId1, source); - ExpectSessionError(ReadActorId1, "no path"); + ExpectSessionError(ReadActorId1, EStatusId::SCHEME_ERROR, "no path"); StopSession(ReadActorId1, source); } Y_UNIT_TEST_F(SlowSession, TFixture) { const TString topicName = "topic8"; PQCreateStream(topicName); - Init(topicName, 50); + Init(topicName, 40); auto source = BuildSource(topicName); StartSession(ReadActorId1, source); StartSession(ReadActorId2, source); // slow session @@ -442,26 +424,26 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { Sleep(TDuration::MilliSeconds(100)); Runtime.DispatchEvents({}, Runtime.GetCurrentTime() - TDuration::MilliSeconds(1)); }; - + writeMessages(); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); auto readMessages = ReadMessages(ReadActorId1); - UNIT_ASSERT(readMessages == messagesSize); + UNIT_ASSERT_VALUES_EQUAL(readMessages, messagesSize); // Reading from yds is stopped. writeMessages(); readMessages = ReadMessages(ReadActorId1); - UNIT_ASSERT(readMessages == 0); + UNIT_ASSERT_VALUES_EQUAL(readMessages, 0); readMessages = ReadMessages(ReadActorId2); - UNIT_ASSERT(readMessages == messagesSize); + UNIT_ASSERT_VALUES_EQUAL(readMessages, messagesSize); Sleep(TDuration::MilliSeconds(100)); Runtime.DispatchEvents({}, Runtime.GetCurrentTime() - TDuration::MilliSeconds(1)); readMessages = ReadMessages(ReadActorId1); - UNIT_ASSERT(readMessages == messagesSize); + UNIT_ASSERT_VALUES_EQUAL(readMessages, messagesSize); writeMessages(); StopSession(ReadActorId2, source); // delete slow client, clear unread buffer @@ -469,11 +451,11 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { Runtime.DispatchEvents({}, Runtime.GetCurrentTime() - TDuration::MilliSeconds(1)); readMessages = ReadMessages(ReadActorId1); - UNIT_ASSERT(readMessages == messagesSize); + UNIT_ASSERT_VALUES_EQUAL(readMessages, messagesSize); StopSession(ReadActorId1, source); } - Y_UNIT_TEST_F(TwoSessionsWithDifferentSchemes, TFixture) { + Y_UNIT_TEST_F(TwoSessionsWithDifferentSchemes, TFixture) { const TString topicName = "dif_schemes"; PQCreateStream(topicName); Init(topicName); @@ -485,13 +467,13 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { StartSession(ReadActorId1, source1); StartSession(ReadActorId2, source2); - TString json1 = "{\"dt\":101,\"value\":\"value1\", \"field1\":\"field1\"}"; - TString json2 = "{\"dt\":102,\"value\":\"value2\", \"field1\":\"field2\"}"; + TString json1 = "{\"dt\":100,\"value\":\"value1\", \"field1\":\"field1\"}"; + TString json2 = "{\"dt\":200,\"value\":\"value2\", \"field1\":\"field2\"}"; PQWrite({ json1, json2 }, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2}); - ExpectMessageBatch(ReadActorId1, { "{\"dt\":101,\"value\":\"value1\"}", "{\"dt\":102,\"value\":\"value2\"}" }); - ExpectMessageBatch(ReadActorId2, { "{\"dt\":101,\"field1\":\"field1\",\"value\":\"value1\"}", "{\"dt\":102,\"field1\":\"field2\",\"value\":\"value2\"}" }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1), JsonMessage(2) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(1).AddString("field1"), JsonMessage(2).AddString("field2") }); auto source3 = BuildSource(topicName); source3.AddColumns("field2"); @@ -499,27 +481,27 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { auto readActorId3 = Runtime.AllocateEdgeActor(); StartSession(readActorId3, source3); - TString json3 = "{\"dt\":103,\"value\":\"value3\", \"field1\":\"value1_field1\", \"field2\":\"value1_field2\"}"; + TString json3 = "{\"dt\":300,\"value\":\"value3\", \"field1\":\"value1_field1\", \"field2\":\"value1_field2\"}"; PQWrite({ json3 }, topicName); ExpectNewDataArrived({ReadActorId1, ReadActorId2, readActorId3}); - ExpectMessageBatch(ReadActorId1, { "{\"dt\":103,\"value\":\"value3\"}" }); - ExpectMessageBatch(ReadActorId2, { "{\"dt\":103,\"field1\":\"value1_field1\",\"value\":\"value3\"}" }); - ExpectMessageBatch(readActorId3, { "{\"dt\":103,\"field2\":\"value1_field2\",\"value\":\"value3\"}" }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(3) }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(3).AddString("value1_field1") }); + ExpectMessageBatch(readActorId3, { JsonMessage(3).AddString("value1_field2") }); StopSession(ReadActorId1, source3); StopSession(readActorId3, source3); - TString json4 = "{\"dt\":104,\"value\":\"value4\", \"field1\":\"value2_field1\", \"field2\":\"value2_field2\"}"; - TString json5 = "{\"dt\":105,\"value\":\"value5\", \"field1\":\"value2_field1\", \"field2\":\"value2_field2\"}"; + TString json4 = "{\"dt\":400,\"value\":\"value4\", \"field1\":\"value2_field1\", \"field2\":\"value2_field2\"}"; + TString json5 = "{\"dt\":500,\"value\":\"value5\", \"field1\":\"value3_field1\", \"field2\":\"value3_field2\"}"; PQWrite({ json4, json5 }, topicName); ExpectNewDataArrived({ReadActorId2}); - ExpectMessageBatch(ReadActorId2, { "{\"dt\":104,\"field1\":\"value2_field1\",\"value\":\"value4\"}", "{\"dt\":105,\"field1\":\"value2_field1\",\"value\":\"value5\"}" }); + ExpectMessageBatch(ReadActorId2, { JsonMessage(4).AddString("value2_field1"), JsonMessage(5).AddString("value3_field1") }); StopSession(ReadActorId1, source1); StopSession(ReadActorId2, source2); } - Y_UNIT_TEST_F(TwoSessionsWithDifferentColumnTypes, TFixture) { + Y_UNIT_TEST_F(TwoSessionsWithDifferentColumnTypes, TFixture) { const TString topicName = "dif_types"; PQCreateStream(topicName); Init(topicName); @@ -529,18 +511,17 @@ Y_UNIT_TEST_SUITE(TopicSessionTests) { source1.AddColumnTypes("[OptionalType; [DataType; String]]"); StartSession(ReadActorId1, source1); - TString json1 = "{\"dt\":101,\"field1\":null,\"value\":\"value1\"}"; + TString json1 = "{\"dt\":100,\"field1\":\"str\",\"value\":\"value1\"}"; PQWrite({ json1 }, topicName); ExpectNewDataArrived({ReadActorId1}); - ExpectMessageBatch(ReadActorId1, { json1 }); + ExpectMessageBatch(ReadActorId1, { JsonMessage(1).AddString("str", true) }); auto source2 = BuildSource(topicName); source2.AddColumns("field1"); source2.AddColumnTypes("[DataType; String]"); StartSession(ReadActorId2, source2, Nothing(), true); - ExpectSessionError(ReadActorId2, "Use the same column type in all queries via RD, current type for column `field1` is [OptionalType; [DataType; String]] (requested type is [DataType; String])"); - } -} - + ExpectSessionError(ReadActorId2, EStatusId::SCHEME_ERROR, "Use the same column type in all queries via RD, current type for column `field1` is [OptionalType; [DataType; String]] (requested type is [DataType; String])"); + } } +} // namespace NFq::NRowDispatcher::NTests diff --git a/ydb/core/fq/libs/row_dispatcher/ut/ya.make b/ydb/core/fq/libs/row_dispatcher/ut/ya.make index 7049d37ee118..17812aa343d2 100644 --- a/ydb/core/fq/libs/row_dispatcher/ut/ya.make +++ b/ydb/core/fq/libs/row_dispatcher/ut/ya.make @@ -4,8 +4,6 @@ INCLUDE(${ARCADIA_ROOT}/ydb/tests/tools/fq_runner/ydb_runner_with_datastreams.in SRCS( coordinator_ut.cpp - json_filter_ut.cpp - json_parser_ut.cpp leader_election_ut.cpp row_dispatcher_ut.cpp topic_session_ut.cpp @@ -13,11 +11,10 @@ SRCS( PEERDIR( library/cpp/testing/unittest + ydb/core/fq/libs/row_dispatcher/format_handler/ut/common ydb/core/fq/libs/row_dispatcher ydb/core/testlib ydb/core/testlib/actors - yql/essentials/udfs/common/json2 - yql/essentials/udfs/common/yson2 ydb/tests/fq/pq_async_io yql/essentials/sql/pg_dummy ) diff --git a/ydb/core/fq/libs/row_dispatcher/ya.make b/ydb/core/fq/libs/row_dispatcher/ya.make index 0b6f68bfc03a..60c23eb21f56 100644 --- a/ydb/core/fq/libs/row_dispatcher/ya.make +++ b/ydb/core/fq/libs/row_dispatcher/ya.make @@ -4,8 +4,6 @@ SRCS( actors_factory.cpp common.cpp coordinator.cpp - json_filter.cpp - json_parser.cpp leader_election.cpp row_dispatcher_service.cpp row_dispatcher.cpp @@ -13,35 +11,35 @@ SRCS( ) PEERDIR( - contrib/libs/fmt - contrib/libs/simdjson ydb/core/fq/libs/actors/logging ydb/core/fq/libs/config/protos ydb/core/fq/libs/row_dispatcher/events + ydb/core/fq/libs/row_dispatcher/format_handler ydb/core/fq/libs/row_dispatcher/purecalc_compilation ydb/core/fq/libs/shared_resources ydb/core/fq/libs/ydb ydb/core/mon + ydb/library/actors/core ydb/library/security + ydb/library/yql/dq/actors ydb/library/yql/dq/actors/common ydb/library/yql/dq/actors/compute ydb/library/yql/dq/proto ydb/library/yql/providers/pq/provider - ydb/core/fq/libs/row_dispatcher/purecalc_no_pg_wrapper + ydb/public/sdk/cpp/client/ydb_scheme ydb/public/sdk/cpp/client/ydb_table ) -CFLAGS( - -Wno-assume -) - YQL_LAST_ABI_VERSION() END() -RECURSE(purecalc_no_pg_wrapper) +RECURSE( + purecalc_no_pg_wrapper + format_handler +) IF(NOT EXPORT_CMAKE) RECURSE_FOR_TESTS( diff --git a/ydb/library/conclusion/generic/yql_status.h b/ydb/library/conclusion/generic/yql_status.h index 5a86b609f812..8b1d4f10107d 100644 --- a/ydb/library/conclusion/generic/yql_status.h +++ b/ydb/library/conclusion/generic/yql_status.h @@ -23,11 +23,11 @@ class TYQLConclusionStatusImpl : public TConclusionStatusGenericImpl(childIssue)); } - TBase::ErrorMessage = {std::move(issue)}; + TBase::ErrorDescription = {std::move(issue)}; return *this; } @@ -37,8 +37,8 @@ class TYQLConclusionStatusImpl : public TConclusionStatusGenericImplAddIssue(std::move(issue)); + Y_ABORT_UNLESS(!!TBase::ErrorDescription); + TBase::ErrorDescription->AddIssue(std::move(issue)); return *this; } diff --git a/ydb/library/yql/dq/actors/common/retry_queue.h b/ydb/library/yql/dq/actors/common/retry_queue.h index ab4e0f84c498..2124a0e9f6e2 100644 --- a/ydb/library/yql/dq/actors/common/retry_queue.h +++ b/ydb/library/yql/dq/actors/common/retry_queue.h @@ -186,6 +186,9 @@ class TRetryEventsQueue { THolder ev = MakeHolder(); ev->Record = Event->Record; ev->Record.MutableTransportMeta()->SetConfirmedSeqNo(confirmedSeqNo); + for (ui32 i = 0; i < Event->GetPayloadCount(); ++i) { + ev->AddPayload(TRope(Event->GetPayload(i))); + } return MakeHolder(Recipient, Sender, ev.Release(), NActors::IEventHandle::FlagTrackDelivery, Cookie); } diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.cpp b/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.cpp index 426a4ab92f79..810d54cd8693 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.cpp +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.cpp @@ -2,6 +2,7 @@ #include "probes.h" #include +#include #include #include #include @@ -11,11 +12,14 @@ #include #include +#include #include +#include #include #include #include #include +#include #include #include #include @@ -119,7 +123,7 @@ class TDqPqRdReadActor : public NActors::TActor, public NYql:: } public: - TVector Data; + TVector Data; i64 UsedSpace = 0; ui64 NextOffset = 0; ui64 PartitionId; @@ -155,7 +159,6 @@ class TDqPqRdReadActor : public NActors::TActor, public NYql:: }; private: - std::vector> MetadataFields; const TString Token; TMaybe CoordinatorActorId; NActors::TActorId LocalRowDispatcherActorId; @@ -166,7 +169,12 @@ class TDqPqRdReadActor : public NActors::TActor, public NYql:: bool SchedulePrintStatePeriod = false; bool ProcessStateScheduled = false; bool InFlyAsyncInputData = false; - TCounters Counters; + TCounters Counters; + + // Parsing info + std::vector> ColumnIndexes; // Output column index in schema passed into RowDispatcher + const TType* InputDataType = nullptr; // Multi type (comes from Row Dispatcher) + std::unique_ptr> DataUnpacker; struct SessionInfo { enum class ESessionStatus { @@ -197,7 +205,7 @@ class TDqPqRdReadActor : public NActors::TActor, public NYql:: ui64 PartitionId; ui64 Generation; }; - + TMap Sessions; const THolderFactory& HolderFactory; const i64 MaxBufferSize; @@ -211,6 +219,7 @@ class TDqPqRdReadActor : public NActors::TActor, public NYql:: const TTxId& txId, ui64 taskId, const THolderFactory& holderFactory, + const TTypeEnvironment& typeEnv, NPq::NProto::TDqPqTopicSource&& sourceParams, NPq::NProto::TDqReadTaskParams&& readParams, const NActors::TActorId& computeActorId, @@ -265,9 +274,9 @@ class TDqPqRdReadActor : public NActors::TActor, public NYql:: void PassAway() override; i64 GetAsyncInputData(NKikimr::NMiniKQL::TUnboxedValueBatch& buffer, TMaybe& watermark, bool&, i64 freeSpace) override; std::vector GetPartitionsToRead() const; - std::pair CreateItem(const TString& data); + void AddMessageBatch(TRope&& serializedBatch, NKikimr::NMiniKQL::TUnboxedValueBatch& buffer); void ProcessState(); - void Stop(const TString& message); + void Stop(NDqProto::StatusIds::StatusCode status, TIssues issues); void StopSessions(); void ReInit(const TString& reason); void PrintInternalState(); @@ -285,6 +294,7 @@ TDqPqRdReadActor::TDqPqRdReadActor( const TTxId& txId, ui64 taskId, const THolderFactory& holderFactory, + const TTypeEnvironment& typeEnv, NPq::NProto::TDqPqTopicSource&& sourceParams, NPq::NProto::TDqReadTaskParams&& readParams, const NActors::TActorId& computeActorId, @@ -300,11 +310,26 @@ TDqPqRdReadActor::TDqPqRdReadActor( , HolderFactory(holderFactory) , MaxBufferSize(bufferSize) { - MetadataFields.reserve(SourceParams.MetadataFieldsSize()); - TPqMetaExtractor fieldsExtractor; - for (const auto& fieldName : SourceParams.GetMetadataFields()) { - MetadataFields.emplace_back(fieldName, fieldsExtractor.FindExtractorLambda(fieldName)); - } + const auto programBuilder = std::make_unique(typeEnv, *holderFactory.GetFunctionRegistry()); + + // Parse output schema (expected struct output type) + const auto& outputTypeYson = SourceParams.GetRowType(); + const auto outputItemType = NCommon::ParseTypeFromYson(TStringBuf(outputTypeYson), *programBuilder, Cerr); + YQL_ENSURE(outputItemType, "Failed to parse output type: " << outputTypeYson); + YQL_ENSURE(outputItemType->IsStruct(), "Output type " << outputTypeYson << " is not struct"); + const auto structType = static_cast(outputItemType); + + // Build input schema and unpacker (for data comes from RowDispatcher) + TVector inputTypeParts; + inputTypeParts.reserve(SourceParams.ColumnsSize()); + ColumnIndexes.resize(structType->GetMembersCount()); + for (size_t i = 0; i < SourceParams.ColumnsSize(); ++i) { + const auto index = structType->GetMemberIndex(SourceParams.GetColumns().Get(i)); + inputTypeParts.emplace_back(structType->GetMemberType(index)); + ColumnIndexes[index] = i; + } + InputDataType = programBuilder->NewMultiType(inputTypeParts); + DataUnpacker = std::make_unique>(InputDataType); IngressStats.Level = statsLevel; SRC_LOG_I("Start read actor, local row dispatcher " << LocalRowDispatcherActorId.ToString() << ", metadatafields: " << JoinSeq(',', SourceParams.GetMetadataFields())); @@ -427,18 +452,18 @@ i64 TDqPqRdReadActor::GetAsyncInputData(NKikimr::NMiniKQL::TUnboxedValueBatch& b i64 usedSpace = 0; buffer.clear(); do { - auto& readyBatch = ReadyBuffer.front(); + auto readyBatch = std::move(ReadyBuffer.front()); + ReadyBuffer.pop(); - for (const auto& message : readyBatch.Data) { - auto [item, size] = CreateItem(message); - buffer.push_back(std::move(item)); + for (auto& messageBatch : readyBatch.Data) { + AddMessageBatch(std::move(messageBatch), buffer); } + usedSpace += readyBatch.UsedSpace; freeSpace -= readyBatch.UsedSpace; TPartitionKey partitionKey{TString{}, readyBatch.PartitionId}; PartitionToOffset[partitionKey] = readyBatch.NextOffset; SRC_LOG_T("NextOffset " << readyBatch.NextOffset); - ReadyBuffer.pop(); } while (freeSpace > 0 && !ReadyBuffer.empty()); ReadyBufferSizeBytes -= usedSpace; @@ -505,7 +530,10 @@ void TDqPqRdReadActor::Handle(NFq::TEvRowDispatcher::TEvSessionError::TPtr& ev) if (!CheckSession(sessionInfo, ev, partitionId)) { return; } - Stop(ev->Get()->Record.GetMessage()); + + NYql::TIssues issues; + IssuesFromMessage(ev->Get()->Record.GetIssues(), issues); + Stop(ev->Get()->Record.GetStatusCode(), issues); } void TDqPqRdReadActor::Handle(NFq::TEvRowDispatcher::TEvStatistics::TPtr& ev) { @@ -642,11 +670,9 @@ void TDqPqRdReadActor::ReInit(const TString& reason) { PrintInternalState(); } -void TDqPqRdReadActor::Stop(const TString& message) { - NYql::TIssues issues; - issues.AddIssue(NYql::TIssue{message}); - SRC_LOG_E("Stop read actor, error: " << message); - Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::BAD_REQUEST)); // TODO: use UNAVAILABLE ? +void TDqPqRdReadActor::Stop(NDqProto::StatusIds::StatusCode status, TIssues issues) { + SRC_LOG_E("Stop read actor, status: " << NYql::NDqProto::StatusIds_StatusCode_Name(status) << ", issues: " << issues.ToOneLineString()); + Send(ComputeActorId, new TEvAsyncInputError(InputIndex, std::move(issues), status)); } void TDqPqRdReadActor::Handle(NFq::TEvRowDispatcher::TEvCoordinatorResult::TPtr& ev) { @@ -664,7 +690,7 @@ void TDqPqRdReadActor::Handle(NFq::TEvRowDispatcher::TEvCoordinatorResult::TPtr& TActorId rowDispatcherActorId = ActorIdFromProto(p.GetActorId()); for (auto partitionId : p.GetPartitionId()) { if (Sessions.contains(partitionId)) { - Stop("Internal error: session already exists"); + Stop(NDqProto::StatusIds::INTERNAL_ERROR, {TIssue("Session already exists")}); return; } SRC_LOG_I("Create session to RD (" << rowDispatcherActorId << "), partitionId " << partitionId); @@ -735,10 +761,16 @@ void TDqPqRdReadActor::Handle(NFq::TEvRowDispatcher::TEvMessageBatch::TPtr& ev) ui64 bytes = 0; for (const auto& message : ev->Get()->Record.GetMessages()) { - SRC_LOG_T("Json: " << message.GetJson()); - activeBatch.Data.emplace_back(message.GetJson()); - sessionInfo.NextOffset = message.GetOffset() + 1; - bytes += message.GetJson().size(); + const auto& offsets = message.GetOffsets(); + if (offsets.empty()) { + Stop(NDqProto::StatusIds::INTERNAL_ERROR, {TIssue("Got unexpected empty batch from row dispatcher")}); + return; + } + + activeBatch.Data.emplace_back(ev->Get()->GetPayload(message.GetPayloadId())); + bytes += activeBatch.Data.back().GetSize(); + + sessionInfo.NextOffset = *offsets.rbegin() + 1; SRC_LOG_T("TEvMessageBatch NextOffset " << sessionInfo.NextOffset); } activeBatch.UsedSpace = bytes; @@ -748,25 +780,30 @@ void TDqPqRdReadActor::Handle(NFq::TEvRowDispatcher::TEvMessageBatch::TPtr& ev) NotifyCA(); } -std::pair TDqPqRdReadActor::CreateItem(const TString& data) { - i64 usedSpace = 0; - NUdf::TUnboxedValuePod item; - if (MetadataFields.empty()) { - item = NKikimr::NMiniKQL::MakeString(NUdf::TStringRef(data.data(), data.size())); - usedSpace += data.size(); - return std::make_pair(item, usedSpace); - } +void TDqPqRdReadActor::AddMessageBatch(TRope&& messageBatch, NKikimr::NMiniKQL::TUnboxedValueBatch& buffer) { + // TDOD: pass multi type directly to CA, without transforming it into struct - NUdf::TUnboxedValue* itemPtr; - item = HolderFactory.CreateDirectArrayHolder(MetadataFields.size() + 1, itemPtr); - *(itemPtr++) = NKikimr::NMiniKQL::MakeString(NUdf::TStringRef(data.data(), data.size())); - usedSpace += data.size(); + NKikimr::NMiniKQL::TUnboxedValueBatch parsedData(InputDataType); + DataUnpacker->UnpackBatch(MakeChunkedBuffer(std::move(messageBatch)), HolderFactory, parsedData); + + while (!parsedData.empty()) { + const auto* parsedRow = parsedData.Head(); + + NUdf::TUnboxedValue* itemPtr; + NUdf::TUnboxedValuePod item = HolderFactory.CreateDirectArrayHolder(ColumnIndexes.size(), itemPtr); + for (const auto index : ColumnIndexes) { + if (index) { + YQL_ENSURE(*index < parsedData.Width(), "Unexpected data width " << parsedData.Width() << ", failed to extract column by index " << index); + *(itemPtr++) = parsedRow[*index]; + } else { + // TODO: support metadata fields here + *(itemPtr++) = NUdf::TUnboxedValue(); + } + } - for ([[maybe_unused]] const auto& [name, extractor] : MetadataFields) { - auto ub = NYql::NUdf::TUnboxedValuePod(0); // TODO: use real values - *(itemPtr++) = std::move(ub); + buffer.emplace_back(std::move(item)); + parsedData.Pop(); } - return std::make_pair(item, usedSpace); } void TDqPqRdReadActor::Handle(NActors::TEvents::TEvPong::TPtr& ev) { @@ -856,6 +893,7 @@ void TDqPqRdReadActor::NotifyCA() { } std::pair CreateDqPqRdReadActor( + const TTypeEnvironment& typeEnv, NPq::NProto::TDqPqTopicSource&& settings, ui64 inputIndex, TCollectStatsLevel statsLevel, @@ -884,6 +922,7 @@ std::pair CreateDqPqRdReadActor( txId, taskId, holderFactory, + typeEnv, std::move(settings), std::move(readTaskParamsMsg), computeActorId, diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.h b/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.h index a218ae18c587..6e09d4637eba 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.h +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_rd_read_actor.h @@ -23,6 +23,7 @@ class TDqAsyncIoFactory; const i64 PQRdReadDefaultFreeSpace = 256_MB; std::pair CreateDqPqRdReadActor( + const NKikimr::NMiniKQL::TTypeEnvironment& typeEnv, NPq::NProto::TDqPqTopicSource&& settings, ui64 inputIndex, TCollectStatsLevel statsLevel, 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 efae17d4d68e..900b0f9916bb 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 @@ -698,6 +698,7 @@ void RegisterDqPqReadActorFactory(TDqAsyncIoFactory& factory, NYdb::TDriver driv } return CreateDqPqRdReadActor( + args.TypeEnv, std::move(settings), args.InputIndex, args.StatsLevel, diff --git a/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor_base.h b/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor_base.h index 5028a93078c2..7b965b7f94b4 100644 --- a/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor_base.h +++ b/ydb/library/yql/providers/pq/async_io/dq_pq_read_actor_base.h @@ -1,6 +1,8 @@ #pragma once #include +#include +#include namespace NYql::NDq::NInternal { diff --git a/ydb/library/yql/providers/pq/expr_nodes/yql_pq_expr_nodes.json b/ydb/library/yql/providers/pq/expr_nodes/yql_pq_expr_nodes.json index 1fe5bf0b57dd..9a6603bd6631 100644 --- a/ydb/library/yql/providers/pq/expr_nodes/yql_pq_expr_nodes.json +++ b/ydb/library/yql/providers/pq/expr_nodes/yql_pq_expr_nodes.json @@ -72,7 +72,8 @@ {"Index": 2, "Name": "Columns", "Type": "TExprBase"}, {"Index": 3, "Name": "Settings", "Type": "TCoNameValueTupleList"}, {"Index": 4, "Name": "Token", "Type": "TCoSecureParam"}, - {"Index": 5, "Name": "FilterPredicate", "Type": "TCoAtom"} + {"Index": 5, "Name": "FilterPredicate", "Type": "TCoAtom"}, + {"Index": 6, "Name": "RowType", "Type": "TExprBase"} ] }, { diff --git a/ydb/library/yql/providers/pq/proto/dq_io.proto b/ydb/library/yql/providers/pq/proto/dq_io.proto index 5cda0945d7a9..93b5d8a06026 100644 --- a/ydb/library/yql/providers/pq/proto/dq_io.proto +++ b/ydb/library/yql/providers/pq/proto/dq_io.proto @@ -40,6 +40,8 @@ message TDqPqTopicSource { string ReconnectPeriod = 16; // disabled by default, example of a parameter: 5m bool EnabledLLVM = 17; string ReadGroup = 18; + string Format = 19; + string RowType = 20; // Final row type with metadata columns } message TDqPqTopicSink { diff --git a/ydb/library/yql/providers/pq/provider/yql_pq_datasource_type_ann.cpp b/ydb/library/yql/providers/pq/provider/yql_pq_datasource_type_ann.cpp index 988cc73bea76..82cebc19e0cc 100644 --- a/ydb/library/yql/providers/pq/provider/yql_pq_datasource_type_ann.cpp +++ b/ydb/library/yql/providers/pq/provider/yql_pq_datasource_type_ann.cpp @@ -1,4 +1,5 @@ #include "yql_pq_provider_impl.h" +#include "yql_pq_helpers.h" #include #include @@ -7,6 +8,7 @@ #include #include #include +#include #include #include @@ -132,7 +134,7 @@ class TPqDataSourceTypeAnnotationTransformer : public TVisitorTransformerBase { } TStatus HandleDqTopicSource(TExprBase input, TExprContext& ctx) { - if (!EnsureArgsCount(input.Ref(), 6, ctx)) { + if (!EnsureArgsCount(input.Ref(), 7, ctx)) { return TStatus::Error; } @@ -156,6 +158,13 @@ class TPqDataSourceTypeAnnotationTransformer : public TVisitorTransformerBase { return TStatus::Error; } + if (const auto maybeSharedReadingSetting = FindSetting(topicSource.Settings().Ptr(), SharedReading)) { + const TExprNode& value = maybeSharedReadingSetting.Cast().Ref(); + if (value.IsAtom() && FromString(value.Content())) { + input.Ptr()->SetTypeAnn(ctx.MakeType(topicSource.RowType().Ref().GetTypeAnn())); + return TStatus::Ok; + } + } if (topic.Metadata().Empty()) { input.Ptr()->SetTypeAnn(ctx.MakeType(ctx.MakeType(EDataSlot::String))); diff --git a/ydb/library/yql/providers/pq/provider/yql_pq_dq_integration.cpp b/ydb/library/yql/providers/pq/provider/yql_pq_dq_integration.cpp index 1932fbe9a9a2..a91029b6566f 100644 --- a/ydb/library/yql/providers/pq/provider/yql_pq_dq_integration.cpp +++ b/ydb/library/yql/providers/pq/provider/yql_pq_dq_integration.cpp @@ -81,48 +81,7 @@ class TPqDqIntegration: public TDqIntegrationBase { ->Cast()->GetItems().back()->Cast() ->GetItemType()->Cast(); const auto& clusterName = pqReadTopic.DataSource().Cluster().StringValue(); - - TVector settings; - settings.push_back(Build(ctx, pqReadTopic.Pos()) - .Name().Build("format") - .Value(pqReadTopic.Format()) - .Done()); - - auto format = pqReadTopic.Format().Ref().Content(); - - TVector innerSettings; - if (pqReadTopic.Compression() != "") { - innerSettings.push_back(Build(ctx, pqReadTopic.Pos()) - .Name().Build("compression") - .Value(pqReadTopic.Compression()) - .Done()); - } - - if (!innerSettings.empty()) { - settings.push_back(Build(ctx, pqReadTopic.Pos()) - .Name().Build("settings") - .Value() - .Add(innerSettings) - .Build() - .Done()); - } - - TExprNode::TListType metadataFieldsList; - for (auto sysColumn : AllowedPqMetaSysColumns()) { - metadataFieldsList.push_back(ctx.NewAtom(pqReadTopic.Pos(), sysColumn)); - } - - settings.push_back(Build(ctx, pqReadTopic.Pos()) - .Name().Build("metadataColumns") - .Value(ctx.NewList(pqReadTopic.Pos(), std::move(metadataFieldsList))) - .Done()); - - - settings.push_back(Build(ctx, pqReadTopic.Pos()) - .Name().Build("formatSettings") - .Value(std::move(pqReadTopic.Settings())) - .Done()); - + const auto format = pqReadTopic.Format().Ref().Content(); const auto token = "cluster:default_" + clusterName; const auto& typeItems = pqReadTopic.Topic().RowSpec().Ref().GetTypeAnn()->Cast()->GetType()->Cast()->GetItems(); @@ -136,25 +95,21 @@ class TPqDqIntegration: public TDqIntegrationBase { }); auto columnNames = ctx.NewList(pos, std::move(colNames)); - auto row = Build(ctx, read->Pos()) - .Name("row") - .Done(); - TString emptyPredicate; - - return Build(ctx, read->Pos()) + return Build(ctx, pos) .Input() .World(pqReadTopic.World()) .Topic(pqReadTopic.Topic()) .Columns(std::move(columnNames)) - .Settings(BuildTopicReadSettings(clusterName, wrSettings, read->Pos(), format, ctx)) + .Settings(BuildTopicReadSettings(clusterName, wrSettings, pos, format, ctx)) .Token() .Name().Build(token) .Build() - .FilterPredicate().Value(emptyPredicate).Build() + .FilterPredicate().Value(TString()).Build() // Empty predicate by default <=> WHERE TRUE + .RowType(ExpandType(pqReadTopic.Pos(), *rowType, ctx)) .Build() .RowType(ExpandType(pqReadTopic.Pos(), *rowType, ctx)) .DataSource(pqReadTopic.DataSource().Cast()) - .Settings(Build(ctx, read->Pos()).Add(settings).Done()) + .Settings(BuildDqSourceWrapSettings(pqReadTopic, pos, ctx)) .Done().Ptr(); } return read; @@ -209,6 +164,9 @@ class TPqDqIntegration: public TDqIntegrationBase { srcDesc.SetClusterType(ToClusterType(clusterDesc->ClusterType)); srcDesc.SetDatabaseId(clusterDesc->DatabaseId); + const TStructExprType* fullRowType = topicSource.RowType().Ref().GetTypeAnn()->Cast()->GetType()->Cast(); + srcDesc.SetRowType(NCommon::WriteTypeToYson(fullRowType, NYT::NYson::EYsonFormat::Text)); + if (const auto& types = State_->Types) { if (const auto& optLLVM = types->OptLLVM) { srcDesc.SetEnabledLLVM(!optLLVM->empty() && *optLLVM != "OFF"); @@ -247,6 +205,7 @@ class TPqDqIntegration: public TDqIntegrationBase { srcDesc.MutableWatermarks()->SetIdlePartitionsEnabled(true); } } + srcDesc.SetFormat(format); if (auto maybeToken = TMaybeNode(topicSource.Token().Raw())) { srcDesc.MutableToken()->SetName(TString(maybeToken.Cast().Name().Value())); @@ -270,12 +229,9 @@ class TPqDqIntegration: public TDqIntegrationBase { auto serializedProto = topicSource.FilterPredicate().Ref().Content(); YQL_ENSURE (predicateProto.ParseFromString(serializedProto)); - sharedReading = sharedReading && (format == "json_each_row" || format == "raw"); TString predicateSql = NYql::FormatWhere(predicateProto); if (sharedReading) { - if (format == "json_each_row") { - srcDesc.SetPredicate(predicateSql); - } + srcDesc.SetPredicate(predicateSql); srcDesc.SetSharedReading(true); } protoSettings.PackFrom(srcDesc); @@ -341,13 +297,10 @@ class TPqDqIntegration: public TDqIntegrationBase { } } - auto clusterConfiguration = State_->Configuration->ClustersConfigurationSettings.FindPtr(cluster); - if (!clusterConfiguration) { - ythrow yexception() << "Unknown pq cluster \"" << cluster << "\""; - } + auto clusterConfiguration = GetClusterConfiguration(cluster); Add(props, EndpointSetting, clusterConfiguration->Endpoint, pos, ctx); - Add(props, SharedReading, ToString(clusterConfiguration->SharedReading), pos, ctx); + Add(props, SharedReading, ToString(UseSharedReading(clusterConfiguration, format)), pos, ctx); Add(props, ReconnectPeriod, ToString(clusterConfiguration->ReconnectPeriod), pos, ctx); Add(props, Format, format, pos, ctx); Add(props, ReadGroup, clusterConfiguration->ReadGroup, pos, ctx); @@ -383,6 +336,65 @@ class TPqDqIntegration: public TDqIntegrationBase { .Done(); } + NNodes::TCoNameValueTupleList BuildDqSourceWrapSettings(const TPqReadTopic& pqReadTopic, TPositionHandle pos, TExprContext& ctx) const { + TVector settings; + settings.push_back(Build(ctx, pos) + .Name().Build("format") + .Value(pqReadTopic.Format()) + .Done()); + + TExprNode::TListType metadataFieldsList; + for (const auto& sysColumn : AllowedPqMetaSysColumns()) { + metadataFieldsList.push_back(ctx.NewAtom(pos, sysColumn)); + } + + settings.push_back(Build(ctx, pos) + .Name().Build("metadataColumns") + .Value(ctx.NewList(pos, std::move(metadataFieldsList))) + .Done()); + + settings.push_back(Build(ctx, pos) + .Name().Build("formatSettings") + .Value(std::move(pqReadTopic.Settings())) + .Done()); + + TVector innerSettings; + if (pqReadTopic.Compression() != "") { + innerSettings.push_back(Build(ctx, pos) + .Name().Build("compression") + .Value(pqReadTopic.Compression()) + .Done()); + } + + const auto clusterConfiguration = GetClusterConfiguration(pqReadTopic.DataSource().Cluster().StringValue()); + Add(innerSettings, SharedReading, ToString(UseSharedReading(clusterConfiguration, pqReadTopic.Format().Ref().Content())), pos, ctx); + + if (!innerSettings.empty()) { + settings.push_back(Build(ctx, pos) + .Name().Build("settings") + .Value() + .Add(innerSettings) + .Build() + .Done()); + } + + return Build(ctx, pos) + .Add(settings) + .Done(); + } + + const TPqClusterConfigurationSettings* GetClusterConfiguration(const TString& cluster) const { + const auto clusterConfiguration = State_->Configuration->ClustersConfigurationSettings.FindPtr(cluster); + if (!clusterConfiguration) { + ythrow yexception() << "Unknown pq cluster \"" << cluster << "\""; + } + return clusterConfiguration; + } + + static bool UseSharedReading(const TPqClusterConfigurationSettings* clusterConfiguration, std::string_view format) { + return clusterConfiguration->SharedReading && (format == "json_each_row" || format == "raw"); + } + private: TPqState* State_; // State owns dq integration, so back reference must be not smart. }; diff --git a/ydb/library/yql/providers/pq/provider/yql_pq_helpers.cpp b/ydb/library/yql/providers/pq/provider/yql_pq_helpers.cpp index c4fc523825f1..de525c933685 100644 --- a/ydb/library/yql/providers/pq/provider/yql_pq_helpers.cpp +++ b/ydb/library/yql/providers/pq/provider/yql_pq_helpers.cpp @@ -101,5 +101,20 @@ void FillSettingsWithResolvedYdsIds( } } +TMaybeNode FindSetting(TExprNode::TPtr settings, TStringBuf name) { + const auto maybeSettingsList = TMaybeNode(settings); + if (!maybeSettingsList) { + return nullptr; + } + const auto settingsList = maybeSettingsList.Cast(); + + for (size_t i = 0; i < settingsList.Size(); ++i) { + TCoNameValueTuple setting = settingsList.Item(i); + if (setting.Name().Value() == name) { + return setting.Value(); + } + } + return nullptr; +} } // namespace NYql diff --git a/ydb/library/yql/providers/pq/provider/yql_pq_helpers.h b/ydb/library/yql/providers/pq/provider/yql_pq_helpers.h index 0a64879f0352..bf328554a817 100644 --- a/ydb/library/yql/providers/pq/provider/yql_pq_helpers.h +++ b/ydb/library/yql/providers/pq/provider/yql_pq_helpers.h @@ -21,4 +21,6 @@ void FillSettingsWithResolvedYdsIds( const TPqState::TPtr& state, const TDatabaseResolverResponse::TDatabaseDescriptionMap& fullResolvedIds); +NNodes::TMaybeNode FindSetting(TExprNode::TPtr settings, TStringBuf name); + } // namespace NYql diff --git a/ydb/library/yql/providers/pq/provider/yql_pq_logical_opt.cpp b/ydb/library/yql/providers/pq/provider/yql_pq_logical_opt.cpp index 60a541aa94c9..fc667dbe6030 100644 --- a/ydb/library/yql/providers/pq/provider/yql_pq_logical_opt.cpp +++ b/ydb/library/yql/providers/pq/provider/yql_pq_logical_opt.cpp @@ -216,6 +216,7 @@ class TPqLogicalOptProposalTransformer : public TOptimizeTransformerBase { .RowSpec(DropUnusedRowItems(pqTopic.RowSpec().Pos(), inputRowType, usedColumnNames, ctx)) .Build() .Columns(DropUnusedColumns(dqPqTopicSource.Columns(), usedColumnNames, ctx)) + .RowType(DropUnusedRowItems(dqPqTopicSource.RowType().Pos(), oldRowType, usedColumnNames, ctx)) .Done() .Ptr(); diff --git a/ydb/library/yql/providers/pq/provider/yql_pq_mkql_compiler.cpp b/ydb/library/yql/providers/pq/provider/yql_pq_mkql_compiler.cpp index cd708f07c0ef..483393d9b3a8 100644 --- a/ydb/library/yql/providers/pq/provider/yql_pq_mkql_compiler.cpp +++ b/ydb/library/yql/providers/pq/provider/yql_pq_mkql_compiler.cpp @@ -1,5 +1,7 @@ #include "yql_pq_mkql_compiler.h" +#include "yql_pq_helpers.h" +#include #include #include #include @@ -9,10 +11,52 @@ namespace NYql { using namespace NKikimr::NMiniKQL; using namespace NNodes; +namespace { + +bool UseSharedReading(TExprNode::TPtr settings) { + const auto maybeInnerSettings = FindSetting(settings, "settings"); + if (!maybeInnerSettings) { + return false; + } + + const auto maybeSharedReadingSetting = FindSetting(maybeInnerSettings.Cast().Ptr(), SharedReading); + if (!maybeSharedReadingSetting) { + return false; + } + + const TExprNode& value = maybeSharedReadingSetting.Cast().Ref(); + return value.IsAtom() && FromString(value.Content()); +} + +TRuntimeNode WrapSharedReading(const TDqSourceWrapBase &wrapper, NCommon::TMkqlBuildContext& ctx) { + const auto input = MkqlBuildExpr(wrapper.Input().Ref(), ctx); + const auto flow = ctx.ProgramBuilder.ToFlow(input); + + const TStructExprType* rowType = wrapper.RowType().Ref().GetTypeAnn()->Cast()->GetType()->Cast(); + const auto* finalItemStructType = static_cast(NCommon::BuildType(wrapper.RowType().Ref(), *rowType, ctx.ProgramBuilder)); + + return ctx.ProgramBuilder.ExpandMap(flow, [&](TRuntimeNode item) -> TRuntimeNode::TList { + TRuntimeNode::TList fields; + fields.reserve(finalItemStructType->GetMembersCount()); + for (ui32 i = 0; i < finalItemStructType->GetMembersCount(); ++i) { + fields.push_back(ctx.ProgramBuilder.Member(item, finalItemStructType->GetMemberName(i))); + } + return fields; + }); +} + +} + void RegisterDqPqMkqlCompilers(NCommon::TMkqlCallableCompilerBase& compiler) { compiler.ChainCallable(TDqSourceWideWrap::CallableName(), [](const TExprNode& node, NCommon::TMkqlBuildContext& ctx) { if (const auto wrapper = TDqSourceWideWrap(&node); wrapper.DataSource().Category().Value() == PqProviderName) { + if (const auto maybeSettings = wrapper.Settings()) { + if (UseSharedReading(maybeSettings.Cast().Ptr())) { + return WrapSharedReading(wrapper, ctx); + } + } + const auto wrapped = TryWrapWithParser(wrapper, ctx); if (wrapped) { return *wrapped; diff --git a/ydb/tests/fq/pq_async_io/ut/dq_pq_rd_read_actor_ut.cpp b/ydb/tests/fq/pq_async_io/ut/dq_pq_rd_read_actor_ut.cpp index 83fc68b55d0b..2b12f60b758c 100644 --- a/ydb/tests/fq/pq_async_io/ut/dq_pq_rd_read_actor_ut.cpp +++ b/ydb/tests/fq/pq_async_io/ut/dq_pq_rd_read_actor_ut.cpp @@ -1,12 +1,16 @@ #include +#include +#include +#include #include -#include #include #include #include +#include #include +#include #include @@ -15,7 +19,6 @@ namespace NYql::NDq { const ui64 PartitionId = 666; struct TFixture : public TPqIoTestFixture { - TFixture() { LocalRowDispatcherId = CaSetup->Runtime->AllocateEdgeActor(); Coordinator1Id = CaSetup->Runtime->AllocateEdgeActor(); @@ -43,6 +46,7 @@ struct TFixture : public TPqIoTestFixture { NYql::NPq::NProto::TDqPqTopicSource copySettings = settings; auto [dqSource, dqSourceAsActor] = CreateDqPqRdReadActor( + actor.TypeEnv, std::move(copySettings), 0, NYql::NDq::TCollectStatsLevel::None, @@ -132,13 +136,32 @@ struct TFixture : public TPqIoTestFixture { }); } - void MockMessageBatch(ui64 offset, const std::vector& jsons, NActors::TActorId rowDispatcherId, ui64 generation = 1) { + TRope SerializeItem(TFakeActor& actor, ui64 intValue, const TString& strValue) { + NKikimr::NMiniKQL::TType* typeMkql = actor.ProgramBuilder.NewMultiType({ + NYql::NCommon::ParseTypeFromYson(TStringBuf("[DataType; Uint64]"), actor.ProgramBuilder, Cerr), + NYql::NCommon::ParseTypeFromYson(TStringBuf("[DataType; String]"), actor.ProgramBuilder, Cerr) + }); + UNIT_ASSERT_C(typeMkql, "Failed to create multi type"); + + NKikimr::NMiniKQL::TValuePackerTransport packer(typeMkql); + + TVector values = { + NUdf::TUnboxedValuePod(intValue), + NKikimr::NMiniKQL::MakeString(strValue) + }; + packer.AddWideItem(values.data(), 2); + + return NYql::MakeReadOnlyRope(packer.Finish()); + } + + // Supported schema (Uint64, String) + void MockMessageBatch(ui64 offset, const std::vector>& messages, NActors::TActorId rowDispatcherId, ui64 generation = 1) { CaSetup->Execute([&](TFakeActor& actor) { auto event = new NFq::TEvRowDispatcher::TEvMessageBatch(); - for (const auto& json :jsons) { + for (const auto& item : messages) { NFq::NRowDispatcherProto::TEvMessage message; - message.SetJson(json); - message.SetOffset(offset++); + message.SetPayloadId(event->AddPayload(SerializeItem(actor, item.first, item.second))); + message.AddOffsets(offset++); *event->Record.AddMessages() = message; } event->Record.SetPartitionId(PartitionId); @@ -150,7 +173,8 @@ struct TFixture : public TPqIoTestFixture { void MockSessionError() { CaSetup->Execute([&](TFakeActor& actor) { auto event = new NFq::TEvRowDispatcher::TEvSessionError(); - event->Record.SetMessage("A problem has been detected and session has been shut down to prevent damage your life"); + event->Record.SetStatusCode(::NYql::NDqProto::StatusIds::BAD_REQUEST); + IssueToMessage(TIssue("A problem has been detected and session has been shut down to prevent damage your life"), event->Record.AddIssues()); event->Record.SetPartitionId(PartitionId); CaSetup->Runtime->Send(new NActors::IEventHandle(*actor.DqAsyncInputActorId, RowDispatcher1, event, 0, 1)); }); @@ -177,7 +201,7 @@ struct TFixture : public TPqIoTestFixture { watermarksBeforeIter == watermarkBeforePositions.end() || *watermarksBeforeIter > expectedPos, "Watermark before item on position " << expectedPos << " was expected"); - UNIT_ASSERT_EQUAL(std::get(item), expected.at(expectedPos)); + UNIT_ASSERT_VALUES_EQUAL(std::get(item), expected.at(expectedPos)); expectedPos++; } } @@ -206,7 +230,7 @@ struct TFixture : public TPqIoTestFixture { void StartSession(NYql::NPq::NProto::TDqPqTopicSource& settings, i64 freeSpace = 1_MB) { InitRdSource(settings, freeSpace); - SourceRead(UVParser); + SourceRead>(UVPairParser); ExpectCoordinatorChangesSubscribe(); MockCoordinatorChanged(Coordinator1Id); @@ -217,21 +241,21 @@ struct TFixture : public TPqIoTestFixture { MockAck(RowDispatcher1); } - void ProcessSomeJsons(ui64 offset, const std::vector& jsons, NActors::TActorId rowDispatcherId, - std::function(const NUdf::TUnboxedValue&)> uvParser = UVParser, ui64 generation = 1) { + void ProcessSomeMessages(ui64 offset, const std::vector>& messages, NActors::TActorId rowDispatcherId, + std::function>(const NUdf::TUnboxedValue&)> uvParser = UVPairParser, ui64 generation = 1) { MockNewDataArrived(rowDispatcherId, generation); ExpectGetNextBatch(rowDispatcherId); - MockMessageBatch(offset, jsons, rowDispatcherId, generation); + MockMessageBatch(offset, messages, rowDispatcherId, generation); - auto result = SourceReadDataUntil(uvParser, jsons.size()); - AssertDataWithWatermarks(result, jsons, {}); + auto result = SourceReadDataUntil>(uvParser, messages.size()); + AssertDataWithWatermarks(result, messages, {}); } - const TString Json1 = "{\"dt\":100,\"value\":\"value1\"}"; - const TString Json2 = "{\"dt\":200,\"value\":\"value2\"}"; - const TString Json3 = "{\"dt\":300,\"value\":\"value3\"}"; - const TString Json4 = "{\"dt\":400,\"value\":\"value4\"}"; + const std::pair Message1 = {100, "value1"}; + const std::pair Message2 = {200, "value2"}; + const std::pair Message3 = {300, "value3"}; + const std::pair Message4 = {400, "value4"}; NYql::NPq::NProto::TDqPqTopicSource Source1 = BuildPqTopicSourceSettings("topicName"); @@ -245,7 +269,7 @@ struct TFixture : public TPqIoTestFixture { Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { Y_UNIT_TEST_F(TestReadFromTopic, TFixture) { StartSession(Source1); - ProcessSomeJsons(0, {Json1, Json2}, RowDispatcher1); + ProcessSomeMessages(0, {Message1, Message2}, RowDispatcher1); } Y_UNIT_TEST_F(SessionError, TFixture) { @@ -257,7 +281,7 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { bool failured = false; while (Now() < deadline) { - SourceRead(UVParser); + SourceRead>(UVPairParser); if (future.HasValue()) { UNIT_ASSERT_STRING_CONTAINS(future.GetValue().ToOneLineString(), "damage your life"); failured = true; @@ -273,17 +297,18 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { MockNewDataArrived(RowDispatcher1); ExpectGetNextBatch(RowDispatcher1); - const std::vector data1 = {Json1, Json2}; + const std::vector> data1 = {Message1, Message2}; MockMessageBatch(0, data1, RowDispatcher1); - const std::vector data2 = {Json3, Json4}; + const std::vector> data2 = {Message3, Message4}; MockMessageBatch(2, data2, RowDispatcher1); - auto result = SourceReadDataUntil(UVParser, 1, 1); - std::vector expected{data1}; + auto result = SourceReadDataUntil>(UVPairParser, 1, 1); + std::vector> expected{data1}; AssertDataWithWatermarks(result, expected, {}); - UNIT_ASSERT_EQUAL(SourceRead(UVParser, 0).size(), 0); + auto readSize = SourceRead>(UVPairParser, 0).size(); + UNIT_ASSERT_VALUES_EQUAL(readSize, 0); } Y_UNIT_TEST(TestSaveLoadPqRdRead) { @@ -292,7 +317,7 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { { TFixture f; f.StartSession(f.Source1); - f.ProcessSomeJsons(0, {f.Json1, f.Json2}, f.RowDispatcher1); // offsets: 0, 1 + f.ProcessSomeMessages(0, {f.Message1, f.Message2}, f.RowDispatcher1); // offsets: 0, 1 f.SaveSourceState(CreateCheckpoint(), state); Cerr << "State saved" << Endl; @@ -300,9 +325,9 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { { TFixture f; f.InitRdSource(f.Source1); - f.SourceRead(UVParser); + f.SourceRead>(UVPairParser); f.LoadSource(state); - f.SourceRead(UVParser); + f.SourceRead>(UVPairParser); f.ExpectCoordinatorChangesSubscribe(); f.MockCoordinatorChanged(f.Coordinator1Id); @@ -312,7 +337,7 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { f.ExpectStartSession(2, f.RowDispatcher1); f.MockAck(f.RowDispatcher1); - f.ProcessSomeJsons(2, {f.Json3}, f.RowDispatcher1); // offsets: 2 + f.ProcessSomeMessages(2, {f.Message3}, f.RowDispatcher1); // offsets: 2 state.Data.clear(); f.SaveSourceState(CreateCheckpoint(), state); Cerr << "State saved" << Endl; @@ -320,9 +345,9 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { { TFixture f; f.InitRdSource(f.Source1); - f.SourceRead(UVParser); + f.SourceRead>(UVPairParser); f.LoadSource(state); - f.SourceRead(UVParser); + f.SourceRead>(UVPairParser); f.ExpectCoordinatorChangesSubscribe(); f.MockCoordinatorChanged(f.Coordinator1Id); @@ -332,21 +357,21 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { f.ExpectStartSession(3, f.RowDispatcher1); f.MockAck(f.RowDispatcher1); - f.ProcessSomeJsons(3, {f.Json4}, f.RowDispatcher1); // offsets: 3 + f.ProcessSomeMessages(3, {f.Message4}, f.RowDispatcher1); // offsets: 3 } } Y_UNIT_TEST_F(CoordinatorChanged, TFixture) { StartSession(Source1); - ProcessSomeJsons(0, {Json1, Json2}, RowDispatcher1); - MockMessageBatch(2, {Json3}, RowDispatcher1); + ProcessSomeMessages(0, {Message1, Message2}, RowDispatcher1); + MockMessageBatch(2, {Message3}, RowDispatcher1); // change active Coordinator MockCoordinatorChanged(Coordinator2Id); ExpectStopSession(RowDispatcher1); - auto result = SourceReadDataUntil(UVParser, 1); - AssertDataWithWatermarks(result, {Json3}, {}); + auto result = SourceReadDataUntil>(UVPairParser, 1); + AssertDataWithWatermarks(result, {Message3}, {}); auto req = ExpectCoordinatorRequest(Coordinator2Id); MockCoordinatorResult(RowDispatcher2, req->Cookie); @@ -354,7 +379,7 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { ExpectStartSession(3, RowDispatcher2, 2); MockAck(RowDispatcher2, 2); - ProcessSomeJsons(3, {Json4}, RowDispatcher2, UVParser, 2); + ProcessSomeMessages(3, {Message4}, RowDispatcher2, UVPairParser, 2); MockHeartbeat(RowDispatcher1, 1); // old generation ExpectStopSession(RowDispatcher1); @@ -363,30 +388,30 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { Y_UNIT_TEST_F(Backpressure, TFixture) { StartSession(Source1, 2_KB); - TString json(900, 'c'); - ProcessSomeJsons(0, {json}, RowDispatcher1); + std::pair message = {100500, TString(900, 'c')}; + ProcessSomeMessages(0, {message}, RowDispatcher1); MockNewDataArrived(RowDispatcher1); ExpectGetNextBatch(RowDispatcher1); - MockMessageBatch(0, {json, json, json}, RowDispatcher1); + MockMessageBatch(0, {message, message, message}, RowDispatcher1); MockNewDataArrived(RowDispatcher1); ASSERT_THROW( CaSetup->Runtime->GrabEdgeEvent(RowDispatcher1, TDuration::Seconds(0)), NActors::TEmptyEventQueueException); - auto result = SourceReadDataUntil(UVParser, 3); - AssertDataWithWatermarks(result, {json, json, json}, {}); + auto result = SourceReadDataUntil>(UVPairParser, 3); + AssertDataWithWatermarks(result, {message, message, message}, {}); ExpectGetNextBatch(RowDispatcher1); - MockMessageBatch(3, {Json1}, RowDispatcher1); - result = SourceReadDataUntil(UVParser, 1); - AssertDataWithWatermarks(result, {Json1}, {}); + MockMessageBatch(3, {Message1}, RowDispatcher1); + result = SourceReadDataUntil>(UVPairParser, 1); + AssertDataWithWatermarks(result, {Message1}, {}); } Y_UNIT_TEST_F(RowDispatcherIsRestarted, TFixture) { StartSession(Source1); - ProcessSomeJsons(0, {Json1, Json2}, RowDispatcher1); + ProcessSomeMessages(0, {Message1, Message2}, RowDispatcher1); MockDisconnected(); MockConnected(); MockUndelivered(); @@ -396,7 +421,7 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { ExpectStartSession(2, RowDispatcher1, 2); MockAck(RowDispatcher1, 2); - ProcessSomeJsons(2, {Json3}, RowDispatcher1, UVParser, 2); + ProcessSomeMessages(2, {Message3}, RowDispatcher1, UVPairParser, 2); } Y_UNIT_TEST_F(IgnoreMessageIfNoSessions, TFixture) { @@ -406,15 +431,22 @@ Y_UNIT_TEST_SUITE(TDqPqRdReadActorTests) { } Y_UNIT_TEST_F(MetadataFields, TFixture) { + auto metadataUVParser = [](const NUdf::TUnboxedValue& item) -> std::vector> { + UNIT_ASSERT_VALUES_EQUAL(item.GetListLength(), 3); + auto stringElement = item.GetElement(2); + return { {item.GetElement(1).Get(), TString(stringElement.AsStringRef())} }; + }; + auto source = BuildPqTopicSourceSettings("topicName"); source.AddMetadataFields("_yql_sys_create_time"); + source.SetRowType("[StructType; [[_yql_sys_create_time; [DataType; Uint32]]; [dt; [DataType; Uint64]]; [value; [DataType; String]]]]"); StartSession(source); - ProcessSomeJsons(0, {Json1}, RowDispatcher1, UVParserWithMetadatafields); + ProcessSomeMessages(0, {Message1}, RowDispatcher1, metadataUVParser); } Y_UNIT_TEST_F(IgnoreCoordinatorResultIfWrongState, TFixture) { StartSession(Source1); - ProcessSomeJsons(0, {Json1, Json2}, RowDispatcher1); + ProcessSomeMessages(0, {Message1, Message2}, RowDispatcher1); MockCoordinatorChanged(Coordinator2Id); auto req = ExpectCoordinatorRequest(Coordinator2Id); diff --git a/ydb/tests/fq/pq_async_io/ut_helpers.cpp b/ydb/tests/fq/pq_async_io/ut_helpers.cpp index a9d716d730c1..df15829e670d 100644 --- a/ydb/tests/fq/pq_async_io/ut_helpers.cpp +++ b/ydb/tests/fq/pq_async_io/ut_helpers.cpp @@ -3,6 +3,7 @@ #include #include +#include #include #include @@ -12,6 +13,16 @@ namespace NYql::NDq { +namespace { + +void SegmentationFaultHandler(int) { + Cerr << "segmentation fault call stack:" << Endl; + FormatBackTrace(&Cerr); + abort(); +} + +} + using namespace NActors; NYql::NPq::NProto::TDqPqTopicSource BuildPqTopicSourceSettings( @@ -26,6 +37,11 @@ NYql::NPq::NProto::TDqPqTopicSource BuildPqTopicSourceSettings( settings.SetEndpoint(GetDefaultPqEndpoint()); settings.MutableToken()->SetName("token"); settings.SetDatabase(GetDefaultPqDatabase()); + settings.SetRowType("[StructType; [[dt; [DataType; Uint64]]; [value; [DataType; String]]]]"); + settings.AddColumns("dt"); + settings.AddColumns("value"); + settings.AddColumnTypes("[DataType; Uint64]"); + settings.AddColumnTypes("[DataType; String]"); if (watermarksPeriod) { settings.MutableWatermarks()->SetEnabled(true); settings.MutableWatermarks()->SetGranularityUs(watermarksPeriod->MicroSeconds()); @@ -48,6 +64,8 @@ NYql::NPq::NProto::TDqPqTopicSink BuildPqTopicSinkSettings(TString topic) { } TPqIoTestFixture::TPqIoTestFixture() { + NKikimr::EnableYDBBacktraceFormat(); + signal(SIGSEGV, &SegmentationFaultHandler); } TPqIoTestFixture::~TPqIoTestFixture() { @@ -239,14 +257,14 @@ void AddReadRule(NYdb::TDriver& driver, const TString& streamName) { UNIT_ASSERT_VALUES_EQUAL(result.IsTransportError(), false); } -std::vector UVParser(const NUdf::TUnboxedValue& item) { - return { TString(item.AsStringRef()) }; +std::vector> UVPairParser(const NUdf::TUnboxedValue& item) { + UNIT_ASSERT_VALUES_EQUAL(item.GetListLength(), 2); + auto stringElement = item.GetElement(1); + return { {item.GetElement(0).Get(), TString(stringElement.AsStringRef())} }; } -std::vector UVParserWithMetadatafields(const NUdf::TUnboxedValue& item) { - const auto& cell = item.GetElement(0); - TString str(cell.AsStringRef()); - return {str}; +std::vector UVParser(const NUdf::TUnboxedValue& item) { + return { TString(item.AsStringRef()) }; } void TPqIoTestFixture::AsyncOutputWrite(std::vector data, TMaybe checkpoint) { diff --git a/ydb/tests/fq/pq_async_io/ut_helpers.h b/ydb/tests/fq/pq_async_io/ut_helpers.h index 114b678686a5..b685412b5d01 100644 --- a/ydb/tests/fq/pq_async_io/ut_helpers.h +++ b/ydb/tests/fq/pq_async_io/ut_helpers.h @@ -124,7 +124,7 @@ void AddReadRule( NYdb::TDriver& driver, const TString& streamName); +std::vector> UVPairParser(const NUdf::TUnboxedValue& item); std::vector UVParser(const NUdf::TUnboxedValue& item); -std::vector UVParserWithMetadatafields(const NUdf::TUnboxedValue& item); } diff --git a/ydb/tests/fq/yds/test_row_dispatcher.py b/ydb/tests/fq/yds/test_row_dispatcher.py index 1351665c122d..7e8ea3909f29 100644 --- a/ydb/tests/fq/yds/test_row_dispatcher.py +++ b/ydb/tests/fq/yds/test_row_dispatcher.py @@ -216,7 +216,7 @@ def test_scheme_error(self, kikimr, client): client.wait_query_status(query_id, fq.QueryMeta.FAILED) issues = str(client.describe_query(query_id).result.query.issue) - assert "Cannot parse JSON string" in issues, "Incorrect Issues: " + issues + assert "Failed to parse json message for offset" in issues, "Incorrect Issues: " + issues wait_actor_count(kikimr, "DQ_PQ_READ_ACTOR", 0) wait_actor_count(kikimr, "FQ_ROW_DISPATCHER_SESSION", 0)