diff --git a/ydb/core/kqp/common/buffer/buffer.h b/ydb/core/kqp/common/buffer/buffer.h new file mode 100644 index 000000000000..226273440269 --- /dev/null +++ b/ydb/core/kqp/common/buffer/buffer.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + +namespace NKikimr { +namespace NKqp { + +struct TKqpBufferWriterSettings { + TActorId SessionActorId; + IKqpTransactionManagerPtr TxManager; + NWilson::TTraceId TraceId; + TIntrusivePtr Counters; + TIntrusivePtr TxProxyMon; +}; + +NActors::IActor* CreateKqpBufferWriterActor(TKqpBufferWriterSettings&& settings); + +} +} diff --git a/ydb/core/kqp/common/buffer/events.cpp b/ydb/core/kqp/common/buffer/events.cpp new file mode 100644 index 000000000000..9b10573c4c94 --- /dev/null +++ b/ydb/core/kqp/common/buffer/events.cpp @@ -0,0 +1,16 @@ +#include "events.h" + +namespace NKikimr { +namespace NKqp { + +TEvKqpBuffer::TEvError::TEvError( + const TString& message, + NYql::NDqProto::StatusIds::StatusCode statusCode, + const NYql::TIssues& subIssues) + : Message(message) + , StatusCode(statusCode) + , SubIssues(subIssues) { +} + +} +} diff --git a/ydb/core/kqp/common/buffer/events.h b/ydb/core/kqp/common/buffer/events.h new file mode 100644 index 000000000000..af790a338239 --- /dev/null +++ b/ydb/core/kqp/common/buffer/events.h @@ -0,0 +1,52 @@ +#pragma once + +#include +#include +#include +#include + + +namespace NKikimr { +namespace NKqp { + +struct TEvKqpBuffer { + +struct TEvPrepare : public TEventLocal { + TActorId ExecuterActorId; +}; + +struct TEvCommit : public TEventLocal { + TActorId ExecuterActorId; + ui64 TxId; +}; + +struct TEvRollback : public TEventLocal { + TActorId ExecuterActorId; +}; + +struct TEvFlush : public TEventLocal { + TActorId ExecuterActorId; +}; + +struct TEvResult : public TEventLocal { + TEvResult() = default; + TEvResult(NYql::NDqProto::TDqTaskStats&& stats) : Stats(std::move(stats)) {} + + std::optional Stats; +}; + +struct TEvError : public TEventLocal { + TString Message; + NYql::NDqProto::StatusIds::StatusCode StatusCode; + NYql::TIssues SubIssues; + + TEvError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues); +}; + +struct TEvTerminate : public TEventLocal { +}; + +}; + +} +} diff --git a/ydb/core/kqp/common/buffer/ya.make b/ydb/core/kqp/common/buffer/ya.make new file mode 100644 index 000000000000..c4fb712d297f --- /dev/null +++ b/ydb/core/kqp/common/buffer/ya.make @@ -0,0 +1,14 @@ +LIBRARY() + +SRCS( + events.cpp +) + +PEERDIR( + ydb/core/kqp/common/simple + ydb/library/yql/public/issue +) + +YQL_LAST_ABI_VERSION() + +END() diff --git a/ydb/core/kqp/common/kqp_tx.cpp b/ydb/core/kqp/common/kqp_tx.cpp index 7dd9c26ef87c..4d9f7ab90574 100644 --- a/ydb/core/kqp/common/kqp_tx.cpp +++ b/ydb/core/kqp/common/kqp_tx.cpp @@ -171,7 +171,6 @@ bool NeedSnapshot(const TKqpTransactionContext& txCtx, const NYql::TKikimrConfig size_t readPhases = 0; bool hasEffects = false; - bool hasSourceRead = false; bool hasStreamLookup = false; bool hasSinkWrite = false; @@ -191,7 +190,6 @@ bool NeedSnapshot(const TKqpTransactionContext& txCtx, const NYql::TKikimrConfig } for (const auto &stage : tx.GetStages()) { - hasSourceRead |= !stage.GetSources().empty(); hasSinkWrite |= !stage.GetSinks().empty(); for (const auto &input : stage.GetInputs()) { @@ -211,9 +209,7 @@ bool NeedSnapshot(const TKqpTransactionContext& txCtx, const NYql::TKikimrConfig return true; } - if ((hasSourceRead || hasStreamLookup) && hasSinkWrite) { - return true; - } + YQL_ENSURE(!hasSinkWrite || hasEffects); // We don't want snapshot when there are effects at the moment, // because it hurts performance when there are multiple single-shard @@ -251,23 +247,12 @@ bool HasOlapTableReadInTx(const NKqpProto::TKqpPhyQuery& physicalQuery) { return false; } -bool HasOlapTableWriteInStage(const NKqpProto::TKqpPhyStage& stage, const google::protobuf::RepeatedPtrField< ::NKqpProto::TKqpPhyTable>& tables) { +bool HasOlapTableWriteInStage(const NKqpProto::TKqpPhyStage& stage) { for (const auto& sink : stage.GetSinks()) { if (sink.GetTypeCase() == NKqpProto::TKqpSink::kInternalSink && sink.GetInternalSink().GetSettings().Is()) { NKikimrKqp::TKqpTableSinkSettings settings; YQL_ENSURE(sink.GetInternalSink().GetSettings().UnpackTo(&settings), "Failed to unpack settings"); - - const bool isOlapSink = std::any_of( - std::begin(tables), - std::end(tables), - [&](const NKqpProto::TKqpPhyTable& table) { - return table.GetKind() == NKqpProto::EKqpPhyTableKind::TABLE_KIND_OLAP - && google::protobuf::util::MessageDifferencer::Equals(table.GetId(), settings.GetTable()); - }); - - if (isOlapSink) { - return true; - } + return settings.GetIsOlap(); } } return false; @@ -276,7 +261,7 @@ bool HasOlapTableWriteInStage(const NKqpProto::TKqpPhyStage& stage, const google bool HasOlapTableWriteInTx(const NKqpProto::TKqpPhyQuery& physicalQuery) { for (const auto &tx : physicalQuery.GetTransactions()) { for (const auto &stage : tx.GetStages()) { - if (HasOlapTableWriteInStage(stage, tx.GetTables())) { + if (HasOlapTableWriteInStage(stage)) { return true; } } @@ -325,18 +310,7 @@ bool HasOltpTableWriteInTx(const NKqpProto::TKqpPhyQuery& physicalQuery) { if (sink.GetTypeCase() == NKqpProto::TKqpSink::kInternalSink && sink.GetInternalSink().GetSettings().Is()) { NKikimrKqp::TKqpTableSinkSettings settings; YQL_ENSURE(sink.GetInternalSink().GetSettings().UnpackTo(&settings), "Failed to unpack settings"); - - const bool isOltpSink = std::any_of( - std::begin(tx.GetTables()), - std::end(tx.GetTables()), - [&](const NKqpProto::TKqpPhyTable& table) { - return table.GetKind() == NKqpProto::EKqpPhyTableKind::TABLE_KIND_DS - && google::protobuf::util::MessageDifferencer::Equals(table.GetId(), settings.GetTable()); - }); - - if (isOltpSink) { - return true; - } + return !settings.GetIsOlap(); } } } diff --git a/ydb/core/kqp/common/kqp_tx.h b/ydb/core/kqp/common/kqp_tx.h index 700e9075236a..8d324e0ae0ec 100644 --- a/ydb/core/kqp/common/kqp_tx.h +++ b/ydb/core/kqp/common/kqp_tx.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -121,12 +122,6 @@ struct TDeferredEffects { friend class TKqpTransactionContext; }; -struct TTableInfo { - bool IsOlap = false; - THashSet Pathes; -}; - - class TShardIdToTableInfo { public: const TTableInfo& Get(ui64 shardId) const { @@ -204,6 +199,8 @@ class TKqpTransactionContext : public NYql::TKikimrTransactionContextBase { void Finish() final { YQL_ENSURE(DeferredEffects.Empty()); YQL_ENSURE(!Locks.HasLocks()); + YQL_ENSURE(!TxManager); + YQL_ENSURE(!BufferActorId); FinishTime = TInstant::Now(); @@ -350,6 +347,9 @@ class TKqpTransactionContext : public NYql::TKikimrTransactionContextBase { bool NeedUncommittedChangesFlush = false; THashSet ModifiedTablesSinceLastFlush; + TActorId BufferActorId; + IKqpTransactionManagerPtr TxManager = nullptr; + TShardIdToTableInfoPtr ShardIdToTableInfo = std::make_shared(); }; @@ -507,9 +507,7 @@ bool NeedSnapshot(const TKqpTransactionContext& txCtx, const NYql::TKikimrConfig bool commitTx, const NKqpProto::TKqpPhyQuery& physicalQuery); bool HasOlapTableReadInTx(const NKqpProto::TKqpPhyQuery& physicalQuery); -bool HasOlapTableWriteInStage( - const NKqpProto::TKqpPhyStage& stage, - const google::protobuf::RepeatedPtrField< ::NKqpProto::TKqpPhyTable>& tables); +bool HasOlapTableWriteInStage(const NKqpProto::TKqpPhyStage& stage); bool HasOlapTableWriteInTx(const NKqpProto::TKqpPhyQuery& physicalQuery); bool HasOltpTableReadInTx(const NKqpProto::TKqpPhyQuery& physicalQuery); bool HasOltpTableWriteInTx(const NKqpProto::TKqpPhyQuery& physicalQuery); diff --git a/ydb/core/kqp/common/kqp_tx_manager.cpp b/ydb/core/kqp/common/kqp_tx_manager.cpp new file mode 100644 index 000000000000..82e280ec25dc --- /dev/null +++ b/ydb/core/kqp/common/kqp_tx_manager.cpp @@ -0,0 +1,431 @@ +#include "kqp_tx_manager.h" + +#include +#include + +namespace NKikimr { +namespace NKqp { + +namespace { + +struct TKqpLock { + using TKey = std::tuple; + TKey GetKey() const { return std::make_tuple(Proto.GetLockId(), Proto.GetDataShard(), Proto.GetSchemeShard(), Proto.GetPathId()); } + + bool Invalidated(const TKqpLock& newLock) const { + AFL_ENSURE(GetKey() == newLock.GetKey()); + return Proto.GetGeneration() != newLock.Proto.GetGeneration() || Proto.GetCounter() != newLock.Proto.GetCounter(); + } + + TKqpLock(const NKikimrDataEvents::TLock& proto) + : Proto(proto) {} + + NKikimrDataEvents::TLock Proto; +}; + +class TKqpTransactionManager : public IKqpTransactionManager { + enum ETransactionState { + COLLECTING, + PREPARING, + EXECUTING, + }; +public: + TKqpTransactionManager(bool collectOnly) + : CollectOnly(collectOnly) {} + + void AddShard(ui64 shardId, bool isOlap, const TString& path) override { + Y_ABORT_UNLESS(State == ETransactionState::COLLECTING); + ShardsIds.insert(shardId); + auto& shardInfo = ShardsInfo[shardId]; + shardInfo.IsOlap = isOlap; + HasOlapTableShard |= isOlap; + + const auto [stringsIter, _] = TablePathes.insert(path); + const TStringBuf pathBuf = *stringsIter; + shardInfo.Pathes.insert(pathBuf); + } + + void AddAction(ui64 shardId, ui8 action) override { + Y_ABORT_UNLESS(State == ETransactionState::COLLECTING); + ShardsInfo.at(shardId).Flags |= action; + if (action & EAction::WRITE) { + ReadOnly = false; + } + } + + bool AddLock(ui64 shardId, const NKikimrDataEvents::TLock& lockProto) override { + Y_ABORT_UNLESS(State == ETransactionState::COLLECTING); + TKqpLock lock(lockProto); + bool isError = (lock.Proto.GetCounter() >= NKikimr::TSysTables::TLocksTable::TLock::ErrorMin); + bool isInvalidated = (lock.Proto.GetCounter() == NKikimr::TSysTables::TLocksTable::TLock::ErrorAlreadyBroken) + || (lock.Proto.GetCounter() == NKikimr::TSysTables::TLocksTable::TLock::ErrorBroken); + bool isLocksAcquireFailure = isError && !isInvalidated; + bool broken = false; + + auto& shardInfo = ShardsInfo.at(shardId); + if (auto lockPtr = shardInfo.Locks.FindPtr(lock.GetKey()); lockPtr) { + if (lock.Proto.GetHasWrites()) { + lockPtr->Lock.Proto.SetHasWrites(true); + } + + lockPtr->LocksAcquireFailure |= isLocksAcquireFailure; + if (!lockPtr->LocksAcquireFailure) { + isInvalidated |= lockPtr->Lock.Invalidated(lock); + lockPtr->Invalidated |= isInvalidated; + } + broken = lockPtr->Invalidated || lockPtr->LocksAcquireFailure; + } else { + shardInfo.Locks.emplace( + lock.GetKey(), + TShardInfo::TLockInfo { + .Lock = std::move(lock), + .Invalidated = isInvalidated, + .LocksAcquireFailure = isLocksAcquireFailure, + }); + broken = isInvalidated || isLocksAcquireFailure; + } + + if (broken && !LocksIssue) { + if (isLocksAcquireFailure) { + LocksIssue = YqlIssue(NYql::TPosition(), NYql::TIssuesIds::KIKIMR_LOCKS_ACQUIRE_FAILURE); + return false; + } else if (isInvalidated) { + MakeLocksIssue(shardInfo); + return false; + } + AFL_ENSURE(false); + } + + return true; + } + + void BreakLock(ui64 shardId) override { + if (LocksIssue) { + return; + } + auto& shardInfo = ShardsInfo.at(shardId); + MakeLocksIssue(shardInfo); + } + + TTableInfo GetShardTableInfo(ui64 shardId) const override { + const auto& info = ShardsInfo.at(shardId); + return TTableInfo{ + .IsOlap = info.IsOlap, + .Pathes = info.Pathes, + }; + } + + EShardState GetState(ui64 shardId) const override { + return ShardsInfo.at(shardId).State; + } + + void SetState(ui64 shardId, EShardState state) override { + ShardsInfo.at(shardId).State = state; + } + + TVector GetLocks() const override { + TVector locks; + for (const auto& [_, shardInfo] : ShardsInfo) { + for (const auto& [_, lockInfo] : shardInfo.Locks) { + locks.push_back(lockInfo.Lock.Proto); + } + } + return locks; + } + + TVector GetLocks(ui64 shardId) const override { + TVector locks; + const auto& shardInfo = ShardsInfo.at(shardId); + for (const auto& [_, lockInfo] : shardInfo.Locks) { + locks.push_back(lockInfo.Lock.Proto); + } + return locks; + } + + bool IsTxPrepared() const override { + for (const auto& [_, shardInfo] : ShardsInfo) { + if (shardInfo.State != EShardState::PREPARED) { + return false; + } + } + return true; + } + + bool IsTxFinished() const override { + for (const auto& [_, shardInfo] : ShardsInfo) { + if (shardInfo.State != EShardState::FINISHED) { + return false; + } + } + return true; + } + + bool IsReadOnly() const override { + return ReadOnly; + } + + bool IsSingleShard() const override { + return GetShardsCount() == 1; + } + + bool HasOlapTable() const override { + return HasOlapTableShard; + } + + bool IsEmpty() const override { + return GetShardsCount() == 0; + } + + bool HasLocks() const override { + for (const auto& [_, shardInfo] : ShardsInfo) { + if (!shardInfo.Locks.empty()) { + return true; + } + } + return false; + } + + bool IsVolatile() const override { + return !HasOlapTable(); + } + + bool HasSnapshot() const override { + return ValidSnapshot; + } + + void SetHasSnapshot(bool hasSnapshot) override { + ValidSnapshot = hasSnapshot; + } + + bool BrokenLocks() const override { + return LocksIssue.has_value() && !(HasSnapshot() && IsReadOnly()); + } + + const std::optional& GetLockIssue() const override { + return LocksIssue; + } + + const THashSet& GetShards() const override { + return ShardsIds; + } + + ui64 GetShardsCount() const override { + return ShardsIds.size(); + } + + void StartPrepare() override { + AFL_ENSURE(!CollectOnly); + AFL_ENSURE(State == ETransactionState::COLLECTING); + AFL_ENSURE(!IsReadOnly()); + + THashSet sendingColumnShardsSet; + THashSet receivingColumnShardsSet; + + for (auto& [shardId, shardInfo] : ShardsInfo) { + if ((shardInfo.Flags & EAction::WRITE)) { + ReceivingShards.insert(shardId); + if (IsVolatile()) { + SendingShards.insert(shardId); + } + if (shardInfo.IsOlap) { + sendingColumnShardsSet.insert(shardId); + } + } + if (!shardInfo.Locks.empty()) { + SendingShards.insert(shardId); + if (shardInfo.IsOlap) { + receivingColumnShardsSet.insert(shardId); + } + } + + AFL_ENSURE(shardInfo.State == EShardState::PROCESSING); + shardInfo.State = EShardState::PREPARING; + } + + Y_ABORT_UNLESS(!ReceivingShards.empty()); + + constexpr size_t minArbiterMeshSize = 5; + if ((IsVolatile() && + ReceivingShards.size() >= minArbiterMeshSize)) + { + std::vector candidates; + candidates.reserve(ReceivingShards.size()); + for (ui64 candidate : ReceivingShards) { + // Note: all receivers are also senders in volatile transactions + if (Y_LIKELY(SendingShards.contains(candidate))) { + candidates.push_back(candidate); + } + } + if (candidates.size() >= minArbiterMeshSize) { + // Select a random arbiter + const ui32 index = RandomNumber(candidates.size()); + Arbiter = candidates.at(index); + } + } + + if (!receivingColumnShardsSet.empty() || !sendingColumnShardsSet.empty()) { + AFL_ENSURE(!IsVolatile()); + const auto& shards = receivingColumnShardsSet.empty() + ? sendingColumnShardsSet + : receivingColumnShardsSet; + + const ui32 index = RandomNumber(shards.size()); + auto arbiterIterator = std::begin(shards); + std::advance(arbiterIterator, index); + ArbiterColumnShard = *arbiterIterator; + } + + ShardsToWaitPrepare = ShardsIds; + + MinStep = std::numeric_limits::min(); + MaxStep = std::numeric_limits::max(); + Coordinator = 0; + + State = ETransactionState::PREPARING; + } + + TPrepareInfo GetPrepareTransactionInfo() override { + AFL_ENSURE(State == ETransactionState::PREPARING); + AFL_ENSURE(!ReceivingShards.empty()); + + TPrepareInfo result { + .SendingShards = SendingShards, + .ReceivingShards = ReceivingShards, + .Arbiter = Arbiter, + .ArbiterColumnShard = ArbiterColumnShard, + }; + + return result; + } + + bool ConsumePrepareTransactionResult(TPrepareResult&& result) override { + AFL_ENSURE(State == ETransactionState::PREPARING); + auto& shardInfo = ShardsInfo.at(result.ShardId); + AFL_ENSURE(shardInfo.State == EShardState::PREPARING); + shardInfo.State = EShardState::PREPARED; + + ShardsToWaitPrepare.erase(result.ShardId); + + MinStep = std::max(MinStep, result.MinStep); + MaxStep = std::min(MaxStep, result.MaxStep); + + if (result.Coordinator && !Coordinator) { + Coordinator = result.Coordinator; + } + + AFL_ENSURE(Coordinator && Coordinator == result.Coordinator)("prev_coordinator", Coordinator)("new_coordinator", result.Coordinator); + + return ShardsToWaitPrepare.empty(); + } + + void StartExecute() override { + AFL_ENSURE(!CollectOnly); + AFL_ENSURE(State == ETransactionState::PREPARING + || (State == ETransactionState::COLLECTING + && IsSingleShard())); + AFL_ENSURE(!IsReadOnly()); + State = ETransactionState::EXECUTING; + + for (auto& [_, shardInfo] : ShardsInfo) { + AFL_ENSURE(shardInfo.State == EShardState::PREPARED + || (shardInfo.State == EShardState::PROCESSING + && IsSingleShard())); + shardInfo.State = EShardState::EXECUTING; + } + + AFL_ENSURE(ReceivingShards.empty() || !IsSingleShard() || HasOlapTable()); + } + + TCommitInfo GetCommitInfo() override { + AFL_ENSURE(State == ETransactionState::EXECUTING); + TCommitInfo result; + result.MinStep = MinStep; + result.MaxStep = MaxStep; + result.Coordinator = Coordinator; + + for (auto& [shardId, shardInfo] : ShardsInfo) { + result.ShardsInfo.push_back(TCommitShardInfo{ + .ShardId = shardId, + .AffectedFlags = shardInfo.Flags, + }); + + AFL_ENSURE(shardInfo.State == EShardState::EXECUTING); + } + return result; + } + + bool ConsumeCommitResult(ui64 shardId) override { + AFL_ENSURE(State == ETransactionState::EXECUTING); + auto& shardInfo = ShardsInfo.at(shardId); + AFL_ENSURE(shardInfo.State == EShardState::EXECUTING); + shardInfo.State = EShardState::FINISHED; + + // Either all shards committed or all shards failed, + // so we need to wait only for one answer from ReceivingShards. + return ReceivingShards.contains(shardId) || IsSingleShard(); + } + +private: + bool CollectOnly = false; + ETransactionState State = ETransactionState::COLLECTING; + + struct TShardInfo { + EShardState State = EShardState::PROCESSING; + TActionFlags Flags = 0; + + struct TLockInfo { + TKqpLock Lock; + bool Invalidated = false; + bool LocksAcquireFailure = false; + }; + + THashMap Locks; + + bool IsOlap = false; + THashSet Pathes; + }; + + void MakeLocksIssue(const TShardInfo& shardInfo) { + TStringBuilder message; + message << "Transaction locks invalidated. Tables: "; + bool first = true; + // TODO: add error by pathid + for (const auto& path : shardInfo.Pathes) { + if (!first) { + message << ", "; + first = false; + } + message << "`" << path << "`"; + } + LocksIssue = YqlIssue(NYql::TPosition(), NYql::TIssuesIds::KIKIMR_LOCKS_INVALIDATED, message); + } + + THashSet ShardsIds; + THashMap ShardsInfo; + std::unordered_set TablePathes; + + bool ReadOnly = true; + bool ValidSnapshot = false; + bool HasOlapTableShard = false; + std::optional LocksIssue; + + THashSet SendingShards; + THashSet ReceivingShards; + std::optional Arbiter; + std::optional ArbiterColumnShard; + + THashSet ShardsToWaitPrepare; + + ui64 MinStep = 0; + ui64 MaxStep = 0; + ui64 Coordinator = 0; +}; + +} + +IKqpTransactionManagerPtr CreateKqpTransactionManager(bool collectOnly) { + return std::make_shared(collectOnly); +} + +} +} diff --git a/ydb/core/kqp/common/kqp_tx_manager.h b/ydb/core/kqp/common/kqp_tx_manager.h new file mode 100644 index 000000000000..03dd73ad034b --- /dev/null +++ b/ydb/core/kqp/common/kqp_tx_manager.h @@ -0,0 +1,118 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace NKikimr { +namespace NKqp { + +struct TTableInfo { + bool IsOlap = false; + THashSet Pathes; +}; + +class IKqpTransactionManager { +public: + virtual ~IKqpTransactionManager() = default; + + enum EShardState { + PROCESSING, + PREPARING, + PREPARED, + EXECUTING, + FINISHED + }; + + enum EAction { + READ = 1, + WRITE = 2, + }; + + using TActionFlags = ui8; + + virtual void AddShard(ui64 shardId, bool isOlap, const TString& path) = 0; + virtual void AddAction(ui64 shardId, ui8 action) = 0; + virtual bool AddLock(ui64 shardId, const NKikimrDataEvents::TLock& lock) = 0; + + virtual void BreakLock(ui64 shardId) = 0; + + virtual TTableInfo GetShardTableInfo(ui64 shardId) const = 0; + + virtual TVector GetLocks() const = 0; + virtual TVector GetLocks(ui64 shardId) const = 0; + + virtual EShardState GetState(ui64 shardId) const = 0; + virtual void SetState(ui64 shardId, EShardState state) = 0; + + virtual bool IsTxPrepared() const = 0; + virtual bool IsTxFinished() const = 0; + + virtual bool IsReadOnly() const = 0; + virtual bool IsSingleShard() const = 0; + virtual bool HasOlapTable() const = 0; + + virtual bool IsEmpty() const = 0; + virtual bool HasLocks() const = 0; + + virtual bool IsVolatile() const = 0; + + virtual bool HasSnapshot() const = 0; + virtual void SetHasSnapshot(bool hasSnapshot) = 0; + + virtual bool BrokenLocks() const = 0; + virtual const std::optional& GetLockIssue() const = 0; + + virtual const THashSet& GetShards() const = 0; + virtual ui64 GetShardsCount() const = 0; + + virtual void StartPrepare() = 0; + + struct TPrepareInfo { + const THashSet& SendingShards; + const THashSet& ReceivingShards; + std::optional Arbiter; + std::optional ArbiterColumnShard; + }; + + virtual TPrepareInfo GetPrepareTransactionInfo() = 0; + + struct TPrepareResult { + ui64 ShardId; + ui64 MinStep; + ui64 MaxStep; + ui64 Coordinator; + }; + + virtual bool ConsumePrepareTransactionResult(TPrepareResult&& result) = 0; + + virtual void StartExecute() = 0; + + struct TCommitShardInfo { + ui64 ShardId; + ui32 AffectedFlags; + }; + + struct TCommitInfo { + ui64 MinStep; + ui64 MaxStep; + ui64 Coordinator; + + TVector ShardsInfo; + }; + + virtual TCommitInfo GetCommitInfo() = 0; + + virtual bool ConsumeCommitResult(ui64 shardId) = 0; +}; + +using IKqpTransactionManagerPtr = std::shared_ptr; + +IKqpTransactionManagerPtr CreateKqpTransactionManager(bool collectOnly = false); + +} +} diff --git a/ydb/core/kqp/common/simple/kqp_event_ids.h b/ydb/core/kqp/common/simple/kqp_event_ids.h index 571944731486..321bf4f5404f 100644 --- a/ydb/core/kqp/common/simple/kqp_event_ids.h +++ b/ydb/core/kqp/common/simple/kqp_event_ids.h @@ -46,7 +46,9 @@ struct TKqpEvents { EvListProxyNodesRequest, EvListProxyNodesResponse, EvUpdateDatabaseInfo, - EvDelayedRequestError + EvDelayedRequestError, + EvBufferWrite, + EvBufferWriteResult, }; static_assert (EvCompileInvalidateRequest + 1 == EvAbortExecution); @@ -182,5 +184,17 @@ struct TKqpWorkloadServiceEvents { }; }; +struct TKqpBufferWriterEvents { + enum EKqpBufferWriterEvents { + EvPrepare = EventSpaceBegin(TKikimrEvents::ES_KQP) + 800, + EvCommit, + EvRollback, + EvFlush, + EvResult, + EvError, + EvTerminate, + }; +}; + } // namespace NKqp } // namespace NKikimr diff --git a/ydb/core/kqp/common/ya.make b/ydb/core/kqp/common/ya.make index 0559e96e994c..0a8050d7f7e4 100644 --- a/ydb/core/kqp/common/ya.make +++ b/ydb/core/kqp/common/ya.make @@ -11,6 +11,7 @@ SRCS( kqp_script_executions.cpp kqp_timeouts.cpp kqp_timeouts.h + kqp_tx_manager.cpp kqp_tx.cpp kqp_types.cpp kqp_types.h diff --git a/ydb/core/kqp/compile_service/kqp_compile_actor.cpp b/ydb/core/kqp/compile_service/kqp_compile_actor.cpp index 00fbecc2726b..21943cfc6145 100644 --- a/ydb/core/kqp/compile_service/kqp_compile_actor.cpp +++ b/ydb/core/kqp/compile_service/kqp_compile_actor.cpp @@ -600,6 +600,7 @@ void ApplyServiceConfig(TKikimrConfiguration& kqpConfig, const TTableServiceConf kqpConfig.ExtractPredicateRangesLimit = serviceConfig.GetExtractPredicateRangesLimit(); kqpConfig.EnablePerStatementQueryExecution = serviceConfig.GetEnablePerStatementQueryExecution(); kqpConfig.EnableCreateTableAs = serviceConfig.GetEnableCreateTableAs(); + kqpConfig.AllowOlapDataQuery = serviceConfig.GetAllowOlapDataQuery(); kqpConfig.EnableOlapSink = serviceConfig.GetEnableOlapSink(); kqpConfig.EnableOltpSink = serviceConfig.GetEnableOltpSink(); kqpConfig.EnableHtapTx = serviceConfig.GetEnableHtapTx(); diff --git a/ydb/core/kqp/compile_service/kqp_compile_service.cpp b/ydb/core/kqp/compile_service/kqp_compile_service.cpp index bde37f8a8c43..6076563d453a 100644 --- a/ydb/core/kqp/compile_service/kqp_compile_service.cpp +++ b/ydb/core/kqp/compile_service/kqp_compile_service.cpp @@ -529,6 +529,7 @@ class TKqpCompileService : public TActorBootstrapped { bool enableSequences = TableServiceConfig.GetEnableSequences(); bool enableColumnsWithDefault = TableServiceConfig.GetEnableColumnsWithDefault(); + bool allowOlapDataQuery = TableServiceConfig.GetAllowOlapDataQuery(); bool enableOlapSink = TableServiceConfig.GetEnableOlapSink(); bool enableOltpSink = TableServiceConfig.GetEnableOltpSink(); bool enableHtapTx = TableServiceConfig.GetEnableHtapTx(); @@ -561,6 +562,7 @@ class TKqpCompileService : public TActorBootstrapped { TableServiceConfig.GetIndexAutoChooseMode() != indexAutoChooser || TableServiceConfig.GetEnableSequences() != enableSequences || TableServiceConfig.GetEnableColumnsWithDefault() != enableColumnsWithDefault || + TableServiceConfig.GetAllowOlapDataQuery() != allowOlapDataQuery || TableServiceConfig.GetEnableOlapSink() != enableOlapSink || TableServiceConfig.GetEnableOltpSink() != enableOltpSink || TableServiceConfig.GetEnableHtapTx() != enableHtapTx || diff --git a/ydb/core/kqp/counters/kqp_counters.cpp b/ydb/core/kqp/counters/kqp_counters.cpp index a9918ca78822..1bedaa6c4743 100644 --- a/ydb/core/kqp/counters/kqp_counters.cpp +++ b/ydb/core/kqp/counters/kqp_counters.cpp @@ -817,14 +817,29 @@ TKqpCounters::TKqpCounters(const ::NMonitoring::TDynamicCounterPtr& counters, co /* sink writes */ WriteActorsShardResolve = KqpGroup->GetCounter("SinkWrites/WriteActorShardResolve", true); WriteActorsCount = KqpGroup->GetCounter("SinkWrites/WriteActorsCount", false); + BufferActorsCount = KqpGroup->GetCounter("SinkWrites/BufferActorsCount", false); + ForwardActorsCount = KqpGroup->GetCounter("SinkWrites/ForwardActorsCount", false); + WriteActorImmediateWrites = KqpGroup->GetCounter("SinkWrites/WriteActorImmediateWrites", true); WriteActorImmediateWritesRetries = KqpGroup->GetCounter("SinkWrites/WriteActorImmediateWritesRetries", true); + WriteActorPrepareWrites = KqpGroup->GetCounter("SinkWrites/WriteActorPrepareWrites", true); + + BufferActorFlushes = KqpGroup->GetCounter("SinkWrites/BufferActorFlushes", true); + BufferActorImmediateCommits = KqpGroup->GetCounter("SinkWrites/BufferActorImmediateCommits", true); + BufferActorDistributedCommits = KqpGroup->GetCounter("SinkWrites/BufferActorDistributedCommits", true); + BufferActorRollbacks = KqpGroup->GetCounter("SinkWrites/BufferActorRollbacks", true); + WriteActorWritesSizeHistogram = KqpGroup->GetHistogram("SinkWrites/WriteActorWritesSize", NMonitoring::ExponentialHistogram(28, 2, 1)); WriteActorWritesOperationsHistogram = KqpGroup->GetHistogram("SinkWrites/WriteActorWritesOperations", NMonitoring::ExponentialHistogram(20, 2, 1)); WriteActorWritesLatencyHistogram = KqpGroup->GetHistogram("SinkWrites/WriteActorWritesLatencyMs", NMonitoring::ExponentialHistogram(20, 2, 1)); + + ForwardActorWritesSizeHistogram = + KqpGroup->GetHistogram("SinkWrites/ForwardActorWritesSize", NMonitoring::ExponentialHistogram(28, 2, 1)); + ForwardActorWritesLatencyHistogram = + KqpGroup->GetHistogram("SinkWrites/ForwardActorWritesLatencyMs", NMonitoring::ExponentialHistogram(20, 2, 1)); /* sequencers */ diff --git a/ydb/core/kqp/counters/kqp_counters.h b/ydb/core/kqp/counters/kqp_counters.h index 356e45b7bdcd..f5d3aa5f8751 100644 --- a/ydb/core/kqp/counters/kqp_counters.h +++ b/ydb/core/kqp/counters/kqp_counters.h @@ -412,12 +412,25 @@ class TKqpCounters : public TKqpCountersBase, public NYql::NDq::TSpillingCounter // Sink write counters ::NMonitoring::TDynamicCounters::TCounterPtr WriteActorsShardResolve; ::NMonitoring::TDynamicCounters::TCounterPtr WriteActorsCount; + ::NMonitoring::TDynamicCounters::TCounterPtr BufferActorsCount; + ::NMonitoring::TDynamicCounters::TCounterPtr ForwardActorsCount; + ::NMonitoring::TDynamicCounters::TCounterPtr WriteActorImmediateWrites; ::NMonitoring::TDynamicCounters::TCounterPtr WriteActorImmediateWritesRetries; + ::NMonitoring::TDynamicCounters::TCounterPtr WriteActorPrepareWrites; + + ::NMonitoring::TDynamicCounters::TCounterPtr BufferActorFlushes; + ::NMonitoring::TDynamicCounters::TCounterPtr BufferActorImmediateCommits; + ::NMonitoring::TDynamicCounters::TCounterPtr BufferActorDistributedCommits; + ::NMonitoring::TDynamicCounters::TCounterPtr BufferActorRollbacks; + NMonitoring::THistogramPtr WriteActorWritesSizeHistogram; NMonitoring::THistogramPtr WriteActorWritesOperationsHistogram; NMonitoring::THistogramPtr WriteActorWritesLatencyHistogram; + NMonitoring::THistogramPtr ForwardActorWritesSizeHistogram; + NMonitoring::THistogramPtr ForwardActorWritesLatencyHistogram; + // Scheduler signals ::NMonitoring::TDynamicCounters::TCounterPtr SchedulerThrottled; ::NMonitoring::TDynamicCounters::TCounterPtr SchedulerCapacity; diff --git a/ydb/core/kqp/executer_actor/kqp_data_executer.cpp b/ydb/core/kqp/executer_actor/kqp_data_executer.cpp index d2b81f080c7a..eef7f68e52d4 100644 --- a/ydb/core/kqp/executer_actor/kqp_data_executer.cpp +++ b/ydb/core/kqp/executer_actor/kqp_data_executer.cpp @@ -10,9 +10,10 @@ #include #include #include -#include -#include +#include #include +#include +#include #include #include #include @@ -128,17 +129,22 @@ class TKqpDataExecuter: public TKqpExecuterBase& userRequestContext, ui32 statementResultIndex, const std::optional& federatedQuerySetup, - const TGUCSettings::TPtr& GUCSettings, const TShardIdToTableInfoPtr& shardIdToTableInfo) + const TGUCSettings::TPtr& GUCSettings, + const TShardIdToTableInfoPtr& shardIdToTableInfo, + const IKqpTransactionManagerPtr& txManager, + const TActorId bufferActorId) : TBase(std::move(request), database, userToken, counters, tableServiceConfig, - userRequestContext, statementResultIndex, TWilsonKqp::DataExecuter, "DataExecuter", streamResult) + userRequestContext, statementResultIndex, TWilsonKqp::DataExecuter, + "DataExecuter", streamResult, bufferActorId, txManager) , AsyncIoFactory(std::move(asyncIoFactory)) - , UseEvWriteForOltp(tableServiceConfig.GetEnableOltpSink()) , FederatedQuerySetup(federatedQuerySetup) , GUCSettings(GUCSettings) , ShardIdToTableInfo(shardIdToTableInfo) + , AllowOlapDataQuery(tableServiceConfig.GetAllowOlapDataQuery()) { Target = creator; + YQL_ENSURE(!TxManager || tableServiceConfig.GetEnableOltpSink()); YQL_ENSURE(Request.IsolationLevel != NKikimrKqp::ISOLATION_LEVEL_UNDEFINED); if (Request.AcquireLocksTxId || Request.LocksOp == ELocksOp::Commit || Request.LocksOp == ELocksOp::Rollback) { @@ -151,6 +157,7 @@ class TKqpDataExecuter: public TKqpExecuterBaseEnableMvccSnapshotWithLegacyDomainRoot)); + const bool forceSnapshot = ( + !GetSnapshot().IsValid() && + ReadOnlyTx && + !ImmediateTx && + !HasPersistentChannels && + !HasOlapTable && + (!Database.empty() || AppData()->EnableMvccSnapshotWithLegacyDomainRoot) + ); return forceSnapshot; } @@ -201,36 +213,53 @@ class TKqpDataExecuter: public TKqpExecuterBaseBrokenLockShardId); return ReplyErrorAndDie(Ydb::StatusIds::ABORTED, {}); } - ResponseEv->Record.MutableResponse()->SetStatus(Ydb::StatusIds::SUCCESS); - Counters->TxProxyMon->ReportStatusOK->Inc(); - auto addLocks = [this](const ui64 taskId, const auto& data) { if (data.GetData().template Is()) { NKikimrTxDataShard::TEvKqpInputActorResultInfo info; YQL_ENSURE(data.GetData().UnpackTo(&info), "Failed to unpack settings"); for (auto& lock : info.GetLocks()) { - Locks.push_back(lock); + if (!TxManager) { + Locks.push_back(lock); + } const auto& task = TasksGraph.GetTask(taskId); const auto& stageInfo = TasksGraph.GetStageInfo(task.StageId); ShardIdToTableInfo->Add(lock.GetDataShard(), stageInfo.Meta.TableKind == ETableKind::Olap, stageInfo.Meta.TablePath); + + if (TxManager) { + TxManager->AddShard(lock.GetDataShard(), stageInfo.Meta.TableKind == ETableKind::Olap, stageInfo.Meta.TablePath); + TxManager->AddAction(lock.GetDataShard(), IKqpTransactionManager::EAction::READ); + TxManager->AddLock(lock.GetDataShard(), lock); + } } } else if (data.GetData().template Is()) { NKikimrKqp::TEvKqpOutputActorResultInfo info; YQL_ENSURE(data.GetData().UnpackTo(&info), "Failed to unpack settings"); for (auto& lock : info.GetLocks()) { - Locks.push_back(lock); + if (!TxManager) { + Locks.push_back(lock); + } const auto& task = TasksGraph.GetTask(taskId); const auto& stageInfo = TasksGraph.GetStageInfo(task.StageId); ShardIdToTableInfo->Add(lock.GetDataShard(), stageInfo.Meta.TableKind == ETableKind::Olap, stageInfo.Meta.TablePath); + if (TxManager) { + YQL_ENSURE(stageInfo.Meta.TableKind == ETableKind::Olap); + IKqpTransactionManager::TActionFlags flags = IKqpTransactionManager::EAction::WRITE; + if (info.GetHasRead()) { + flags |= IKqpTransactionManager::EAction::READ; + } + + TxManager->AddShard(lock.GetDataShard(), stageInfo.Meta.TableKind == ETableKind::Olap, stageInfo.Meta.TablePath); + TxManager->AddAction(lock.GetDataShard(), flags); + TxManager->AddLock(lock.GetDataShard(), lock); + } } } }; @@ -250,13 +279,78 @@ class TKqpDataExecuter: public TKqpExecuterBaseSetHasSnapshot(GetSnapshot().IsValid()); + } + + if (!BufferActorId || (ReadOnlyTx && Request.LocksOp != ELocksOp::Rollback)) { + Become(&TKqpDataExecuter::FinalizeState); + MakeResponseAndPassAway(); + return; + } else if (Request.LocksOp == ELocksOp::Commit && !ReadOnlyTx) { + Become(&TKqpDataExecuter::FinalizeState); + LOG_D("Send Commit to BufferActor=" << BufferActorId); + + auto event = std::make_unique(); + event->ExecuterActorId = SelfId(); + event->TxId = TxId; + Send(BufferActorId, event.release()); + return; + } else if (Request.LocksOp == ELocksOp::Rollback) { + Become(&TKqpDataExecuter::FinalizeState); + LOG_D("Send Rollback to BufferActor=" << BufferActorId); + + auto event = std::make_unique(); + event->ExecuterActorId = SelfId(); + Send(BufferActorId, event.release()); + MakeResponseAndPassAway(); + return; + } else if (Request.UseImmediateEffects) { + Become(&TKqpDataExecuter::FinalizeState); + LOG_D("Send Flush to BufferActor=" << BufferActorId); + + auto event = std::make_unique(); + event->ExecuterActorId = SelfId(); + Send(BufferActorId, event.release()); + return; + } else { + Become(&TKqpDataExecuter::FinalizeState); + MakeResponseAndPassAway(); + return; + } + } + + STATEFN(FinalizeState) { + switch(ev->GetTypeRewrite()) { + hFunc(TEvKqp::TEvAbortExecution, HandleAbortExecution); + hFunc(TEvKqpBuffer::TEvResult, HandleFinalize); + default: + LOG_W("Unexpected event: " << ev->GetTypeName() << ", at state: FinalizeState"); + } + } + + void HandleFinalize(TEvKqpBuffer::TEvResult::TPtr& ev) { + if (ev->Get()->Stats) { + if (Stats) { + Stats->AddBufferStats(std::move(*ev->Get()->Stats)); + } + } + MakeResponseAndPassAway(); + } + + void MakeResponseAndPassAway() { + ResponseEv->Record.MutableResponse()->SetStatus(Ydb::StatusIds::SUCCESS); + Counters->TxProxyMon->ReportStatusOK->Inc(); + ResponseEv->Snapshot = GetSnapshot(); - if (!Locks.empty()) { + if (!Locks.empty() || (TxManager && TxManager->HasLocks())) { if (LockHandle) { ResponseEv->LockHandle = std::move(LockHandle); } - BuildLocks(*ResponseEv->Record.MutableResponse()->MutableResult()->MutableLocks(), Locks); + if (!TxManager) { + BuildLocks(*ResponseEv->Record.MutableResponse()->MutableResult()->MutableLocks(), Locks); + } } auto resultSize = ResponseEv->GetByteSize(); @@ -323,6 +417,8 @@ class TKqpDataExecuter: public TKqpExecuterBase(); @@ -971,7 +1069,7 @@ class TKqpDataExecuter: public TKqpExecuterBaseBrokenLockPathId = NYql::TKikimrPathId(res->Record.GetTxLocks(0).GetSchemeShard(), res->Record.GetTxLocks(0).GetPathId()); ReplyErrorAndDie(Ydb::StatusIds::ABORTED, {}); + return; } CheckExecutionComplete(); return; @@ -1161,6 +1260,7 @@ class TKqpDataExecuter: public TKqpExecuterBaseGet(); ResponseEv->Orbit.Join(res->Orbit); const ui64 shardId = res->GetOrigin(); @@ -1423,7 +1523,7 @@ class TKqpDataExecuter: public TKqpExecuterBase TTask& { - YQL_ENSURE(!UseEvWriteForOltp); + YQL_ENSURE(!TxManager); auto it = shardTasks.find(shardId); if (it != shardTasks.end()) { return TasksGraph.GetTask(it->second); @@ -1557,7 +1657,7 @@ class TKqpDataExecuter: public TKqpExecuterBaseShardReadLocks = locksCount > 0; - LOG_D("State: " << CurrentStateFuncName() << ", Executing KQP transaction on shard: " << shardId << ", tasks: [" - << JoinStrings(shardState.TaskIds.begin(), shardState.TaskIds.end(), ",") << "]" << ", lockTxId: " << lockTxId - << ", locks: " << dataTransaction.GetKqpTransaction().GetLocks().ShortDebugString()); + LOG_D("State: " << CurrentStateFuncName() + << ", Executing KQP transaction on shard: " << shardId + << ", tasks: [" << JoinStrings(shardState.TaskIds.begin(), shardState.TaskIds.end(), ",") << "]" + << ", lockTxId: " << lockTxId + << ", locks: " << dataTransaction.GetKqpTransaction().GetLocks().ShortDebugString() + << ", immediate: " << ImmediateTx); std::unique_ptr ev; if (isOlap) { @@ -1631,6 +1734,7 @@ class TKqpDataExecuter: public TKqpExecuterBase& tables) { - return NKqp::HasOlapTableWriteInStage(stage, tables); - } - void Execute() { LWTRACK(KqpDataExecuterStartExecute, ResponseEv->Orbit, TxId); @@ -1837,7 +1937,7 @@ class TKqpDataExecuter: public TKqpExecuterBase shardIds; for (auto& [stageId, stageInfo] : TasksGraph.GetStagesInfo()) { if (stageInfo.Meta.IsOlap()) { @@ -2162,7 +2264,7 @@ class TKqpDataExecuter: public TKqpExecuterBaseGet(shardId).IsOlap) { + if (TxManager || ShardIdToTableInfo->Get(shardId).IsOlap) { if (auto it = evWriteTxs.find(shardId); it != evWriteTxs.end()) { locks = it->second->MutableLocks(); } else { @@ -2554,7 +2656,9 @@ class TKqpDataExecuter: public TKqpExecuterBase writeId; if (Request.TopicOperations.HasWriteId()) { writeId = Request.TopicOperations.GetWriteId(); @@ -2689,7 +2794,7 @@ class TKqpDataExecuter: public TKqpExecuterBaseSender); if (ev->Sender == SelfId()) { PassAway(); @@ -2748,10 +2853,10 @@ class TKqpDataExecuter: public TKqpExecuterBase FederatedQuerySetup; const TGUCSettings::TPtr GUCSettings; TShardIdToTableInfoPtr ShardIdToTableInfo; + const bool AllowOlapDataQuery = false; bool HasExternalSources = false; bool SecretSnapshotRequired = false; @@ -2796,11 +2901,11 @@ IActor* CreateKqpDataExecuter(IKqpGateway::TExecPhysicalRequest&& request, const NYql::NDq::IDqAsyncIoFactory::TPtr asyncIoFactory, const TActorId& creator, const TIntrusivePtr& userRequestContext, ui32 statementResultIndex, const std::optional& federatedQuerySetup, const TGUCSettings::TPtr& GUCSettings, - const TShardIdToTableInfoPtr& shardIdToTableInfo) + const TShardIdToTableInfoPtr& shardIdToTableInfo, const IKqpTransactionManagerPtr& txManager, const TActorId bufferActorId) { - return new TKqpDataExecuter(std::move(request), database, userToken, counters, streamResult, - tableServiceConfig, std::move(asyncIoFactory), creator, userRequestContext, - statementResultIndex, federatedQuerySetup, GUCSettings, shardIdToTableInfo); + return new TKqpDataExecuter(std::move(request), database, userToken, counters, streamResult, tableServiceConfig, + std::move(asyncIoFactory), creator, userRequestContext, statementResultIndex, federatedQuerySetup, GUCSettings, + shardIdToTableInfo, txManager, bufferActorId); } } // namespace NKqp diff --git a/ydb/core/kqp/executer_actor/kqp_executer.h b/ydb/core/kqp/executer_actor/kqp_executer.h index 9c0ef4fae41e..28404d117bdc 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer.h +++ b/ydb/core/kqp/executer_actor/kqp_executer.h @@ -106,7 +106,7 @@ IActor* CreateKqpExecuter(IKqpGateway::TExecPhysicalRequest&& request, const TSt NYql::NDq::IDqAsyncIoFactory::TPtr asyncIoFactory, TPreparedQueryHolder::TConstPtr preparedQuery, const TActorId& creator, const TIntrusivePtr& userRequestContext, ui32 statementResultIndex, const std::optional& federatedQuerySetup, const TGUCSettings::TPtr& GUCSettings, - const TShardIdToTableInfoPtr& shardIdToTableInfo); + const TShardIdToTableInfoPtr& shardIdToTableInfo, const IKqpTransactionManagerPtr& txManager, const TActorId bufferActorId); IActor* CreateKqpSchemeExecuter( TKqpPhyTxHolder::TConstPtr phyTx, NKikimrKqp::EQueryType queryType, const TActorId& target, diff --git a/ydb/core/kqp/executer_actor/kqp_executer_impl.cpp b/ydb/core/kqp/executer_actor/kqp_executer_impl.cpp index 89da1bd869d6..ba8c81ca520d 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_impl.cpp +++ b/ydb/core/kqp/executer_actor/kqp_executer_impl.cpp @@ -82,7 +82,7 @@ IActor* CreateKqpExecuter(IKqpGateway::TExecPhysicalRequest&& request, const TSt NYql::NDq::IDqAsyncIoFactory::TPtr asyncIoFactory, TPreparedQueryHolder::TConstPtr preparedQuery, const TActorId& creator, const TIntrusivePtr& userRequestContext, ui32 statementResultIndex, const std::optional& federatedQuerySetup, const TGUCSettings::TPtr& GUCSettings, - const TShardIdToTableInfoPtr& shardIdToTableInfo) + const TShardIdToTableInfoPtr& shardIdToTableInfo, const IKqpTransactionManagerPtr& txManager, const TActorId bufferActorId) { if (request.Transactions.empty()) { // commit-only or rollback-only data transaction @@ -90,7 +90,9 @@ IActor* CreateKqpExecuter(IKqpGateway::TExecPhysicalRequest&& request, const TSt std::move(request), database, userToken, counters, false, tableServiceConfig, std::move(asyncIoFactory), creator, userRequestContext, statementResultIndex, - federatedQuerySetup, /*GUCSettings*/nullptr, shardIdToTableInfo); + federatedQuerySetup, /*GUCSettings*/nullptr, + shardIdToTableInfo, txManager, bufferActorId + ); } TMaybe txsType; @@ -112,7 +114,8 @@ IActor* CreateKqpExecuter(IKqpGateway::TExecPhysicalRequest&& request, const TSt std::move(request), database, userToken, counters, false, tableServiceConfig, std::move(asyncIoFactory), creator, userRequestContext, statementResultIndex, - federatedQuerySetup, /*GUCSettings*/nullptr, shardIdToTableInfo + federatedQuerySetup, /*GUCSettings*/nullptr, + shardIdToTableInfo, txManager, bufferActorId ); case NKqpProto::TKqpPhyTx::TYPE_SCAN: @@ -127,7 +130,8 @@ IActor* CreateKqpExecuter(IKqpGateway::TExecPhysicalRequest&& request, const TSt std::move(request), database, userToken, counters, true, tableServiceConfig, std::move(asyncIoFactory), creator, userRequestContext, statementResultIndex, - federatedQuerySetup, GUCSettings, shardIdToTableInfo + federatedQuerySetup, GUCSettings, + shardIdToTableInfo, txManager, bufferActorId ); default: diff --git a/ydb/core/kqp/executer_actor/kqp_executer_impl.h b/ydb/core/kqp/executer_actor/kqp_executer_impl.h index 11031146b295..c58311fd3be3 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_impl.h +++ b/ydb/core/kqp/executer_actor/kqp_executer_impl.h @@ -125,8 +125,11 @@ class TKqpExecuterBase : public TActorBootstrapped { TKqpRequestCounters::TPtr counters, const NKikimrConfig::TTableServiceConfig tableServiceConfig, const TIntrusivePtr& userRequestContext, - ui32 statementResultIndex, ui64 spanVerbosity = 0, TString spanName = "KqpExecuterBase", bool streamResult = false) + ui32 statementResultIndex, ui64 spanVerbosity = 0, TString spanName = "KqpExecuterBase", + bool streamResult = false, const TActorId bufferActorId = {}, const IKqpTransactionManagerPtr& txManager = nullptr) : Request(std::move(request)) + , BufferActorId(bufferActorId) + , TxManager(txManager) , Database(database) , UserToken(userToken) , Counters(counters) @@ -501,6 +504,12 @@ class TKqpExecuterBase : public TActorBootstrapped { } } + if (BufferActorId && Request.LocksOp == ELocksOp::Rollback) { + YQL_ENSURE(Request.Transactions.empty()); + static_cast(this)->Finalize(); + return; + } + ExecuterStateSpan = NWilson::TSpan(TWilsonKqp::ExecuterTableResolve, ExecuterSpan.GetTraceId(), "WaitForTableResolve", NWilson::EFlags::AUTO_END); auto kqpTableResolver = CreateKqpTableResolver(this->SelfId(), TxId, UserToken, Request.Transactions, @@ -890,6 +899,9 @@ class TKqpExecuterBase : public TActorBootstrapped { settings.SetLockTxId(*lockTxId); settings.SetLockNodeId(SelfId().NodeId()); } + if (!settings.GetInconsistentTx() && !settings.GetIsOlap()) { + ActorIdToProto(BufferActorId, settings.MutableBufferActorId()); + } output.SinkSettings.ConstructInPlace(); output.SinkSettings->PackFrom(settings); } else { @@ -1925,6 +1937,8 @@ class TKqpExecuterBase : public TActorBootstrapped { protected: IKqpGateway::TExecPhysicalRequest Request; + TActorId BufferActorId; + IKqpTransactionManagerPtr TxManager; const TString Database; const TIntrusiveConstPtr UserToken; TKqpRequestCounters::TPtr Counters; @@ -1992,7 +2006,7 @@ IActor* CreateKqpDataExecuter(IKqpGateway::TExecPhysicalRequest&& request, const NYql::NDq::IDqAsyncIoFactory::TPtr asyncIoFactory, const TActorId& creator, const TIntrusivePtr& userRequestContext, ui32 statementResultIndex, const std::optional& federatedQuerySetup, const TGUCSettings::TPtr& GUCSettings, - const TShardIdToTableInfoPtr& shardIdToTableInfo); + const TShardIdToTableInfoPtr& shardIdToTableInfo, const IKqpTransactionManagerPtr& txManager, const TActorId bufferActorId); IActor* CreateKqpScanExecuter(IKqpGateway::TExecPhysicalRequest&& request, const TString& database, const TIntrusiveConstPtr& userToken, TKqpRequestCounters::TPtr counters, diff --git a/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp b/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp index 13589071bbb6..ffb35db8ff93 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp +++ b/ydb/core/kqp/executer_actor/kqp_executer_stats.cpp @@ -862,6 +862,26 @@ void TQueryExecutionStats::AddDatashardStats(NKikimrQueryStats::TTxStats&& txSta } } +void TQueryExecutionStats::AddBufferStats(NYql::NDqProto::TDqTaskStats&& taskStats) { + for (auto& table : taskStats.GetTables()) { + NYql::NDqProto::TDqTableStats* tableAggr = nullptr; + if (auto it = TableStats.find(table.GetTablePath()); it != TableStats.end()) { + tableAggr = it->second; + } else { + tableAggr = Result->AddTables(); + tableAggr->SetTablePath(table.GetTablePath()); + TableStats.emplace(table.GetTablePath(), tableAggr); + } + + tableAggr->SetReadRows(tableAggr->GetReadRows() + table.GetReadRows()); + tableAggr->SetReadBytes(tableAggr->GetReadBytes() + table.GetReadBytes()); + tableAggr->SetWriteRows(tableAggr->GetWriteRows() + table.GetWriteRows()); + tableAggr->SetWriteBytes(tableAggr->GetWriteBytes() + table.GetWriteBytes()); + tableAggr->SetEraseRows(tableAggr->GetEraseRows() + table.GetEraseRows()); + tableAggr->SetAffectedPartitions(table.GetAffectedPartitions()); + } +} + void TQueryExecutionStats::UpdateTaskStats(ui64 taskId, const NYql::NDqProto::TDqComputeActorStats& stats) { Y_ASSERT(stats.GetTasks().size() == 1); const NYql::NDqProto::TDqTaskStats& taskStats = stats.GetTasks(0); diff --git a/ydb/core/kqp/executer_actor/kqp_executer_stats.h b/ydb/core/kqp/executer_actor/kqp_executer_stats.h index ad297bf6a8b3..6ed028db5092 100644 --- a/ydb/core/kqp/executer_actor/kqp_executer_stats.h +++ b/ydb/core/kqp/executer_actor/kqp_executer_stats.h @@ -195,6 +195,7 @@ struct TQueryExecutionStats { TDuration collectLongTaskStatsTimeout = TDuration::Max() ); void AddDatashardStats(NKikimrQueryStats::TTxStats&& txStats); + void AddBufferStats(NYql::NDqProto::TDqTaskStats&& taskStats); void UpdateTaskStats(ui64 taskId, const NYql::NDqProto::TDqComputeActorStats& stats); void ExportExecStats(NYql::NDqProto::TDqExecutionStats& stats); diff --git a/ydb/core/kqp/executer_actor/kqp_planner.cpp b/ydb/core/kqp/executer_actor/kqp_planner.cpp index 929a3f01863a..9365aad0eb26 100644 --- a/ydb/core/kqp/executer_actor/kqp_planner.cpp +++ b/ydb/core/kqp/executer_actor/kqp_planner.cpp @@ -53,6 +53,23 @@ void BuildInitialTaskResources(const TKqpTasksGraph& graph, ui64 taskId, TTaskRe ret.HeavyProgram = opts.GetHasMapJoin(); } +bool NeedToRunLocally(const TTask& task) { + for (const auto& output : task.Outputs) { + if (output.Type == TTaskOutputType::Sink && output.SinkType == KqpTableSinkName) { + YQL_ENSURE(output.SinkSettings); + const google::protobuf::Any& settingsAny = *output.SinkSettings; + YQL_ENSURE(settingsAny.Is()); + NKikimrKqp::TKqpTableSinkSettings settings; + YQL_ENSURE(settingsAny.UnpackTo(&settings)); + if (ActorIdFromProto(settings.GetBufferActorId())) { + // We need to run compute actor locally if it uses buffer actor. + return true; + } + } + } + return false; +} + bool LimitCPU(TIntrusivePtr ctx) { return ctx->PoolId && ctx->PoolConfig.has_value() && ctx->PoolConfig->TotalCpuLimitPercentPerNode > 0; } @@ -417,7 +434,12 @@ std::unique_ptr TKqpPlanner::AssignTasksToNodes() { for(ui64 taskId: group.TaskIds) { auto [it, success] = alreadyAssigned.emplace(taskId, group.NodeId); if (success) { - TasksPerNode[group.NodeId].push_back(taskId); + if (NeedToRunLocally(TasksGraph.GetTask(taskId))) { + const ui64 selfNodeId = ExecuterId.NodeId(); + TasksPerNode[selfNodeId].push_back(taskId); + } else { + TasksPerNode[group.NodeId].push_back(taskId); + } } } } @@ -463,7 +485,7 @@ TString TKqpPlanner::ExecuteDataComputeTask(ui64 taskId, ui32 computeTasksSize) .WithSpilling = WithSpilling, .StatsMode = GetDqStatsMode(StatsMode), .Deadline = Deadline, - .ShareMailbox = (computeTasksSize <= 1), + .ShareMailbox = (computeTasksSize <= 1) || NeedToRunLocally(task), .RlPath = Nothing(), }); diff --git a/ydb/core/kqp/expr_nodes/kqp_expr_nodes.json b/ydb/core/kqp/expr_nodes/kqp_expr_nodes.json index 63ec4a7dc929..b8f78803d853 100644 --- a/ydb/core/kqp/expr_nodes/kqp_expr_nodes.json +++ b/ydb/core/kqp/expr_nodes/kqp_expr_nodes.json @@ -548,7 +548,9 @@ {"Index": 0, "Name": "Table", "Type": "TKqpTable"}, {"Index": 1, "Name": "InconsistentWrite", "Type": "TCoAtom"}, {"Index": 2, "Name": "Mode", "Type": "TCoAtom"}, - {"Index": 3, "Name": "Settings", "Type": "TCoNameValueTupleList", "Optional": true} + {"Index": 3, "Name": "Priority", "Type": "TCoAtom"}, + {"Index": 4, "Name": "TableType", "Type": "TCoAtom"}, + {"Index": 5, "Name": "Settings", "Type": "TCoNameValueTupleList", "Optional": true} ] }, { diff --git a/ydb/core/kqp/host/kqp_type_ann.cpp b/ydb/core/kqp/host/kqp_type_ann.cpp index 36249024fea4..c1eac4a75c47 100644 --- a/ydb/core/kqp/host/kqp_type_ann.cpp +++ b/ydb/core/kqp/host/kqp_type_ann.cpp @@ -1859,7 +1859,7 @@ TStatus AnnotateKqpSinkEffect(const TExprNode::TPtr& node, TExprContext& ctx) { } TStatus AnnotateTableSinkSettings(const TExprNode::TPtr& input, TExprContext& ctx) { - if (!EnsureMinMaxArgsCount(*input, 4, 5, ctx)) { + if (!EnsureMinMaxArgsCount(*input, 5, 6, ctx)) { return TStatus::Error; } input->SetTypeAnn(ctx.MakeType()); diff --git a/ydb/core/kqp/opt/kqp_opt.cpp b/ydb/core/kqp/opt/kqp_opt.cpp index 8af6ac69ad4b..0c153b525efa 100644 --- a/ydb/core/kqp/opt/kqp_opt.cpp +++ b/ydb/core/kqp/opt/kqp_opt.cpp @@ -83,7 +83,8 @@ bool IsKqpEffectsStage(const TDqStageBase& stage) { } bool NeedSinks(const TKikimrTableDescription& table, const TKqpOptimizeContext& kqpCtx) { - return (kqpCtx.IsGenericQuery() || (kqpCtx.IsDataQuery() && table.Metadata->Kind != EKikimrTableKind::Olap)) + return (kqpCtx.IsGenericQuery() + || (kqpCtx.IsDataQuery() && (table.Metadata->Kind != EKikimrTableKind::Olap || kqpCtx.Config->AllowOlapDataQuery))) && (table.Metadata->Kind != EKikimrTableKind::Olap || kqpCtx.Config->EnableOlapSink) && (table.Metadata->Kind != EKikimrTableKind::Datashard || kqpCtx.Config->EnableOltpSink); } diff --git a/ydb/core/kqp/opt/kqp_opt_build_txs.cpp b/ydb/core/kqp/opt/kqp_opt_build_txs.cpp index 95e276543040..298869dc452e 100644 --- a/ydb/core/kqp/opt/kqp_opt_build_txs.cpp +++ b/ydb/core/kqp/opt/kqp_opt_build_txs.cpp @@ -558,7 +558,7 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { } if (!query.Effects().Empty()) { - auto collectedEffects = CollectEffects(query.Effects(), ctx); + auto collectedEffects = CollectEffects(query.Effects(), ctx, *KqpCtx); for (auto& effects : collectedEffects) { auto tx = BuildTx(effects.Ptr(), ctx, /* isPrecompute */ false); @@ -583,11 +583,12 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { } private: - TVector CollectEffects(const TExprList& list, TExprContext& ctx) { + TVector CollectEffects(const TExprList& list, TExprContext& ctx, TKqpOptimizeContext& kqpCtx) { struct TEffectsInfo { enum class EType { KQP_EFFECT, KQP_SINK, + KQP_BATCH_SINK, EXTERNAL_SINK, }; @@ -615,23 +616,38 @@ class TKqpBuildTxsTransformer : public TSyncTransformerBase { effectsInfos.back().Type = TEffectsInfo::EType::EXTERNAL_SINK; effectsInfos.back().Exprs.push_back(expr.Ptr()); } else { - // Two table sinks can't be executed in one physical transaction if they write into one table. - const TStringBuf tablePathId = sinkSettings.Cast().Table().PathId().Value(); - - auto it = std::find_if( - std::begin(effectsInfos), - std::end(effectsInfos), - [&tablePathId](const auto& effectsInfo) { - return effectsInfo.Type == TEffectsInfo::EType::KQP_SINK - && !effectsInfo.TablesPathIds.contains(tablePathId); - }); - if (it == std::end(effectsInfos)) { - effectsInfos.emplace_back(); - it = std::prev(std::end(effectsInfos)); - it->Type = TEffectsInfo::EType::KQP_SINK; + // Two table sinks can't be executed in one physical transaction if they write into same table and have same priority. + + const auto& tableDescription = kqpCtx.Tables->ExistingTable(kqpCtx.Cluster, sinkSettings.Cast().Table().Path()); + if (tableDescription.Metadata->Kind == EKikimrTableKind::Olap) { + const TStringBuf tablePathId = sinkSettings.Cast().Table().PathId().Value(); + + auto it = std::find_if( + std::begin(effectsInfos), + std::end(effectsInfos), + [&tablePathId](const auto& effectsInfo) { + return effectsInfo.Type == TEffectsInfo::EType::KQP_SINK + && !effectsInfo.TablesPathIds.contains(tablePathId); + }); + if (it == std::end(effectsInfos)) { + effectsInfos.emplace_back(); + it = std::prev(std::end(effectsInfos)); + it->Type = TEffectsInfo::EType::KQP_SINK; + } + it->TablesPathIds.insert(tablePathId); + it->Exprs.push_back(expr.Ptr()); + } else { + auto it = std::find_if( + std::begin(effectsInfos), + std::end(effectsInfos), + [](const auto& effectsInfo) { return effectsInfo.Type == TEffectsInfo::EType::KQP_BATCH_SINK; }); + if (it == std::end(effectsInfos)) { + effectsInfos.emplace_back(); + it = std::prev(std::end(effectsInfos)); + it->Type = TEffectsInfo::EType::KQP_BATCH_SINK; + } + it->Exprs.push_back(expr.Ptr()); } - it->TablesPathIds.insert(tablePathId); - it->Exprs.push_back(expr.Ptr()); } } else { // Table effects are executed all in one physical transaction. diff --git a/ydb/core/kqp/opt/kqp_opt_effects.cpp b/ydb/core/kqp/opt/kqp_opt_effects.cpp index 0c0e818d6853..2dd9dd91ee40 100644 --- a/ydb/core/kqp/opt/kqp_opt_effects.cpp +++ b/ydb/core/kqp/opt/kqp_opt_effects.cpp @@ -232,7 +232,7 @@ TCoAtomList BuildKeyColumnsList(const TKikimrTableDescription& table, TPositionH } TDqStage RebuildPureStageWithSink(TExprBase expr, const TKqpTable& table, - const bool allowInconsistentWrites, const TStringBuf mode, TExprContext& ctx) { + const bool allowInconsistentWrites, const TStringBuf mode, const i64 order, const bool isOlap, TExprContext& ctx) { Y_DEBUG_ABORT_UNLESS(IsDqPureExpr(expr)); return Build(ctx, expr.Pos()) @@ -257,6 +257,8 @@ TDqStage RebuildPureStageWithSink(TExprBase expr, const TKqpTable& table, ? ctx.NewAtom(expr.Pos(), "true") : ctx.NewAtom(expr.Pos(), "false")) .Mode(ctx.NewAtom(expr.Pos(), mode)) + .Priority(ctx.NewAtom(expr.Pos(), ToString(order))) + .TableType(ctx.NewAtom(expr.Pos(), isOlap ? "olap" : "oltp")) .Settings() .Build() .Build() @@ -296,7 +298,7 @@ TDqPhyPrecompute BuildPrecomputeStage(TExprBase expr, TExprContext& ctx) { } bool BuildUpsertRowsEffect(const TKqlUpsertRows& node, TExprContext& ctx, const TKqpOptimizeContext& kqpCtx, - const TCoArgument& inputArg, TMaybeNode& stageInput, TMaybeNode& effect, bool& sinkEffect) + const TCoArgument& inputArg, TMaybeNode& stageInput, TMaybeNode& effect, bool& sinkEffect, const i64 order) { const auto& table = kqpCtx.Tables->ExistingTable(kqpCtx.Cluster, node.Table().Path()); @@ -306,12 +308,14 @@ bool BuildUpsertRowsEffect(const TKqlUpsertRows& node, TExprContext& ctx, const } sinkEffect = NeedSinks(table, kqpCtx) || (kqpCtx.IsGenericQuery() && settings.AllowInconsistentWrites); + const bool isOlap = (table.Metadata->Kind == EKikimrTableKind::Olap); + const i64 priority = isOlap ? 0 : order; if (IsDqPureExpr(node.Input())) { if (sinkEffect) { stageInput = RebuildPureStageWithSink( node.Input(), node.Table(), - settings.AllowInconsistentWrites, settings.Mode, ctx); + settings.AllowInconsistentWrites, settings.Mode, priority, isOlap, ctx); effect = Build(ctx, node.Pos()) .Stage(stageInput.Cast().Ptr()) .SinkIndex().Build("0") @@ -352,6 +356,8 @@ bool BuildUpsertRowsEffect(const TKqlUpsertRows& node, TExprContext& ctx, const ? ctx.NewAtom(node.Pos(), "true") : ctx.NewAtom(node.Pos(), "false")) .Mode(ctx.NewAtom(node.Pos(), settings.Mode)) + .Priority(ctx.NewAtom(node.Pos(), ToString(priority))) + .TableType(ctx.NewAtom(node.Pos(), isOlap ? "olap" : "oltp")) .Settings() .Build() .Build() @@ -448,16 +454,17 @@ bool BuildUpsertRowsEffect(const TKqlUpsertRows& node, TExprContext& ctx, const } bool BuildDeleteRowsEffect(const TKqlDeleteRows& node, TExprContext& ctx, const TKqpOptimizeContext& kqpCtx, - const TCoArgument& inputArg, TMaybeNode& stageInput, TMaybeNode& effect, bool& sinkEffect) + const TCoArgument& inputArg, TMaybeNode& stageInput, TMaybeNode& effect, bool& sinkEffect, const i64 order) { const auto& table = kqpCtx.Tables->ExistingTable(kqpCtx.Cluster, node.Table().Path()); sinkEffect = NeedSinks(table, kqpCtx); - + const bool isOlap = (table.Metadata->Kind == EKikimrTableKind::Olap); + const i64 priority = isOlap ? 0 : order; if (IsDqPureExpr(node.Input())) { if (sinkEffect) { const auto keyColumns = BuildKeyColumnsList(table, node.Pos(), ctx); - stageInput = RebuildPureStageWithSink(node.Input(), node.Table(), false, "delete", ctx); + stageInput = RebuildPureStageWithSink(node.Input(), node.Table(), false, "delete", priority, isOlap, ctx); effect = Build(ctx, node.Pos()) .Stage(stageInput.Cast().Ptr()) .SinkIndex().Build("0") @@ -494,6 +501,8 @@ bool BuildDeleteRowsEffect(const TKqlDeleteRows& node, TExprContext& ctx, const .Table(node.Table()) .InconsistentWrite(ctx.NewAtom(node.Pos(), "false")) .Mode(ctx.NewAtom(node.Pos(), "delete")) + .Priority(ctx.NewAtom(node.Pos(), ToString(priority))) + .TableType(ctx.NewAtom(node.Pos(), isOlap ? "olap" : "oltp")) .Settings() .Build() .Build() @@ -584,6 +593,7 @@ bool BuildEffects(TPositionHandle pos, const TVector& effects, TVector newSinkEffects; newEffects.reserve(effects.size()); newSinkEffects.reserve(effects.size()); + i64 order = builtEffects.size(); for (const auto& effect : effects) { TMaybeNode newEffect; @@ -596,15 +606,17 @@ bool BuildEffects(TPositionHandle pos, const TVector& effects, .Done(); if (auto maybeUpsertRows = effect.Maybe()) { - if (!BuildUpsertRowsEffect(maybeUpsertRows.Cast(), ctx, kqpCtx, inputArg, input, newEffect, sinkEffect)) { + if (!BuildUpsertRowsEffect(maybeUpsertRows.Cast(), ctx, kqpCtx, inputArg, input, newEffect, sinkEffect, order)) { return false; } + ++order; } if (auto maybeDeleteRows = effect.Maybe()) { - if (!BuildDeleteRowsEffect(maybeDeleteRows.Cast(), ctx, kqpCtx, inputArg, input, newEffect, sinkEffect)) { + if (!BuildDeleteRowsEffect(maybeDeleteRows.Cast(), ctx, kqpCtx, inputArg, input, newEffect, sinkEffect, order)) { return false; } + ++order; } if (input) { @@ -696,7 +708,6 @@ TMaybeNode BuildEffects(const TKqlQuery& query, TExprContext& ctx, const TKqpOptimizeContext& kqpCtx) { TVector builtEffects; - if constexpr (GroupEffectsByTable) { TMap> tableEffectsMap; ExploreEffectLists( diff --git a/ydb/core/kqp/provider/yql_kikimr_settings.h b/ydb/core/kqp/provider/yql_kikimr_settings.h index 4ba9627c485a..6be159a847c7 100644 --- a/ydb/core/kqp/provider/yql_kikimr_settings.h +++ b/ydb/core/kqp/provider/yql_kikimr_settings.h @@ -170,6 +170,7 @@ struct TKikimrConfiguration : public TKikimrSettings, public NCommon::TSettingDi bool EnableCreateTableAs = false; ui64 IdxLookupJoinsPrefixPointLimit = 1; bool OldLookupJoinBehaviour = true; + bool AllowOlapDataQuery = false; bool EnableOlapSink = false; bool EnableOltpSink = false; bool EnableHtapTx = false; diff --git a/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp b/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp index 75a83e2337cb..6deffdf98133 100644 --- a/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp +++ b/ydb/core/kqp/query_compiler/kqp_query_compiler.cpp @@ -1098,6 +1098,8 @@ class TKqpQueryCompiler : public IKqpQueryCompiler { if (const auto inconsistentWrite = settings.InconsistentWrite().Cast(); inconsistentWrite.StringValue() == "true") { settingsProto.SetInconsistentTx(true); } + settingsProto.SetIsOlap(settings.TableType().Cast().StringValue() == "olap"); + settingsProto.SetPriority(FromString(settings.Priority().Cast().StringValue())); if (settings.Mode().Cast().StringValue() == "replace") { settingsProto.SetType(NKikimrKqp::TKqpTableSinkSettings::MODE_REPLACE); diff --git a/ydb/core/kqp/runtime/kqp_read_actor.cpp b/ydb/core/kqp/runtime/kqp_read_actor.cpp index aa17440a948d..fd9114ea1aee 100644 --- a/ydb/core/kqp/runtime/kqp_read_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_read_actor.cpp @@ -593,8 +593,8 @@ class TKqpReadActor : public TActorBootstrapped, public NYql::NDq PendingShards.PushBack(state.Release()); return; } - } else if (!Snapshot.IsValid() && !Settings->GetAllowInconsistentReads()) { - return RuntimeError("Inconsistent reads after shards split", NDqProto::StatusIds::UNAVAILABLE); + } else if (!Snapshot.IsValid() && !Settings->HasLockTxId() && !Settings->GetAllowInconsistentReads()) { + return RuntimeError("Inconsistent reads without locks", NDqProto::StatusIds::UNAVAILABLE); } const auto& tr = *AppData()->TypeRegistry; diff --git a/ydb/core/kqp/runtime/kqp_write_actor.cpp b/ydb/core/kqp/runtime/kqp_write_actor.cpp index ef862fa62078..33cf47057797 100644 --- a/ydb/core/kqp/runtime/kqp_write_actor.cpp +++ b/ydb/core/kqp/runtime/kqp_write_actor.cpp @@ -8,14 +8,17 @@ #include #include #include +#include +#include #include +#include #include +#include #include #include #include #include #include -#include #include #include #include @@ -37,65 +40,114 @@ namespace { return delay; } - struct TLockInfo { - bool AddAndCheckLock(const NKikimrDataEvents::TLock& lock) { - if (!Lock) { - Lock = lock; - return true; - } else { - return lock.GetLockId() == Lock->GetLockId() - && lock.GetDataShard() == Lock->GetDataShard() - && lock.GetSchemeShard() == Lock->GetSchemeShard() - && lock.GetPathId() == Lock->GetPathId() - && lock.GetGeneration() == Lock->GetGeneration() - && lock.GetCounter() == Lock->GetCounter(); + NKikimrDataEvents::TEvWrite::TOperation::EOperationType GetOperation(NKikimrKqp::TKqpTableSinkSettings::EType type) { + switch (type) { + case NKikimrKqp::TKqpTableSinkSettings::MODE_REPLACE: + return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_REPLACE; + case NKikimrKqp::TKqpTableSinkSettings::MODE_UPSERT: + return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPSERT; + case NKikimrKqp::TKqpTableSinkSettings::MODE_INSERT: + return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_INSERT; + case NKikimrKqp::TKqpTableSinkSettings::MODE_DELETE: + return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_DELETE; + case NKikimrKqp::TKqpTableSinkSettings::MODE_UPDATE: + return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPDATE; + default: + return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UNSPECIFIED; + } + } + + void FillEvWritePrepare(NKikimr::NEvents::TDataEvents::TEvWrite* evWrite, ui64 shardId, ui64 txId, const NKikimr::NKqp::IKqpTransactionManagerPtr& txManager) { + evWrite->Record.SetTxId(txId); + auto* protoLocks = evWrite->Record.MutableLocks(); + protoLocks->SetOp(NKikimrDataEvents::TKqpLocks::Commit); + + const auto prepareSettings = txManager->GetPrepareTransactionInfo(); + if (!prepareSettings.ArbiterColumnShard) { + for (const ui64 sendingShardId : prepareSettings.SendingShards) { + protoLocks->AddSendingShards(sendingShardId); + } + for (const ui64 receivingShardId : prepareSettings.ReceivingShards) { + protoLocks->AddReceivingShards(receivingShardId); + } + if (prepareSettings.Arbiter) { + protoLocks->SetArbiterShard(*prepareSettings.Arbiter); + } + } else if (prepareSettings.ArbiterColumnShard == shardId) { + protoLocks->SetArbiterColumnShard(*prepareSettings.ArbiterColumnShard); + for (const ui64 sendingShardId : prepareSettings.SendingShards) { + protoLocks->AddSendingShards(sendingShardId); + } + for (const ui64 receivingShardId : prepareSettings.ReceivingShards) { + protoLocks->AddReceivingShards(receivingShardId); + } + } else { + protoLocks->SetArbiterColumnShard(*prepareSettings.ArbiterColumnShard); + protoLocks->AddSendingShards(*prepareSettings.ArbiterColumnShard); + protoLocks->AddReceivingShards(*prepareSettings.ArbiterColumnShard); + if (prepareSettings.SendingShards.contains(shardId)) { + protoLocks->AddSendingShards(shardId); + } + if (prepareSettings.ReceivingShards.contains(shardId)) { + protoLocks->AddReceivingShards(shardId); } } - const std::optional& GetLock() const { - return Lock; + const auto locks = txManager->GetLocks(shardId); + for (const auto& lock : locks) { + *protoLocks->AddLocks() = lock; } + } - private: - std::optional Lock; - }; + void FillEvWriteRollback(NKikimr::NEvents::TDataEvents::TEvWrite* evWrite, ui64 shardId, const NKikimr::NKqp::IKqpTransactionManagerPtr& txManager) { + auto* protoLocks = evWrite->Record.MutableLocks(); + protoLocks->SetOp(NKikimrDataEvents::TKqpLocks::Rollback); + + const auto locks = txManager->GetLocks(shardId); + for (const auto& lock : locks) { + *protoLocks->AddLocks() = lock; + } + } } namespace NKikimr { namespace NKqp { -class TKqpDirectWriteActor : public TActorBootstrapped, public NYql::NDq::IDqComputeActorAsyncOutput { - using TBase = TActorBootstrapped; +struct IKqpTableWriterCallbacks { + virtual ~IKqpTableWriterCallbacks() = default; - class TResumeNotificationManager { - public: - TResumeNotificationManager(TKqpDirectWriteActor& writer) - : Writer(writer) { - CheckMemory(); - } + // Ready to accept writes + virtual void OnReady() = 0; - void CheckMemory() { - const auto freeSpace = Writer.GetFreeSpace(); - const auto targetMemory = Writer.MemoryLimit / 2; - if (freeSpace >= targetMemory && targetMemory > LastFreeMemory) { - YQL_ENSURE(freeSpace > 0); - Writer.ResumeExecution(); - } - LastFreeMemory = freeSpace; - } + // EvWrite statuses + virtual void OnPrepared(IKqpTransactionManager::TPrepareResult&& preparedInfo, ui64 dataSize) = 0; + virtual void OnCommitted(ui64 shardId, ui64 dataSize) = 0; + virtual void OnMessageAcknowledged(ui64 dataSize) = 0; + + virtual void OnError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues) = 0; +}; + +struct TKqpTableWriterStatistics { + ui64 ReadRows = 0; + ui64 ReadBytes = 0; + ui64 WriteRows = 0; + ui64 WriteBytes = 0; + ui64 EraseRows = 0; + ui64 EraseBytes = 0; + + THashSet AffectedPartitions; +}; - private: - TKqpDirectWriteActor& Writer; - i64 LastFreeMemory = std::numeric_limits::max(); - }; - friend class TResumeNotificationManager; +class TKqpTableWriteActor : public TActorBootstrapped { + using TBase = TActorBootstrapped; struct TEvPrivate { enum EEv { EvShardRequestTimeout = EventSpaceBegin(TKikimrEvents::ES_PRIVATE), EvResolveRequestPlanned, + EvTerminate, }; struct TEvShardRequestTimeout : public TEventLocal { @@ -108,114 +160,184 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu struct TEvResolveRequestPlanned : public TEventLocal { }; + + struct TEvTerminate : public TEventLocal { + }; + }; + + enum class EMode { + WRITE, + PREPARE, + COMMIT, + IMMEDIATE_COMMIT, }; public: - TKqpDirectWriteActor( - NKikimrKqp::TKqpTableSinkSettings&& settings, - NYql::NDq::TDqAsyncIoFactory::TSinkArguments&& args, - TIntrusivePtr counters) - : LogPrefix(TStringBuilder() << "TxId: " << args.TxId << ", task: " << args.TaskId << ". ") - , Settings(std::move(settings)) - , MessageSettings(GetWriteActorSettings()) - , OutputIndex(args.OutputIndex) - , Callbacks(args.Callback) + TKqpTableWriteActor( + IKqpTableWriterCallbacks* callbacks, + const TTableId& tableId, + const TStringBuf tablePath, + const ui64 lockTxId, + const ui64 lockNodeId, + const bool inconsistentTx, + const NMiniKQL::TTypeEnvironment& typeEnv, + std::shared_ptr alloc, + const IKqpTransactionManagerPtr& txManager, + const TActorId sessionActorId, + TIntrusivePtr counters, + NWilson::TTraceId traceId) + : TypeEnv(typeEnv) + , Alloc(alloc) + , TableId(tableId) + , TablePath(tablePath) + , LockTxId(lockTxId) + , LockNodeId(lockNodeId) + , InconsistentTx(inconsistentTx) + , Callbacks(callbacks) + , TxManager(txManager ? txManager : CreateKqpTransactionManager(/* collectOnly= */ true)) , Counters(counters) - , TypeEnv(args.TypeEnv) - , Alloc(args.Alloc) - , TxId(args.TxId) - , TableId( - Settings.GetTable().GetOwnerId(), - Settings.GetTable().GetTableId(), - Settings.GetTable().GetVersion()) - , FinalTx( - Settings.GetFinalTx()) - , ImmediateTx( - Settings.GetImmediateTx()) - , InconsistentTx( - Settings.GetInconsistentTx()) - , MemoryLimit(MessageSettings.InFlightMemoryLimitPerActorBytes) - , WriteActorSpan(TWilsonKqp::WriteActor, NWilson::TTraceId(args.TraceId), "WriteActor") + , TableWriteActorSpan(TWilsonKqp::TableWriteActor, NWilson::TTraceId(traceId), "TKqpTableWriteActor") { - YQL_ENSURE(std::holds_alternative(TxId)); - YQL_ENSURE(!ImmediateTx); - EgressStats.Level = args.StatsLevel; + LogPrefix = TStringBuilder() << "SessionActorId: " << sessionActorId; + try { + ShardedWriteController = CreateShardedWriteController( + TShardedWriteControllerSettings { + .MemoryLimitTotal = MessageSettings.InFlightMemoryLimitPerActorBytes, + .MemoryLimitPerMessage = MessageSettings.MemoryLimitPerMessageBytes, + .MaxBatchesPerMessage = MessageSettings.MaxBatchesPerMessage, + }, + TypeEnv, + Alloc); + } catch (...) { + RuntimeError( + CurrentExceptionMessage(), + NYql::NDqProto::StatusIds::INTERNAL_ERROR); + } Counters->WriteActorsCount->Inc(); } void Bootstrap() { - LogPrefix = TStringBuilder() << "SelfId: " << this->SelfId() << ", " << LogPrefix; + LogPrefix = TStringBuilder() << "SelfId: " << this->SelfId() << ", Table: `" << TablePath << "` (" << TableId << "), "<< LogPrefix; ResolveTable(); - Become(&TKqpDirectWriteActor::StateFunc); + Become(&TKqpTableWriteActor::StateProcessing); } - static constexpr char ActorName[] = "KQP_WRITE_ACTOR"; + static constexpr char ActorName[] = "KQP_TABLE_WRITE_ACTOR"; -private: - virtual ~TKqpDirectWriteActor() { + i64 GetMemory() const { + return IsReady() + ? ShardedWriteController->GetMemory() + : 0; } - void CommitState(const NYql::NDqProto::TCheckpoint&) final {}; - void LoadState(const NYql::NDq::TSinkState&) final {}; + bool IsReady() const { + return ShardedWriteController->IsReady(); + } - ui64 GetOutputIndex() const final { - return OutputIndex; + bool IsEmpty() const { + return ShardedWriteController->IsEmpty(); } - const NYql::NDq::TDqAsyncStats& GetEgressStats() const final { - return EgressStats; + bool IsOlap() const { + YQL_ENSURE(SchemeEntry); + return SchemeEntry->Kind == NSchemeCache::TSchemeCacheNavigate::KindColumnTable; } - i64 GetFreeSpace() const final { - const i64 result = (ShardedWriteController && !IsResolving()) - ? MemoryLimit - ShardedWriteController->GetMemory() - : std::numeric_limits::min(); // Can't use zero here because compute can use overcommit! - return result; + TVector GetLocks() const { + return TxManager->GetLocks(); } - TMaybe ExtraData() override { - NKikimrKqp::TEvKqpOutputActorResultInfo resultInfo; - for (const auto& [_, lockInfo] : LocksInfo) { - if (const auto& lock = lockInfo.GetLock(); lock) { - resultInfo.AddLocks()->CopyFrom(*lock); - } - } - google::protobuf::Any result; - result.PackFrom(resultInfo); - return result; + TVector GetShardsIds() const { + return ShardedWriteController->GetShardsIds(); } - void SendData(NMiniKQL::TUnboxedValueBatch&& data, i64 size, const TMaybe&, bool finished) final { - YQL_ENSURE(!data.IsWide(), "Wide stream is not supported yet"); - YQL_ENSURE(!Finished); - Finished = finished; - EgressStats.Resume(); + std::optional GetShardsCount() const { + return InconsistentTx + ? std::nullopt + : std::optional(ShardedWriteController->GetShardsCount()); + } - CA_LOG_D("New data: size=" << size << ", finished=" << finished << ", used memory=" << ShardedWriteController->GetMemory() << "."); + using TWriteToken = IShardedWriteController::TWriteToken; + + TWriteToken Open( + NKikimrDataEvents::TEvWrite::TOperation::EOperationType operationType, + TVector&& columnsMetadata, + i64 priority) { + YQL_ENSURE(!Closed); + auto token = ShardedWriteController->Open( + TableId, + operationType, + std::move(columnsMetadata), + priority); + CA_LOG_D("Open: token=" << token); + return token; + } + void Write(TWriteToken token, const NMiniKQL::TUnboxedValueBatch& data) { + YQL_ENSURE(!data.IsWide(), "Wide stream is not supported yet"); + YQL_ENSURE(!Closed); YQL_ENSURE(ShardedWriteController); + CA_LOG_D("Write: token=" << token); try { - ShardedWriteController->AddData(std::move(data)); - if (Finished) { - ShardedWriteController->Close(); - } + ShardedWriteController->Write(token, data); + UpdateShards(); + } catch (...) { + RuntimeError( + CurrentExceptionMessage(), + NYql::NDqProto::StatusIds::INTERNAL_ERROR); + } + } + + void Close(TWriteToken token) { + YQL_ENSURE(!Closed); + YQL_ENSURE(ShardedWriteController); + CA_LOG_D("Close: token=" << token); + try { + ShardedWriteController->Close(token); + UpdateShards(); } catch (...) { RuntimeError( CurrentExceptionMessage(), NYql::NDqProto::StatusIds::INTERNAL_ERROR); } - ProcessBatches(); } - STFUNC(StateFunc) { + void Close() { + YQL_ENSURE(!Closed); + YQL_ENSURE(ShardedWriteController); + YQL_ENSURE(ShardedWriteController->IsAllWritesClosed()); + Closed = true; + ShardedWriteController->Close(); + } + + void UpdateShards() { + // TODO: Maybe there are better ways to initialize new shards... + for (const auto& shardInfo : ShardedWriteController->GetPendingShards()) { + TxManager->AddShard(shardInfo.ShardId, IsOlap(), TablePath); + IKqpTransactionManager::TActionFlags flags = IKqpTransactionManager::EAction::WRITE; + if (shardInfo.HasRead) { + flags |= IKqpTransactionManager::EAction::READ; + } + TxManager->AddAction(shardInfo.ShardId, flags); + } + } + + bool IsClosed() const { + return Closed; + } + + bool IsFinished() const { + return IsClosed() && ShardedWriteController->IsAllWritesFinished(); + } + + STFUNC(StateProcessing) { try { switch (ev->GetTypeRewrite()) { hFunc(NKikimr::NEvents::TDataEvents::TEvWriteResult, Handle); hFunc(TEvTxProxySchemeCache::TEvNavigateKeySetResult, Handle); hFunc(TEvTxProxySchemeCache::TEvResolveKeySetResult, Handle); hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); - IgnoreFunc(TEvTxUserProxy::TEvAllocateTxIdResult); hFunc(TEvPrivate::TEvShardRequestTimeout, Handle); hFunc(TEvPrivate::TEvResolveRequestPlanned, Handle); IgnoreFunc(TEvInterconnect::TEvNodeConnected); @@ -226,6 +348,16 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } } + STFUNC(StateTerminating) { + try { + switch (ev->GetTypeRewrite()) { + hFunc(TEvPrivate::TEvTerminate, Handle); + } + } catch (const yexception& e) { + CA_LOG_W(e.what()); + } + } + bool IsResolving() const { return ResolveAttempts > 0; } @@ -254,10 +386,10 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu if (ResolveAttempts++ >= MessageSettings.MaxResolveAttempts) { CA_LOG_E(TStringBuilder() - << "Too many table resolve attempts for table " << TableId << "."); + << "Too many table resolve attempts for table `" << TablePath << "` (" << TableId << ")."); RuntimeError( TStringBuilder() - << "Too many table resolve attempts for table `" << Settings.GetTable().GetPath() << "`.", + << "Too many table resolve attempts for table `" << TablePath << "`.", NYql::NDqProto::StatusIds::SCHEME_ERROR); return; } @@ -272,14 +404,15 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu entry.ShowPrivatePath = true; request->ResultSet.emplace_back(entry); - WriteActorStateSpan = NWilson::TSpan(TWilsonKqp::WriteActorTableNavigate, WriteActorSpan.GetTraceId(), + TableWriteActorStateSpan = NWilson::TSpan(TWilsonKqp::TableWriteActorTableNavigate, TableWriteActorSpan.GetTraceId(), "WaitForShardsResolve", NWilson::EFlags::AUTO_END); - Send(MakeSchemeCacheID(), new TEvTxProxySchemeCache::TEvInvalidateTable(TableId, {}), 0, 0, WriteActorSpan.GetTraceId()); - Send(MakeSchemeCacheID(), new TEvTxProxySchemeCache::TEvNavigateKeySet(request), 0, 0, WriteActorSpan.GetTraceId()); + Send(MakeSchemeCacheID(), new TEvTxProxySchemeCache::TEvInvalidateTable(TableId, {}), 0, 0, TableWriteActorStateSpan.GetTraceId()); + Send(MakeSchemeCacheID(), new TEvTxProxySchemeCache::TEvNavigateKeySet(request), 0, 0, TableWriteActorStateSpan.GetTraceId()); } void Handle(TEvTxProxySchemeCache::TEvNavigateKeySetResult::TPtr& ev) { + YQL_ENSURE(!SchemeRequest || InconsistentTx); auto& resultSet = ev->Get()->Request->ResultSet; YQL_ENSURE(resultSet.size() == 1); @@ -302,7 +435,6 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } if (SchemeEntry->Kind == NSchemeCache::TSchemeCacheNavigate::KindColumnTable) { - YQL_ENSURE(!ImmediateTx); Prepare(); } else { ResolveShards(); @@ -335,7 +467,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu request->ResultSet.emplace_back(std::move(keyRange)); TAutoPtr resolveReq(new TEvTxProxySchemeCache::TEvResolveKeySet(request)); - Send(MakeSchemeCacheID(), resolveReq.Release(), 0, 0, WriteActorSpan.GetTraceId()); + Send(MakeSchemeCacheID(), resolveReq.Release(), 0, 0, TableWriteActorStateSpan.GetTraceId()); } void Handle(TEvTxProxySchemeCache::TEvResolveKeySetResult::TPtr& ev) { @@ -375,8 +507,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu return builder; }() << ", Cookie=" << ev->Cookie); - - + UpdateStats(ev->Get()->Record.GetTxStats()); switch (ev->Get()->GetStatus()) { case NKikimrDataEvents::TEvWriteResult::STATUS_UNSPECIFIED: { @@ -387,14 +518,15 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << getIssues().ToOneLineString()); RuntimeError( TStringBuilder() << "Unspecified error for table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::UNSPECIFIED, getIssues()); return; } case NKikimrDataEvents::TEvWriteResult::STATUS_PREPARED: { - YQL_ENSURE(false); + ProcessWritePreparedShard(ev); + return; } case NKikimrDataEvents::TEvWriteResult::STATUS_COMPLETED: { ProcessWriteCompletedShard(ev); @@ -408,7 +540,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << getIssues().ToOneLineString()); RuntimeError( TStringBuilder() << "Aborted for table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::ABORTED, getIssues()); @@ -420,7 +552,6 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << " ShardID=" << ev->Get()->Record.GetOrigin() << "," << " Sink=" << this->SelfId() << "." << getIssues().ToOneLineString()); - // TODO: Add new status for splits in datashard. This is tmp solution. if (getIssues().ToOneLineString().Contains("in a pre/offline state assuming this is due to a finished split (wrong shard state)")) { ResetShardRetries(ev->Get()->Record.GetOrigin(), ev->Cookie); @@ -428,7 +559,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } else { RuntimeError( TStringBuilder() << "Internal error for table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::INTERNAL_ERROR, getIssues()); @@ -441,12 +572,12 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << " ShardID=" << ev->Get()->Record.GetOrigin() << "," << " Sink=" << this->SelfId() << "." << getIssues().ToOneLineString()); - - RuntimeError( - TStringBuilder() << "Got DISK_SPACE_EXHAUSTED for table `" - << SchemeEntry->TableId.PathId.ToString() << "`.", - NYql::NDqProto::StatusIds::PRECONDITION_FAILED, - getIssues()); + RuntimeError( + TStringBuilder() << "Disk space exhausted for table `" + << TablePath << "`. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::UNAVAILABLE, + getIssues()); return; } case NKikimrDataEvents::TEvWriteResult::STATUS_OVERLOADED: { @@ -460,7 +591,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu if (!InconsistentTx) { RuntimeError( TStringBuilder() << "Tablet " << ev->Get()->Record.GetOrigin() << " is overloaded. Table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::OVERLOADED, getIssues()); @@ -475,7 +606,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << getIssues().ToOneLineString()); RuntimeError( TStringBuilder() << "Cancelled request to table `" - << SchemeEntry->TableId.PathId.ToString() << "`." + << TablePath << "`." << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::CANCELLED, getIssues()); @@ -489,7 +620,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << getIssues().ToOneLineString()); RuntimeError( TStringBuilder() << "Bad request. Table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::BAD_REQUEST, getIssues()); @@ -507,7 +638,7 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } else { RuntimeError( TStringBuilder() << "Scheme changed. Table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::SCHEME_ERROR, getIssues()); @@ -520,9 +651,12 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu << " ShardID=" << ev->Get()->Record.GetOrigin() << "," << " Sink=" << this->SelfId() << "." << getIssues().ToOneLineString()); + + TxManager->BreakLock(ev->Get()->Record.GetOrigin()); + YQL_ENSURE(TxManager->BrokenLocks()); RuntimeError( TStringBuilder() << "Transaction locks invalidated. Table `" - << SchemeEntry->TableId.PathId.ToString() << "`. " + << TablePath << "`. " << getIssues().ToOneLineString(), NYql::NDqProto::StatusIds::ABORTED, getIssues()); @@ -531,10 +665,36 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } } + void ProcessWritePreparedShard(NKikimr::NEvents::TDataEvents::TEvWriteResult::TPtr& ev) { + YQL_ENSURE(Mode == EMode::PREPARE); + const auto& record = ev->Get()->Record; + IKqpTransactionManager::TPrepareResult preparedInfo; + preparedInfo.ShardId = record.GetOrigin(); + preparedInfo.MinStep = record.GetMinStep(); + preparedInfo.MaxStep = record.GetMaxStep(); + + preparedInfo.Coordinator = 0; + if (record.DomainCoordinatorsSize()) { + auto domainCoordinators = TCoordinators(TVector(record.GetDomainCoordinators().begin(), + record.GetDomainCoordinators().end())); + preparedInfo.Coordinator = domainCoordinators.Select(*TxId); + } + + OnMessageReceived(ev->Get()->Record.GetOrigin()); + const auto result = ShardedWriteController->OnMessageAcknowledged( + ev->Get()->Record.GetOrigin(), ev->Cookie); + if (result) { + YQL_ENSURE(result->IsShardEmpty); + Callbacks->OnPrepared(std::move(preparedInfo), result->DataSize); + } + } + void ProcessWriteCompletedShard(NKikimr::NEvents::TDataEvents::TEvWriteResult::TPtr& ev) { + YQL_ENSURE(SchemeEntry); CA_LOG_D("Got completed result TxId=" << ev->Get()->Record.GetTxId() << ", TabletId=" << ev->Get()->Record.GetOrigin() << ", Cookie=" << ev->Cookie + << ", Mode=" << static_cast(Mode) << ", Locks=" << [&]() { TStringBuilder builder; for (const auto& lock : ev->Get()->Record.GetTxLocks()) { @@ -543,116 +703,147 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu return builder; }()); - OnMessageAcknowledged(ev->Get()->Record.GetOrigin(), ev->Cookie); - for (const auto& lock : ev->Get()->Record.GetTxLocks()) { - if (!LocksInfo[ev->Get()->Record.GetOrigin()].AddAndCheckLock(lock)) { + Y_ABORT_UNLESS(Mode == EMode::WRITE); + if (!TxManager->AddLock(ev->Get()->Record.GetOrigin(), lock)) { + YQL_ENSURE(TxManager->BrokenLocks()); + NYql::TIssues issues; + issues.AddIssue(*TxManager->GetLockIssue()); RuntimeError( TStringBuilder() << "Transaction locks invalidated. Table `" - << SchemeEntry->TableId.PathId.ToString() << "`.", + << TablePath << "`.", NYql::NDqProto::StatusIds::ABORTED, - NYql::TIssues{}); + issues); + return; } } - ProcessBatches(); - } - - void OnMessageAcknowledged(ui64 shardId, ui64 cookie) { - TResumeNotificationManager resumeNotificator(*this); - const auto removedDataSize = ShardedWriteController->OnMessageAcknowledged(shardId, cookie); - if (removedDataSize) { - EgressStats.Bytes += *removedDataSize; - EgressStats.Chunks++; - EgressStats.Splits++; - EgressStats.Resume(); + if (Mode == EMode::COMMIT) { + Callbacks->OnCommitted(ev->Get()->Record.GetOrigin(), 0); + return; + } - if (auto it = SendTime.find(shardId); it != std::end(SendTime)) { - Counters->WriteActorWritesLatencyHistogram->Collect((TInstant::Now() - it->second).MilliSeconds()); - SendTime.erase(it); - } + OnMessageReceived(ev->Get()->Record.GetOrigin()); + const auto result = ShardedWriteController->OnMessageAcknowledged( + ev->Get()->Record.GetOrigin(), ev->Cookie); + if (result && result->IsShardEmpty && Mode == EMode::IMMEDIATE_COMMIT) { + Callbacks->OnCommitted(ev->Get()->Record.GetOrigin(), result->DataSize); + } else if (result) { + Callbacks->OnMessageAcknowledged(result->DataSize); } - resumeNotificator.CheckMemory(); } - void ProcessBatches() { - if (!ImmediateTx || Finished || GetFreeSpace() <= 0) { - SendBatchesToShards(); + void OnMessageReceived(const ui64 shardId) { + if (auto it = SendTime.find(shardId); it != std::end(SendTime)) { + Counters->WriteActorWritesLatencyHistogram->Collect((TInstant::Now() - it->second).MilliSeconds()); + SendTime.erase(it); } + } - if (Finished && ShardedWriteController->IsFinished()) { - CA_LOG_D("Write actor finished"); - Callbacks->OnAsyncOutputFinished(GetOutputIndex()); + void SetPrepare(ui64 txId) { + CA_LOG_D("SetPrepare; txId=" << txId); + YQL_ENSURE(Mode == EMode::WRITE); + Mode = EMode::PREPARE; + TxId = txId; + ShardedWriteController->AddCoveringMessages(); + } + + void SetDistributedCommit() { + CA_LOG_D("SetDistributedCommit; txId=" << *TxId); + YQL_ENSURE(Mode == EMode::PREPARE); + Mode = EMode::COMMIT; + } + + void SetImmediateCommit() { + CA_LOG_D("SetImmediateCommit"); + YQL_ENSURE(Mode == EMode::WRITE); + Mode = EMode::IMMEDIATE_COMMIT; + + if (ShardedWriteController->GetShardsCount() == 1) { + ShardedWriteController->AddCoveringMessages(); + } else { + YQL_ENSURE(ShardedWriteController->GetShardsCount() == 0); } } - void SendBatchesToShards() { - for (const size_t shardId : ShardedWriteController->GetPendingShards()) { - SendDataToShard(shardId); + void FlushBuffers() { + ShardedWriteController->FlushBuffers(); + UpdateShards(); + } + + void Flush() { + for (const auto& shardInfo : ShardedWriteController->GetPendingShards()) { + SendDataToShard(shardInfo.ShardId); } } void SendDataToShard(const ui64 shardId) { + YQL_ENSURE(Mode != EMode::COMMIT); + const auto metadata = ShardedWriteController->GetMessageMetadata(shardId); YQL_ENSURE(metadata); if (metadata->SendAttempts >= MessageSettings.MaxWriteAttempts) { CA_LOG_E("ShardId=" << shardId - << " for table '" << Settings.GetTable().GetPath() + << " for table '" << TablePath << "': retry limit exceeded." << " Sink=" << this->SelfId() << "."); RuntimeError( TStringBuilder() << "ShardId=" << shardId - << " for table '" << Settings.GetTable().GetPath() + << " for table '" << TablePath << "': retry limit exceeded.", NYql::NDqProto::StatusIds::UNAVAILABLE); return; } - auto evWrite = std::make_unique( - NKikimrDataEvents::TEvWrite::MODE_IMMEDIATE); + + const bool isPrepare = metadata->IsFinal && Mode == EMode::PREPARE; + const bool isImmediateCommit = metadata->IsFinal && Mode == EMode::IMMEDIATE_COMMIT; + + auto evWrite = std::make_unique(); + + evWrite->Record.SetTxMode(isPrepare + ? (TxManager->IsVolatile() + ? NKikimrDataEvents::TEvWrite::MODE_VOLATILE_PREPARE + : NKikimrDataEvents::TEvWrite::MODE_PREPARE) + : NKikimrDataEvents::TEvWrite::MODE_IMMEDIATE); - if (ImmediateTx && FinalTx && Finished && metadata->IsFinal) { - // Last immediate write (only for datashard) - if (LocksInfo[shardId].GetLock()) { - // multi immediate evwrite - auto* locks = evWrite->Record.MutableLocks(); - locks->SetOp(NKikimrDataEvents::TKqpLocks::Commit); - locks->AddSendingShards(shardId); - locks->AddReceivingShards(shardId); - *locks->AddLocks() = *LocksInfo.at(shardId).GetLock(); + if (isImmediateCommit) { + const auto locks = TxManager->GetLocks(shardId); + if (!locks.empty()) { + auto* protoLocks = evWrite->Record.MutableLocks(); + protoLocks->SetOp(NKikimrDataEvents::TKqpLocks::Commit); + protoLocks->AddSendingShards(shardId); + protoLocks->AddReceivingShards(shardId); + for (const auto& lock : locks) { + *protoLocks->AddLocks() = lock; + } } + } else if (isPrepare) { + YQL_ENSURE(TxId); + FillEvWritePrepare(evWrite.get(), shardId, *TxId, TxManager); } else if (!InconsistentTx) { - evWrite->SetLockId(Settings.GetLockTxId(), Settings.GetLockNodeId()); + evWrite->SetLockId(LockTxId, LockNodeId); } const auto serializationResult = ShardedWriteController->SerializeMessageToPayload(shardId, *evWrite); - YQL_ENSURE(serializationResult.TotalDataSize > 0); - - for (size_t payloadIndex : serializationResult.PayloadIndexes) { - evWrite->AddOperation( - GetOperation(), - { - Settings.GetTable().GetOwnerId(), - Settings.GetTable().GetTableId(), - Settings.GetTable().GetVersion(), - }, - ShardedWriteController->GetWriteColumnIds(), - payloadIndex, - ShardedWriteController->GetDataFormat()); - } + YQL_ENSURE(isPrepare || isImmediateCommit || serializationResult.TotalDataSize > 0); if (metadata->SendAttempts == 0) { - Counters->WriteActorImmediateWrites->Inc(); + if (!isPrepare) { + Counters->WriteActorImmediateWrites->Inc(); + } else { + Counters->WriteActorPrepareWrites->Inc(); + } Counters->WriteActorWritesSizeHistogram->Collect(serializationResult.TotalDataSize); Counters->WriteActorWritesOperationsHistogram->Collect(metadata->OperationsCount); SendTime[shardId] = TInstant::Now(); } else { + YQL_ENSURE(!isPrepare); Counters->WriteActorImmediateWritesRetries->Inc(); } - CA_LOG_D("Send EvWrite to ShardID=" << shardId << ", TxId=" << evWrite->Record.GetTxId() - << ", TxMode=" << evWrite->Record.GetTxMode() + CA_LOG_D("Send EvWrite to ShardID=" << shardId << ", isPrepare=" << isPrepare << ", isImmediateCommit=" << isImmediateCommit << ", TxId=" << evWrite->Record.GetTxId() << ", LockTxId=" << evWrite->Record.GetLockTxId() << ", LockNodeId=" << evWrite->Record.GetLockNodeId() << ", Locks= " << [&]() { TStringBuilder builder; @@ -662,13 +853,14 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu return builder; }() << ", Size=" << serializationResult.TotalDataSize << ", Cookie=" << metadata->Cookie - << ", OperationsCount=" << metadata->OperationsCount << ", IsFinal=" << metadata->IsFinal - << ", Attempts=" << metadata->SendAttempts); + << ", OperationsCount=" << evWrite->Record.OperationsSize() << ", IsFinal=" << metadata->IsFinal + << ", Attempts=" << metadata->SendAttempts << ", Mode=" << static_cast(Mode)); Send( PipeCacheId, - new TEvPipeCache::TEvForward(evWrite.release(), shardId, true), - 0, - metadata->Cookie); + new TEvPipeCache::TEvForward(evWrite.release(), shardId, /* subscribe */ true), + IEventHandle::FlagTrackDelivery, + metadata->Cookie, + TableWriteActorSpan.GetTraceId()); ShardedWriteController->OnMessageSent(shardId, metadata->Cookie); @@ -684,25 +876,6 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } } - NKikimrDataEvents::TEvWrite::TOperation::EOperationType GetOperation() { - switch (Settings.GetType()) { - case NKikimrKqp::TKqpTableSinkSettings::MODE_REPLACE: - return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_REPLACE; - case NKikimrKqp::TKqpTableSinkSettings::MODE_UPSERT: - return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPSERT; - case NKikimrKqp::TKqpTableSinkSettings::MODE_INSERT: - return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_INSERT; - case NKikimrKqp::TKqpTableSinkSettings::MODE_DELETE: - return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_DELETE; - case NKikimrKqp::TKqpTableSinkSettings::MODE_UPDATE: - return NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPDATE; - default: - RuntimeError( - TStringBuilder() << "Unknown operation.", - NYql::NDqProto::StatusIds::INTERNAL_ERROR); - } - } - void RetryShard(const ui64 shardId, const std::optional ifCookieEqual) { const auto metadata = ShardedWriteController->GetMessageMetadata(shardId); if (!metadata || (ifCookieEqual && metadata->Cookie != ifCookieEqual)) { @@ -724,6 +897,10 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu RetryShard(ev->Get()->ShardId, ev->Cookie); } + void Handle(TEvPrivate::TEvTerminate::TPtr&) { + PassAway(); + } + void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr& ev) { CA_LOG_W("TEvDeliveryProblem was received from tablet: " << ev->Get()->TabletId); if (InconsistentTx) { @@ -737,122 +914,1443 @@ class TKqpDirectWriteActor : public TActorBootstrapped, pu } } - void RuntimeError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues = {}) { - NYql::TIssue issue(message); - for (const auto& i : subIssues) { - issue.AddSubIssue(MakeIntrusive(i)); - } - - NYql::TIssues issues; - issues.AddIssue(std::move(issue)); - - if (WriteActorStateSpan) { - WriteActorStateSpan.EndError(issues.ToOneLineString()); - } - if (WriteActorSpan) { - WriteActorSpan.EndError(issues.ToOneLineString()); - } - - Callbacks->OnAsyncOutputError(OutputIndex, std::move(issues), statusCode); - } - - void PassAway() override { - Send(PipeCacheId, new TEvPipeCache::TEvUnlink(0)); - TActorBootstrapped::PassAway(); - } - void Prepare() { - WriteActorStateSpan.EndOk(); - + TableWriteActorStateSpan.EndOk(); YQL_ENSURE(SchemeEntry); ResolveAttempts = 0; - if (!ShardedWriteController) { - TVector columnsMetadata; - columnsMetadata.reserve(Settings.GetColumns().size()); - for (const auto & column : Settings.GetColumns()) { - columnsMetadata.push_back(column); - } - - try { - ShardedWriteController = CreateShardedWriteController( - TShardedWriteControllerSettings { - .MemoryLimitTotal = MessageSettings.InFlightMemoryLimitPerActorBytes, - .MemoryLimitPerMessage = MessageSettings.MemoryLimitPerMessageBytes, - .MaxBatchesPerMessage = (SchemeEntry->Kind == NSchemeCache::TSchemeCacheNavigate::KindColumnTable - ? 1 - : MessageSettings.MaxBatchesPerMessage), - }, - std::move(columnsMetadata), - TypeEnv, - Alloc); - } catch (...) { - RuntimeError( - CurrentExceptionMessage(), - NYql::NDqProto::StatusIds::INTERNAL_ERROR); - } - } - try { if (SchemeEntry->Kind == NSchemeCache::TSchemeCacheNavigate::KindColumnTable) { ShardedWriteController->OnPartitioningChanged(*SchemeEntry); } else { ShardedWriteController->OnPartitioningChanged(*SchemeEntry, std::move(*SchemeRequest)); + SchemeRequest.reset(); } - ResumeExecution(); } catch (...) { RuntimeError( CurrentExceptionMessage(), NYql::NDqProto::StatusIds::INTERNAL_ERROR); } - ProcessBatches(); + + Callbacks->OnReady(); } - void ResumeExecution() { - CA_LOG_D("Resuming execution."); - Callbacks->ResumeExecution(); + void RuntimeError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues = {}) { + if (TableWriteActorStateSpan) { + TableWriteActorStateSpan.EndError(message); + } + if (TableWriteActorSpan) { + TableWriteActorSpan.EndError(message); + } + + Callbacks->OnError(message, statusCode, subIssues); + } + + void PassAway() override {; + CA_LOG_D("PassAway"); + Send(PipeCacheId, new TEvPipeCache::TEvUnlink(0)); + TActorBootstrapped::PassAway(); + } + + void Terminate() { + Become(&TKqpTableWriteActor::StateTerminating); + Send(this->SelfId(), new TEvPrivate::TEvTerminate{}); + } + + void UpdateStats(const NKikimrQueryStats::TTxStats& txStats) { + for (const auto& tableAccessStats : txStats.GetTableAccessStats()) { + YQL_ENSURE(tableAccessStats.GetTableInfo().GetPathId() == TableId.PathId.LocalPathId); + Stats.ReadRows += tableAccessStats.GetSelectRow().GetRows(); + Stats.ReadRows += tableAccessStats.GetSelectRange().GetRows(); + Stats.ReadBytes += tableAccessStats.GetSelectRow().GetBytes(); + Stats.ReadBytes += tableAccessStats.GetSelectRange().GetBytes(); + Stats.WriteRows += tableAccessStats.GetUpdateRow().GetRows(); + Stats.WriteBytes += tableAccessStats.GetUpdateRow().GetBytes(); + Stats.EraseRows += tableAccessStats.GetEraseRow().GetRows(); + Stats.EraseBytes += tableAccessStats.GetEraseRow().GetRows(); + } + + for (const auto& perShardStats : txStats.GetPerShardStats()) { + Stats.AffectedPartitions.insert(perShardStats.GetShardId()); + } + } + + void FillStats(NYql::NDqProto::TDqTaskStats* stats) { + NYql::NDqProto::TDqTableStats* tableStats = nullptr; + for (size_t i = 0; i < stats->TablesSize(); ++i) { + auto* table = stats->MutableTables(i); + if (table->GetTablePath() == TablePath) { + tableStats = table; + } + } + if (!tableStats) { + tableStats = stats->AddTables(); + tableStats->SetTablePath(TablePath); + } + + tableStats->SetReadRows(tableStats->GetReadRows() + Stats.ReadRows); + tableStats->SetReadBytes(tableStats->GetReadBytes() + Stats.ReadBytes); + tableStats->SetWriteRows(tableStats->GetWriteRows() + Stats.WriteRows); + tableStats->SetWriteBytes(tableStats->GetWriteBytes() + Stats.WriteBytes); + tableStats->SetEraseRows(tableStats->GetEraseRows() + Stats.EraseRows); + tableStats->SetEraseBytes(tableStats->GetEraseBytes() + Stats.EraseBytes); + + Stats.ReadRows = 0; + Stats.ReadBytes = 0; + Stats.WriteRows = 0; + Stats.WriteBytes = 0; + Stats.EraseRows = 0; + Stats.EraseBytes = 0; + + tableStats->SetAffectedPartitions( + tableStats->GetAffectedPartitions() + Stats.AffectedPartitions.size()); + Stats.AffectedPartitions.clear(); } NActors::TActorId PipeCacheId = NKikimr::MakePipePerNodeCacheID(false); TString LogPrefix; - const NKikimrKqp::TKqpTableSinkSettings Settings; TWriteActorSettings MessageSettings; - const ui64 OutputIndex; - NYql::NDq::TDqAsyncStats EgressStats; - NYql::NDq::IDqComputeActorAsyncOutput::ICallbacks * Callbacks = nullptr; - TIntrusivePtr Counters; const NMiniKQL::TTypeEnvironment& TypeEnv; std::shared_ptr Alloc; - const NYql::NDq::TTxId TxId; const TTableId TableId; - const bool FinalTx; - const bool ImmediateTx; + const TString TablePath; + + std::optional TxId; + const ui64 LockTxId; + const ui64 LockNodeId; const bool InconsistentTx; + IKqpTableWriterCallbacks* Callbacks; + std::optional SchemeEntry; std::optional SchemeRequest; ui64 ResolveAttempts = 0; + IKqpTransactionManagerPtr TxManager; + bool Closed = false; + EMode Mode = EMode::WRITE; THashMap SendTime; - THashMap LocksInfo; - bool Finished = false; - - const i64 MemoryLimit; IShardedWriteControllerPtr ShardedWriteController = nullptr; - NWilson::TSpan WriteActorSpan; - NWilson::TSpan WriteActorStateSpan; + TIntrusivePtr Counters; + + TKqpTableWriterStatistics Stats; + + NWilson::TSpan TableWriteActorSpan; + NWilson::TSpan TableWriteActorStateSpan; }; -void RegisterKqpWriteActor(NYql::NDq::TDqAsyncIoFactory& factory, TIntrusivePtr counters) { - factory.RegisterSink( - TString(NYql::KqpTableSinkName), - [counters] (NKikimrKqp::TKqpTableSinkSettings&& settings, NYql::NDq::TDqAsyncIoFactory::TSinkArguments&& args) { - auto* actor = new TKqpDirectWriteActor(std::move(settings), std::move(args), counters); - return std::make_pair(actor, actor); +class TKqpDirectWriteActor : public TActorBootstrapped, public NYql::NDq::IDqComputeActorAsyncOutput, public IKqpTableWriterCallbacks { + using TBase = TActorBootstrapped; + +public: + TKqpDirectWriteActor( + NKikimrKqp::TKqpTableSinkSettings&& settings, + NYql::NDq::TDqAsyncIoFactory::TSinkArguments&& args, + TIntrusivePtr counters) + : LogPrefix(TStringBuilder() << "TxId: " << args.TxId << ", task: " << args.TaskId << ". ") + , Settings(std::move(settings)) + , MessageSettings(GetWriteActorSettings()) + , OutputIndex(args.OutputIndex) + , Callbacks(args.Callback) + , Counters(counters) + , TypeEnv(args.TypeEnv) + , Alloc(args.Alloc) + , TxId(std::get(args.TxId)) + , TableId( + Settings.GetTable().GetOwnerId(), + Settings.GetTable().GetTableId(), + Settings.GetTable().GetVersion()) + , DirectWriteActorSpan(TWilsonKqp::DirectWriteActor, NWilson::TTraceId(args.TraceId), "TKqpDirectWriteActor") + { + EgressStats.Level = args.StatsLevel; + } + + void Bootstrap() { + LogPrefix = TStringBuilder() << "SelfId: " << this->SelfId() << ", " << LogPrefix; + + WriteTableActor = new TKqpTableWriteActor( + this, + TableId, + Settings.GetTable().GetPath(), + Settings.GetLockTxId(), + Settings.GetLockNodeId(), + Settings.GetInconsistentTx(), + TypeEnv, + Alloc, + nullptr, + TActorId{}, + Counters, + DirectWriteActorSpan.GetTraceId()); + + WriteTableActorId = RegisterWithSameMailbox(WriteTableActor); + + TVector columnsMetadata; + columnsMetadata.reserve(Settings.GetColumns().size()); + for (const auto & column : Settings.GetColumns()) { + columnsMetadata.push_back(column); + } + YQL_ENSURE(Settings.GetPriority() == 0); + WriteToken = WriteTableActor->Open(GetOperation(Settings.GetType()), std::move(columnsMetadata), Settings.GetPriority()); + WaitingForTableActor = true; + } + + static constexpr char ActorName[] = "KQP_DIRECT_WRITE_ACTOR"; + +private: + virtual ~TKqpDirectWriteActor() { + } + + void CommitState(const NYql::NDqProto::TCheckpoint&) final {}; + void LoadState(const NYql::NDq::TSinkState&) final {}; + + ui64 GetOutputIndex() const final { + return OutputIndex; + } + + const NYql::NDq::TDqAsyncStats& GetEgressStats() const final { + return EgressStats; + } + + i64 GetFreeSpace() const final { + return (WriteTableActor && WriteTableActor->IsReady()) + ? MessageSettings.InFlightMemoryLimitPerActorBytes - GetMemory() + : std::numeric_limits::min(); // Can't use zero here because compute can use overcommit! + } + + i64 GetMemory() const { + return (WriteTableActor && WriteTableActor->IsReady()) + ? WriteTableActor->GetMemory() + : 0; + } + + TMaybe ExtraData() override { + if (!WriteTableActor) { + return {}; + } + NKikimrKqp::TEvKqpOutputActorResultInfo resultInfo; + for (const auto& lock : WriteTableActor->GetLocks()) { + resultInfo.AddLocks()->CopyFrom(lock); + } + resultInfo.SetHasRead( + GetOperation(Settings.GetType()) == NKikimrDataEvents::TEvWrite::TOperation::OPERATION_INSERT || + GetOperation(Settings.GetType()) == NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPDATE); + google::protobuf::Any result; + result.PackFrom(resultInfo); + return result; + } + + void SendData(NMiniKQL::TUnboxedValueBatch&& data, i64 size, const TMaybe&, bool finished) final { + YQL_ENSURE(!data.IsWide(), "Wide stream is not supported yet"); + YQL_ENSURE(!Closed); + Closed = finished; + EgressStats.Resume(); + Y_UNUSED(size); + + YQL_ENSURE(WriteTableActor); + WriteTableActor->Write(*WriteToken, data); + if (Closed) { + WriteTableActor->Close(*WriteToken); + WriteTableActor->Close(); + } + Process(); + } + + void Process() { + if (GetFreeSpace() <= 0) { + WaitingForTableActor = true; + } else if (WaitingForTableActor && GetFreeSpace() > MessageSettings.InFlightMemoryLimitPerActorBytes / 2) { + ResumeExecution(); + } + + if (Closed || GetFreeSpace() <= 0) { + WriteTableActor->Flush(); + } + + if (Closed && WriteTableActor->IsFinished()) { + CA_LOG_D("Write actor finished"); + Callbacks->OnAsyncOutputFinished(GetOutputIndex()); + } + } + + void RuntimeError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues = {}) { + DirectWriteActorSpan.EndError(message); + + NYql::TIssue issue(message); + for (const auto& i : subIssues) { + issue.AddSubIssue(MakeIntrusive(i)); + } + + NYql::TIssues issues; + issues.AddIssue(std::move(issue)); + + Callbacks->OnAsyncOutputError(OutputIndex, std::move(issues), statusCode); + } + + void PassAway() override { + if (WriteTableActor) { + WriteTableActor->Terminate(); + } + TActorBootstrapped::PassAway(); + } + + void ResumeExecution() { + CA_LOG_D("Resuming execution."); + WaitingForTableActor = false; + Callbacks->ResumeExecution(); + } + + void OnReady() override { + Process(); + } + + void OnPrepared(IKqpTransactionManager::TPrepareResult&&, ui64) override { + AFL_ENSURE(false); + } + + void OnCommitted(ui64, ui64) override { + AFL_ENSURE(false); + } + + void OnMessageAcknowledged(ui64 dataSize) override { + EgressStats.Bytes += dataSize; + EgressStats.Chunks++; + EgressStats.Splits++; + EgressStats.Resume(); + Process(); + } + + void OnError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues) override { + RuntimeError(message, statusCode, subIssues); + } + + void FillExtraStats(NYql::NDqProto::TDqTaskStats* stats, bool last, const NYql::NDq::TDqMeteringStats*) override { + if (last && WriteTableActor) { + WriteTableActor->FillStats(stats); + } + } + + TString LogPrefix; + const NKikimrKqp::TKqpTableSinkSettings Settings; + TWriteActorSettings MessageSettings; + const ui64 OutputIndex; + NYql::NDq::TDqAsyncStats EgressStats; + NYql::NDq::IDqComputeActorAsyncOutput::ICallbacks * Callbacks = nullptr; + TIntrusivePtr Counters; + const NMiniKQL::TTypeEnvironment& TypeEnv; + std::shared_ptr Alloc; + + const ui64 TxId; + const TTableId TableId; + TKqpTableWriteActor* WriteTableActor = nullptr; + TActorId WriteTableActorId; + + std::optional WriteToken; + + bool Closed = false; + + bool WaitingForTableActor = false; + + NWilson::TSpan DirectWriteActorSpan; +}; + + +namespace { + +struct TWriteToken { + TTableId TableId; + ui64 Cookie; + + bool IsEmpty() const { + return !TableId; + } +}; + +struct TTransactionSettings { + ui64 TxId = 0; + ui64 LockTxId = 0; + ui64 LockNodeId = 0; + bool InconsistentTx = false; +}; + +struct TWriteSettings { + TTableId TableId; + TString TablePath; // for error messages + NKikimrDataEvents::TEvWrite::TOperation::EOperationType OperationType; + TVector Columns; + TTransactionSettings TransactionSettings; + i64 Priority; +}; + +struct TBufferWriteMessage { + TActorId From; + TWriteToken Token; + bool Close = false; + // TODO: move to serialized data + std::shared_ptr> Data; + std::shared_ptr Alloc; +}; + +struct TEvBufferWrite : public TEventLocal { + bool Close = false; + std::optional Token; + std::optional Settings; + std::shared_ptr> Data; + std::shared_ptr Alloc; + + ~TEvBufferWrite() { + if (Alloc) { + TGuard guard(*Alloc); + Data = nullptr; + } + } +}; + +struct TEvBufferWriteResult : public TEventLocal { + TWriteToken Token; +}; + +} + + +class TKqpBufferWriteActor :public TActorBootstrapped, public IKqpTableWriterCallbacks { + using TBase = TActorBootstrapped; + +public: + enum class EState { + WRITING, // Allow to write data to buffer. + FLUSHING, // Force flush (for uncommitted changes visibility). Can't accept any writes in this state. + PREPARING, // Do preparation for commit. All writers are closed. New writes wouldn't be accepted. + COMMITTING, // Do commit. All writers are closed. New writes wouldn't be accepted. + ROLLINGBACK, // Do rollback. New writes wouldn't be accepted. + FINISHED, + }; + +public: + TKqpBufferWriteActor( + TKqpBufferWriterSettings&& settings) + : SessionActorId(settings.SessionActorId) + , MessageSettings(GetWriteActorSettings()) + , TxManager(settings.TxManager) + , Alloc(std::make_shared(__LOCATION__)) + , TypeEnv(*Alloc) + , Counters(settings.Counters) + , TxProxyMon(settings.TxProxyMon) + , BufferWriteActor(TWilsonKqp::BufferWriteActor, NWilson::TTraceId(settings.TraceId), "TKqpBufferWriteActor", NWilson::EFlags::AUTO_END) + , BufferWriteActorState(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::Writing", NWilson::EFlags::AUTO_END) + { + State = EState::WRITING; + Alloc->Release(); + Counters->BufferActorsCount->Inc(); + } + + void Bootstrap() { + LogPrefix = TStringBuilder() << "SelfId: " << this->SelfId() << ", SessionActorId: " << SessionActorId << ", " << LogPrefix; + Become(&TKqpBufferWriteActor::StateWrite); + } + + static constexpr char ActorName[] = "KQP_BUFFER_WRITE_ACTOR"; + + // TODO: split states + STFUNC(StateWrite) { + try { + switch (ev->GetTypeRewrite()) { + hFunc(TEvKqpBuffer::TEvTerminate, Handle); + hFunc(TEvKqpBuffer::TEvFlush, Handle); + hFunc(TEvKqpBuffer::TEvCommit, Handle); + hFunc(TEvKqpBuffer::TEvRollback, Handle); + hFunc(TEvBufferWrite, Handle); + + hFunc(TEvTxProxy::TEvProposeTransactionStatus, Handle); + hFunc(NKikimr::NEvents::TDataEvents::TEvWriteResult, Handle); + hFunc(TEvPipeCache::TEvDeliveryProblem, Handle); + default: + AFL_ENSURE(false)("unknown message", ev->GetTypeRewrite()); + } + } catch (const yexception& e) { + ReplyErrorAndDie(e.what(), NYql::NDqProto::StatusIds::INTERNAL_ERROR, {}); + } + } + + void Handle(TEvBufferWrite::TPtr& ev) { + TWriteToken token; + if (!ev->Get()->Token) { + AFL_ENSURE(ev->Get()->Settings); + auto& settings = *ev->Get()->Settings; + if (!WriteInfos.empty()) { + AFL_ENSURE(LockTxId == settings.TransactionSettings.LockTxId); + AFL_ENSURE(LockNodeId == settings.TransactionSettings.LockNodeId); + AFL_ENSURE(InconsistentTx == settings.TransactionSettings.InconsistentTx); + } else { + LockTxId = settings.TransactionSettings.LockTxId; + LockNodeId = settings.TransactionSettings.LockNodeId; + InconsistentTx = settings.TransactionSettings.InconsistentTx; + } + + auto& writeInfo = WriteInfos[settings.TableId]; + if (!writeInfo.WriteTableActor) { + writeInfo.WriteTableActor = new TKqpTableWriteActor( + this, + settings.TableId, + settings.TablePath, + LockTxId, + LockNodeId, + InconsistentTx, + TypeEnv, + Alloc, + TxManager, + SessionActorId, + Counters, + BufferWriteActor.GetTraceId()); + writeInfo.WriteTableActorId = RegisterWithSameMailbox(writeInfo.WriteTableActor); + CA_LOG_D("Create new TableWriteActor for table `" << settings.TablePath << "` (" << settings.TableId << "). lockId=" << LockTxId << " " << writeInfo.WriteTableActorId); + } + + auto cookie = writeInfo.WriteTableActor->Open(settings.OperationType, std::move(settings.Columns), settings.Priority); + token = TWriteToken{settings.TableId, cookie}; + } else { + token = *ev->Get()->Token; + } + + auto& queue = DataQueues[token.TableId]; + queue.emplace(); + auto& message = queue.back(); + + message.Token = token; + message.From = ev->Sender; + message.Close = ev->Get()->Close; + message.Data = ev->Get()->Data; + message.Alloc = ev->Get()->Alloc; + + ev->Get()->Data = nullptr; + ev->Get()->Alloc = nullptr; + + Process(); + } + + void Process() { + ProcessRequestQueue(); + ProcessWrite(); + ProcessAckQueue(); + + if (State == EState::FLUSHING) { + bool isEmpty = true; + for (auto& [_, info] : WriteInfos) { + isEmpty = isEmpty && info.WriteTableActor->IsReady() && info.WriteTableActor->IsEmpty(); + } + if (isEmpty) { + OnFlushed(); + } + } + } + + void ProcessRequestQueue() { + for (auto& [tableId, queue] : DataQueues) { + auto& writeInfo = WriteInfos.at(tableId); + + if (!writeInfo.WriteTableActor->IsReady()) { + CA_LOG_D("ProcessRequestQueue " << tableId << " NOT READY queue=" << queue.size()); + return; + } + + while (!queue.empty()) { + auto& message = queue.front(); + + if (!message.Data->empty()) { + for (const auto& data : *message.Data) { + writeInfo.WriteTableActor->Write(message.Token.Cookie, data); + } + } + if (message.Close) { + writeInfo.WriteTableActor->Close(message.Token.Cookie); + } + + AckQueue.push(TAckMessage{ + .ForwardActorId = message.From, + .Token = message.Token, + .DataSize = 0, + }); + + { + TGuard guard(*message.Alloc); + message.Data = nullptr; + } + queue.pop(); + } + } + } + + void ProcessAckQueue() { + while (!AckQueue.empty()) { + const auto& item = AckQueue.front(); + if (GetTotalFreeSpace() >= item.DataSize) { + auto result = std::make_unique(); + result->Token = AckQueue.front().Token; + Send(AckQueue.front().ForwardActorId, result.release()); + AckQueue.pop(); + } else { + YQL_ENSURE(false); + return; + } + } + } + + void ProcessWrite() { + const bool needToFlush = GetTotalFreeSpace() <= 0 + || State == EState::FLUSHING + || State == EState::PREPARING + || State == EState::COMMITTING + || State == EState::ROLLINGBACK; + + if (needToFlush) { + CA_LOG_D("Flush data"); + for (auto& [_, info] : WriteInfos) { + if (info.WriteTableActor->IsReady()) { + info.WriteTableActor->Flush(); + } + } + } + } + + void Flush() { + Counters->BufferActorFlushes->Inc(); + BufferWriteActorState = NWilson::TSpan(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::Flushing", NWilson::EFlags::AUTO_END); + + CA_LOG_D("Start flush"); + YQL_ENSURE(State == EState::WRITING); + State = EState::FLUSHING; + for (auto& [_, queue] : DataQueues) { + YQL_ENSURE(queue.empty()); + } + Process(); + } + + void Prepare(const ui64 txId) { + BufferWriteActorState = NWilson::TSpan(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::Preparing", NWilson::EFlags::AUTO_END); + + CA_LOG_D("Start prepare for distributed commit"); + YQL_ENSURE(State == EState::WRITING); + State = EState::PREPARING; + for (auto& [_, queue] : DataQueues) { + YQL_ENSURE(queue.empty()); + } + TxId = txId; + for (auto& [_, info] : WriteInfos) { + info.WriteTableActor->SetPrepare(txId); + } + Close(); + Process(); + SendToExternalShards(false); + } + + void ImmediateCommit() { + Counters->BufferActorImmediateCommits->Inc(); + BufferWriteActorState = NWilson::TSpan(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::Committing", NWilson::EFlags::AUTO_END); + + CA_LOG_D("Start immediate commit"); + YQL_ENSURE(State == EState::WRITING); + State = EState::COMMITTING; + for (auto& [_, queue] : DataQueues) { + YQL_ENSURE(queue.empty()); + } + for (auto& [_, info] : WriteInfos) { + info.WriteTableActor->SetImmediateCommit(); + } + Close(); + Process(); + } + + void DistributedCommit() { + Counters->BufferActorDistributedCommits->Inc(); + BufferWriteActorState = NWilson::TSpan(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::Committing", NWilson::EFlags::AUTO_END); + + CA_LOG_D("Start distributed commit with TxId=" << *TxId); + YQL_ENSURE(State == EState::PREPARING); + State = EState::COMMITTING; + for (auto& [_, queue] : DataQueues) { + YQL_ENSURE(queue.empty()); + } + for (auto& [_, info] : WriteInfos) { + info.WriteTableActor->SetDistributedCommit(); + } + SendCommitToCoordinator(); + } + + void Rollback() { + Counters->BufferActorRollbacks->Inc(); + BufferWriteActorState = NWilson::TSpan(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::RollingBack", NWilson::EFlags::AUTO_END); + + CA_LOG_D("Start rollback"); + State = EState::ROLLINGBACK; + SendToExternalShards(true); + } + + void SendToExternalShards(bool isRollback) { + THashSet shards = TxManager->GetShards(); + if (!isRollback) { + for (auto& [_, info] : WriteInfos) { + for (const auto& shardId : info.WriteTableActor->GetShardsIds()) { + shards.erase(shardId); + } + } + } + + for (const ui64 shardId : shards) { + if (TxManager->GetLocks(shardId).empty()) { + continue; + } + auto evWrite = std::make_unique(isRollback + ? NKikimrDataEvents::TEvWrite::MODE_IMMEDIATE + : (TxManager->IsVolatile() + ? NKikimrDataEvents::TEvWrite::MODE_VOLATILE_PREPARE + : NKikimrDataEvents::TEvWrite::MODE_PREPARE)); + + if (isRollback) { + FillEvWriteRollback(evWrite.get(), shardId, TxManager); + } else { + YQL_ENSURE(TxId); + FillEvWritePrepare(evWrite.get(), shardId, *TxId, TxManager); + } + + SendTime[shardId] = TInstant::Now(); + CA_LOG_D("Send EvWrite (external) to ShardID=" << shardId << ", isPrepare=" << !isRollback << ", isImmediateCommit=" << isRollback << ", TxId=" << evWrite->Record.GetTxId() + << ", LockTxId=" << evWrite->Record.GetLockTxId() << ", LockNodeId=" << evWrite->Record.GetLockNodeId() + << ", Locks= " << [&]() { + TStringBuilder builder; + for (const auto& lock : evWrite->Record.GetLocks().GetLocks()) { + builder << lock.ShortDebugString(); + } + return builder; + }() + << ", Size=" << 0 << ", Cookie=" << 0 + << ", OperationsCount=" << 0 << ", IsFinal=" << 1 + << ", Attempts=" << 0); + + // TODO: Track latecy + Send( + NKikimr::MakePipePerNodeCacheID(false), + new TEvPipeCache::TEvForward(evWrite.release(), shardId, /* subscribe */ true), + IEventHandle::FlagTrackDelivery, + 0); + } + } + + void SendCommitToCoordinator() { + const auto commitInfo = TxManager->GetCommitInfo(); + + auto ev = MakeHolder(); + + YQL_ENSURE(commitInfo.Coordinator); + ev->Record.SetCoordinatorID(commitInfo.Coordinator); + + auto& transaction = *ev->Record.MutableTransaction(); + auto& affectedSet = *transaction.MutableAffectedSet(); + affectedSet.Reserve(commitInfo.ShardsInfo.size()); + + YQL_ENSURE(TxId); + transaction.SetTxId(*TxId); + transaction.SetMinStep(commitInfo.MinStep); + transaction.SetMaxStep(commitInfo.MaxStep); + if (TxManager->IsVolatile()) { + transaction.SetFlags(TEvTxProxy::TEvProposeTransaction::FlagVolatile); + } + + for (const auto& shardInfo : commitInfo.ShardsInfo) { + auto& item = *affectedSet.Add(); + item.SetTabletId(shardInfo.ShardId); + Y_ABORT_UNLESS(shardInfo.AffectedFlags != 0); + item.SetFlags(shardInfo.AffectedFlags); + } + + //TODO: NDataIntegrity + CA_LOG_D("Execute planned transaction, coordinator: " << commitInfo.Coordinator + << ", volitale: " << ((transaction.GetFlags() & TEvTxProxy::TEvProposeTransaction::FlagVolatile) != 0) + << ", shards: " << affectedSet.size()); + Send(MakePipePerNodeCacheID(false), new TEvPipeCache::TEvForward(ev.Release(), commitInfo.Coordinator, /* subscribe */ true)); + } + + void Close() { + for (auto& [_, info] : WriteInfos) { + if (!info.WriteTableActor->IsClosed()) { + info.WriteTableActor->Close(); + } + } + } + + i64 GetFreeSpace(TWriteToken token) const { + auto& info = WriteInfos.at(token.TableId); + return info.WriteTableActor->IsReady() + ? MessageSettings.InFlightMemoryLimitPerActorBytes - info.WriteTableActor->GetMemory() + : std::numeric_limits::min(); // Can't use zero here because compute can use overcommit! + } + + i64 GetTotalFreeSpace() const { + return MessageSettings.InFlightMemoryLimitPerActorBytes - GetTotalMemory(); + } + + i64 GetTotalMemory() const { + i64 totalMemory = 0; + for (auto& [_, info] : WriteInfos) { + totalMemory += info.WriteTableActor->IsReady() + ? info.WriteTableActor->GetMemory() + : 0; + } + return totalMemory; + } + + THashSet GetShardsIds() const { + THashSet shardIds; + for (auto& [_, info] : WriteInfos) { + for (const auto& id : info.WriteTableActor->GetShardsIds()) { + shardIds.insert(id); + } + } + return shardIds; + } + + void PassAway() override { + for (auto& [_, queue] : DataQueues) { + while (!queue.empty()) { + auto& message = queue.front(); + { + TGuard guard(*message.Alloc); + message.Data = nullptr; + } + queue.pop(); + } + } + + for (auto& [_, info] : WriteInfos) { + if (info.WriteTableActor) { + info.WriteTableActor->Terminate(); + } + } + TActorBootstrapped::PassAway(); + } + + void Handle(TEvTxProxy::TEvProposeTransactionStatus::TPtr &ev) { + TEvTxProxy::TEvProposeTransactionStatus* res = ev->Get(); + CA_LOG_D("Got transaction status, status: " << res->GetStatus()); + + switch (res->GetStatus()) { + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusAccepted: + TxProxyMon->ClientTxStatusAccepted->Inc(); + break; + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusProcessed: + TxProxyMon->ClientTxStatusProcessed->Inc(); + break; + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusConfirmed: + TxProxyMon->ClientTxStatusConfirmed->Inc(); + break; + + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusPlanned: + TxProxyMon->ClientTxStatusPlanned->Inc(); + break; + + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusOutdated: + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusDeclined: + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusDeclinedNoSpace: + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusRestarting: + TxProxyMon->ClientTxStatusCoordinatorDeclined->Inc(); + ReplyErrorAndDie(TStringBuilder() << "Failed to plan transaction, status: " << res->GetStatus(), NYql::NDqProto::StatusIds::UNAVAILABLE, {}); + break; + + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusUnknown: + case TEvTxProxy::TEvProposeTransactionStatus::EStatus::StatusAborted: + TxProxyMon->ClientTxStatusCoordinatorDeclined->Inc(); + ReplyErrorAndDie(TStringBuilder() << "Unexpected TEvProposeTransactionStatus status: " << res->GetStatus(), NYql::NDqProto::StatusIds::INTERNAL_ERROR, {}); + break; + } + } + + void Handle(TEvPipeCache::TEvDeliveryProblem::TPtr& ev) { + CA_LOG_W("TEvDeliveryProblem was received from tablet: " << ev->Get()->TabletId); + ReplyErrorAndDie(TStringBuilder() << "Failed to deviler message.", NYql::NDqProto::StatusIds::UNAVAILABLE, {}); + } + + void Handle(TEvKqpBuffer::TEvTerminate::TPtr&) { + PassAway(); + } + + void Handle(TEvKqpBuffer::TEvFlush::TPtr& ev) { + ExecuterActorId = ev->Get()->ExecuterActorId; + for (auto& [_, info] : WriteInfos) { + info.WriteTableActor->FlushBuffers(); + } + Flush(); + } + + void Handle(TEvKqpBuffer::TEvCommit::TPtr& ev) { + ExecuterActorId = ev->Get()->ExecuterActorId; + for (auto& [_, info] : WriteInfos) { + info.WriteTableActor->FlushBuffers(); + } + if (TxManager->IsReadOnly()) { + Rollback(); + State = EState::FINISHED; + Send(ExecuterActorId, new TEvKqpBuffer::TEvResult{}); + } else if (TxManager->IsSingleShard() && !TxManager->HasOlapTable() && !WriteInfos.empty()) { + TxManager->StartExecute(); + ImmediateCommit(); + } else { + TxManager->StartPrepare(); + Prepare(ev->Get()->TxId); + } + } + + void Handle(TEvKqpBuffer::TEvRollback::TPtr& ev) { + ExecuterActorId = ev->Get()->ExecuterActorId; + Rollback(); + State = EState::FINISHED; + Send(ExecuterActorId, new TEvKqpBuffer::TEvResult{}); + } + + void Handle(NKikimr::NEvents::TDataEvents::TEvWriteResult::TPtr& ev) { + auto getIssues = [&ev]() { + NYql::TIssues issues; + NYql::IssuesFromMessage(ev->Get()->Record.GetIssues(), issues); + return issues; + }; + + CA_LOG_D("Recv EvWriteResult (external) from ShardID=" << ev->Get()->Record.GetOrigin() + << ", Status=" << NKikimrDataEvents::TEvWriteResult::EStatus_Name(ev->Get()->GetStatus()) + << ", TxId=" << ev->Get()->Record.GetTxId() + << ", Locks= " << [&]() { + TStringBuilder builder; + for (const auto& lock : ev->Get()->Record.GetTxLocks()) { + builder << lock.ShortDebugString(); + } + return builder; + }() + << ", Cookie=" << ev->Cookie); + + // TODO: get rid of copy-paste + switch (ev->Get()->GetStatus()) { + case NKikimrDataEvents::TEvWriteResult::STATUS_UNSPECIFIED: { + CA_LOG_E("Got UNSPECIFIED for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Unspecified error for table. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::UNSPECIFIED, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_PREPARED: { + ProcessWritePreparedShard(ev); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_COMPLETED: { + ProcessWriteCompletedShard(ev); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_ABORTED: { + CA_LOG_E("Got ABORTED for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Aborted for table. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::ABORTED, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_INTERNAL_ERROR: { + CA_LOG_E("Got INTERNAL ERROR for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Internal error for table. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::INTERNAL_ERROR, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_DISK_SPACE_EXHAUSTED: { + CA_LOG_E("Got DISK_SPACE_EXHAUSTED for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Disk space exhausted for table. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::UNAVAILABLE, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_OVERLOADED: { + CA_LOG_W("Got OVERLOADED for table ." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << " Ignored this error." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Tablet " << ev->Get()->Record.GetOrigin() << " is overloaded." + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::OVERLOADED, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_CANCELLED: { + CA_LOG_E("Got CANCELLED for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Cancelled request to table." + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::CANCELLED, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_BAD_REQUEST: { + CA_LOG_E("Got BAD REQUEST for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Bad request. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::BAD_REQUEST, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_SCHEME_CHANGED: { + CA_LOG_E("Got SCHEME CHANGED for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + ReplyErrorAndDie( + TStringBuilder() << "Scheme changed. " + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::SCHEME_ERROR, + getIssues()); + return; + } + case NKikimrDataEvents::TEvWriteResult::STATUS_LOCKS_BROKEN: { + CA_LOG_E("Got LOCKS BROKEN for table." + << " ShardID=" << ev->Get()->Record.GetOrigin() << "," + << " Sink=" << this->SelfId() << "." + << getIssues().ToOneLineString()); + + TxManager->BreakLock(ev->Get()->Record.GetOrigin()); + YQL_ENSURE(TxManager->BrokenLocks()); + ReplyErrorAndDie( + TStringBuilder() << "Transaction locks invalidated." + << getIssues().ToOneLineString(), + NYql::NDqProto::StatusIds::ABORTED, + getIssues()); + return; + } + } + } + + void OnMessageReceived(const ui64 shardId) { + if (auto it = SendTime.find(shardId); it != std::end(SendTime)) { + Counters->WriteActorWritesLatencyHistogram->Collect((TInstant::Now() - it->second).MilliSeconds()); + SendTime.erase(it); + } + } + + void ProcessWritePreparedShard(NKikimr::NEvents::TDataEvents::TEvWriteResult::TPtr& ev) { + if (State != EState::PREPARING) { + CA_LOG_D("Ignored write prepared event."); + return; + } + OnMessageReceived(ev->Get()->Record.GetOrigin()); + CA_LOG_D("Got prepared result TxId=" << ev->Get()->Record.GetTxId() + << ", TabletId=" << ev->Get()->Record.GetOrigin() + << ", Cookie=" << ev->Cookie); + + const auto& record = ev->Get()->Record; + IKqpTransactionManager::TPrepareResult preparedInfo; + preparedInfo.ShardId = record.GetOrigin(); + preparedInfo.MinStep = record.GetMinStep(); + preparedInfo.MaxStep = record.GetMaxStep(); + + preparedInfo.Coordinator = 0; + if (record.DomainCoordinatorsSize()) { + auto domainCoordinators = TCoordinators(TVector(record.GetDomainCoordinators().begin(), + record.GetDomainCoordinators().end())); + preparedInfo.Coordinator = domainCoordinators.Select(*TxId); + } + + OnPrepared(std::move(preparedInfo), 0); + } + + void ProcessWriteCompletedShard(NKikimr::NEvents::TDataEvents::TEvWriteResult::TPtr& ev) { + if (State != EState::COMMITTING) { + CA_LOG_D("Ignored write completed event."); + return; + } + OnMessageReceived(ev->Get()->Record.GetOrigin()); + CA_LOG_D("Got completed result TxId=" << ev->Get()->Record.GetTxId() + << ", TabletId=" << ev->Get()->Record.GetOrigin() + << ", Cookie=" << ev->Cookie + << ", Locks=" << [&]() { + TStringBuilder builder; + for (const auto& lock : ev->Get()->Record.GetTxLocks()) { + builder << lock.ShortDebugString(); + } + return builder; + }()); + + OnCommitted(ev->Get()->Record.GetOrigin(), 0); + } + + void OnReady() override { + Process(); + } + + void OnPrepared(IKqpTransactionManager::TPrepareResult&& preparedInfo, ui64 dataSize) override { + if (State != EState::PREPARING) { + return; + } + Y_UNUSED(preparedInfo, dataSize); + if (TxManager->ConsumePrepareTransactionResult(std::move(preparedInfo))) { + TxManager->StartExecute(); + Y_ABORT_UNLESS(GetTotalMemory() == 0); + DistributedCommit(); + return; + } + Process(); + } + + void OnCommitted(ui64 shardId, ui64 dataSize) override { + if (State != EState::COMMITTING) { + return; + } + Y_UNUSED(dataSize); + if (TxManager->ConsumeCommitResult(shardId)) { + CA_LOG_D("Committed"); + State = EState::FINISHED; + Send(ExecuterActorId, new TEvKqpBuffer::TEvResult{ + BuildStats() + }); + ExecuterActorId = {}; + Y_ABORT_UNLESS(GetTotalMemory() == 0); + return; + } + } + + void OnMessageAcknowledged(ui64 dataSize) override { + Y_UNUSED(dataSize); + Process(); + } + + void OnFlushed() { + BufferWriteActorState = NWilson::TSpan(TWilsonKqp::BufferWriteActorState, BufferWriteActor.GetTraceId(), + "BufferWriteActorState::Writing", NWilson::EFlags::AUTO_END); + CA_LOG_D("Flushed"); + State = EState::WRITING; + Send(ExecuterActorId, new TEvKqpBuffer::TEvResult{ + BuildStats() + }); + ExecuterActorId = {}; + Y_ABORT_UNLESS(GetTotalMemory() == 0); + } + + void OnError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues) override { + ReplyErrorAndDie(message, statusCode, subIssues); + } + + void ReplyErrorAndDie(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues = {}) { + BufferWriteActorState.EndError(message); + BufferWriteActor.EndError(message); + CA_LOG_E(message << ". statusCode=" << NYql::NDqProto::StatusIds_StatusCode_Name(statusCode) << ". subIssues=" << subIssues.ToString() << ". sessionActorId=" << SessionActorId << ". isRollback=" << (State == EState::ROLLINGBACK)); + + Y_ABORT_UNLESS(!HasError); + HasError = true; + if (State != EState::ROLLINGBACK) { + // Rollback can't finish with error + Send(SessionActorId, new TEvKqpBuffer::TEvError{ + message, + statusCode, + subIssues, + }); + } + PassAway(); + } + + NYql::NDqProto::TDqTaskStats BuildStats() { + NYql::NDqProto::TDqTaskStats result; + for (const auto& [_, writeInfo] : WriteInfos) { + writeInfo.WriteTableActor->FillStats(&result); + } + return result; + } + +private: + TString LogPrefix; + const TActorId SessionActorId; + TWriteActorSettings MessageSettings; + + TActorId ExecuterActorId; + IKqpTransactionManagerPtr TxManager; + + std::optional TxId; + ui64 LockTxId = 0; + ui64 LockNodeId = 0; + bool InconsistentTx = false; + + std::shared_ptr Alloc; + NMiniKQL::TTypeEnvironment TypeEnv; + + struct TWriteInfo { + TKqpTableWriteActor* WriteTableActor = nullptr; + TActorId WriteTableActorId; + }; + + THashMap WriteInfos; + + EState State; + bool HasError = false; + THashMap> DataQueues; + + struct TAckMessage { + TActorId ForwardActorId; + TWriteToken Token; + i64 DataSize; + }; + std::queue AckQueue; + + IShardedWriteControllerPtr ShardedWriteController = nullptr; + + TIntrusivePtr Counters; + TIntrusivePtr TxProxyMon; + THashMap SendTime; + + NWilson::TSpan BufferWriteActor; + NWilson::TSpan BufferWriteActorState; +}; + +class TKqpForwardWriteActor : public TActorBootstrapped, public NYql::NDq::IDqComputeActorAsyncOutput { + using TBase = TActorBootstrapped; + +public: + TKqpForwardWriteActor( + NKikimrKqp::TKqpTableSinkSettings&& settings, + NYql::NDq::TDqAsyncIoFactory::TSinkArguments&& args, + TIntrusivePtr counters) + : LogPrefix(TStringBuilder() << "TxId: " << args.TxId << ", task: " << args.TaskId << ". ") + , Settings(std::move(settings)) + , MessageSettings(GetWriteActorSettings()) + , OutputIndex(args.OutputIndex) + , Callbacks(args.Callback) + , Counters(counters) + , TypeEnv(args.TypeEnv) + , Alloc(args.Alloc) + , BufferActorId(ActorIdFromProto(Settings.GetBufferActorId())) + , TxId(std::get(args.TxId)) + , TableId( + Settings.GetTable().GetOwnerId(), + Settings.GetTable().GetTableId(), + Settings.GetTable().GetVersion()) + , ForwardWriteActorSpan(TWilsonKqp::ForwardWriteActor, NWilson::TTraceId(args.TraceId), "TKqpForwardWriteActor") + { + EgressStats.Level = args.StatsLevel; + Counters->ForwardActorsCount->Inc(); + } + + void Bootstrap() { + LogPrefix = TStringBuilder() << "SelfId: " << this->SelfId() << ", " << LogPrefix; + Become(&TKqpForwardWriteActor::StateFuncFwd); + } + + static constexpr char ActorName[] = "KQP_FORWARD_WRITE_ACTOR"; + +private: + STFUNC(StateFuncFwd) { + try { + switch (ev->GetTypeRewrite()) { + hFunc(TEvBufferWriteResult, Handle); + default: + AFL_ENSURE(false)("unknown message", ev->GetTypeRewrite()); + } + } catch (const yexception& e) { + RuntimeError(e.what(), NYql::NDqProto::StatusIds::INTERNAL_ERROR); + } + } + + void Handle(TEvBufferWriteResult::TPtr& result) { + CA_LOG_D("TKqpForwardWriteActor recieve EvBufferWriteResult from " << BufferActorId); + EgressStats.Bytes += DataSize; + EgressStats.Chunks++; + EgressStats.Splits++; + EgressStats.Resume(); + + Counters->ForwardActorWritesSizeHistogram->Collect(DataSize); + Counters->ForwardActorWritesLatencyHistogram->Collect((TInstant::Now() - SendTime).MilliSeconds()); + + WriteToken = result->Get()->Token; + DataSize = 0; + { + auto alloc = TypeEnv.BindAllocator(); + Data = nullptr; + } + + if (Closed) { + CA_LOG_D("Finished"); + Callbacks->OnAsyncOutputFinished(GetOutputIndex()); + return; + } + CA_LOG_D("Resume with freeSpace=" << GetFreeSpace()); + Callbacks->ResumeExecution(); + } + + void WriteToBuffer() { + auto ev = std::make_unique(); + + ev->Data = Data; + ev->Close = Closed; + ev->Alloc = Alloc; + + if (!WriteToken.IsEmpty()) { + ev->Token = WriteToken; + } else { + TVector columnsMetadata; + columnsMetadata.reserve(Settings.GetColumns().size()); + for (const auto & column : Settings.GetColumns()) { + columnsMetadata.push_back(column); + } + + ev->Settings = TWriteSettings{ + .TableId = TableId, + .TablePath = Settings.GetTable().GetPath(), + .OperationType = GetOperation(Settings.GetType()), + .Columns = std::move(columnsMetadata), + .TransactionSettings = TTransactionSettings{ + .TxId = TxId, + .LockTxId = Settings.GetLockTxId(), + .LockNodeId = Settings.GetLockNodeId(), + .InconsistentTx = Settings.GetInconsistentTx(), + }, + .Priority = Settings.GetPriority(), + }; + } + + SendTime = TInstant::Now(); + + CA_LOG_D("Send data=" << DataSize << ", closed=" << Closed << ", bufferActorId=" << BufferActorId); + AFL_ENSURE(Send(BufferActorId, ev.release())); + } + + void CommitState(const NYql::NDqProto::TCheckpoint&) final {}; + void LoadState(const NYql::NDq::TSinkState&) final {}; + + ui64 GetOutputIndex() const final { + return OutputIndex; + } + + const NYql::NDq::TDqAsyncStats& GetEgressStats() const final { + return EgressStats; + } + + i64 GetFreeSpace() const final { + return MessageSettings.MaxForwardedSize - DataSize > 0 + ? MessageSettings.MaxForwardedSize - DataSize + : std::numeric_limits::min(); + } + + TMaybe ExtraData() override { + return {}; + } + + void SendData(NMiniKQL::TUnboxedValueBatch&& data, i64 size, const TMaybe&, bool finished) final { + YQL_ENSURE(!data.IsWide(), "Wide stream is not supported yet"); + Closed |= finished; + if (!Data) { + Data = std::make_shared>(); + } + Data->emplace_back(std::move(data)); + DataSize += size; + + CA_LOG_D("Add data: " << size << " / " << DataSize); + if (Closed || GetFreeSpace() <= 0) { + WriteToBuffer(); + } + } + + void RuntimeError(const TString& message, NYql::NDqProto::StatusIds::StatusCode statusCode, const NYql::TIssues& subIssues = {}) { + ForwardWriteActorSpan.EndError(message); + + CA_LOG_E("RuntimeError: " << message); + NYql::TIssue issue(message); + for (const auto& i : subIssues) { + issue.AddSubIssue(MakeIntrusive(i)); + } + + NYql::TIssues issues; + issues.AddIssue(std::move(issue)); + + Callbacks->OnAsyncOutputError(OutputIndex, std::move(issues), statusCode); + } + + ~TKqpForwardWriteActor() { + { + TGuard guard(*Alloc); + Data = nullptr; + } + } + + void PassAway() override { + TActorBootstrapped::PassAway(); + } + + TString LogPrefix; + const NKikimrKqp::TKqpTableSinkSettings Settings; + TWriteActorSettings MessageSettings; + const ui64 OutputIndex; + NYql::NDq::TDqAsyncStats EgressStats; + NYql::NDq::IDqComputeActorAsyncOutput::ICallbacks * Callbacks = nullptr; + TIntrusivePtr Counters; + const NMiniKQL::TTypeEnvironment& TypeEnv; + std::shared_ptr Alloc; + + TActorId BufferActorId; + + std::shared_ptr> Data; + i64 DataSize = 0; + bool Closed = false; + + const ui64 TxId; + const TTableId TableId; + + TInstant SendTime; + + TWriteToken WriteToken; + NWilson::TSpan ForwardWriteActorSpan; +}; + +NActors::IActor* CreateKqpBufferWriterActor(TKqpBufferWriterSettings&& settings) { + return new TKqpBufferWriteActor(std::move(settings)); +} + + +void RegisterKqpWriteActor(NYql::NDq::TDqAsyncIoFactory& factory, TIntrusivePtr counters) { + factory.RegisterSink( + TString(NYql::KqpTableSinkName), + [counters] (NKikimrKqp::TKqpTableSinkSettings&& settings, NYql::NDq::TDqAsyncIoFactory::TSinkArguments&& args) { + if (!ActorIdFromProto(settings.GetBufferActorId())) { + auto* actor = new TKqpDirectWriteActor(std::move(settings), std::move(args), counters); + return std::make_pair(actor, actor); + } else { + auto* actor = new TKqpForwardWriteActor(std::move(settings), std::move(args), counters); + return std::make_pair(actor, actor); + } }); } diff --git a/ydb/core/kqp/runtime/kqp_write_actor.h b/ydb/core/kqp/runtime/kqp_write_actor.h index 844309a70a77..f87062db2289 100644 --- a/ydb/core/kqp/runtime/kqp_write_actor.h +++ b/ydb/core/kqp/runtime/kqp_write_actor.h @@ -1,6 +1,10 @@ #pragma once +#include #include +#include +#include +#include #include namespace NKikimr { diff --git a/ydb/core/kqp/runtime/kqp_write_actor_settings.h b/ydb/core/kqp/runtime/kqp_write_actor_settings.h index 328dcd5120a7..37e8bfe91055 100644 --- a/ydb/core/kqp/runtime/kqp_write_actor_settings.h +++ b/ydb/core/kqp/runtime/kqp_write_actor_settings.h @@ -11,6 +11,7 @@ struct TWriteActorSettings : TAtomicRefCount { i64 InFlightMemoryLimitPerActorBytes = 64_MB; i64 MemoryLimitPerMessageBytes = 64_MB; i64 MaxBatchesPerMessage = 1000; + i64 MaxForwardedSize = 64_MB; TDuration StartRetryDelay = TDuration::Seconds(1); TDuration MaxRetryDelay = TDuration::Seconds(10); diff --git a/ydb/core/kqp/runtime/kqp_write_table.cpp b/ydb/core/kqp/runtime/kqp_write_table.cpp index d62485a4e085..d5894af099af 100644 --- a/ydb/core/kqp/runtime/kqp_write_table.cpp +++ b/ydb/core/kqp/runtime/kqp_write_table.cpp @@ -19,7 +19,6 @@ namespace { constexpr ui64 DataShardMaxOperationBytes = 8_MB; constexpr ui64 ColumnShardMaxOperationBytes = 64_MB; -constexpr ui64 MaxUnshardedBatchBytes = 0_MB; class IPayloadSerializer : public TThrRefBase { public: @@ -32,7 +31,7 @@ class IPayloadSerializer : public TThrRefBase { using IBatchPtr = TIntrusivePtr; - virtual void AddData(NMiniKQL::TUnboxedValueBatch&& data) = 0; + virtual void AddData(const NMiniKQL::TUnboxedValueBatch& data) = 0; virtual void AddBatch(const IBatchPtr& batch) = 0; virtual void Close() = 0; @@ -183,7 +182,7 @@ std::vector> BuildBatchBuilderColumns( result.reserve(columns.size()); for (const auto& column : columns) { if (inputColumnsIds.contains(column.GetId())) { - Y_ABORT_UNLESS(column.HasTypeId()); + YQL_ENSURE(column.HasTypeId()); auto typeInfoMod = NScheme::TypeInfoModFromProtoColumnType(column.GetTypeId(), column.HasTypeInfo() ? &column.GetTypeInfo() : nullptr); result.emplace_back(column.GetName(), typeInfoMod.TypeInfo); @@ -398,7 +397,7 @@ class TColumnShardPayloadSerializer : public IPayloadSerializer { Sharding = shardingConclusion.DetachResult(); } - void AddData(NMiniKQL::TUnboxedValueBatch&& data) override { + void AddData(const NMiniKQL::TUnboxedValueBatch& data) override { YQL_ENSURE(!Closed); if (data.empty()) { return; @@ -425,7 +424,7 @@ class TColumnShardPayloadSerializer : public IPayloadSerializer { } void FlushUnsharded(bool force) { - if ((BatchBuilder.Bytes() > 0 && force) || BatchBuilder.Bytes() > MaxUnshardedBatchBytes) { + if (BatchBuilder.Bytes() > 0 && force) { const auto unshardedBatch = BatchBuilder.FlushBatch(true); YQL_ENSURE(unshardedBatch); ShardAndFlushBatch(unshardedBatch, force); @@ -437,14 +436,13 @@ class TColumnShardPayloadSerializer : public IPayloadSerializer { const i64 shardBatchMemory = NArrow::GetBatchDataSize(shardBatch); YQL_ENSURE(shardBatchMemory != 0); + ShardIds.insert(shardId); auto& unpreparedBatch = UnpreparedBatches[shardId]; unpreparedBatch.TotalDataSize += shardBatchMemory; unpreparedBatch.Batches.emplace_back(shardBatch); Memory += shardBatchMemory; FlushUnpreparedBatch(shardId, unpreparedBatch, force); - - ShardIds.insert(shardId); } } @@ -654,7 +652,7 @@ class TDataShardPayloadSerializer : public IPayloadSerializer { } ui64 AddRow(TRowWithData&& rowWithData) { - Y_ABORT_UNLESS(rowWithData.Cells.size() == ColumnCount); + YQL_ENSURE(rowWithData.Cells.size() == ColumnCount); ui64 newMemory = 0; for (const auto& cell : rowWithData.Cells) { newMemory += cell.Size(); @@ -686,10 +684,10 @@ class TDataShardPayloadSerializer : public IPayloadSerializer { public: TDataShardPayloadSerializer( const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry, - NSchemeCache::TSchemeCacheRequest::TEntry&& partitionsEntry, + const NSchemeCache::TSchemeCacheRequest::TEntry& partitionsEntry, const TConstArrayRef inputColumns) : SchemeEntry(schemeEntry) - , KeyDescription(std::move(partitionsEntry.KeyDescription)) + , KeyDescription(partitionsEntry.KeyDescription) , Columns(BuildColumns(inputColumns)) , WriteIndex(BuildWriteIndexKeyFirst(SchemeEntry, inputColumns)) , WriteColumnIds(BuildWriteColumnIds(inputColumns, WriteIndex)) @@ -720,7 +718,7 @@ class TDataShardPayloadSerializer : public IPayloadSerializer { ShardIds.insert(shardIter->ShardId); } - void AddData(NMiniKQL::TUnboxedValueBatch&& data) override { + void AddData(const NMiniKQL::TUnboxedValueBatch& data) override { YQL_ENSURE(!Closed); TRowBuilder rowBuilder(Columns.size()); @@ -825,7 +823,7 @@ class TDataShardPayloadSerializer : public IPayloadSerializer { } const NSchemeCache::TSchemeCacheNavigate::TEntry SchemeEntry; - THolder KeyDescription; + const THolder& KeyDescription; const TVector Columns; const std::vector WriteIndex; @@ -840,8 +838,6 @@ class TDataShardPayloadSerializer : public IPayloadSerializer { bool Closed = false; }; -} - bool IPayloadSerializer::IBatch::IsEmpty() const { return GetMemory() == 0; } @@ -855,14 +851,37 @@ IPayloadSerializerPtr CreateColumnShardPayloadSerializer( IPayloadSerializerPtr CreateDataShardPayloadSerializer( const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry, - NSchemeCache::TSchemeCacheRequest::TEntry&& partitionsEntry, + const NSchemeCache::TSchemeCacheRequest::TEntry& partitionsEntry, const TConstArrayRef inputColumns) { return MakeIntrusive( - schemeEntry, std::move(partitionsEntry), inputColumns); + schemeEntry, partitionsEntry, inputColumns); +} + } namespace { +struct TMetadata { + const TTableId TableId; + const NKikimrDataEvents::TEvWrite::TOperation::EOperationType OperationType; + const TVector InputColumnsMetadata; + const i64 Priority; +}; + +struct TBatchWithMetadata { + IShardedWriteController::TWriteToken Token = std::numeric_limits::max(); + IPayloadSerializer::IBatchPtr Data = nullptr; + bool HasRead = false; + + bool IsCoveringBatch() const { + return Data == nullptr; + } + + i64 GetMemory() const { + return IsCoveringBatch() ? 0 : Data->GetMemory(); + } +}; + class TShardsInfo { public: class TShardInfo { @@ -894,27 +913,31 @@ class TShardsInfo { void MakeNextBatches(i64 maxDataSize, ui64 maxCount) { YQL_ENSURE(BatchesInFlight == 0); YQL_ENSURE(!IsEmpty()); + YQL_ENSURE(maxCount != 0); i64 dataSize = 0; // For columnshard batch can be slightly larger than the limit. while (BatchesInFlight < maxCount && BatchesInFlight < Batches.size() - && (dataSize + GetBatch(BatchesInFlight)->GetMemory() <= maxDataSize || BatchesInFlight == 0)) { - dataSize += GetBatch(BatchesInFlight)->GetMemory(); + && (dataSize + GetBatch(BatchesInFlight).GetMemory() <= maxDataSize || BatchesInFlight == 0)) { + dataSize += GetBatch(BatchesInFlight).GetMemory(); ++BatchesInFlight; } YQL_ENSURE(BatchesInFlight != 0); - YQL_ENSURE(BatchesInFlight == maxCount || BatchesInFlight == Batches.size() || dataSize + GetBatch(BatchesInFlight)->GetMemory() >= maxDataSize); + YQL_ENSURE(BatchesInFlight == Batches.size() || BatchesInFlight >= maxCount || dataSize + GetBatch(BatchesInFlight).GetMemory() > maxDataSize); } - const IPayloadSerializer::IBatchPtr& GetBatch(size_t index) const { + const TBatchWithMetadata& GetBatch(size_t index) const { return Batches.at(index); } - std::optional PopBatches(const ui64 cookie) { + struct TBatchInfo { + ui64 DataSize = 0; + }; + std::optional PopBatches(const ui64 cookie) { if (BatchesInFlight != 0 && Cookie == cookie) { - ui64 dataSize = 0; + TBatchInfo result; for (size_t index = 0; index < BatchesInFlight; ++index) { - dataSize += Batches.front()->GetMemory(); + result.DataSize += Batches.front().GetMemory(); Batches.pop_front(); } @@ -922,16 +945,17 @@ class TShardsInfo { SendAttempts = 0; BatchesInFlight = 0; - Memory -= dataSize; - return dataSize; + Memory -= result.DataSize; + return result; } return std::nullopt; } - void PushBatch(IPayloadSerializer::IBatchPtr&& batch) { + void PushBatch(TBatchWithMetadata&& batch) { YQL_ENSURE(!IsClosed()); Batches.emplace_back(std::move(batch)); - Memory += Batches.back()->GetMemory(); + Memory += Batches.back().GetMemory(); + HasReadInBatch |= Batches.back().HasRead; } ui64 GetCookie() const { @@ -954,9 +978,14 @@ class TShardsInfo { SendAttempts = 0; } + bool HasRead() const { + return HasReadInBatch; + } + private: - std::deque Batches; + std::deque Batches; i64& Memory; + bool HasReadInBatch = false; ui64& NextCookie; ui64 Cookie; @@ -977,11 +1006,14 @@ class TShardsInfo { return insertIt->second; } - TVector GetPendingShards() const { - TVector result; + TVector GetPendingShards() const { + TVector result; for (const auto& [id, shard] : ShardsInfo) { if (!shard.IsEmpty() && shard.GetSendAttempts() == 0) { - result.push_back(id); + result.push_back(IShardedWriteController::TPendingShardInfo{ + .ShardId = id, + .HasRead = shard.HasRead(), + }); } } return result; @@ -1013,6 +1045,10 @@ class TShardsInfo { return ShardsInfo; } + const THashMap& GetShards() const { + return ShardsInfo; + } + i64 GetMemory() const { return Memory; } @@ -1037,69 +1073,176 @@ class TShardsInfo { class TShardedWriteController : public IShardedWriteController { public: void OnPartitioningChanged(const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry) override { + IsOlap = true; + SchemeEntry = schemeEntry; BeforePartitioningChanged(); - Serializer = CreateColumnShardPayloadSerializer( - schemeEntry, - InputColumnsMetadata); + for (auto& [_, writeInfo] : WriteInfos) { + writeInfo.Serializer = CreateColumnShardPayloadSerializer( + *SchemeEntry, + writeInfo.Metadata.InputColumnsMetadata); + } AfterPartitioningChanged(); } void OnPartitioningChanged( const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry, NSchemeCache::TSchemeCacheRequest::TEntry&& partitionsEntry) override { + IsOlap = false; + SchemeEntry = schemeEntry; + PartitionsEntry = std::move(partitionsEntry); BeforePartitioningChanged(); - Serializer = CreateDataShardPayloadSerializer( - schemeEntry, - std::move(partitionsEntry), - InputColumnsMetadata); + for (auto& [_, writeInfo] : WriteInfos) { + writeInfo.Serializer = CreateDataShardPayloadSerializer( + *SchemeEntry, + *PartitionsEntry, + writeInfo.Metadata.InputColumnsMetadata); + } AfterPartitioningChanged(); } void BeforePartitioningChanged() { - if (Serializer) { - if (!Closed) { - Serializer->Close(); + if (!Settings.Inconsistent) { + return; + } + for (auto& [token, writeInfo] : WriteInfos) { + if (writeInfo.Serializer) { + if (!writeInfo.Closed) { + writeInfo.Serializer->Close(); + } + FlushSerializer(token, true); + writeInfo.Serializer = nullptr; } - FlushSerializer(true); } } void AfterPartitioningChanged() { - ShardsInfo.Close(); - ReshardData(); - ShardsInfo.Clear(); - if (Closed) { - Close(); - } else { - FlushSerializer(GetMemory() >= Settings.MemoryLimitTotal); + if (!Settings.Inconsistent) { + return; + } + if (!WriteInfos.empty()) { + ShardsInfo.Close(); + ReshardData(); + ShardsInfo.Clear(); + for (const auto& [token, writeInfo] : WriteInfos) { + if (writeInfo.Closed) { + Close(token); + } else { + FlushSerializer(token, GetMemory() >= Settings.MemoryLimitTotal); + } + } } } - void AddData(NMiniKQL::TUnboxedValueBatch&& data) override { + TWriteToken Open( + const TTableId tableId, + const NKikimrDataEvents::TEvWrite::TOperation::EOperationType operationType, + TVector&& inputColumns, + const i64 priority) override { + auto token = CurrentWriteToken++; + auto iter = WriteInfos.emplace( + token, + TWriteInfo { + .Metadata = TMetadata { + .TableId = tableId, + .OperationType = operationType, + .InputColumnsMetadata = std::move(inputColumns), + .Priority = priority, + }, + .Serializer = nullptr, + .Closed = false, + }).first; + if (PartitionsEntry) { + iter->second.Serializer = CreateDataShardPayloadSerializer( + *SchemeEntry, + *PartitionsEntry, + iter->second.Metadata.InputColumnsMetadata); + } else if (SchemeEntry) { + iter->second.Serializer = CreateColumnShardPayloadSerializer( + *SchemeEntry, + iter->second.Metadata.InputColumnsMetadata); + } + return token; + } + + void Write(TWriteToken token, const NMiniKQL::TUnboxedValueBatch& data) override { YQL_ENSURE(!data.IsWide(), "Wide stream is not supported yet"); - YQL_ENSURE(!Closed); + auto& info = WriteInfos.at(token); + YQL_ENSURE(!info.Closed); auto allocGuard = TypeEnv.BindAllocator(); - YQL_ENSURE(Serializer); - Serializer->AddData(std::move(data)); + YQL_ENSURE(info.Serializer); + info.Serializer->AddData(data); - FlushSerializer(GetMemory() >= Settings.MemoryLimitTotal); + if (info.Metadata.Priority == 0) { + FlushSerializer(token, GetMemory() >= Settings.MemoryLimitTotal); + } else { + YQL_ENSURE(GetMemory() <= Settings.MemoryLimitTotal); + } } - void Close() override { + void Close(TWriteToken token) override { auto allocGuard = TypeEnv.BindAllocator(); - YQL_ENSURE(Serializer); - Closed = true; - Serializer->Close(); - FlushSerializer(true); - YQL_ENSURE(Serializer->IsFinished()); + auto& info = WriteInfos.at(token); + YQL_ENSURE(info.Serializer); + info.Closed = true; + info.Serializer->Close(); + if (info.Metadata.Priority == 0) { + FlushSerializer(token, true); + YQL_ENSURE(info.Serializer->IsFinished()); + } + } + + void FlushBuffers() override { + TVector writeTokensFoFlush; + for (const auto& [token, writeInfo] : WriteInfos) { + YQL_ENSURE(writeInfo.Closed); + if (writeInfo.Metadata.Priority != 0) { + if (!writeInfo.Serializer->IsFinished()) { + writeTokensFoFlush.push_back(token); + } + } else { + YQL_ENSURE(writeInfo.Serializer->IsFinished()); + } + } + + std::sort( + std::begin(writeTokensFoFlush), + std::end(writeTokensFoFlush), + [&](const TWriteToken& lhs, const TWriteToken& rhs) { + const auto& leftWriteInfo = WriteInfos.at(lhs); + const auto& rightWriteInfo = WriteInfos.at(rhs); + return leftWriteInfo.Metadata.Priority < rightWriteInfo.Metadata.Priority; + }); + + for (const TWriteToken token : writeTokensFoFlush) { + FlushSerializer(token, true); + YQL_ENSURE(WriteInfos.at(token).Serializer->IsFinished()); + } + } + + void Close() override { ShardsInfo.Close(); } - TVector GetPendingShards() const override { + void AddCoveringMessages() override { + for (auto& [_, shardInfo] : ShardsInfo.GetShards()) { + shardInfo.PushBatch(TBatchWithMetadata{}); + } + } + + TVector GetPendingShards() const override { return ShardsInfo.GetPendingShards(); } + TVector GetShardsIds() const override { + TVector result; + result.reserve(ShardsInfo.GetShards().size()); + for (const auto& [id, _] : ShardsInfo.GetShards()) { + result.push_back(id); + } + return result; + } + std::optional GetMessageMetadata(ui64 shardId) override { auto& shardInfo = ShardsInfo.GetShard(shardId); if (shardInfo.IsEmpty()) { @@ -1126,29 +1269,37 @@ class TShardedWriteController : public IShardedWriteController { for (size_t index = 0; index < shardInfo.GetBatchesInFlight(); ++index) { const auto& inFlightBatch = shardInfo.GetBatch(index); - YQL_ENSURE(!inFlightBatch->IsEmpty()); - result.TotalDataSize += inFlightBatch->GetMemory(); - const ui64 payloadIndex = NKikimr::NEvWrite::TPayloadWriter(evWrite) - .AddDataToPayload(inFlightBatch->SerializeToString()); - result.PayloadIndexes.push_back(payloadIndex); + if (inFlightBatch.Data) { + YQL_ENSURE(!inFlightBatch.Data->IsEmpty()); + result.TotalDataSize += inFlightBatch.Data->GetMemory(); + const ui64 payloadIndex = NKikimr::NEvWrite::TPayloadWriter(evWrite) + .AddDataToPayload(inFlightBatch.Data->SerializeToString()); + const auto& writeInfo = WriteInfos.at(inFlightBatch.Token); + evWrite.AddOperation( + writeInfo.Metadata.OperationType, + writeInfo.Metadata.TableId, + writeInfo.Serializer->GetWriteColumnIds(), + payloadIndex, + writeInfo.Serializer->GetDataFormat()); + } else { + YQL_ENSURE(index + 1 == shardInfo.GetBatchesInFlight()); + } } return result; } - NKikimrDataEvents::EDataFormat GetDataFormat() override { - return Serializer->GetDataFormat(); - } - - std::vector GetWriteColumnIds() override { - return Serializer->GetWriteColumnIds(); - } - - std::optional OnMessageAcknowledged(ui64 shardId, ui64 cookie) override { + std::optional OnMessageAcknowledged(ui64 shardId, ui64 cookie) override { auto allocGuard = TypeEnv.BindAllocator(); auto& shardInfo = ShardsInfo.GetShard(shardId); - const auto removedDataSize = shardInfo.PopBatches(cookie); - return removedDataSize; + const auto result = shardInfo.PopBatches(cookie); + if (result) { + return TMessageAcknowledgedResult { + .DataSize = result->DataSize, + .IsShardEmpty = shardInfo.IsEmpty(), + }; + } + return std::nullopt; } void OnMessageSent(ui64 shardId, ui64 cookie) override { @@ -1168,29 +1319,62 @@ class TShardedWriteController : public IShardedWriteController { } i64 GetMemory() const override { - YQL_ENSURE(Serializer); - return Serializer->GetMemory() + ShardsInfo.GetMemory(); + i64 total = ShardsInfo.GetMemory(); + for (const auto& [_, writeInfo] : WriteInfos) { + if (writeInfo.Serializer) { + total += writeInfo.Serializer->GetMemory(); + } else { + YQL_ENSURE(writeInfo.Closed); + } + } + return total; } - bool IsClosed() const override { - return Closed; + bool IsAllWritesClosed() const override { + for (const auto& [_, writeInfo] : WriteInfos) { + if (!writeInfo.Closed) { + return false; + } + } + return true; } - bool IsFinished() const override { - return IsClosed() && Serializer->IsFinished() && ShardsInfo.IsFinished(); + bool IsAllWritesFinished() const override { + for (const auto& [_, writeInfo] : WriteInfos) { + if (!writeInfo.Closed || !writeInfo.Serializer->IsFinished()) { + return false; + } + } + return ShardsInfo.IsFinished(); } bool IsReady() const override { - return Serializer != nullptr; + for (const auto& [_, writeInfo] : WriteInfos) { + if (!writeInfo.Serializer && !writeInfo.Closed) { + return false; + } + } + return true; + } + + bool IsEmpty() const override { + for (const auto& [_, writeInfo] : WriteInfos) { + if (writeInfo.Serializer && !writeInfo.Serializer->IsEmpty()) { + return false; + } + } + return ShardsInfo.IsEmpty(); + } + + ui64 GetShardsCount() const override { + return ShardsInfo.GetShards().size(); } TShardedWriteController( const TShardedWriteControllerSettings settings, - TVector&& inputColumnsMetadata, const NMiniKQL::TTypeEnvironment& typeEnv, std::shared_ptr alloc) : Settings(settings) - , InputColumnsMetadata(std::move(inputColumnsMetadata)) , TypeEnv(typeEnv) , Alloc(alloc) { } @@ -1199,28 +1383,42 @@ class TShardedWriteController : public IShardedWriteController { Y_ABORT_UNLESS(Alloc); TGuard allocGuard(*Alloc); ShardsInfo.Clear(); - Serializer = nullptr; + for (auto& [_, writeInfo] : WriteInfos) { + writeInfo.Serializer = nullptr; + } } private: - void FlushSerializer(bool force) { + void FlushSerializer(TWriteToken token, bool force) { if (force) { - for (auto& [shardId, batches] : Serializer->FlushBatchesForce()) { + const auto& writeInfo = WriteInfos.at(token); + for (auto& [shardId, batches] : writeInfo.Serializer->FlushBatchesForce()) { for (auto& batch : batches) { if (batch && !batch->IsEmpty()) { - ShardsInfo.GetShard(shardId).PushBatch(std::move(batch)); + ShardsInfo.GetShard(shardId).PushBatch(TBatchWithMetadata{ + .Token = token, + .Data = std::move(batch), + .HasRead = (writeInfo.Metadata.OperationType != NKikimrDataEvents::TEvWrite::TOperation::OPERATION_REPLACE + && writeInfo.Metadata.OperationType != NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPSERT), + }); } } } } else { - for (const ui64 shardId : Serializer->GetShardIds()) { + const auto& writeInfo = WriteInfos.at(token); + for (const ui64 shardId : writeInfo.Serializer->GetShardIds()) { auto& shard = ShardsInfo.GetShard(shardId); while (true) { - auto batch = Serializer->FlushBatch(shardId); + auto batch = writeInfo.Serializer->FlushBatch(shardId); if (!batch || batch->IsEmpty()) { break; } - shard.PushBatch(std::move(batch)); + shard.PushBatch(TBatchWithMetadata{ + .Token = token, + .Data = std::move(batch), + .HasRead = (writeInfo.Metadata.OperationType != NKikimrDataEvents::TEvWrite::TOperation::OPERATION_REPLACE + && writeInfo.Metadata.OperationType != NKikimrDataEvents::TEvWrite::TOperation::OPERATION_UPSERT), + }); } } } @@ -1228,29 +1426,45 @@ class TShardedWriteController : public IShardedWriteController { void BuildBatchesForShard(TShardsInfo::TShardInfo& shard) { if (shard.GetBatchesInFlight() == 0) { + YQL_ENSURE(IsOlap != std::nullopt); shard.MakeNextBatches( Settings.MemoryLimitPerMessage, - Settings.MaxBatchesPerMessage); + (*IsOlap) ? 1 : Settings.MaxBatchesPerMessage); } } void ReshardData() { + YQL_ENSURE(!Settings.Inconsistent); for (auto& [_, shardInfo] : ShardsInfo.GetShards()) { for (size_t index = 0; index < shardInfo.Size(); ++index) { - Serializer->AddBatch(shardInfo.GetBatch(index)); + const auto& batch = shardInfo.GetBatch(index); + const auto& writeInfo = WriteInfos.at(batch.Token); + // Resharding supported only for inconsistent write, + // so convering empty batches don't exist in this case. + YQL_ENSURE(batch.Data); + writeInfo.Serializer->AddBatch(batch.Data); } } } TShardedWriteControllerSettings Settings; - TVector InputColumnsMetadata; const NMiniKQL::TTypeEnvironment& TypeEnv; std::shared_ptr Alloc; + struct TWriteInfo { + TMetadata Metadata; + IPayloadSerializerPtr Serializer = nullptr; + bool Closed = false; + }; + + std::map WriteInfos; + TWriteToken CurrentWriteToken = 0; + TShardsInfo ShardsInfo; - bool Closed = false; - IPayloadSerializerPtr Serializer = nullptr; + std::optional SchemeEntry; + std::optional PartitionsEntry; + std::optional IsOlap; }; } @@ -1258,11 +1472,10 @@ class TShardedWriteController : public IShardedWriteController { IShardedWriteControllerPtr CreateShardedWriteController( const TShardedWriteControllerSettings& settings, - TVector&& inputColumns, const NMiniKQL::TTypeEnvironment& typeEnv, std::shared_ptr alloc) { return MakeIntrusive( - settings, std::move(inputColumns), typeEnv, alloc); + settings, typeEnv, alloc); } } diff --git a/ydb/core/kqp/runtime/kqp_write_table.h b/ydb/core/kqp/runtime/kqp_write_table.h index 46e5ac4f7308..e3e645541310 100644 --- a/ydb/core/kqp/runtime/kqp_write_table.h +++ b/ydb/core/kqp/runtime/kqp_write_table.h @@ -12,15 +12,39 @@ namespace NKqp { class IShardedWriteController : public TThrRefBase { public: - virtual void OnPartitioningChanged(const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry) = 0; + virtual void OnPartitioningChanged( + const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry) = 0; virtual void OnPartitioningChanged( const NSchemeCache::TSchemeCacheNavigate::TEntry& schemeEntry, NSchemeCache::TSchemeCacheRequest::TEntry&& partitionsEntry) = 0; - virtual void AddData(NMiniKQL::TUnboxedValueBatch&& data) = 0; + using TWriteToken = ui64; + + // Data ordering invariant: + // For two writes A and B: + // A happend before B <=> Close(A) happend before Open(B) otherwise Priority(A) < Priority(B). + + virtual TWriteToken Open( + const TTableId TableId, + const NKikimrDataEvents::TEvWrite::TOperation::EOperationType operationType, + TVector&& inputColumns, + const i64 priority) = 0; + virtual void Write(TWriteToken token, const NMiniKQL::TUnboxedValueBatch& data) = 0; + virtual void Close(TWriteToken token) = 0; + + virtual void FlushBuffers() = 0; + virtual void Close() = 0; - virtual TVector GetPendingShards() const = 0; + virtual void AddCoveringMessages() = 0; + + struct TPendingShardInfo { + ui64 ShardId; + bool HasRead; + }; + virtual TVector GetPendingShards() const = 0; + virtual ui64 GetShardsCount() const = 0; + virtual TVector GetShardsIds() const = 0; struct TMessageMetadata { ui64 Cookie = 0; @@ -36,20 +60,24 @@ class IShardedWriteController : public TThrRefBase { }; virtual TSerializationResult SerializeMessageToPayload(ui64 shardId, NKikimr::NEvents::TDataEvents::TEvWrite& evWrite) = 0; - virtual NKikimrDataEvents::EDataFormat GetDataFormat() = 0; - virtual std::vector GetWriteColumnIds() = 0; - virtual std::optional OnMessageAcknowledged(ui64 shardId, ui64 cookie) = 0; + struct TMessageAcknowledgedResult { + ui64 DataSize = 0; + bool IsShardEmpty = 0; + }; + + virtual std::optional OnMessageAcknowledged(ui64 shardId, ui64 cookie) = 0; virtual void OnMessageSent(ui64 shardId, ui64 cookie) = 0; virtual void ResetRetries(ui64 shardId, ui64 cookie) = 0; virtual i64 GetMemory() const = 0; - virtual bool IsClosed() const = 0; - virtual bool IsFinished() const = 0; + virtual bool IsAllWritesClosed() const = 0; + virtual bool IsAllWritesFinished() const = 0; virtual bool IsReady() const = 0; + virtual bool IsEmpty() const = 0; }; using IShardedWriteControllerPtr = TIntrusivePtr; @@ -59,11 +87,11 @@ struct TShardedWriteControllerSettings { i64 MemoryLimitTotal; i64 MemoryLimitPerMessage; i64 MaxBatchesPerMessage; + bool Inconsistent; }; IShardedWriteControllerPtr CreateShardedWriteController( const TShardedWriteControllerSettings& settings, - TVector&& inputColumns, const NMiniKQL::TTypeEnvironment& typeEnv, std::shared_ptr alloc); diff --git a/ydb/core/kqp/runtime/ya.make b/ydb/core/kqp/runtime/ya.make index d4192b460ff7..75b7ecb6931b 100644 --- a/ydb/core/kqp/runtime/ya.make +++ b/ydb/core/kqp/runtime/ya.make @@ -29,23 +29,24 @@ SRCS( PEERDIR( contrib/libs/apache/arrow + library/cpp/threading/hot_swap ydb/core/actorlib_impl ydb/core/base ydb/core/engine ydb/core/engine/minikql ydb/core/formats ydb/core/kqp/common + ydb/core/kqp/common/buffer ydb/core/protos ydb/core/scheme ydb/core/ydb_convert - ydb/library/yql/minikql/computation/llvm14 - ydb/library/yql/minikql/comp_nodes - ydb/library/yql/utils ydb/library/yql/dq/actors/protos ydb/library/yql/dq/actors/spilling ydb/library/yql/dq/common ydb/library/yql/dq/runtime - library/cpp/threading/hot_swap + ydb/library/yql/minikql/comp_nodes + ydb/library/yql/minikql/computation/llvm14 + ydb/library/yql/utils ) YQL_LAST_ABI_VERSION() diff --git a/ydb/core/kqp/session_actor/kqp_query_state.h b/ydb/core/kqp/session_actor/kqp_query_state.h index 6c31e5660ac8..a4320654032e 100644 --- a/ydb/core/kqp/session_actor/kqp_query_state.h +++ b/ydb/core/kqp/session_actor/kqp_query_state.h @@ -345,13 +345,8 @@ class TKqpQueryState : public TNonCopyable { return true; } - if (HasTxSinkInTx(tx)) { - // At current time transactional internal sinks require separate tnx with commit. - return false; - } - if (TxCtx->HasOlapTable) { - // HTAP/OLAP transactions always use separate commit. + // Olap sink results can't be committed with changes return false; } @@ -370,12 +365,13 @@ class TKqpQueryState : public TNonCopyable { } bool ShouldAcquireLocks(const TKqpPhyTxHolder::TConstPtr& tx) { + Y_UNUSED(tx); if (*TxCtx->EffectiveIsolationLevel != NKikimrKqp::ISOLATION_LEVEL_SERIALIZABLE) { return false; } // Inconsistent writes (CTAS) don't require locks. - if (IsSplitted() && !HasTxSinkInTx(tx)) { + if (IsSplitted()) { return false; } @@ -414,9 +410,9 @@ class TKqpQueryState : public TNonCopyable { const auto& phyQuery = PreparedQuery->GetPhysicalQuery(); auto tx = PreparedQuery->GetPhyTxOrEmpty(CurrentTx); - if (TxCtx->CanDeferEffects()) { + if (TxCtx->CanDeferEffects()) { // At current time sinks require separate tnx with commit. - while (tx && tx->GetHasEffects() && !HasTxSinkInTx(tx)) { + while (tx && tx->GetHasEffects() && !TxCtx->HasOlapTable) { QueryData->CreateKqpValueMap(tx); bool success = TxCtx->AddDeferredEffect(tx, QueryData); YQL_ENSURE(success); @@ -433,40 +429,6 @@ class TKqpQueryState : public TNonCopyable { return tx; } - bool HasTxSinkInStage(const ::NKqpProto::TKqpPhyStage& stage) const { - for (const auto& sink : stage.GetSinks()) { - if (sink.GetTypeCase() == NKqpProto::TKqpSink::kInternalSink && sink.GetInternalSink().GetSettings().Is()) { - NKikimrKqp::TKqpTableSinkSettings settings; - YQL_ENSURE(sink.GetInternalSink().GetSettings().UnpackTo(&settings), "Failed to unpack settings"); - if (!settings.GetInconsistentTx()) { - return true; - } - } - } - return false; - } - - bool HasTxSink() const { - const auto& query = PreparedQuery->GetPhysicalQuery(); - for (auto& tx : query.GetTransactions()) { - for (const auto& stage : tx.GetStages()) { - if (HasTxSinkInStage(stage)) { - return true; - } - } - } - return false; - } - - bool HasTxSinkInTx(const TKqpPhyTxHolder::TConstPtr& tx) const { - for (const auto& stage : tx->GetStages()) { - if (HasTxSinkInStage(stage)) { - return true; - } - } - return false; - } - bool HasTxControl() const { return RequestEv->HasTxControl(); } diff --git a/ydb/core/kqp/session_actor/kqp_session_actor.cpp b/ydb/core/kqp/session_actor/kqp_session_actor.cpp index cb935e2adcc3..f583ddc4b662 100644 --- a/ydb/core/kqp/session_actor/kqp_session_actor.cpp +++ b/ydb/core/kqp/session_actor/kqp_session_actor.cpp @@ -3,6 +3,8 @@ #include "kqp_query_state.h" #include "kqp_query_stats.h" +#include +#include #include #include #include @@ -853,9 +855,11 @@ class TKqpSessionActor : public TActorBootstrapped { } const NKqpProto::TKqpPhyQuery& phyQuery = QueryState->PreparedQuery->GetPhysicalQuery(); - QueryState->TxCtx->HasOlapTable |= ::NKikimr::NKqp::HasOlapTableReadInTx(phyQuery) || ::NKikimr::NKqp::HasOlapTableWriteInTx(phyQuery); - QueryState->TxCtx->HasOltpTable |= ::NKikimr::NKqp::HasOltpTableReadInTx(phyQuery) || ::NKikimr::NKqp::HasOltpTableWriteInTx(phyQuery); - QueryState->TxCtx->HasTableWrite |= ::NKikimr::NKqp::HasOlapTableWriteInTx(phyQuery) || ::NKikimr::NKqp::HasOltpTableWriteInTx(phyQuery); + const bool hasOlapWrite = ::NKikimr::NKqp::HasOlapTableWriteInTx(phyQuery); + const bool hasOltpWrite = ::NKikimr::NKqp::HasOltpTableWriteInTx(phyQuery); + QueryState->TxCtx->HasOlapTable |= hasOlapWrite || ::NKikimr::NKqp::HasOlapTableReadInTx(phyQuery); + QueryState->TxCtx->HasOltpTable |= hasOltpWrite || ::NKikimr::NKqp::HasOltpTableReadInTx(phyQuery); + QueryState->TxCtx->HasTableWrite |= hasOlapWrite || hasOltpWrite; if (QueryState->TxCtx->HasOlapTable && QueryState->TxCtx->HasOltpTable && QueryState->TxCtx->HasTableWrite && !Settings.TableService.GetEnableHtapTx() && !QueryState->IsSplitted()) { ReplyQueryError(Ydb::StatusIds::PRECONDITION_FAILED, @@ -1032,15 +1036,19 @@ class TKqpSessionActor : public TActorBootstrapped { bool CheckTransactionLocks(const TKqpPhyTxHolder::TConstPtr& tx) { auto& txCtx = *QueryState->TxCtx; - if (!txCtx.DeferredEffects.Empty() && txCtx.Locks.Broken()) { + const bool broken = txCtx.TxManager + ? !!txCtx.TxManager->GetLockIssue() + : txCtx.Locks.Broken(); + + if (!txCtx.DeferredEffects.Empty() && broken) { ReplyQueryError(Ydb::StatusIds::ABORTED, "tx has deferred effects, but locks are broken", - MessageFromIssues(std::vector{txCtx.Locks.GetIssue()})); + MessageFromIssues(std::vector{txCtx.TxManager ? *txCtx.TxManager->GetLockIssue() : txCtx.Locks.GetIssue()})); return false; } - if (tx && tx->GetHasEffects() && txCtx.Locks.Broken()) { + if (tx && tx->GetHasEffects() && broken) { ReplyQueryError(Ydb::StatusIds::ABORTED, "tx has effects, but locks are broken", - MessageFromIssues(std::vector{txCtx.Locks.GetIssue()})); + MessageFromIssues(std::vector{txCtx.TxManager ? *txCtx.TxManager->GetLockIssue() : txCtx.Locks.GetIssue()})); return false; } @@ -1148,9 +1156,10 @@ class TKqpSessionActor : public TActorBootstrapped { auto& txCtx = *QueryState->TxCtx; bool literal = tx && tx->IsLiteralTx(); + const bool hasLocks = txCtx.TxManager ? txCtx.TxManager->HasLocks() : txCtx.Locks.HasLocks(); if (commit) { - if (txCtx.TxHasEffects() || txCtx.Locks.HasLocks() || txCtx.TopicOperations.HasOperations()) { + if (txCtx.TxHasEffects() || hasLocks || txCtx.TopicOperations.HasOperations()) { // Cannot perform commit in literal execution literal = false; } else if (!tx) { @@ -1214,20 +1223,37 @@ class TKqpSessionActor : public TActorBootstrapped { request.PerShardKeysSizeLimitBytes = Config->_CommitPerShardKeysSizeLimitBytes.Get().GetRef(); } - if (txCtx.Locks.HasLocks() || txCtx.TopicOperations.HasOperations()) { - if (!txCtx.GetSnapshot().IsValid() || txCtx.TxHasEffects() || txCtx.TopicOperations.HasOperations()) { - LOG_D("TExecPhysicalRequest, tx has commit locks"); - request.LocksOp = ELocksOp::Commit; - } else { - LOG_D("TExecPhysicalRequest, tx has rollback locks"); - request.LocksOp = ELocksOp::Rollback; + if (Settings.TableService.GetEnableOltpSink()) { + if (txCtx.TxHasEffects() || hasLocks || txCtx.TopicOperations.HasOperations()) { + request.AcquireLocksTxId = txCtx.Locks.GetLockTxId(); } - for (auto& [lockId, lock] : txCtx.Locks.LocksMap) { - auto dsLock = ExtractLock(lock.GetValueRef(txCtx.Locks.LockType)); - request.DataShardLocks[dsLock.GetDataShard()].emplace_back(dsLock); + if (hasLocks) { + if (!txCtx.GetSnapshot().IsValid() || txCtx.TxHasEffects() || txCtx.TopicOperations.HasOperations()) { + LOG_D("TExecPhysicalRequest, tx has commit locks"); + request.LocksOp = ELocksOp::Commit; + } else { + LOG_D("TExecPhysicalRequest, tx has rollback locks"); + request.LocksOp = ELocksOp::Rollback; + } + } else if (txCtx.TxHasEffects()) { + LOG_D("TExecPhysicalRequest, need commit locks"); + request.LocksOp = ELocksOp::Commit; + } + } else { + if (hasLocks || txCtx.TopicOperations.HasOperations()) { + if (!txCtx.GetSnapshot().IsValid() || txCtx.TxHasEffects() || txCtx.TopicOperations.HasOperations()) { + LOG_D("TExecPhysicalRequest, tx has commit locks"); + request.LocksOp = ELocksOp::Commit; + } else { + LOG_D("TExecPhysicalRequest, tx has rollback locks"); + request.LocksOp = ELocksOp::Rollback; + } + for (auto& [lockId, lock] : txCtx.Locks.LocksMap) { + auto dsLock = ExtractLock(lock.GetValueRef(txCtx.Locks.LockType)); + request.DataShardLocks[dsLock.GetDataShard()].emplace_back(dsLock); + } } - } request.TopicOperations = std::move(txCtx.TopicOperations); @@ -1243,7 +1269,7 @@ class TKqpSessionActor : public TActorBootstrapped { QueryState->Orbit, QueryState->CurrentTx, request.Transactions.size(), - txCtx.Locks.Size(), + (txCtx.TxManager ? txCtx.TxManager->GetShardsCount() : txCtx.Locks.Size()), request.AcquireLocksTxId.Defined()); SendToExecuter(QueryState->TxCtx.Get(), std::move(request)); @@ -1294,19 +1320,34 @@ class TKqpSessionActor : public TActorBootstrapped { request.ResourceManager_ = ResourceManager_; LOG_D("Sending to Executer TraceId: " << request.TraceId.GetTraceId() << " " << request.TraceId.GetSpanIdSize()); + if (Settings.TableService.GetEnableOltpSink() && !txCtx->TxManager) { + txCtx->TxManager = CreateKqpTransactionManager(); + } + if (Settings.TableService.GetEnableOltpSink() && !txCtx->BufferActorId && txCtx->HasTableWrite) { + TKqpBufferWriterSettings settings { + .SessionActorId = SelfId(), + .TxManager = txCtx->TxManager, + .TraceId = request.TraceId.GetTraceId(), + .Counters = Counters, + .TxProxyMon = RequestCounters->TxProxyMon, + }; + auto* actor = CreateKqpBufferWriterActor(std::move(settings)); + txCtx->BufferActorId = RegisterWithSameMailbox(actor); + } auto executerActor = CreateKqpExecuter(std::move(request), Settings.Database, QueryState ? QueryState->UserToken : TIntrusiveConstPtr(), RequestCounters, Settings.TableService, AsyncIoFactory, QueryState ? QueryState->PreparedQuery : nullptr, SelfId(), QueryState ? QueryState->UserRequestContext : MakeIntrusive("", Settings.Database, SessionId), - QueryState ? QueryState->StatementResultIndex : 0, FederatedQuerySetup, GUCSettings, txCtx->ShardIdToTableInfo); + QueryState ? QueryState->StatementResultIndex : 0, FederatedQuerySetup, GUCSettings, + txCtx->ShardIdToTableInfo, txCtx->TxManager, txCtx->BufferActorId); auto exId = RegisterWithSameMailbox(executerActor); LOG_D("Created new KQP executer: " << exId << " isRollback: " << isRollback); auto ev = std::make_unique(exId); Send(MakeTxProxyID(), ev.release()); if (!isRollback) { - Y_ABORT_UNLESS(!ExecuterId); + YQL_ENSURE(!ExecuterId); } ExecuterId = exId; } @@ -1456,9 +1497,13 @@ class TKqpSessionActor : public TActorBootstrapped { // Invalidate query cache on scheme/internal errors switch (status) { case Ydb::StatusIds::ABORTED: { - if (ev->BrokenLockPathId) { + if (QueryState->TxCtx->TxManager && QueryState->TxCtx->TxManager->BrokenLocks()) { + issues.AddIssue(*QueryState->TxCtx->TxManager->GetLockIssue()); + } else if (ev->BrokenLockPathId) { + YQL_ENSURE(!QueryState->TxCtx->TxManager); issues.AddIssue(GetLocksInvalidatedIssue(*QueryState->TxCtx, *ev->BrokenLockPathId)); } else if (ev->BrokenLockShardId) { + YQL_ENSURE(!QueryState->TxCtx->TxManager); issues.AddIssue(GetLocksInvalidatedIssue(*QueryState->TxCtx->ShardIdToTableInfo, *ev->BrokenLockShardId)); } break; @@ -1506,7 +1551,7 @@ class TKqpSessionActor : public TActorBootstrapped { QueryState->TxCtx->Locks.LockHandle = std::move(ev->LockHandle); } - if (!MergeLocksWithTxResult(executerResults)) { + if (!QueryState->TxCtx->TxManager && !MergeLocksWithTxResult(executerResults)) { return; } @@ -1546,6 +1591,29 @@ class TKqpSessionActor : public TActorBootstrapped { } } + void Handle(TEvKqpBuffer::TEvError::TPtr& ev) { + const auto& msg = *ev->Get(); + + TString logMsg = TStringBuilder() << "got TEvKqpBuffer::TEvError in " << CurrentStateFuncName() + << ", status: " << NYql::NDqProto::StatusIds_StatusCode_Name(msg.StatusCode) << " send to: " << ExecuterId << " from: " << ev->Sender; + + if (!QueryState || !QueryState->TxCtx || QueryState->TxCtx->BufferActorId != ev->Sender) { + LOG_E(logMsg << ": Old error."); + return; + } else { + LOG_W(logMsg); + } + + TString reason = TStringBuilder() << msg.Message << "; " << msg.SubIssues.ToString(); + + if (ExecuterId) { + auto abortEv = MakeHolder(msg.StatusCode, reason); + Send(ExecuterId, abortEv.Release(), IEventHandle::FlagTrackDelivery); + } else { + ReplyQueryError(NYql::NDq::DqStatusToYdbStatus(msg.StatusCode), logMsg, MessageFromIssues(msg.SubIssues)); + } + } + void CollectSystemViewQueryStats(const TKqpQueryStats* stats, TDuration queryDuration, const TString& database, ui64 requestUnits) { @@ -1880,7 +1948,7 @@ class TKqpSessionActor : public TActorBootstrapped { } void Reply() { - YQL_ENSURE(QueryState); + Y_ABORT_UNLESS(QueryState); YQL_ENSURE(Counters); auto& record = QueryResponse->Record.GetRef(); @@ -2016,10 +2084,12 @@ class TKqpSessionActor : public TActorBootstrapped { request.LocksOp = ELocksOp::Rollback; - // Should tx with empty LocksMap be aborted? - for (auto& [lockId, lock] : txCtx->Locks.LocksMap) { - auto dsLock = ExtractLock(lock.GetValueRef(txCtx->Locks.LockType)); - request.DataShardLocks[dsLock.GetDataShard()].emplace_back(dsLock); + if (!txCtx->TxManager) { + // Should tx with empty LocksMap be aborted? + for (auto& [lockId, lock] : txCtx->Locks.LocksMap) { + auto dsLock = ExtractLock(lock.GetValueRef(txCtx->Locks.LockType)); + request.DataShardLocks[dsLock.GetDataShard()].emplace_back(dsLock); + } } SendToExecuter(txCtx, std::move(request), true); @@ -2029,6 +2099,13 @@ class TKqpSessionActor : public TActorBootstrapped { if (QueryState->TxCtx) { QueryState->TxCtx->ClearDeferredEffects(); QueryState->TxCtx->Locks.Clear(); + QueryState->TxCtx->TxManager.reset(); + + if (QueryState->TxCtx->BufferActorId) { + Send(QueryState->TxCtx->BufferActorId, new TEvKqpBuffer::TEvTerminate{}); + QueryState->TxCtx->BufferActorId = {}; + } + QueryState->TxCtx->Finish(); } } @@ -2304,6 +2381,7 @@ class TKqpSessionActor : public TActorBootstrapped { hFunc(NWorkload::TEvContinueRequest, HandleNoop); // message from KQP proxy in case of our reply just after kqp proxy timer tick hFunc(NYql::NDq::TEvDq::TEvAbortExecution, HandleNoop); + hFunc(TEvKqpBuffer::TEvError, Handle); hFunc(TEvTxUserProxy::TEvAllocateTxIdResult, HandleNoop); default: @@ -2338,6 +2416,7 @@ class TKqpSessionActor : public TActorBootstrapped { hFunc(TEvKqpExecuter::TEvStreamDataAck, HandleExecute); hFunc(NYql::NDq::TEvDq::TEvAbortExecution, HandleExecute); + hFunc(TEvKqpBuffer::TEvError, Handle); hFunc(TEvKqp::TEvCloseSessionRequest, HandleExecute); hFunc(NGRpcService::TEvClientLost, HandleClientLost); @@ -2385,6 +2464,7 @@ class TKqpSessionActor : public TActorBootstrapped { hFunc(TEvKqp::TEvCompileResponse, HandleNoop); hFunc(TEvKqp::TEvSplitResponse, HandleNoop); hFunc(NYql::NDq::TEvDq::TEvAbortExecution, HandleNoop); + hFunc(TEvKqpBuffer::TEvError, Handle); hFunc(TEvTxProxySchemeCache::TEvNavigateKeySetResult, HandleNoop); hFunc(TEvents::TEvUndelivered, HandleNoop); hFunc(TEvTxUserProxy::TEvAllocateTxIdResult, HandleNoop); diff --git a/ydb/core/kqp/session_actor/kqp_session_actor.h b/ydb/core/kqp/session_actor/kqp_session_actor.h index f26fff2b00ca..50ad957d2453 100644 --- a/ydb/core/kqp/session_actor/kqp_session_actor.h +++ b/ydb/core/kqp/session_actor/kqp_session_actor.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include diff --git a/ydb/core/kqp/ut/cost/kqp_cost_ut.cpp b/ydb/core/kqp/ut/cost/kqp_cost_ut.cpp index 2077c10116e0..e10bb7f9e560 100644 --- a/ydb/core/kqp/ut/cost/kqp_cost_ut.cpp +++ b/ydb/core/kqp/ut/cost/kqp_cost_ut.cpp @@ -1,5 +1,6 @@ #include +#include #include #include @@ -11,12 +12,14 @@ namespace NKqp { using namespace NYdb; using namespace NYdb::NTable; -static NKikimrConfig::TAppConfig GetAppConfig(bool sourceRead, bool streamLookup = true, bool streamLookupJoin = false) { +static NKikimrConfig::TAppConfig GetAppConfig(bool sourceRead = false, bool streamLookup = true, bool streamLookupJoin = false, bool enableOltpSink = false) { auto app = NKikimrConfig::TAppConfig(); app.MutableTableServiceConfig()->SetEnableKqpDataQuerySourceRead(sourceRead); app.MutableTableServiceConfig()->SetEnableKqpScanQuerySourceRead(sourceRead); app.MutableTableServiceConfig()->SetEnableKqpDataQueryStreamLookup(streamLookup); app.MutableTableServiceConfig()->SetEnableKqpDataQueryStreamIdxLookupJoin(streamLookupJoin); + app.MutableTableServiceConfig()->SetEnableOlapSink(true); + app.MutableTableServiceConfig()->SetEnableOltpSink(enableOltpSink); return app; } @@ -26,6 +29,11 @@ static NYdb::NTable::TExecDataQuerySettings GetDataQuerySettings() { return execSettings; } +static NYdb::NQuery::TExecuteQuerySettings GetQuerySettings() { + NYdb::NQuery::TExecuteQuerySettings execSettings; + execSettings.StatsMode(NYdb::NQuery::EStatsMode::Basic); + return execSettings; +} static void CreateSampleTables(TSession session) { UNIT_ASSERT(session.ExecuteSchemeQuery(R"( @@ -103,8 +111,11 @@ Y_UNIT_TEST_SUITE(KqpCost) { //runtime->SetLogPriority(NKikimrServices::GRPC_SERVER, NActors::NLog::PRI_DEBUG); } - Y_UNIT_TEST_TWIN(IndexLookup, StreamLookup) { - TKikimrRunner kikimr(GetAppConfig(true, StreamLookup)); + Y_UNIT_TEST_QUAD(IndexLookup, StreamLookup, useSink) { + if (useSink && !StreamLookup) { + return; + } + TKikimrRunner kikimr(GetAppConfig(true, StreamLookup, false, useSink)); auto db = kikimr.GetTableClient(); auto session = db.CreateSession().GetValueSync().GetSession(); @@ -118,10 +129,9 @@ Y_UNIT_TEST_SUITE(KqpCost) { PRIMARY KEY (Key), INDEX Index GLOBAL ON (Fk) ); - )").GetValueSync(); - session.ExecuteDataQuery(R"( + auto prepare = session.ExecuteDataQuery(R"( REPLACE INTO `/Root/SecondaryKeys` (Key, Fk, Value, ValueInt) VALUES (1, 1, "Payload1", 100), (2, 2, "Payload2", 200), @@ -130,6 +140,7 @@ Y_UNIT_TEST_SUITE(KqpCost) { (7, NULL, "Payload7", 700), (NULL, NULL, "Payload8", 800); )", TTxControl::BeginTx().CommitTx()).GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(prepare.GetStatus(), EStatus::SUCCESS, prepare.GetIssues().ToString()); auto query = Q_(R"( SELECT Value FROM `/Root/SecondaryKeys` VIEW Index WHERE Fk = 1; @@ -168,8 +179,11 @@ Y_UNIT_TEST_SUITE(KqpCost) { UNIT_ASSERT_VALUES_EQUAL(readsByTable.at("/Root/SecondaryKeys/Index/indexImplTable").second, 8); } - Y_UNIT_TEST_TWIN(IndexLookupAtLeast8BytesInStorage, StreamLookup) { - TKikimrRunner kikimr(GetAppConfig(true, StreamLookup)); + Y_UNIT_TEST_QUAD(IndexLookupAtLeast8BytesInStorage, StreamLookup, useSink) { + if (useSink && !StreamLookup) { + return; + } + TKikimrRunner kikimr(GetAppConfig(true, StreamLookup, false, useSink)); auto db = kikimr.GetTableClient(); auto session = db.CreateSession().GetValueSync().GetSession(); @@ -234,8 +248,11 @@ Y_UNIT_TEST_SUITE(KqpCost) { UNIT_ASSERT_VALUES_EQUAL(readsByTable.at("/Root/SecondaryKeys/Index/indexImplTable").second, 8); } - Y_UNIT_TEST_TWIN(IndexLookupAndTake, StreamLookup) { - TKikimrRunner kikimr(GetAppConfig(true, StreamLookup)); + Y_UNIT_TEST_QUAD(IndexLookupAndTake, StreamLookup, useSink) { + if (useSink && !StreamLookup) { + return; + } + TKikimrRunner kikimr(GetAppConfig(true, StreamLookup, false, useSink)); auto db = kikimr.GetTableClient(); auto session = db.CreateSession().GetValueSync().GetSession(); @@ -354,7 +371,7 @@ Y_UNIT_TEST_SUITE(KqpCost) { } Y_UNIT_TEST_TWIN(IndexLookupJoin, StreamLookupJoin) { - TKikimrRunner kikimr(GetAppConfig(true, true, StreamLookupJoin)); + TKikimrRunner kikimr(GetAppConfig(true, true, StreamLookupJoin, false)); auto db = kikimr.GetTableClient(); auto session = db.CreateSession().GetValueSync().GetSession(); @@ -487,6 +504,466 @@ Y_UNIT_TEST_SUITE(KqpCost) { CompareYson(Expected, res.ResultSetYson); } + void CreateTestTable(auto session, bool isColumn) { + UNIT_ASSERT(session.ExecuteQuery(std::format(R"( + --!syntax_v1 + CREATE TABLE `/Root/TestTable` ( + Group Uint32 not null, + Name String not null, + Amount Uint64, + Comment String, + PRIMARY KEY (Group, Name) + ) WITH ( + STORE = {}, + AUTO_PARTITIONING_MIN_PARTITIONS_COUNT = 10 + ); + + )", isColumn ? "COLUMN" : "ROW"), NYdb::NQuery::TTxControl::NoTx()).GetValueSync().IsSuccess()); + + auto result = session.ExecuteQuery(R"( + REPLACE INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES + (1u, "Anna", 3500ul, "None"), + (1u, "Paul", 300ul, "None"), + (2u, "Tony", 7200ul, "None"); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).GetValueSync(); + UNIT_ASSERT_C(result.IsSuccess(), result.GetIssues().ToString()); + } + + Y_UNIT_TEST(OlapPointLookup) { + TKikimrRunner kikimr(GetAppConfig(false, false)); + auto db = kikimr.GetQueryClient(); + auto session = db.GetSession().GetValueSync().GetSession(); + + CreateTestTable(session, true); + + auto query = Q_(R"( + SELECT * FROM `/Root/TestTable` WHERE Group = 1u AND Name = "Anna"; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + CompareYson(R"( + [ + [[3500u];["None"]; + 1u;"Anna"] + ] + )", NYdb::FormatResultSetYson(result.GetResultSet(0))); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.query_phases().size() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).reads().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).reads().bytes(), 36); + } + + Y_UNIT_TEST(OlapRange) { + TKikimrRunner kikimr(GetAppConfig()); + auto db = kikimr.GetQueryClient(); + auto session = db.GetSession().GetValueSync().GetSession(); + + CreateTestTable(session, true); + + auto query = Q_(R"( + SELECT * FROM `/Root/TestTable` WHERE Group < 2u ORDER BY Group, Name; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + CompareYson(R"( + [ + [[3500u];["None"];1u;"Anna"]; + [[300u];["None"];1u;"Paul"] + ] + )", NYdb::FormatResultSetYson(result.GetResultSet(0))); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).reads().rows(), 2); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).reads().bytes(), 72); + } + + Y_UNIT_TEST(OlapRangeFullScan) { + TKikimrRunner kikimr(GetAppConfig()); + + auto db = kikimr.GetQueryClient(); + auto session = db.GetSession().GetValueSync().GetSession(); + + CreateTestTable(session, true); + + auto query = Q_(R"( + SELECT * FROM `/Root/TestTable` WHERE Amount < 5000ul ORDER BY Group, Name LIMIT 1; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + CompareYson(R"( + [ + [[3500u];["None"];1u;"Anna"] + ] + )", NYdb::FormatResultSetYson(result.GetResultSet(0))); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(0).table_access(0).reads().rows(), 2); // Limit??? + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(0).table_access(0).reads().bytes(), 72); + } + + Y_UNIT_TEST(OlapWriteRow) { + TKikimrRunner kikimr(GetAppConfig(false, false, false, true)); + auto db = kikimr.GetQueryClient(); + auto session = db.GetSession().GetValueSync().GetSession(); + + CreateTestTable(session, true); + + { + auto query = Q_(R"( + REPLACE INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (1u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + } + + { + auto query = Q_(R"( + UPSERT INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (1u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + } + + { + // INSERT EXISTS + auto query = Q_(R"( + INSERT INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (1u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + // TODO: fix status? + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::BAD_REQUEST); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + // TODO: reads??? + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().bytes(), 8); + } + + { + // INSERT NEW + auto query = Q_(R"( + INSERT INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (3u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + } + + { + // UPDATE empty + auto query = Q_(R"( + UPDATE `/Root/TestTable` ON SELECT 4u AS Group, "Anna" AS Name, 4000u AS Amount, "None" AS Comment; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + // No reads & no writes + for (int phase = 0; phase < stats.query_phases_size(); ++phase) { + if (stats.query_phases(phase).table_access_size() > 0) { + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access_size(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 0); + } + } + } + + { + // UPDATE + auto query = Q_(R"( + UPDATE `/Root/TestTable` ON SELECT 3u AS Group, "Anna" AS Name, 4000u AS Amount, "None" AS Comment; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + // TODO: reads??? + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().bytes(), 8); + } + + { + // DELETE empty + auto query = Q_(R"( + DELETE FROM `/Root/TestTable` ON SELECT 4u AS Group, "Anna" AS Name; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().bytes(), 0); + } + + { + // DELETE + auto query = Q_(R"( + DELETE FROM `/Root/TestTable` ON SELECT 3u AS Group, "Anna" AS Name; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().bytes(), 0); + } + } + + Y_UNIT_TEST_TWIN(OltpWriteRow, isSink) { + TKikimrRunner kikimr(GetAppConfig(false, false, false, isSink)); + auto db = kikimr.GetQueryClient(); + auto session = db.GetSession().GetValueSync().GetSession(); + + CreateTestTable(session, false); + + { + auto query = Q_(R"( + REPLACE INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (1u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + } + + { + auto query = Q_(R"( + UPSERT INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (1u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + } + + { + // INSERT EXISTS + auto query = Q_(R"( + INSERT INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (1u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + if (isSink) { // TODO: fix status? + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::BAD_REQUEST); + } else { + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::PRECONDITION_FAILED); + } + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + if (isSink) { + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases_size(), 1); + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access_size(), 0); + // TODO: reads??? + } else { + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().bytes(), 8); + } + } + + { + // INSERT NEW + auto query = Q_(R"( + INSERT INTO `/Root/TestTable` (Group, Name, Amount, Comment) VALUES (3u, "Anna", 3500u, "None"); + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + } + + { + // UPDATE empty + auto query = Q_(R"( + UPDATE `/Root/TestTable` ON SELECT 4u AS Group, "Anna" AS Name, 4000u AS Amount, "None" AS Comment; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + // No reads & no writes + for (int phase = 0; phase < stats.query_phases_size(); ++phase) { + if (stats.query_phases(phase).table_access_size() > 0) { + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access_size(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 0); + } + } + } + + { + // UPDATE + auto query = Q_(R"( + UPDATE `/Root/TestTable` ON SELECT 3u AS Group, "Anna" AS Name, 4000u AS Amount, "None" AS Comment; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).updates().bytes(), 20); + if (!isSink) { + // TODO: reads??? + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(1).table_access(0).reads().bytes(), 8); + } + } + + { + // DELETE empty + auto query = Q_(R"( + DELETE FROM `/Root/TestTable` ON SELECT 4u AS Group, "Anna" AS Name; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().bytes(), 0); + } + + { + // DELETE + auto query = Q_(R"( + DELETE FROM `/Root/TestTable` ON SELECT 3u AS Group, "Anna" AS Name; + )"); + + auto txControl = NYdb::NQuery::TTxControl::BeginTx().CommitTx(); + + auto result = session.ExecuteQuery(query, txControl, GetQuerySettings()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL(result.GetStatus(), EStatus::SUCCESS); + + auto stats = NYdb::TProtoAccessor::GetProto(*result.GetStats()); + + Cerr << stats.DebugString() << Endl; + size_t phase = stats.query_phases_size() - 1; + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().rows(), 1); + UNIT_ASSERT_VALUES_EQUAL(stats.query_phases(phase).table_access(0).deletes().bytes(), 0); + } + } + } } diff --git a/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp b/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp index 9165c99f37ea..61664eed6fdf 100644 --- a/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp +++ b/ydb/core/kqp/ut/olap/kqp_olap_ut.cpp @@ -1824,7 +1824,11 @@ Y_UNIT_TEST_SUITE(KqpOlap) { } void TestOlapUpsert(ui32 numShards) { + NKikimrConfig::TAppConfig appConfig; + appConfig.MutableTableServiceConfig()->SetEnableOlapSink(true); + appConfig.MutableTableServiceConfig()->SetAllowOlapDataQuery(true); auto settings = TKikimrSettings() + .SetAppConfig(appConfig) .SetWithSampleTables(false); TKikimrRunner kikimr(settings); @@ -1867,27 +1871,20 @@ Y_UNIT_TEST_SUITE(KqpOlap) { ORDER BY CounterID, WatchID )"; - auto it = tableClient.StreamExecuteScanQuery(query).GetValueSync(); + auto it = session.ExecuteDataQuery(query, TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).GetValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); - TString result = StreamResultToYson(it); + TString result = FormatResultSetYson(it.GetResultSet(0)); Cout << result << Endl; - //CompareYson(result, R"([[0;15];[1;15]])"); - CompareYson(result, R"([])"); // FIXME + CompareYson(result, R"([[15;0];[15;1]])"); } } Y_UNIT_TEST(OlapUpsertImmediate) { - // Should be fixed in KIKIMR-17646 - return; - TestOlapUpsert(1); } Y_UNIT_TEST(OlapUpsert) { - // Should be fixed in KIKIMR-17646 - return; - TestOlapUpsert(2); } diff --git a/ydb/core/kqp/ut/scheme/kqp_acl_ut.cpp b/ydb/core/kqp/ut/scheme/kqp_acl_ut.cpp index 1ea3da6bfe1f..690fb89ee928 100644 --- a/ydb/core/kqp/ut/scheme/kqp_acl_ut.cpp +++ b/ydb/core/kqp/ut/scheme/kqp_acl_ut.cpp @@ -154,6 +154,154 @@ Y_UNIT_TEST_SUITE(KqpAcl) { driver.Stop(true); } + + Y_UNIT_TEST_TWIN(AclForOltpAndOlap, isOlap) { + const TString query = Sprintf(R"( + CREATE TABLE `/Root/test_acl` ( + id Int64 NOT NULL, + name String, + primary key (id) + ) WITH (STORE=%s); + )", isOlap ? "COLUMN" : "ROW"); + + TKikimrRunner kikimr; + + { + auto driverConfig = TDriverConfig() + .SetEndpoint(kikimr.GetEndpoint()) + .SetAuthToken("root@builtin"); + auto driver = TDriver(driverConfig); + auto client = NYdb::NQuery::TQueryClient(driver); + + AssertSuccessResult(client.ExecuteQuery(query, NYdb::NQuery::TTxControl::NoTx()).ExtractValueSync()); + + driver.Stop(true); + } + + { + auto schemeClient = kikimr.GetSchemeClient(); + NYdb::NScheme::TPermissions permissions("user0@builtin", {}); + AssertSuccessResult(schemeClient.ModifyPermissions("/Root/test_acl", + NYdb::NScheme::TModifyPermissionsSettings().AddGrantPermissions(permissions) + ).ExtractValueSync() + ); + } + + { + auto driverConfig = TDriverConfig() + .SetEndpoint(kikimr.GetEndpoint()) + .SetAuthToken("user0@builtin"); + auto driver = TDriver(driverConfig); + auto client = NYdb::NQuery::TQueryClient(driver); + + auto result = client.ExecuteQuery(R"( + SELECT * FROM `/Root/test_acl`; + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(!result.IsSuccess(), result.GetIssues().ToString()); + const auto expectedIssueMessage = "Cannot find table 'db.[/Root/test_acl]' because it does not exist or you do not have access permissions."; + UNIT_ASSERT_C(result.GetIssues().ToString().Contains(expectedIssueMessage), result.GetIssues().ToString()); + + auto resultWrite = client.ExecuteQuery(R"( + REPLACE INTO `/Root/test_acl` (id, name) VALUES (1, 'test'); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(!resultWrite.IsSuccess(), resultWrite.GetIssues().ToString()); + UNIT_ASSERT_C(resultWrite.GetIssues().ToString().Contains(expectedIssueMessage), resultWrite.GetIssues().ToString()); + + driver.Stop(true); + } + + { + auto schemeClient = kikimr.GetSchemeClient(); + NYdb::NScheme::TPermissions permissions("user0@builtin", {"ydb.deprecated.describe_schema"}); + AssertSuccessResult(schemeClient.ModifyPermissions("/Root/test_acl", + NYdb::NScheme::TModifyPermissionsSettings().AddGrantPermissions(permissions) + ).ExtractValueSync() + ); + } + + { + auto driverConfig = TDriverConfig() + .SetEndpoint(kikimr.GetEndpoint()) + .SetAuthToken("user0@builtin"); + auto driver = TDriver(driverConfig); + auto client = NYdb::NQuery::TQueryClient(driver); + + auto result = client.ExecuteQuery(R"( + SELECT * FROM `/Root/test_acl`; + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(!result.IsSuccess(), result.GetIssues().ToString()); + const auto expectedIssueMessage = "Failed to resolve table `/Root/test_acl` status: AccessDenied., code: 2028"; + UNIT_ASSERT_C(result.GetIssues().ToString().Contains(expectedIssueMessage), result.GetIssues().ToString()); + + auto resultWrite = client.ExecuteQuery(R"( + REPLACE INTO `/Root/test_acl` (id, name) VALUES (1, 'test'); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(!resultWrite.IsSuccess(), resultWrite.GetIssues().ToString()); + UNIT_ASSERT_C(resultWrite.GetIssues().ToString().Contains(expectedIssueMessage), resultWrite.GetIssues().ToString()); + + driver.Stop(true); + } + + { + auto schemeClient = kikimr.GetSchemeClient(); + NYdb::NScheme::TPermissions permissions("user0@builtin", {"ydb.deprecated.describe_schema", "ydb.deprecated.select_row"}); + AssertSuccessResult(schemeClient.ModifyPermissions("/Root/test_acl", + NYdb::NScheme::TModifyPermissionsSettings().AddGrantPermissions(permissions) + ).ExtractValueSync() + ); + } + + { + auto driverConfig = TDriverConfig() + .SetEndpoint(kikimr.GetEndpoint()) + .SetAuthToken("user0@builtin"); + auto driver = TDriver(driverConfig); + auto client = NYdb::NQuery::TQueryClient(driver); + + auto result = client.ExecuteQuery(R"( + SELECT * FROM `/Root/test_acl`; + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(result.IsSuccess(), result.GetIssues().ToString()); + + auto resultWrite = client.ExecuteQuery(R"( + REPLACE INTO `/Root/test_acl` (id, name) VALUES (1, 'test'); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(!resultWrite.IsSuccess(), resultWrite.GetIssues().ToString()); + const auto expectedIssueMessage = "Failed to resolve table `/Root/test_acl` status: AccessDenied., code: 2028"; + UNIT_ASSERT_C(resultWrite.GetIssues().ToString().Contains(expectedIssueMessage), resultWrite.GetIssues().ToString()); + + driver.Stop(true); + } + + { + auto schemeClient = kikimr.GetSchemeClient(); + NYdb::NScheme::TPermissions permissions("user0@builtin", {"ydb.deprecated.update_row"}); + AssertSuccessResult(schemeClient.ModifyPermissions("/Root/test_acl", + NYdb::NScheme::TModifyPermissionsSettings().AddGrantPermissions(permissions) + ).ExtractValueSync() + ); + } + + { + auto driverConfig = TDriverConfig() + .SetEndpoint(kikimr.GetEndpoint()) + .SetAuthToken("user0@builtin"); + auto driver = TDriver(driverConfig); + auto client = NYdb::NQuery::TQueryClient(driver); + + auto result = client.ExecuteQuery(R"( + SELECT * FROM `/Root/test_acl`; + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(result.IsSuccess(), result.GetIssues().ToString()); + + auto resultWrite = client.ExecuteQuery(R"( + REPLACE INTO `/Root/test_acl` (id, name) VALUES (1, 'test'); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + UNIT_ASSERT_C(resultWrite.IsSuccess(), resultWrite.GetIssues().ToString()); + + driver.Stop(true); + } + } } } // namespace NKqp diff --git a/ydb/core/kqp/ut/service/kqp_qs_queries_ut.cpp b/ydb/core/kqp/ut/service/kqp_qs_queries_ut.cpp index a8803bee347d..f52c9fa427b8 100644 --- a/ydb/core/kqp/ut/service/kqp_qs_queries_ut.cpp +++ b/ydb/core/kqp/ut/service/kqp_qs_queries_ut.cpp @@ -3634,7 +3634,7 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { } } - Y_UNIT_TEST(TableSink_ReplaceColumnShard) { + Y_UNIT_TEST(TableSink_Olap_Replace) { NKikimrConfig::TAppConfig appConfig; appConfig.MutableTableServiceConfig()->SetEnableOlapSink(true); auto settings = TKikimrSettings() @@ -3717,14 +3717,14 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { auto it = client.ExecuteQuery(R"( REPLACE INTO `/Root/DataShard` (Col1, Col2) VALUES (0u, 0); REPLACE INTO `/Root/DataShard` (Col1, Col3) VALUES (1u, 'test'); - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;[0];#];[1u;#;["test"]]])"); @@ -3734,7 +3734,7 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { auto it = client.ExecuteQuery(R"( REPLACE INTO `/Root/DataShard` (Col1, Col3) VALUES (0u, 'null'); REPLACE INTO `/Root/DataShard` (Col1) VALUES (1u); - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } @@ -3742,7 +3742,7 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { { auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;#;["null"]];[1u;#;#]])"); @@ -3757,8 +3757,8 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { virtual void DoExecute() = 0; public: void Execute() { - AppConfig.MutableTableServiceConfig()->SetEnableOlapSink(true); - AppConfig.MutableTableServiceConfig()->SetEnableOltpSink(true); + AppConfig.MutableTableServiceConfig()->SetEnableOlapSink(IsOlap); + AppConfig.MutableTableServiceConfig()->SetEnableOltpSink(!IsOlap); AppConfig.MutableTableServiceConfig()->SetEnableKqpDataQueryStreamLookup(true); AppConfig.MutableTableServiceConfig()->SetOldLookupJoinBehaviour(false); auto settings = TKikimrSettings().SetAppConfig(AppConfig).SetWithSampleTables(false); @@ -3864,14 +3864,14 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { INSERT INTO `/Root/DataShard` (Col1, Col2) VALUES (0u, 0); INSERT INTO `/Root/DataShard` (Col1, Col3) VALUES (1u, 'test'); INSERT INTO `/Root/DataShard` (Col1, Col3, Col2) VALUES (2u, 't', 3); - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard` ORDER BY Col1; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;[0];#];[1u;#;["test"]];[2u;[3];["t"]]])"); @@ -3880,7 +3880,7 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { { auto it = client.ExecuteQuery(R"( INSERT INTO `/Root/DataShard` (Col1, Col3) VALUES (0u, 'null'); - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(!it.IsSuccess(), it.GetIssues().ToString()); UNIT_ASSERT_C( it.GetIssues().ToString().Contains("Operation is aborting because an duplicate key") @@ -3891,7 +3891,7 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { { auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard` ORDER BY Col1; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;[0];#];[1u;#;["test"]];[2u;[3];["t"]]])"); @@ -3996,14 +3996,14 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { INSERT INTO `/Root/DataShard` (Col1, Col2) VALUES (0u, 0); INSERT INTO `/Root/DataShard` (Col1, Col3) VALUES (1u, 'test'); INSERT INTO `/Root/DataShard` (Col1, Col3, Col2) VALUES (2u, 't', 3); - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard` ORDER BY Col1; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;[0];#];[1u;#;["test"]];[2u;[3];["t"]]])"); @@ -4012,21 +4012,21 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { { auto it = client.ExecuteQuery(R"( UPDATE `/Root/DataShard` SET Col2 = 42 WHERE Col3 == 'not found'; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } { auto it = client.ExecuteQuery(R"( UPDATE `/Root/DataShard` SET Col2 = 42 WHERE Col3 == 't'; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard` ORDER BY Col1; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;[0];#];[1u;#;["test"]];[2u;[42];["t"]]])"); @@ -4035,20 +4035,20 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { { auto it = client.ExecuteQuery(R"( UPDATE `/Root/DataShard` ON SELECT 0u AS Col1, 1 AS Col2, 'text' AS Col3; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } { auto it = client.ExecuteQuery(R"( UPDATE `/Root/DataShard` ON SELECT 10u AS Col1, 1 AS Col2, 'text' AS Col3; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(it.IsSuccess(), it.GetIssues().ToString()); } auto it = client.StreamExecuteQuery(R"( SELECT * FROM `/Root/DataShard` ORDER BY Col1; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson(output, R"([[0u;[1];["text"]];[1u;#;["test"]];[2u;[42];["t"]]])"); @@ -4164,7 +4164,8 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { } } - Y_UNIT_TEST_TWIN(TableSink_ReplaceDataShard, UseSink) { + Y_UNIT_TEST_TWIN(TableSink_Oltp_Replace, UseSink) { + //UseSink = true; NKikimrConfig::TAppConfig appConfig; appConfig.MutableTableServiceConfig()->SetEnableOlapSink(UseSink); appConfig.MutableTableServiceConfig()->SetEnableOltpSink(UseSink); @@ -4210,14 +4211,14 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { auto prepareResult = client.ExecuteQuery(R"( REPLACE INTO `/Root/DataShard` (Col1, Col2, Col3) VALUES (10u, "test1", 10), (20u, "test2", 11), (2147483647u, "test3", 12), (2147483640u, NULL, 13); - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT COUNT(*) FROM `/Root/DataShard`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson( @@ -4228,14 +4229,14 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { { auto prepareResult = client.ExecuteQuery(R"( REPLACE INTO `/Root/DataShard2` SELECT * FROM `/Root/DataShard`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT COUNT(*) FROM `/Root/DataShard2`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson( @@ -4248,14 +4249,14 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { REPLACE INTO `/Root/DataShard2` (Col1, Col2, Col3) VALUES (11u, "test1", 10), (21u, "test2", 11), (2147483646u, "test3", 12), (2147483641u, NULL, 13); SELECT COUNT(*) FROM `/Root/DataShard`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); } { auto it = client.StreamExecuteQuery(R"( SELECT COUNT(*) FROM `/Root/DataShard2`; - )", NYdb::NQuery::TTxControl::BeginTx().CommitTx()).ExtractValueSync(); + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); TString output = StreamResultToYson(it); CompareYson( @@ -4264,6 +4265,109 @@ Y_UNIT_TEST_SUITE(KqpQueryService) { } } + Y_UNIT_TEST(TableSink_OltpInteractive) { + NKikimrConfig::TAppConfig appConfig; + appConfig.MutableTableServiceConfig()->SetEnableOlapSink(true); + appConfig.MutableTableServiceConfig()->SetEnableOltpSink(true); + auto settings = TKikimrSettings() + .SetAppConfig(appConfig) + .SetWithSampleTables(false); + TKikimrRunner kikimr(settings); + Tests::NCommon::TLoggerInit(kikimr).Initialize(); + + auto session = kikimr.GetTableClient().CreateSession().GetValueSync().GetSession(); + + const TString query = R"( + CREATE TABLE `/Root/DataShard` ( + Col1 Uint32 NOT NULL, + Col2 String, + Col3 Int32 NOT NULL, + PRIMARY KEY (Col1) + ) + WITH ( + AUTO_PARTITIONING_BY_SIZE = DISABLED, + AUTO_PARTITIONING_MIN_PARTITIONS_COUNT = 16, + AUTO_PARTITIONING_MAX_PARTITIONS_COUNT = 16, + UNIFORM_PARTITIONS = 16); + + CREATE TABLE `/Root/DataShard2` ( + Col1 Uint32 NOT NULL, + Col2 String, + Col3 Int32 NOT NULL, + PRIMARY KEY (Col1) + ) + WITH ( + AUTO_PARTITIONING_BY_SIZE = DISABLED, + AUTO_PARTITIONING_MIN_PARTITIONS_COUNT = 17, + AUTO_PARTITIONING_MAX_PARTITIONS_COUNT = 17, + UNIFORM_PARTITIONS = 17); + )"; + + auto result = session.ExecuteSchemeQuery(query).GetValueSync(); + UNIT_ASSERT_C(result.GetStatus() == NYdb::EStatus::SUCCESS, result.GetIssues().ToString()); + + auto client = kikimr.GetQueryClient(); + auto session2 = client.GetSession().GetValueSync().GetSession(); + + auto tx = session2.BeginTransaction(NYdb::NQuery::TTxSettings::SerializableRW()) + .ExtractValueSync() + .GetTransaction(); + UNIT_ASSERT(tx.IsActive()); + { + auto prepareResult = session2.ExecuteQuery(R"( + REPLACE INTO `/Root/DataShard` (Col1, Col2, Col3) VALUES + (10u, "test1", 10), (20u, "test2", 11), (2147483647u, "test3", 12), (2147483640u, NULL, 13); + )", TTxControl::Tx(tx.GetId()), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); + UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); + } + + { + auto prepareResult = session2.ExecuteQuery(R"( + REPLACE INTO `/Root/DataShard2` (Col1, Col2, Col3) VALUES + (11u, "test1", 10), (21u, "test2", 11), (2147483646u, "test3", 12), (2147483641u, NULL, 13); + )", TTxControl::Tx(tx.GetId()), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); + UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); + } + + { + auto it = session2.StreamExecuteQuery(R"( + SELECT COUNT(*) FROM `/Root/DataShard`; + )", TTxControl::Tx(tx.GetId()), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(it.GetStatus(), EStatus::SUCCESS, it.GetIssues().ToString()); + TString output = StreamResultToYson(it); + CompareYson( + output, + R"([[4u]])"); + } + + { + auto prepareResult = session2.ExecuteQuery(R"( + SELECT * FROM `/Root/DataShard2`; + )", TTxControl::Tx(tx.GetId()), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); + UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); + } + + { + auto prepareResult = session2.ExecuteQuery(R"( + REPLACE INTO `/Root/DataShard2` (Col1, Col2, Col3) VALUES + (11u, "test1", 10), (21u, "test2", 11), (2147483646u, "test3", 12), (2147483641u, NULL, 13); + )", TTxControl::Tx(tx.GetId()), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); + UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); + } + + { + auto commitResult = tx.Commit().ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(commitResult.GetStatus(), EStatus::SUCCESS, commitResult.GetIssues().ToString()); + } + + { + auto prepareResult = client.ExecuteQuery(R"( + REPLACE INTO `/Root/DataShard2` SELECT * FROM `/Root/DataShard`; + )", NYdb::NQuery::TTxControl::BeginTx().CommitTx(), TExecuteQuerySettings().ClientTimeout(TDuration::MilliSeconds(1000))).ExtractValueSync(); + UNIT_ASSERT_C(prepareResult.IsSuccess(), prepareResult.GetIssues().ToString()); + } + } + Y_UNIT_TEST(ReadDatashardAndColumnshard) { NKikimrConfig::TAppConfig appConfig; appConfig.MutableTableServiceConfig()->SetEnableOlapSink(true); diff --git a/ydb/core/kqp/ut/tx/kqp_mvcc_ut.cpp b/ydb/core/kqp/ut/tx/kqp_mvcc_ut.cpp index d7497273491f..04365d4b326e 100644 --- a/ydb/core/kqp/ut/tx/kqp_mvcc_ut.cpp +++ b/ydb/core/kqp/ut/tx/kqp_mvcc_ut.cpp @@ -9,6 +9,51 @@ using namespace NYdb; using namespace NYdb::NTable; Y_UNIT_TEST_SUITE(KqpSnapshotRead) { + + Y_UNIT_TEST_TWIN(TestReadOnly, withSink) { + NKikimrConfig::TAppConfig appConfig; + appConfig.MutableTableServiceConfig()->SetEnableOltpSink(withSink); + auto settings = TKikimrSettings() + .SetKeepSnapshotTimeout(TDuration::Seconds(1)) + .SetAppConfig(appConfig); + + TKikimrRunner kikimr(settings); + + auto db = kikimr.GetTableClient(); + auto session1 = db.CreateSession().GetValueSync().GetSession(); + auto session2 = db.CreateSession().GetValueSync().GetSession(); + + auto result = session1.ExecuteDataQuery(Q_(R"( + SELECT * FROM `/Root/EightShard` WHERE Key = 101u OR Key = 801u ORDER BY Key; + )"), TTxControl::BeginTx(TTxSettings::SerializableRW())).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + CompareYson(R"([ + [[1];[101u];["Value1"]]; + [[2];[801u];["Value1"]] + ])", FormatResultSetYson(result.GetResultSet(0))); + + auto tx = result.GetTransaction(); + + result = session2.ExecuteDataQuery(Q_(R"( + UPSERT INTO `/Root/EightShard` (Key, Text) VALUES (101u, "Changed"), (801u, "Changed"); + UPSERT INTO `/Root/TwoShard` (Key, Value1, Value2) VALUES (1u, "Changed", 1), (4000000001u, "Changed", 2); + )"), TTxControl::BeginTx(TTxSettings::SerializableRW()).CommitTx()).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + + result = session1.ExecuteDataQuery(Q_(R"( + SELECT * FROM `/Root/TwoShard` WHERE Key = 1u OR Key = 4000000001u ORDER BY Key; + )"), TTxControl::Tx(*tx)).ExtractValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + CompareYson(R"([ + [[1u];["One"];[-1]]; + [[4000000001u];["BigOne"];[-1]] + ])", FormatResultSetYson(result.GetResultSet(0))); + + auto commitResult = tx->Commit().GetValueSync(); + UNIT_ASSERT_VALUES_EQUAL_C(commitResult.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); + } + + Y_UNIT_TEST_TWIN(TestSnapshotExpiration, withSink) { NKikimrConfig::TAppConfig appConfig; appConfig.MutableTableServiceConfig()->SetEnableOltpSink(withSink); diff --git a/ydb/core/kqp/ut/tx/kqp_sink_tx_ut.cpp b/ydb/core/kqp/ut/tx/kqp_sink_tx_ut.cpp index bc4f31996137..a780ea2f9260 100644 --- a/ydb/core/kqp/ut/tx/kqp_sink_tx_ut.cpp +++ b/ydb/core/kqp/ut/tx/kqp_sink_tx_ut.cpp @@ -144,11 +144,11 @@ Y_UNIT_TEST_SUITE(KqpSinkTx) { result = session.ExecuteQuery(Q_(R"( UPDATE `/Root/KV` SET Value = "third" WHERE Key = 4; )"), TTxControl::Tx(tx->GetId())).ExtractValueSync(); - UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::ABORTED, result.GetIssues().ToString()); + UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::SUCCESS, result.GetIssues().ToString()); auto commitResult = tx->Commit().ExtractValueSync(); - UNIT_ASSERT_VALUES_EQUAL_C(commitResult.GetStatus(), EStatus::NOT_FOUND, commitResult.GetIssues().ToString()); + UNIT_ASSERT_VALUES_EQUAL_C(commitResult.GetStatus(), EStatus::ABORTED, commitResult.GetIssues().ToString()); } }; @@ -177,6 +177,7 @@ Y_UNIT_TEST_SUITE(KqpSinkTx) { auto result = session.ExecuteQuery(Q_(R"( INSERT INTO `/Root/KV` (Key, Value) VALUES (1u, "New"); + SELECT COUNT(*) FROM `/Root/KV`; )"), TTxControl::Tx(tx.GetId())).ExtractValueSync(); result.GetIssues().PrintTo(Cerr); UNIT_ASSERT_VALUES_EQUAL_C(result.GetStatus(), EStatus::BAD_REQUEST, result.GetIssues().ToString()); diff --git a/ydb/core/protos/kqp.proto b/ydb/core/protos/kqp.proto index 932f5b669e05..212eb14418ef 100644 --- a/ydb/core/protos/kqp.proto +++ b/ydb/core/protos/kqp.proto @@ -748,13 +748,9 @@ message TEvKillScanTablet { message TEvKqpOutputActorResultInfo { repeated NKikimrDataEvents.TLock Locks = 1; + optional bool HasRead = 2; } -message TKqpTableSinkLocks { - repeated NKikimrDataEvents.TLock Locks = 1; - repeated uint64 SendingShards = 2; - repeated uint64 ReceivingShards = 3; -} message TKqpTableSinkSettings { enum EType { @@ -770,11 +766,11 @@ message TKqpTableSinkSettings { repeated TKqpColumnMetadataProto Columns = 5; optional uint64 LockTxId = 6; optional uint64 LockNodeId = 7; - optional bool FinalTx = 8; // If tx is immediate then commit, otherwise send prepare - optional bool ImmediateTx = 9; // Try to be immediate tx - optional bool InconsistentTx = 10; // Write each batch in it's own single immediate tx - optional EType Type = 11; - optional TKqpTableSinkLocks Locks = 12; + optional bool InconsistentTx = 8; // Write each batch in it's own single immediate tx + optional EType Type = 9; + optional NActorsProto.TActorId BufferActorId = 10; + optional int64 Priority = 11; + optional bool IsOlap = 12; } message TKqpStreamLookupSettings { diff --git a/ydb/core/protos/table_service_config.proto b/ydb/core/protos/table_service_config.proto index 6a08a0c28179..40b8e16036e9 100644 --- a/ydb/core/protos/table_service_config.proto +++ b/ydb/core/protos/table_service_config.proto @@ -339,4 +339,5 @@ message TTableServiceConfig { } optional TWriteActorSettings WriteActorSettings = 72; + optional bool AllowOlapDataQuery = 74 [default = false]; }; diff --git a/ydb/library/wilson_ids/wilson.h b/ydb/library/wilson_ids/wilson.h index d8a176ccfa9e..632e1da1b5f0 100644 --- a/ydb/library/wilson_ids/wilson.h +++ b/ydb/library/wilson_ids/wilson.h @@ -79,8 +79,14 @@ namespace NKikimr { LookupActor = TComponentTracingLevels::TQueryProcessor::Basic, LookupActorShardsResolve = TComponentTracingLevels::TQueryProcessor::Detailed, - WriteActor = TComponentTracingLevels::TQueryProcessor::Basic, - WriteActorTableNavigate = TComponentTracingLevels::TQueryProcessor::Detailed, + TableWriteActor = TComponentTracingLevels::TQueryProcessor::Basic, + TableWriteActorTableNavigate = TComponentTracingLevels::TQueryProcessor::Detailed, + + DirectWriteActor = TComponentTracingLevels::TQueryProcessor::Basic, + ForwardWriteActor = TComponentTracingLevels::TQueryProcessor::Basic, + + BufferWriteActor = TComponentTracingLevels::TQueryProcessor::Basic, + BufferWriteActorState = TComponentTracingLevels::TQueryProcessor::Basic, BulkUpsertActor = TComponentTracingLevels::TQueryProcessor::TopLevel, }; diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h b/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h index 8a59470318f0..00ebf9e3538b 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_async_io.h @@ -198,6 +198,8 @@ struct IDqComputeActorAsyncOutput { virtual TMaybe ExtraData() { return {}; } + virtual void FillExtraStats(NDqProto::TDqTaskStats* /* stats */, bool /* finalized stats */, const NYql::NDq::TDqMeteringStats*) { } + virtual void PassAway() = 0; // The same signature as IActor::PassAway() virtual ~IDqComputeActorAsyncOutput() = default; diff --git a/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h b/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h index af7a36ba6b62..db16f8ea9db2 100644 --- a/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h +++ b/ydb/library/yql/dq/actors/compute/dq_compute_actor_impl.h @@ -1741,6 +1741,10 @@ class TDqComputeActorBase : public NActors::TActorBootstrapped egressRows += egressStats.Rows ? egressStats.Rows : pushStats.Rows; // p.s. sink == sinkInfo.Buffer } + + if (auto* source = sinkInfo.AsyncOutput) { + source->FillExtraStats(protoTask, last, GetMeteringStats()); + } } protoTask->SetFinishTimeMs(finishTimeMs);