From b8304ea93a6d8bad75f6adbae4a6676a3de5469f Mon Sep 17 00:00:00 2001 From: Aleksandr Khoroshilov Date: Thu, 9 May 2024 14:41:51 +0000 Subject: [PATCH] Refactor S3 Reader to split huge file to parts and simplify development --- ydb/core/base/events.h | 2 +- .../yql/providers/s3/actors/parquet_cache.cpp | 14 + .../yql/providers/s3/actors/parquet_cache.h | 11 + .../providers/s3/actors/raw_read_actor.cpp | 554 +++++ .../yql/providers/s3/actors/raw_read_actor.h | 43 + .../s3/actors/read_json_each_row.cpp | 366 ++++ .../providers/s3/actors/read_json_each_row.h | 11 + .../yql/providers/s3/actors/read_parquet.cpp | 511 +++++ .../yql/providers/s3/actors/read_parquet.h | 16 + .../yql/providers/s3/actors/source_queue.cpp | 659 ++++++ .../yql/providers/s3/actors/source_queue.h | 31 + ydb/library/yql/providers/s3/actors/ya.make | 8 + .../s3/actors/yql_arrow_column_converters.cpp | 50 + .../s3/actors/yql_arrow_column_converters.h | 13 + .../providers/s3/actors/yql_s3_read_actor.cpp | 1810 +++-------------- .../providers/s3/actors/yql_s3_read_actor.h | 2 +- .../providers/s3/common/source_context.cpp | 133 ++ .../yql/providers/s3/common/source_context.h | 188 ++ ydb/library/yql/providers/s3/common/ya.make | 13 + .../yql/providers/s3/events/events.cpp | 1 + ydb/library/yql/providers/s3/events/events.h | 264 +++ ydb/library/yql/providers/s3/events/ya.make | 26 + .../s3/expr_nodes/yql_s3_expr_nodes.json | 5 +- .../yql/providers/s3/json/json_row_parser.cpp | 416 ++++ .../yql/providers/s3/json/json_row_parser.h | 89 + .../providers/s3/json/json_row_parser_ut.cpp | 139 ++ ydb/library/yql/providers/s3/json/ut/ya.make | 15 + ydb/library/yql/providers/s3/json/ya.make | 20 + .../yql/providers/s3/proto/source.proto | 4 +- .../provider/yql_s3_datasource_type_ann.cpp | 9 +- .../s3/provider/yql_s3_dq_integration.cpp | 15 +- .../s3/provider/yql_s3_logical_opt.cpp | 3 +- .../providers/s3/provider/yql_s3_settings.cpp | 1 + .../providers/s3/provider/yql_s3_settings.h | 3 +- ydb/library/yql/providers/s3/ya.make | 2 + 35 files changed, 3877 insertions(+), 1570 deletions(-) create mode 100644 ydb/library/yql/providers/s3/actors/parquet_cache.cpp create mode 100644 ydb/library/yql/providers/s3/actors/parquet_cache.h create mode 100644 ydb/library/yql/providers/s3/actors/raw_read_actor.cpp create mode 100644 ydb/library/yql/providers/s3/actors/raw_read_actor.h create mode 100644 ydb/library/yql/providers/s3/actors/read_json_each_row.cpp create mode 100644 ydb/library/yql/providers/s3/actors/read_json_each_row.h create mode 100644 ydb/library/yql/providers/s3/actors/read_parquet.cpp create mode 100644 ydb/library/yql/providers/s3/actors/read_parquet.h create mode 100644 ydb/library/yql/providers/s3/actors/source_queue.cpp create mode 100644 ydb/library/yql/providers/s3/actors/source_queue.h create mode 100644 ydb/library/yql/providers/s3/common/source_context.cpp create mode 100644 ydb/library/yql/providers/s3/common/source_context.h create mode 100644 ydb/library/yql/providers/s3/events/events.cpp create mode 100644 ydb/library/yql/providers/s3/events/events.h create mode 100644 ydb/library/yql/providers/s3/events/ya.make create mode 100644 ydb/library/yql/providers/s3/json/json_row_parser.cpp create mode 100644 ydb/library/yql/providers/s3/json/json_row_parser.h create mode 100644 ydb/library/yql/providers/s3/json/json_row_parser_ut.cpp create mode 100644 ydb/library/yql/providers/s3/json/ut/ya.make create mode 100644 ydb/library/yql/providers/s3/json/ya.make diff --git a/ydb/core/base/events.h b/ydb/core/base/events.h index 6ccf5f838254..a73efb723307 100644 --- a/ydb/core/base/events.h +++ b/ydb/core/base/events.h @@ -173,7 +173,7 @@ struct TKikimrEvents : TEvents { ES_GRAPH, ES_REPLICATION_WORKER, ES_CHANGE_EXCHANGE, - ES_S3_FILE_QUEUE, + ES_S3_PROVIDER, ES_NEBIUS_ACCESS_SERVICE, ES_REPLICATION_SERVICE, ES_BACKUP_SERVICE, diff --git a/ydb/library/yql/providers/s3/actors/parquet_cache.cpp b/ydb/library/yql/providers/s3/actors/parquet_cache.cpp new file mode 100644 index 000000000000..c947e38a345b --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/parquet_cache.cpp @@ -0,0 +1,14 @@ +#include "parquet_cache.h" + +namespace NYql::NDq { + +NActors::TActorId ParquetCacheActorId() { + constexpr TStringBuf name = "PRQTCACH"; + return NActors::TActorId(0, name); +} + +NActors::IActor* CreateParquetCache() { + return nullptr; +} + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/parquet_cache.h b/ydb/library/yql/providers/s3/actors/parquet_cache.h new file mode 100644 index 000000000000..47ff6ccaeb54 --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/parquet_cache.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +namespace NYql::NDq { + +NActors::TActorId ParquetCacheActorId(); + +NActors::IActor* CreateParquetCache(); + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/raw_read_actor.cpp b/ydb/library/yql/providers/s3/actors/raw_read_actor.cpp new file mode 100644 index 000000000000..8adfa7902839 --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/raw_read_actor.cpp @@ -0,0 +1,554 @@ +#include + +#include "raw_read_actor.h" +#include "source_queue.h" +#include "yql_s3_actors_util.cpp" + +#include +#include +#include + +#include +#include +#include + +#include + +#include + +#define LOG_E(name, stream) \ + LOG_ERROR_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_W(name, stream) \ + LOG_WARN_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_I(name, stream) \ + LOG_INFO_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_D(name, stream) \ + LOG_DEBUG_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_T(name, stream) \ + LOG_TRACE_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) + +namespace NYql::NDq { + +ui64 SubtractSaturating(ui64 lhs, ui64 rhs) { + return (lhs > rhs) ? lhs - rhs : 0; +} + +class TS3ReadActor : public NActors::TActorBootstrapped, public IDqComputeActorAsyncInput { +public: + TS3ReadActor( + ui64 inputIndex, + TCollectStatsLevel statsLevel, + const TTxId& txId, + IHTTPGateway::TPtr gateway, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const TString& url, + const TS3Credentials::TAuthInfo& authInfo, + const TString& pattern, + NYql::NS3Lister::ES3PatternVariant patternVariant, + NYql::NS3Details::TPathList&& paths, + bool addPathIndex, + const NActors::TActorId& computeActorId, + ui64 sizeLimit, + const IHTTPGateway::TRetryPolicy::TPtr& retryPolicy, + const TS3ReadActorFactoryConfig& readActorFactoryCfg, + ::NMonitoring::TDynamicCounterPtr counters, + ::NMonitoring::TDynamicCounterPtr taskCounters, + ui64 fileSizeLimit, + std::optional rowsLimitHint, + bool useRuntimeListing, + NActors::TActorId fileQueueActor, + ui64 fileQueueBatchSizeLimit, + ui64 fileQueueBatchObjectCountLimit, + ui64 fileQueueConsumersCountDelta) + : ReadActorFactoryCfg(readActorFactoryCfg) + , Gateway(std::move(gateway)) + , HolderFactory(holderFactory) + , InputIndex(inputIndex) + , TxId(txId) + , ComputeActorId(computeActorId) + , RetryPolicy(retryPolicy) + , ActorSystem(NActors::TActivationContext::ActorSystem()) + , Url(url) + , AuthInfo(authInfo) + , Pattern(pattern) + , PatternVariant(patternVariant) + , Paths(std::move(paths)) + , FileQueueActor(fileQueueActor) + , AddPathIndex(addPathIndex) + , SizeLimit(sizeLimit) + , Counters(counters) + , TaskCounters(taskCounters) + , FileSizeLimit(fileSizeLimit) + , FilesRemained(rowsLimitHint) + , UseRuntimeListing(useRuntimeListing) + , FileQueueBatchSizeLimit(fileQueueBatchSizeLimit) + , FileQueueBatchObjectCountLimit(fileQueueBatchObjectCountLimit) + , FileQueueConsumersCountDelta(fileQueueConsumersCountDelta) { + if (Counters) { + QueueDataSize = Counters->GetCounter("QueueDataSize"); + QueueDataLimit = Counters->GetCounter("QueueDataLimit"); + QueueBlockCount = Counters->GetCounter("QueueBlockCount"); + QueueDataLimit->Add(ReadActorFactoryCfg.DataInflight); + } + if (TaskCounters) { + TaskQueueDataSize = TaskCounters->GetCounter("QueueDataSize"); + TaskQueueDataLimit = TaskCounters->GetCounter("QueueDataLimit"); + TaskQueueDataLimit->Add(ReadActorFactoryCfg.DataInflight); + } + IngressStats.Level = statsLevel; + } + + void Bootstrap() { + if (!UseRuntimeListing) { + FileQueueActor = RegisterWithSameMailbox(CreateS3FileQueueActor( + TxId, + std::move(Paths), + ReadActorFactoryCfg.MaxInflight * 2, + FileSizeLimit, + SizeLimit, + false, + 1, + FileQueueBatchSizeLimit, + FileQueueBatchObjectCountLimit, + Gateway, + Url, + AuthInfo, + Pattern, + PatternVariant, + NYql::NS3Lister::ES3PatternType::Wildcard)); + } + + LOG_D("TS3ReadActor", "Bootstrap" << ", InputIndex: " << InputIndex << ", FileQueue: " << FileQueueActor << (UseRuntimeListing ? " (remote)" : " (local")); + + FileQueueEvents.Init(TxId, SelfId(), SelfId()); + FileQueueEvents.OnNewRecipientId(FileQueueActor); + if (UseRuntimeListing && FileQueueConsumersCountDelta > 0) { + FileQueueEvents.Send(new TEvS3Provider::TEvUpdateConsumersCount(FileQueueConsumersCountDelta)); + } + SendPathBatchRequest(); + + Become(&TS3ReadActor::StateFunc); + } + + bool TryStartDownload() { + TrySendPathBatchRequest(); + if (PathBatchQueue.empty()) { + // no path is pending + return false; + } + if (IsCurrentBatchEmpty) { + // waiting for batch to finish + return false; + } + if (QueueTotalDataSize > ReadActorFactoryCfg.DataInflight) { + // too large data inflight + return false; + } + if (DownloadInflight >= ReadActorFactoryCfg.MaxInflight) { + // too large download inflight + return false; + } + if (ConsumedEnoughFiles()) { + // started enough downloads + return false; + } + + StartDownload(); + return true; + } + + void StartDownload() { + DownloadInflight++; + const auto& object = ReadPathFromCache(); + auto url = Url + object.GetPath(); + auto id = object.GetPathIndex(); + const TString requestId = CreateGuidAsString(); + LOG_D("TS3ReadActor", "Download: " << url << ", ID: " << id << ", request id: [" << requestId << "]"); + Gateway->Download( + UrlEscapeRet(url, true), + IHTTPGateway::MakeYcHeaders(requestId, AuthInfo.GetToken(), {}, AuthInfo.GetAwsUserPwd(), AuthInfo.GetAwsSigV4()), + 0U, + std::min(object.GetSize(), SizeLimit), + std::bind(&TS3ReadActor::OnDownloadFinished, ActorSystem, SelfId(), requestId, std::placeholders::_1, id, object.GetPath()), + {}, + RetryPolicy); + } + + NS3::FileQueue::TObjectPath ReadPathFromCache() { + Y_ENSURE(!PathBatchQueue.empty()); + auto& currentBatch = PathBatchQueue.front(); + Y_ENSURE(!currentBatch.empty()); + auto object = currentBatch.back(); + currentBatch.pop_back(); + if (currentBatch.empty()) { + PathBatchQueue.pop_front(); + IsCurrentBatchEmpty = true; + } + TrySendPathBatchRequest(); + return object; + } + void TrySendPathBatchRequest() { + if (PathBatchQueue.size() < 2 && !IsFileQueueEmpty && !ConsumedEnoughFiles() && !IsWaitingFileQueueResponse) { + SendPathBatchRequest(); + } + } + void SendPathBatchRequest() { + FileQueueEvents.Send(new TEvS3Provider::TEvGetNextBatch()); + IsWaitingFileQueueResponse = true; + } + + static constexpr char ActorName[] = "S3_READ_ACTOR"; + +private: + void SaveState(const NDqProto::TCheckpoint&, TSourceState&) final {} + void LoadState(const TSourceState&) final {} + void CommitState(const NDqProto::TCheckpoint&) final {} + + ui64 GetInputIndex() const final { + return InputIndex; + } + + const TDqAsyncStats& GetIngressStats() const override { + return IngressStats; + } + + TDuration GetCpuTime() override { + return CpuTime; + } + + bool ConsumedEnoughFiles() const { + return FilesRemained && (*FilesRemained == 0); + } + + STRICT_STFUNC_EXC(StateFunc, + hFunc(TEvS3Provider::TEvReadResult, Handle); + hFunc(TEvS3Provider::TEvReadError, Handle); + hFunc(TEvS3Provider::TEvObjectPathBatch, HandleObjectPathBatch); + hFunc(TEvS3Provider::TEvObjectPathReadError, HandleObjectPathReadError); + hFunc(TEvS3Provider::TEvAck, HandleAck); + hFunc(NYql::NDq::TEvRetryQueuePrivate::TEvRetry, Handle); + hFunc(NActors::TEvInterconnect::TEvNodeDisconnected, Handle); + hFunc(NActors::TEvInterconnect::TEvNodeConnected, Handle); + hFunc(NActors::TEvents::TEvUndelivered, Handle); + , catch (const std::exception& e) { + TIssues issues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; + Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + } + ) + + void HandleObjectPathBatch(TEvS3Provider::TEvObjectPathBatch::TPtr& objectPathBatch) { + if (!FileQueueEvents.OnEventReceived(objectPathBatch)) { + LOG_W("TS3ReadActor", "Duplicated TEvObjectPathBatch (likely resent) from " << FileQueueActor); + return; + } + + Y_ENSURE(IsWaitingFileQueueResponse); + IsWaitingFileQueueResponse = false; + auto& objectBatch = objectPathBatch->Get()->Record; + ListedFiles += objectBatch.GetObjectPaths().size(); + IsFileQueueEmpty = objectBatch.GetNoMoreFiles(); + if (IsFileQueueEmpty && !IsConfirmedFileQueueFinish) { + LOG_D("TS3ReadActor", "Confirm finish to " << FileQueueActor); + SendPathBatchRequest(); + IsConfirmedFileQueueFinish = true; + } + if (!objectBatch.GetObjectPaths().empty()) { + PathBatchQueue.emplace_back( + std::make_move_iterator(objectBatch.MutableObjectPaths()->begin()), + std::make_move_iterator(objectBatch.MutableObjectPaths()->end())); + } + while (TryStartDownload()) {} + + if (LastFileWasProcessed()) { + Send(ComputeActorId, new TEvNewAsyncInputDataArrived(InputIndex)); + } + } + void HandleObjectPathReadError(TEvS3Provider::TEvObjectPathReadError::TPtr& result) { + if (!FileQueueEvents.OnEventReceived(result)) { + LOG_W("TS3ReadActor", "Duplicated TEvObjectPathReadError (likely resent) from " << FileQueueActor); + return; + } + + IsFileQueueEmpty = true; + if (!IsConfirmedFileQueueFinish) { + LOG_D("TS3ReadActor", "Confirm finish (with errors) to " << FileQueueActor); + SendPathBatchRequest(); + IsConfirmedFileQueueFinish = true; + } + TIssues issues; + IssuesFromMessage(result->Get()->Record.GetIssues(), issues); + LOG_E("TS3ReadActor", "Error while object listing, details: TEvObjectPathReadError: " << issues.ToOneLineString()); + issues = NS3Util::AddParentIssue(TStringBuilder{} << "Error while object listing", std::move(issues)); + Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); + } + + void HandleAck(TEvS3Provider::TEvAck::TPtr& ev) { + FileQueueEvents.OnEventReceived(ev); + } + + static void OnDownloadFinished(NActors::TActorSystem* actorSystem, NActors::TActorId selfId, const TString& requestId, IHTTPGateway::TResult&& result, size_t pathInd, const TString path) { + if (!result.Issues) { + actorSystem->Send(new NActors::IEventHandle(selfId, NActors::TActorId(), new TEvS3Provider::TEvReadResult(std::move(result.Content), requestId, pathInd, path))); + } else { + actorSystem->Send(new NActors::IEventHandle(selfId, NActors::TActorId(), new TEvS3Provider::TEvReadError(std::move(result.Issues), requestId, pathInd, path))); + } + } + + i64 GetAsyncInputData(NKikimr::NMiniKQL::TUnboxedValueBatch& buffer, TMaybe&, bool& finished, i64 freeSpace) final { + i64 total = 0LL; + if (!Blocks.empty()) { + do { + auto& content = std::get(Blocks.front()); + const auto size = content.size(); + auto value = NKikimr::NMiniKQL::MakeString(std::string_view(content)); + if (AddPathIndex) { + NUdf::TUnboxedValue* tupleItems = nullptr; + auto tuple = ContainerCache.NewArray(HolderFactory, 2, tupleItems); + *tupleItems++ = value; + *tupleItems++ = NUdf::TUnboxedValuePod(std::get(Blocks.front())); + value = tuple; + } + + buffer.emplace_back(std::move(value)); + Blocks.pop(); + total += size; + freeSpace -= size; + + QueueTotalDataSize -= size; + if (Counters) { + QueueDataSize->Sub(size); + QueueBlockCount->Dec(); + } + if (TaskCounters) { + TaskQueueDataSize->Sub(size); + } + TryStartDownload(); + } while (!Blocks.empty() && freeSpace > 0LL); + } + + if ((LastFileWasProcessed() || ConsumedEnoughFiles()) && !FileQueueEvents.RemoveConfirmedEvents()) { + finished = true; + ContainerCache.Clear(); + } + + if (!total) { + IngressStats.TryPause(); + } + + return total; + } + bool LastFileWasProcessed() const { + return Blocks.empty() && (ListedFiles == CompletedFiles) && IsFileQueueEmpty; + } + + void Handle(TEvS3Provider::TEvReadResult::TPtr& result) { + ++CompletedFiles; + const auto id = result->Get()->PathIndex; + const auto path = result->Get()->Path; + const auto httpCode = result->Get()->Result.HttpResponseCode; + const auto requestId = result->Get()->RequestId; + LOG_D("TS3ReadActor", "ID: " << id << ", Path: " << path << ", read size: " << result->Get()->Result.size() << ", HTTP response code: " << httpCode << ", request id: [" << requestId << "]"); + if (200 == httpCode || 206 == httpCode) { + auto size = result->Get()->Result.size(); + + // in TS3ReadActor all files (aka Splits) are loaded in single Chunks + IngressStats.Bytes += size; + IngressStats.Rows++; + IngressStats.Chunks++; + IngressStats.Splits++; + IngressStats.Resume(); + + QueueTotalDataSize += size; + if (Counters) { + QueueBlockCount->Inc(); + QueueDataSize->Add(size); + } + if (TaskCounters) { + TaskQueueDataSize->Add(size); + } + Blocks.emplace(std::make_tuple(std::move(result->Get()->Result), id)); + DownloadInflight--; + if (IsCurrentBatchEmpty && DownloadInflight == 0) { + IsCurrentBatchEmpty = false; + } + if (FilesRemained) { + *FilesRemained = SubtractSaturating(*FilesRemained, 1); + } + TryStartDownload(); + Send(ComputeActorId, new TEvNewAsyncInputDataArrived(InputIndex)); + } else { + TString errorText = result->Get()->Result.Extract(); + TString errorCode; + TString message; + if (!ParseS3ErrorResponse(errorText, errorCode, message)) { + message = errorText; + } + message = TStringBuilder{} << "Error while reading file " << path << ", details: " << message << ", request id: [" << requestId << "]"; + Send(ComputeActorId, new TEvAsyncInputError(InputIndex, BuildIssues(httpCode, errorCode, message), NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); + } + } + + void Handle(TEvS3Provider::TEvReadError::TPtr& result) { + ++CompletedFiles; + auto id = result->Get()->PathIndex; + const auto requestId = result->Get()->RequestId; + const auto path = result->Get()->Path; + LOG_W("TS3ReadActor", "Error while reading file " << path << ", details: ID: " << id << ", TEvReadError: " << result->Get()->Error.ToOneLineString() << ", request id: [" << requestId << "]"); + auto issues = NS3Util::AddParentIssue(TStringBuilder{} << "Error while reading file " << path << " with request id [" << requestId << "]", TIssues{result->Get()->Error}); + Send(ComputeActorId, new TEvAsyncInputError(InputIndex, std::move(issues), NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); + } + + void Handle(const NYql::NDq::TEvRetryQueuePrivate::TEvRetry::TPtr&) { + FileQueueEvents.Retry(); + } + + void Handle(NActors::TEvInterconnect::TEvNodeDisconnected::TPtr& ev) { + LOG_T("TS3ReadActor", "Handle disconnected FileQueue " << ev->Get()->NodeId); + FileQueueEvents.HandleNodeDisconnected(ev->Get()->NodeId); + } + + void Handle(NActors::TEvInterconnect::TEvNodeConnected::TPtr& ev) { + LOG_T("TS3ReadActor", "Handle connected FileQueue " << ev->Get()->NodeId); + FileQueueEvents.HandleNodeConnected(ev->Get()->NodeId); + } + + void Handle(NActors::TEvents::TEvUndelivered::TPtr& ev) { + LOG_T("TS3ReadActor", "Handle undelivered FileQueue "); + if (!FileQueueEvents.HandleUndelivered(ev)) { + TIssues issues{TIssue{TStringBuilder() << "FileQueue was lost"}}; + Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::UNAVAILABLE)); + } + } + + // IActor & IDqComputeActorAsyncInput + void PassAway() override { // Is called from Compute Actor + LOG_D("TS3ReadActor", "PassAway"); + + if (Counters) { + QueueDataSize->Sub(QueueTotalDataSize); + QueueBlockCount->Sub(Blocks.size()); + QueueDataLimit->Sub(ReadActorFactoryCfg.DataInflight); + } + if (TaskCounters) { + TaskQueueDataSize->Sub(QueueTotalDataSize); + TaskQueueDataLimit->Sub(ReadActorFactoryCfg.DataInflight); + } + QueueTotalDataSize = 0; + + ContainerCache.Clear(); + FileQueueEvents.Unsubscribe(); + TActorBootstrapped::PassAway(); + } + +private: + const TS3ReadActorFactoryConfig ReadActorFactoryCfg; + const IHTTPGateway::TPtr Gateway; + const NKikimr::NMiniKQL::THolderFactory& HolderFactory; + NKikimr::NMiniKQL::TPlainContainerCache ContainerCache; + + const ui64 InputIndex; + TDqAsyncStats IngressStats; + const TTxId TxId; + const NActors::TActorId ComputeActorId; + const IHTTPGateway::TRetryPolicy::TPtr RetryPolicy; + + NActors::TActorSystem* const ActorSystem; + + const TString Url; + const TS3Credentials::TAuthInfo AuthInfo; + const TString Pattern; + const NYql::NS3Lister::ES3PatternVariant PatternVariant; + NYql::NS3Details::TPathList Paths; + size_t ListedFiles = 0; + size_t CompletedFiles = 0; + NActors::TActorId FileQueueActor; + const bool AddPathIndex; + const ui64 SizeLimit; + TDuration CpuTime; + + std::queue> Blocks; + + ::NMonitoring::TDynamicCounters::TCounterPtr QueueDataSize; + ::NMonitoring::TDynamicCounters::TCounterPtr QueueDataLimit; + ::NMonitoring::TDynamicCounters::TCounterPtr QueueBlockCount; + ::NMonitoring::TDynamicCounters::TCounterPtr DownloadPaused; + ::NMonitoring::TDynamicCounters::TCounterPtr DeferredQueueSize; + ::NMonitoring::TDynamicCounters::TCounterPtr TaskDownloadPaused; + ::NMonitoring::TDynamicCounters::TCounterPtr TaskQueueDataSize; + ::NMonitoring::TDynamicCounters::TCounterPtr TaskQueueDataLimit; + ::NMonitoring::TDynamicCounterPtr Counters; + ::NMonitoring::TDynamicCounterPtr TaskCounters; + ui64 QueueTotalDataSize = 0; + ui64 DownloadInflight = 0; + const ui64 FileSizeLimit; + std::optional FilesRemained; + + bool UseRuntimeListing; + ui64 FileQueueBatchSizeLimit; + ui64 FileQueueBatchObjectCountLimit; + ui64 FileQueueConsumersCountDelta; + bool IsFileQueueEmpty = false; + bool IsCurrentBatchEmpty = false; + bool IsWaitingFileQueueResponse = false; + bool IsConfirmedFileQueueFinish = false; + TRetryEventsQueue FileQueueEvents; + TDeque> PathBatchQueue; +}; + +std::pair CreateRawReadActor( + ui64 inputIndex, + TCollectStatsLevel statsLevel, + const TTxId& txId, + IHTTPGateway::TPtr gateway, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const TString& url, + const TS3Credentials::TAuthInfo& authInfo, + const TString& pattern, + NYql::NS3Lister::ES3PatternVariant patternVariant, + NYql::NS3Details::TPathList&& paths, + bool addPathIndex, + const NActors::TActorId& computeActorId, + ui64 sizeLimit, + const IHTTPGateway::TRetryPolicy::TPtr& retryPolicy, + const TS3ReadActorFactoryConfig& readActorFactoryCfg, + ::NMonitoring::TDynamicCounterPtr counters, + ::NMonitoring::TDynamicCounterPtr taskCounters, + ui64 fileSizeLimit, + std::optional rowsLimitHint, + bool useRuntimeListing, + NActors::TActorId fileQueueActor, + ui64 fileQueueBatchSizeLimit, + ui64 fileQueueBatchObjectCountLimit, + ui64 fileQueueConsumersCountDelta +) { + const auto actor = new TS3ReadActor( + inputIndex, + statsLevel, + txId, + std::move(gateway), + holderFactory, + url, + authInfo, + pattern, + patternVariant, + std::move(paths), + addPathIndex, + computeActorId, + sizeLimit, + retryPolicy, + readActorFactoryCfg, + counters, + taskCounters, + fileSizeLimit, + rowsLimitHint, + useRuntimeListing, + fileQueueActor, + fileQueueBatchSizeLimit, + fileQueueBatchObjectCountLimit, + fileQueueConsumersCountDelta + ); + + return {actor, actor}; +} + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/raw_read_actor.h b/ydb/library/yql/providers/s3/actors/raw_read_actor.h new file mode 100644 index 000000000000..832c07f41287 --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/raw_read_actor.h @@ -0,0 +1,43 @@ +#pragma once + +#include + +#include + +#include +#include +#include + +#include +#include + +namespace NYql::NDq { + +std::pair CreateRawReadActor( + ui64 inputIndex, + TCollectStatsLevel statsLevel, + const TTxId& txId, + IHTTPGateway::TPtr gateway, + const NKikimr::NMiniKQL::THolderFactory& holderFactory, + const TString& url, + const TS3Credentials::TAuthInfo& authInfo, + const TString& pattern, + NYql::NS3Lister::ES3PatternVariant patternVariant, + NYql::NS3Details::TPathList&& paths, + bool addPathIndex, + const NActors::TActorId& computeActorId, + ui64 sizeLimit, + const IHTTPGateway::TRetryPolicy::TPtr& retryPolicy, + const TS3ReadActorFactoryConfig& readActorFactoryCfg, + ::NMonitoring::TDynamicCounterPtr counters, + ::NMonitoring::TDynamicCounterPtr taskCounters, + ui64 fileSizeLimit, + std::optional rowsLimitHint, + bool useRuntimeListing, + NActors::TActorId fileQueueActor, + ui64 fileQueueBatchSizeLimit, + ui64 fileQueueBatchObjectCountLimit, + ui64 fileQueueConsumersCountDelta +); + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/read_json_each_row.cpp b/ydb/library/yql/providers/s3/actors/read_json_each_row.cpp new file mode 100644 index 000000000000..503abf967cfa --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/read_json_each_row.cpp @@ -0,0 +1,366 @@ +#include "read_json_each_row.h" +#include "yql_s3_actors_util.h" + +#include + +#include + +#include + +#include + +#include + +#define LOG_E(stream) \ + LOG_ERROR_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadJsonEachRowActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_W(stream) \ + LOG_WARN_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadJsonEachRowActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_I(stream) \ + LOG_INFO_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadJsonEachRowActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_D(stream) \ + LOG_DEBUG_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadJsonEachRowActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_T(stream) \ + LOG_TRACE_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadJsonEachRowActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) + +namespace NYql::NDq { + +void OnSplitStart(NActors::TActorSystem* actorSystem, const NActors::TActorId& self, CURLcode curlResponseCode, long httpResponseCode) { + actorSystem->Send(new NActors::IEventHandle(self, NActors::TActorId{}, new TEvS3Provider::TEvDownloadStart(curlResponseCode, httpResponseCode))); +} + +void OnSplitData(NActors::TActorSystem* actorSystem, const NActors::TActorId& self, IHTTPGateway::TCountedContent&& data) { + actorSystem->Send(new NActors::IEventHandle(self, NActors::TActorId{}, new TEvS3Provider::TEvDownloadData(std::move(data)))); +} + +void OnSplitFinish(NActors::TActorSystem* actorSystem, const NActors::TActorId& self, CURLcode curlResponseCode, TIssues issues) { + actorSystem->Send(new NActors::IEventHandle(self, NActors::TActorId{}, new TEvS3Provider::TEvDownloadFinish(0, curlResponseCode, std::move(issues)))); +} + +class TS3ReadJsonEachRowActor : public NActors::TActorBootstrapped { + +public: + TS3ReadJsonEachRowActor(TSplitReadContext::TPtr context) + : Context(context) + , Parser(Context->SourceContext->Schema) { + } + + void Bootstrap() { + if (Context->SplitOffset != 0 || Context->SplitSize != Context->FileSize) { + TIssues issues{TIssue{TStringBuilder() << "Partial file splits are not supported"}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + return; + } + Offset = Context->SplitOffset; + Size = Context->FileSize; + + LOG_D("Bootstrap"); + Become(&TS3ReadJsonEachRowActor::StateFunc); + + if (Context->SourceContext->Compression && Context->SourceContext->Compression != "gzip") { + TIssues issues{TIssue{TStringBuilder() << "Unsupported compression \"" << Context->SourceContext->Compression << "\""}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::BAD_REQUEST)); + } else if (Context->SourceContext->Format != "json_each_row") { + TIssues issues{TIssue{TStringBuilder() << "Unexpected format \"" << Context->SourceContext->Format << "\" (\"json_each_row\" expected)"}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::BAD_REQUEST)); + } else { + if (Context->SourceContext->Compression == "gzip") { +// Cerr << "GZIP INIT" << Endl; + ProcessData = &TS3ReadJsonEachRowActor::ProcessGzipData; + FinishData = &TS3ReadJsonEachRowActor::FinishGzip; + Zero(ZStream); + if (inflateInit2(&ZStream, 31) != Z_OK) { + TIssues issues{TIssue{TStringBuilder() << "Can't initialize ZLIB"}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + return; + } + } else { + ProcessData = &TS3ReadJsonEachRowActor::ProcessRawData; + FinishData = nullptr; + } + StartDownload(); + } + } + + STRICT_STFUNC(StateFunc, + cFunc(NActors::TEvents::TEvWakeup::EventType, StartDownload); + hFunc(TEvS3Provider::TEvDownloadStart, Handle); + hFunc(TEvS3Provider::TEvDownloadData, Handle); + hFunc(TEvS3Provider::TEvDownloadFinish, Handle); + cFunc(TEvS3Provider::TEvContinue::EventType, HandleContinue); + cFunc(NActors::TEvents::TEvPoison::EventType, HandlePoison); + ) + +private: + static constexpr std::string_view TruncatedSuffix = "... [truncated]"sv; +private: + void StartDownload() { + Context->CancelHook = Context->Gateway->Download( + Context->Url, + Context->Headers, + 0, + Context->FileSize, + std::bind(&OnSplitStart, Context->SourceContext->ActorSystem, SelfId(), std::placeholders::_1, std::placeholders::_2), + std::bind(&OnSplitData, Context->SourceContext->ActorSystem, SelfId(), std::placeholders::_1), + std::bind(&OnSplitFinish, Context->SourceContext->ActorSystem, SelfId(), std::placeholders::_1, std::placeholders::_2), + Context->SourceContext->HttpInflightSize); + } + + void Handle(TEvS3Provider::TEvDownloadStart::TPtr& ev) { + HttpResponseCode = ev->Get()->HttpResponseCode; + CurlResponseCode = ev->Get()->CurlResponseCode; + LOG_D("TEvDownloadStart, HTTP code: " << HttpResponseCode << ", CURL code: " << static_cast(CurlResponseCode)); + } + + void Handle(TEvS3Provider::TEvDownloadData::TPtr& ev) { + if (Context->SourceContext->HttpDataRps) { + Context->SourceContext->HttpDataRps->Inc(); + } + if (HttpResponseCode == 200L || HttpResponseCode == 206L) { + if (Paused || !DeferredDataParts.empty()) { + DeferredDataParts.push(std::move(ev->Release())); + if (Context->SourceContext->DeferredQueueSize) { + Context->SourceContext->DeferredQueueSize->Inc(); + } + } else { + ExtractDataPart(*ev->Get(), false); + } + } else if (HttpResponseCode && !Context->IsCancelled() && !Context->NextRetryDelay) { + ServerReturnedError = true; + if (ErrorText.size() < 256_KB) + ErrorText.append(ev->Get()->Result.Extract()); + else if (!ErrorText.EndsWith(TruncatedSuffix)) + ErrorText.append(TruncatedSuffix); + LOG_W("TEvDownloadData, ERROR: " << ErrorText << ", Offset: " << Offset); + } + } + + void Handle(TEvS3Provider::TEvDownloadFinish::TPtr& ev) { + + if (CurlResponseCode == CURLE_OK) { + CurlResponseCode = ev->Get()->CurlResponseCode; + } + + Issues.Clear(); + if (!ErrorText.empty()) { + TString errorCode; + TString message; + if (!ParseS3ErrorResponse(ErrorText, errorCode, message)) { + message = ErrorText; + } + Issues.AddIssues(BuildIssues(HttpResponseCode, errorCode, message)); + } + + if (ev->Get()->Issues) { + Issues.AddIssues(ev->Get()->Issues); + } + + if (HttpResponseCode >= 300) { + ServerReturnedError = true; + Issues.AddIssue(TIssue{TStringBuilder() << "HTTP error code: " << HttpResponseCode}); + } + + if (Issues) { + Context->NextRetryDelay = Context->GetRetryState()->GetNextRetryDelay(CurlResponseCode, HttpResponseCode); + LOG_D("TEvDownloadFinish with Issues (try to retry): " << Issues.ToOneLineString()); + if (Context->NextRetryDelay) { + // inplace retry: report problem to TransientIssues and repeat + // Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvAsyncInputError(InputIndex, Issues, NYql::NDqProto::StatusIds::UNSPECIFIED)); + } else { + // can't retry here: fail download + Context->RetryState = nullptr; + // InputFinished = true; + // LOG_W("ReadError: " << Issues.ToOneLineString() << ", Offset: " << Offset); + // throw TS3ReadError(); // Don't pass control to data parsing, because it may validate eof and show wrong issues about incorrect data format + } + } + + if (!Context->IsCancelled() && Context->NextRetryDelay && Size > 0ULL) { + Schedule(*Context->NextRetryDelay, new NActors::TEvents::TEvWakeup()); + if (DeferredDataParts.size()) { + if (Context->SourceContext->DeferredQueueSize) { + Context->SourceContext->DeferredQueueSize->Sub(DeferredDataParts.size()); + } + std::queue> tmp; + DeferredDataParts.swap(tmp); + } + } else { + LOG_D("TEvDownloadFinish, Offset: " << Offset << ", Error: " << ServerReturnedError); + if (ServerReturnedError) { + // throw TS3ReadError(); // Don't pass control to data parsing, because it may validate eof and show wrong issues about incorrect data format + } else { + // Cerr << "JER finished" << Endl; + if (FinishData) { + (this->*FinishData)(); + } + Send(Context->SourceContext->SourceId, new TEvS3Provider::TEvFileFinished(Context->PathIndex, TakeIngressDelta(), TakeCpuTimeDelta(), Context->FileSize)); + } + } + } + + void HandleContinue() { + LOG_D("TEvContinue"); + Paused = false; + if (!Paused && !DeferredDataParts.empty()) { + ExtractDataPart(*DeferredDataParts.front(), true); + DeferredDataParts.pop(); + if (Context->SourceContext->DeferredQueueSize) { + Context->SourceContext->DeferredQueueSize->Dec(); + } + } + } + + void HandlePoison() { + LOG_D("TEvPoison"); + Context->Cancel(); + PassAway(); + } + + void ExtractDataPart(TEvS3Provider::TEvDownloadData& event, bool deferred = false) { + auto data = event.Result.Extract(); + IngressBytes += data.size(); + Offset += data.size(); + Size -= data.size(); + LOG_T("TEvDownloadData (" << (deferred ? "deferred" : "instant") << "), size: " << data.size()); + // Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvNewAsyncInputDataArrived(InputIndex)); + (this->*ProcessData)(data); + } + + void ProcessGzipData(const TString& data) { + ZStream.next_in = const_cast(reinterpret_cast(data.data())); + ZStream.avail_in = data.size(); + while (ZStream.avail_in) { + TString output; + output.resize(1_MB); + ZStream.next_out = const_cast(reinterpret_cast(output.data())); + ZStream.avail_out = output.size(); + switch (const auto code = inflate(&ZStream, Z_SYNC_FLUSH)) { + case Z_NEED_DICT: + { + TIssues issues{TIssue{TStringBuilder() << "Can't initialize ZLIB"}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::BAD_REQUEST)); + } + return; + case Z_OK: + output.resize(1_MB - ZStream.avail_out); +// Cerr << "GZIP OUTPUT " << output << Endl; + ProcessRawData(output); + break; + case Z_STREAM_END: + output.resize(1_MB - ZStream.avail_out); +// Cerr << "GZIP FINAL " << output << Endl; + ProcessRawData(output); + return; + default: + { + TIssues issues{TIssue{TStringBuilder() << "Unexpected ZLIB code " << code}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::BAD_REQUEST)); + } + return; + } + } + } + + void FinishGzip() { + if (inflateReset(&ZStream) != Z_OK) { + TIssues issues{TIssue{TStringBuilder() << "Inflate reset error: " << (ZStream.msg ? ZStream.msg : "Unknown error.")}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::BAD_REQUEST)); + } + } + + void ProcessRawData(const TString& data) { + ParseRawData(data); + +// Cerr << "JER take batch" << Endl; + auto batch = Parser.TakeBatch(); + auto size = NUdf::GetSizeOfArrowBatchInBytes(*batch); +// Cerr << "JER got batch of " << size << "bytes" << Endl; + Paused = Context->SourceContext->Add(size, SelfId(), Paused); + Send(Context->SourceContext->SourceId, new TEvS3Provider::TEvNextRecordBatch( + batch, Context->PathIndex, TakeIngressDelta(), TakeCpuTimeDelta() + )); + } + + void ParseRawData(const TString& data) { +// Cerr << "JER Parse " << data.size() << " bytes" << Endl; + if (data.empty()) { + return; + } + + auto n = data.find('\n'); + if (n == data.npos) { +// Cerr << "JER no EOL" << Endl; + // no EOL found - just wait for next data + PreviousData += data; + return; + } + +// Cerr << "JER found EOL at " << n << Endl; + + if (PreviousData) { + TString mergedLine; + mergedLine.reserve(PreviousData.size() - PreviousPos + n); + std::copy(PreviousData.cbegin() + PreviousPos, PreviousData.cend(), std::back_inserter(mergedLine)); + std::copy(data.cbegin(), data.cbegin() + n, std::back_inserter(mergedLine)); + Parser.ParseNextRow(mergedLine); + PreviousData.clear(); + PreviousPos = 0; + } else { + Parser.ParseNextRow(TStringBuf(data.begin(), n)); + } + + while (true) { + auto n1 = data.find('\n', n + 1); + if (n1 == data.npos) { + PreviousData = data; + PreviousPos = n + 1; + return; + } +// Cerr << "JER found EOL at " << n1 << Endl; + Parser.ParseNextRow(TStringBuf(data.begin() + n + 1, n1 - n - 1)); + if (n1 + 1 == data.size()) + { + // end of data + return; + } + n = n1; + } + } + + ui64 TakeIngressDelta() { + auto currentIngressBytes = IngressBytes; + IngressBytes = 0; + return currentIngressBytes; + } + + TDuration TakeCpuTimeDelta() { + auto currentCpuTime = CpuTime; + CpuTime = TDuration::Zero(); + return currentCpuTime; + } + + TSplitReadContext::TPtr Context; + std::size_t Offset = 0; + std::size_t Size = 0; + NYql::NJson::TJsonRowParser Parser; + long HttpResponseCode = 0L; + CURLcode CurlResponseCode = CURLE_OK; + bool Paused = false; + bool ServerReturnedError = false; + TString ErrorText; + TIssues Issues; + std::queue> DeferredDataParts; + ui64 IngressBytes = 0; + TDuration CpuTime; + TString PreviousData; + std::size_t PreviousPos = 0; + + void (TS3ReadJsonEachRowActor::*ProcessData)(const TString& data) = nullptr; + void (TS3ReadJsonEachRowActor::*FinishData)() = nullptr; + z_stream ZStream; +}; + +NActors::IActor* CreateS3ReadJsonEachRowActor(TSplitReadContext::TPtr context) { + return new TS3ReadJsonEachRowActor(context); +} + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/read_json_each_row.h b/ydb/library/yql/providers/s3/actors/read_json_each_row.h new file mode 100644 index 000000000000..5ae40021b394 --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/read_json_each_row.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +#include + +namespace NYql::NDq { + +NActors::IActor* CreateS3ReadJsonEachRowActor(TSplitReadContext::TPtr context); + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/read_parquet.cpp b/ydb/library/yql/providers/s3/actors/read_parquet.cpp new file mode 100644 index 000000000000..34bbb36f4f5d --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/read_parquet.cpp @@ -0,0 +1,511 @@ +#include "parquet_cache.h" +#include "read_parquet.h" +#include "yql_arrow_column_converters.h" + +#include + +#include +#include + +#include +#include +#include +#include + +#define LOG_E(stream) \ + LOG_ERROR_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadParquetActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_W(stream) \ + LOG_WARN_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadParquetActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_I(stream) \ + LOG_INFO_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadParquetActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_D(stream) \ + LOG_DEBUG_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadParquetActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) +#define LOG_T(stream) \ + LOG_TRACE_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, "TS3ReadParquetActor: " << this->SelfId() << ", TxId: " << Context->TxId << ". " << stream) + +namespace NYql::NDq { + +#define THROW_ARROW_NOT_OK(status) \ + do \ + { \ + if (::arrow::Status _s = (status); !_s.ok()) \ + throw yexception() << _s.ToString(); \ + } while (false) + +struct TS3ReadForMetadata : public yexception { + using yexception::yexception; +}; + +struct TS3ReadInternalError : public yexception { + using yexception::yexception; +}; + +struct TReadCache { + ui64 Cookie = 0; + TString Data; + std::optional RowGroupIndex; + bool Ready = false; +}; + +struct TReadRangeCompare { + bool operator() (const TEvS3Provider::TReadRange& lhs, const TEvS3Provider::TReadRange& rhs) const + { + return (lhs.Offset < rhs.Offset) || (lhs.Offset == rhs.Offset && lhs.Length < rhs.Length); + } +}; + +struct TReaderSlot { + std::unique_ptr Reader; + bool Pending = false; + ui64 RowGroup = 0; +}; + +class TS3ReadParquetActor : public NActors::TActorBootstrapped { + +public: + + class THttpRandomAccessFile : public arrow::io::RandomAccessFile { + public: + THttpRandomAccessFile(TS3ReadParquetActor* reader, size_t fileSize) : Reader(reader), FileSize(fileSize) { + } + + // has no meaning and use + arrow::Result GetSize() override { return FileSize; } + arrow::Result Tell() const override { return InnerPos; } + arrow::Status Seek(int64_t position) override { InnerPos = position; return {}; } + arrow::Status Close() override { return {}; } + bool closed() const override { return false; } + // must not be used currently + arrow::Result Read(int64_t, void*) override { + Y_ABORT_UNLESS(0); + return arrow::Result(); + } + arrow::Result> Read(int64_t) override { + Y_ABORT_UNLESS(0); + return arrow::Result>(); + } + // useful ones + arrow::Future> ReadAsync(const arrow::io::IOContext&, int64_t position, int64_t nbytes) override { + return arrow::Future>::MakeFinished(ReadAt(position, nbytes)); + } + arrow::Result> ReadAt(int64_t position, int64_t nbytes) override { + return Reader->ReadAt(position, nbytes); + } + arrow::Status WillNeed(const std::vector& readRanges) override { + return Reader->WillNeed(readRanges); + } + + private: + TS3ReadParquetActor *const Reader; + const size_t FileSize; + int64_t InnerPos = 0; + }; + + TS3ReadParquetActor( + TSplitReadContext::TPtr context + , ui64 parallelRowGroupCount + , bool rowGroupReordering + , bool useParquetCache + ) + : Context(context) + , ParallelRowGroupCount(parallelRowGroupCount) + , RowGroupReordering(rowGroupReordering) + , UseParquetCache(useParquetCache) { + + } + + void Bootstrap() { + if (Context->SplitOffset != 0 || Context->SplitSize != Context->FileSize) { + TIssues issues{TIssue{TStringBuilder() << "Partial file splits are not supported"}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + return; + } + + LOG_D("Bootstrap"); + Become(&TS3ReadParquetActor::StateFunc); + + FooterSize = std::min(Context->FileSize, 65536u); + FooterOffset = Context->FileSize - FooterSize; + StartDownload(FooterOffset, FooterSize); + } + + STRICT_STFUNC(StateFunc, + hFunc(TEvS3Provider::TEvReadResult2, Handle); + cFunc(NActors::TEvents::TEvPoison::EventType, HandlePoison); + hFunc(TEvS3Provider::TEvCacheCheckResult, Handle); + ) + + void Handle(TEvS3Provider::TEvReadResult2::TPtr& ev) { + + // gateway error is always EXTERNAL_ERROR + + if (ev->Get()->Failure) { + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, ev->Get()->Issues, NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); + return; + } + + auto data = ev->Get()->Result.Extract(); + auto httpCode = ev->Get()->Result.HttpResponseCode; + + // not 20X code may include error response from S3 + + if (httpCode != 200 && httpCode != 206) { + TString errorCode; + TString message; + NDqProto::StatusIds::StatusCode statusCode = NDqProto::StatusIds::EXTERNAL_ERROR; + if (ParseS3ErrorResponse(data, errorCode, message)) { + auto code = StatusFromS3ErrorCode(errorCode); + if (code != NDqProto::StatusIds::UNSPECIFIED) { + statusCode = code; + } + } else { + message = data; + } + message = TStringBuilder{} << "Error while reading file " << Context->Url << ", details: " << message /* TODO << ", request id: [" << requestId << "]" */; + Send(Context->SourceContext->SourceId, new IDqComputeActorAsyncInput::TEvAsyncInputError(0, BuildIssues(httpCode, errorCode, message), statusCode)); + return; + } + + // all validations are passed, may use result as data + + if (MetadataParsed) { + ProcessReadRange(ev); + return; + } else if (WaitForFooter) { + WaitForFooter = false; + RawFooter = data; + } else if (WaitForMetadata) { + WaitForMetadata = false; + RawMetadata = data; + } else { + TIssues issues{TIssue{TStringBuilder() << "Unknown read result"}}; + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + return; + } + + // try to parse metadata + + parquet::arrow::FileReaderBuilder builder; + builder.memory_pool(arrow::default_memory_pool()); + parquet::ArrowReaderProperties properties; + properties.set_cache_options(arrow::io::CacheOptions::LazyDefaults()); + properties.set_pre_buffer(true); + builder.properties(properties); + + try { + THROW_ARROW_NOT_OK(builder.Open(std::make_shared(this, Context->FileSize))); + } catch(const TS3ReadForMetadata& ex) { + if (WaitForMetadata) { + return; + } + TIssues issues; + issues.AddIssue(TIssue(ex.what())); + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + return; + } catch(const TS3ReadInternalError& ex) { + TIssues issues; + issues.AddIssue(TIssue(ex.what())); + Send(Context->SourceContext->SourceId, new NYql::NDq::IDqComputeActorAsyncInput::TEvAsyncInputError(0, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); + return; + } + + MetadataParsed = true; + ui64 readerCount = 1; + ReaderSlots.resize(1); + + // download row groups + + THROW_ARROW_NOT_OK(builder.Build(&ReaderSlots[0].Reader)); + auto fileMetadata = ReaderSlots[0].Reader->parquet_reader()->metadata(); + NumGroups = ReaderSlots[0].Reader->num_row_groups(); + + if (NumGroups == 0) { + Send(Context->SourceContext->SourceId, new TEvS3Provider::TEvFileFinished(Context->PathIndex, TakeIngressDelta(), TakeCpuTimeDelta(), Context->SplitSize)); + return; + } + + std::shared_ptr schema; + THROW_ARROW_NOT_OK(ReaderSlots[0].Reader->GetSchema(&schema)); + + BuildColumnConverters(Context->SourceContext->Schema, schema, ColumnIndices, + ColumnConverters, Context->SourceContext->RowTypes, Context->SourceContext->Settings); + + // select count(*) case - single reader is enough + if (!ColumnIndices.empty()) { + if (ParallelRowGroupCount) { + readerCount = ParallelRowGroupCount; + } else { + // we want to read in parallel as much as 1/2 of fair share bytes + // (it's compressed size, after decoding it will grow) + ui64 compressedSize = 0; + for (int i = 0; i < fileMetadata->num_row_groups(); i++) { + auto rowGroup = fileMetadata->RowGroup(i); + for (const auto columIndex : ColumnIndices) { + compressedSize += rowGroup->ColumnChunk(columIndex)->total_compressed_size(); + } + } + // count = (fair_share / 2) / (compressed_size / num_group) + auto desiredReaderCount = (Context->SourceContext->FairShare() * NumGroups) / (compressedSize * 2); + // min is 1 + // max is 5 (should be also tuned probably) + if (desiredReaderCount) { + readerCount = std::min(desiredReaderCount, 5ul); + } + } + if (readerCount > NumGroups) { + readerCount = NumGroups; + } + } + + if (readerCount > 1) { + // init other readers if any + ReaderSlots.resize(readerCount); + for (ui64 i = 1; i < readerCount; i++) { + THROW_ARROW_NOT_OK(builder.Open(std::make_shared(this, Context->FileSize), + parquet::default_reader_properties(), + fileMetadata)); + THROW_ARROW_NOT_OK(builder.Build(&ReaderSlots[i].Reader)); + } + } + + if (ColumnIndices.empty()) { + for (ui64 i = 0; i < NumGroups; i++) { + ReadRowGroup(0, i); + Send(Context->SourceContext->SourceId, new TEvS3Provider::TEvFileFinished(Context->PathIndex, TakeIngressDelta(), TakeCpuTimeDelta(), Context->SplitSize)); + return; + } + } else { + for (ui64 i = 0; i < readerCount; i++) { + AssignReader(i); + } + } + } + + void HandlePoison() { + LOG_D("TEvPoison"); + PassAway(); + } + + void AssignReader(ui64 readerIndex) { + auto rowGroup = NextGroup++; + RowGroupReaderIndex[rowGroup] = readerIndex; + ReaderSlots[readerIndex].RowGroup = rowGroup; + if (UseParquetCache) { + Send(ParquetCacheActorId(), new TEvS3Provider::TEvCacheCheckRequest( + Context->SourceContext->SourceId, Context->Url, rowGroup + ), 0, readerIndex); + } else { + CurrentRowGroupIndex = rowGroup; + THROW_ARROW_NOT_OK(ReaderSlots[readerIndex].Reader->WillNeedRowGroups({ static_cast(rowGroup) }, ColumnIndices)); + } + Context->SourceContext->IncChunkCount(); + } + + void Handle(TEvS3Provider::TEvCacheCheckResult::TPtr& ev) { + auto readerIndex = ev->Cookie; + if (ev->Get()->Hit) { + // take data + AssignReader(readerIndex); + } else { + CurrentRowGroupIndex = ReaderSlots[readerIndex].RowGroup; + THROW_ARROW_NOT_OK(ReaderSlots[readerIndex].Reader->WillNeedRowGroups({ static_cast(*CurrentRowGroupIndex) }, ColumnIndices)); + } + } + + static void OnDownloadResult(NActors::TActorSystem* actorSystem, NActors::TActorId selfId, TEvS3Provider::TReadRange range, ui64 cookie, IHTTPGateway::TResult&& result) { + if (!result.Issues) { + actorSystem->Send(new NActors::IEventHandle(selfId, NActors::TActorId{}, new TEvS3Provider::TEvReadResult2(range, std::move(result.Content)), 0, cookie)); + } else { + actorSystem->Send(new NActors::IEventHandle(selfId, NActors::TActorId{}, new TEvS3Provider::TEvReadResult2(range, std::move(result.Issues)), 0, cookie)); + } + } + + void StartDownload(std::size_t offset, std::size_t size) { + WaitForFooter = true; + Context->Gateway->Download( + Context->Url, + Context->Headers, + offset, + size, + std::bind(&OnDownloadResult, Context->SourceContext->ActorSystem, SelfId(), TEvS3Provider::TReadRange{static_cast(offset), static_cast(size)}, 0, std::placeholders::_1), + {}, + Context->RetryPolicy); + } + + void ProcessReadRange(TEvS3Provider::TEvReadResult2::TPtr& ev) { + auto readyRange = ev->Get()->ReadRange; + LOG_D("Download FINISHED [" << readyRange.Offset << "-" << readyRange.Length << "], cookie: " << ev->Cookie); + IngressBytes += readyRange.Length; + + auto it = RangeCache.find(readyRange); + if (it == RangeCache.end()) { + LOG_W("Download completed for unknown/discarded range [" << readyRange.Offset << "-" << readyRange.Length << "]"); + return; + } + + if (it->second.Cookie != ev->Cookie) { + LOG_W("Mistmatched cookie for range [" << readyRange.Offset << "-" << readyRange.Length << "], received " << ev->Cookie << ", expected " << it->second.Cookie); + return; + } + + it->second.Data = ev->Get()->Result.Extract(); + ui64 size = it->second.Data.size(); + it->second.Ready = true; + if (it->second.RowGroupIndex) { + if (!DecreaseRowGroupInflight(*it->second.RowGroupIndex)) { + LOG_D("RowGroup #" << *it->second.RowGroupIndex << " is READY"); + ReadyRowGroups.push(*it->second.RowGroupIndex); + } + ReadInflightSize[*it->second.RowGroupIndex] += size; + // if (RawInflightSize) { + // RawInflightSize->Add(size); + // } + } + } + + void ReadRowGroup(std::size_t, std::size_t) { + + } + + ui64 TakeIngressDelta() { + auto currentIngressBytes = IngressBytes; + IngressBytes = 0; + return currentIngressBytes; + } + + TDuration TakeCpuTimeDelta() { + return TDuration::Zero(); + } + + ui64 DecreaseRowGroupInflight(ui64 rowGroupIndex) { + auto inflight = RowGroupRangeInflight[rowGroupIndex]; + if (inflight > 1) { + RowGroupRangeInflight[rowGroupIndex] = --inflight; + } else { + inflight = 0; + RowGroupRangeInflight.erase(rowGroupIndex); + } + return inflight; + } + + TReadCache& GetOrCreate(TEvS3Provider::TReadRange range) { + auto it = RangeCache.find(range); + if (it != RangeCache.end()) { + return it->second; + } + Context->Gateway->Download( + Context->Url, + Context->Headers, + range.Offset, + range.Length, + std::bind(&OnDownloadResult, Context->SourceContext->ActorSystem, SelfId(), range, ++RangeCookie, std::placeholders::_1), + {}, + Context->RetryPolicy); + + LOG_D("Download STARTED [" << range.Offset << "-" << range.Length << "], cookie: " << RangeCookie); + auto& result = RangeCache[range]; + if (result.Cookie) { + // may overwrite old range in case of desync? + if (result.RowGroupIndex) { + LOG_W("RangeInfo DISCARDED [" << range.Offset << "-" << range.Length << "], cookie: " << RangeCookie << ", rowGroup " << *result.RowGroupIndex); + DecreaseRowGroupInflight(*result.RowGroupIndex); + } + } + result.RowGroupIndex = CurrentRowGroupIndex; + result.Cookie = RangeCookie; + if (CurrentRowGroupIndex) { + RowGroupRangeInflight[*CurrentRowGroupIndex]++; + } + return result; + } + + arrow::Status WillNeed(const std::vector& readRanges) { + if (readRanges.empty()) { // select count(*) case + if (CurrentRowGroupIndex) { + ReadyRowGroups.push(*CurrentRowGroupIndex); + } + } else { + for (auto& range : readRanges) { + GetOrCreate(TEvS3Provider::TReadRange{ .Offset = range.offset, .Length = range.length }); + } + } + return {}; + } + + arrow::Result> ReadAt(int64_t position, int64_t nbytes) { + if (!MetadataParsed) { + if (WaitForFooter) { + throw TS3ReadInternalError() << "Footer is not loaded"; + } + if (static_cast(position) == FooterOffset && static_cast(nbytes) == FooterSize) { + return arrow::Buffer::FromString(RawFooter); + } + if (WaitForMetadata) { + throw TS3ReadInternalError() << "Metadata is not loaded"; + } + if (MetadataSize == 0) { + MetadataOffset = position; + MetadataSize = nbytes; + WaitForMetadata = true; + StartDownload(MetadataOffset, MetadataSize); + throw TS3ReadForMetadata(); + } + if (static_cast(position) == MetadataOffset && static_cast(nbytes) == MetadataSize) { + return arrow::Buffer::FromString(RawMetadata); + } + } + + TEvS3Provider::TReadRange range { .Offset = position, .Length = nbytes }; + auto& cache = GetOrCreate(range); + + if (!cache.Ready) { + throw TS3ReadInternalError() << "Cache for range of position = " << position << ", nbytes = " << nbytes << " is not ready"; + } + + TString data = cache.Data; + RangeCache.erase(range); + + return arrow::Buffer::FromString(data); + } + + TSplitReadContext::TPtr Context; + const ui64 ParallelRowGroupCount; + const bool RowGroupReordering; + const bool UseParquetCache; + + bool WaitForFooter = false; + std::size_t FooterOffset = 0; + std::size_t FooterSize = 0; + TString RawFooter; + bool WaitForMetadata = false; + std::size_t MetadataOffset = 0; + std::size_t MetadataSize = 0; + TString RawMetadata; + bool MetadataParsed = false; + std::vector ReaderSlots; + std::vector ColumnIndices; + std::vector ColumnConverters; + + ui64 RangeCookie = 0; + std::map RangeCache; + std::map ReadInflightSize; + std::optional CurrentRowGroupIndex; + std::map RowGroupRangeInflight; + std::priority_queue, std::greater> ReadyRowGroups; + std::map RowGroupReaderIndex; + + ui64 IngressBytes = 0; + ui64 NumGroups = 0; + ui64 NextGroup = 0; +}; + +NActors::IActor* CreateS3ReadParquetActor( + TSplitReadContext::TPtr context, + ui64 parallelRowGroupCount, + bool rowGroupReordering, + bool useParquetCache +) { + return new TS3ReadParquetActor(context, parallelRowGroupCount, rowGroupReordering, useParquetCache); +} + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/read_parquet.h b/ydb/library/yql/providers/s3/actors/read_parquet.h new file mode 100644 index 000000000000..272ddcd9340b --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/read_parquet.h @@ -0,0 +1,16 @@ +#pragma once + +#include + +#include + +namespace NYql::NDq { + +NActors::IActor* CreateS3ReadParquetActor( + TSplitReadContext::TPtr context, + ui64 parallelRowGroupCount, + bool rowGroupReordering, + bool useParquetCache +); + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/source_queue.cpp b/ydb/library/yql/providers/s3/actors/source_queue.cpp new file mode 100644 index 000000000000..dc747a61e12c --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/source_queue.cpp @@ -0,0 +1,659 @@ +#include +#if defined(_linux_) || defined(_darwin_) +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include + +#endif + +#include "yql_arrow_column_converters.h" +#include "yql_s3_actors_util.h" +#include "yql_s3_read_actor.h" +#include "yql_s3_source_factory.h" + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +#include +#include + +#ifdef THROW +#undef THROW +#endif +#include +#include + +#define LOG_E(name, stream) \ + LOG_ERROR_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_W(name, stream) \ + LOG_WARN_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_I(name, stream) \ + LOG_INFO_S (*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_D(name, stream) \ + LOG_DEBUG_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) +#define LOG_T(name, stream) \ + LOG_TRACE_S(*NActors::TlsActivationContext, NKikimrServices::KQP_COMPUTE, name << ": " << this->SelfId() << ", TxId: " << TxId << ". " << stream) + +#define LOG_CORO_E(stream) \ + LOG_ERROR_S(GetActorContext(), NKikimrServices::KQP_COMPUTE, "TS3ReadCoroImpl: " << SelfActorId << ", CA: " << ComputeActorId << ", TxId: " << TxId \ + << " [" << Path << "]. RETRY{ Offset: " << RetryStuff->Offset << ", Delay: " << RetryStuff->NextRetryDelay << ", RequestId: " << RetryStuff->RequestId << "}. " << stream) +#define LOG_CORO_W(stream) \ + LOG_WARN_S (GetActorContext(), NKikimrServices::KQP_COMPUTE, "TS3ReadCoroImpl: " << SelfActorId << ", CA: " << ComputeActorId << ", TxId: " << TxId \ + << " [" << Path << "]. RETRY{ Offset: " << RetryStuff->Offset << ", Delay: " << RetryStuff->NextRetryDelay << ", RequestId: " << RetryStuff->RequestId << "}. " << stream) +#define LOG_CORO_I(stream) \ + LOG_INFO_S (GetActorContext(), NKikimrServices::KQP_COMPUTE, "TS3ReadCoroImpl: " << SelfActorId << ", CA: " << ComputeActorId << ", TxId: " << TxId \ + << " [" << Path << "]. RETRY{ Offset: " << RetryStuff->Offset << ", Delay: " << RetryStuff->NextRetryDelay << ", RequestId: " << RetryStuff->RequestId << "}. " << stream) +#define LOG_CORO_D(stream) \ + LOG_DEBUG_S(GetActorContext(), NKikimrServices::KQP_COMPUTE, "TS3ReadCoroImpl: " << SelfActorId << ", CA: " << ComputeActorId << ", TxId: " << TxId \ + << " [" << Path << "]. RETRY{ Offset: " << RetryStuff->Offset << ", Delay: " << RetryStuff->NextRetryDelay << ", RequestId: " << RetryStuff->RequestId << "}. " << stream) +#define LOG_CORO_T(stream) \ + LOG_TRACE_S(GetActorContext(), NKikimrServices::KQP_COMPUTE, "TS3ReadCoroImpl: " << SelfActorId << ", CA: " << ComputeActorId << ", TxId: " << TxId \ + << " [" << Path << "]. RETRY{ Offset: " << RetryStuff->Offset << ", Delay: " << RetryStuff->NextRetryDelay << ", RequestId: " << RetryStuff->RequestId << "}. " << stream) + +namespace NYql::NDq { + +class TS3FileQueueActor : public NActors::TActorBootstrapped { +public: + static constexpr char ActorName[] = "YQ_S3_FILE_QUEUE_ACTOR"; + + struct TEvPrivatePrivate { + enum { + EvBegin = TEvRetryQueuePrivate::EvEnd, // Leave space for RetryQueue events + + EvNextListingChunkReceived = EvBegin, + EvRoundRobinStageTimeout, + EvTransitToErrorState, + + EvEnd + }; + static_assert( + EvEnd <= EventSpaceEnd(NActors::TEvents::ES_PRIVATE), + "expected EvEnd <= EventSpaceEnd(TEvents::ES_PRIVATE)"); + + struct TEvNextListingChunkReceived : public NActors::TEventLocal { + NS3Lister::TListResult ListingResult; + TEvNextListingChunkReceived(NS3Lister::TListResult listingResult) + : ListingResult(std::move(listingResult)){}; + }; + + struct TEvRoundRobinStageTimeout : public NActors::TEventLocal { + }; + + struct TEvTransitToErrorState : public NActors::TEventLocal { + explicit TEvTransitToErrorState(TIssues&& issues) + : Issues(issues) { + } + TIssues Issues; + }; + }; + using TBase = TActorBootstrapped; + + TS3FileQueueActor( + TTxId txId, + NS3Details::TPathList paths, + size_t prefetchSize, + ui64 fileSizeLimit, + ui64 readLimit, + bool useRuntimeListing, + ui64 consumersCount, + ui64 batchSizeLimit, + ui64 batchObjectCountLimit, + IHTTPGateway::TPtr gateway, + TString url, + TS3Credentials::TAuthInfo authInfo, + TString pattern, + NS3Lister::ES3PatternVariant patternVariant, + NS3Lister::ES3PatternType patternType) + : TxId(std::move(txId)) + , PrefetchSize(prefetchSize) + , FileSizeLimit(fileSizeLimit) + , ReadLimit(readLimit) + , MaybeIssues(Nothing()) + , UseRuntimeListing(useRuntimeListing) + , ConsumersCount(consumersCount) + , BatchSizeLimit(batchSizeLimit) + , BatchObjectCountLimit(batchObjectCountLimit) + , Gateway(std::move(gateway)) + , Url(std::move(url)) + , AuthInfo(std::move(authInfo)) + , Pattern(std::move(pattern)) + , PatternVariant(patternVariant) + , PatternType(patternType) { + for (size_t i = 0; i < paths.size(); ++i) { + NS3::FileQueue::TObjectPath object; + object.SetPath(paths[i].Path); + object.SetPathIndex(paths[i].PathIndex); + if (paths[i].IsDirectory) { + object.SetSize(0); + Directories.emplace_back(std::move(object)); + } else { + object.SetSize(paths[i].Size); + Objects.emplace_back(std::move(object)); + ObjectsTotalSize += paths[i].Size; + } + } + } + + void Bootstrap() { + if (UseRuntimeListing) { + Schedule(PoisonTimeout, new NActors::TEvents::TEvPoison()); + } + if (Directories.empty()) { + LOG_I("TS3FileQueueActor", "Bootstrap there is no directories to list, consumersCount=" << ConsumersCount); + Become(&TS3FileQueueActor::NoMoreDirectoriesState); + } else { + LOG_I("TS3FileQueueActor", "Bootstrap there are directories to list, consumersCount=" << ConsumersCount); + TryPreFetch(); + Become(&TS3FileQueueActor::ThereAreDirectoriesToListState); + } + } + + STATEFN(ThereAreDirectoriesToListState) { + try { + switch (const auto etype = ev->GetTypeRewrite()) { + hFunc(TEvS3Provider::TEvUpdateConsumersCount, HandleUpdateConsumersCount); + hFunc(TEvS3Provider::TEvGetNextBatch, HandleGetNextBatch); + hFunc(TEvPrivatePrivate::TEvNextListingChunkReceived, HandleNextListingChunkReceived); + cFunc(TEvPrivatePrivate::EvRoundRobinStageTimeout, HandleRoundRobinStageTimeout); + hFunc(TEvPrivatePrivate::TEvTransitToErrorState, HandleTransitToErrorState); + cFunc(NActors::TEvents::TSystem::Poison, HandlePoison); + default: + MaybeIssues = TIssues{TIssue{TStringBuilder() << "An event with unknown type has been received: '" << etype << "'"}}; + TransitToErrorState(); + break; + } + } catch (const std::exception& e) { + MaybeIssues = TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; + TransitToErrorState(); + } + } + + void HandleGetNextBatch(TEvS3Provider::TEvGetNextBatch::TPtr& ev) { + if (HasEnoughToSend()) { + LOG_D("TS3FileQueueActor", "HandleGetNextBatch sending right away"); + TrySendObjects(ev->Sender, ev->Get()->Record.GetTransportMeta()); + TryPreFetch(); + } else { + LOG_D("TS3FileQueueActor", "HandleGetNextBatch have not enough objects cached. Start fetching"); + ScheduleRequest(ev->Sender, ev->Get()->Record.GetTransportMeta()); + TryFetch(); + } + } + + void HandleNextListingChunkReceived(TEvPrivatePrivate::TEvNextListingChunkReceived::TPtr& ev) { + Y_ENSURE(FetchingInProgress()); + ListingFuture = Nothing(); + LOG_D("TS3FileQueueActor", "HandleNextListingChunkReceived"); + if (SaveRetrievedResults(ev->Get()->ListingResult)) { + AnswerPendingRequests(true); + if (!HasPendingRequests) { + LOG_D("TS3FileQueueActor", "HandleNextListingChunkReceived no pending requests. Trying to prefetch"); + TryPreFetch(); + } else { + LOG_D("TS3FileQueueActor", "HandleNextListingChunkReceived there are pending requests. Fetching more objects"); + TryFetch(); + } + } else { + TransitToErrorState(); + } + } + + void HandleTransitToErrorState(TEvPrivatePrivate::TEvTransitToErrorState::TPtr& ev) { + MaybeIssues = ev->Get()->Issues; + TransitToErrorState(); + } + + bool SaveRetrievedResults(const NS3Lister::TListResult& listingResult) { + LOG_T("TS3FileQueueActor", "SaveRetrievedResults"); + if (std::holds_alternative(listingResult)) { + MaybeIssues = std::get(listingResult).Issues; + return false; + } + + auto listingChunk = std::get(listingResult); + LOG_D("TS3FileQueueActor", "SaveRetrievedResults saving: " << listingChunk.Objects.size() << " entries"); + Y_ENSURE(listingChunk.Directories.empty()); + for (auto& object: listingChunk.Objects) { + if (object.Path.EndsWith('/')) { + // skip 'directories' + continue; + } + const ui64 bytesUsed = std::min(object.Size, ReadLimit); + if (bytesUsed > FileSizeLimit) { + auto errorMessage = TStringBuilder() + << "Size of object " << object.Path << " = " + << object.Size + << " and exceeds limit = " << FileSizeLimit; + LOG_E("TS3FileQueueActor", errorMessage); + MaybeIssues = TIssues{TIssue{errorMessage}}; + return false; + } + LOG_T("TS3FileQueueActor", "SaveRetrievedResults adding path: " << object.Path); + NS3::FileQueue::TObjectPath objectPath; + objectPath.SetPath(object.Path); + objectPath.SetSize(bytesUsed); + objectPath.SetPathIndex(CurrentDirectoryPathIndex); + Objects.emplace_back(std::move(objectPath)); + ObjectsTotalSize += bytesUsed; + } + return true; + } + + bool FetchingInProgress() const { return ListingFuture.Defined(); } + + void TransitToNoMoreDirectoriesToListState() { + LOG_I("TS3FileQueueActor", "TransitToNoMoreDirectoriesToListState no more directories to list"); + AnswerPendingRequests(); + Become(&TS3FileQueueActor::NoMoreDirectoriesState); + } + + void TransitToErrorState() { + Y_ENSURE(MaybeIssues.Defined()); + LOG_I("TS3FileQueueActor", "TransitToErrorState an error occurred sending "); + AnswerPendingRequests(); + Objects.clear(); + Directories.clear(); + Become(&TS3FileQueueActor::AnErrorOccurredState); + } + + STATEFN(NoMoreDirectoriesState) { + try { + switch (const auto etype = ev->GetTypeRewrite()) { + hFunc(TEvS3Provider::TEvUpdateConsumersCount, HandleUpdateConsumersCount); + hFunc(TEvS3Provider::TEvGetNextBatch, HandleGetNextBatchForEmptyState); + cFunc(TEvPrivatePrivate::EvRoundRobinStageTimeout, HandleRoundRobinStageTimeout); + cFunc(NActors::TEvents::TSystem::Poison, HandlePoison); + default: + MaybeIssues = TIssues{TIssue{TStringBuilder() << "An event with unknown type has been received: '" << etype << "'"}}; + TransitToErrorState(); + break; + } + } catch (const std::exception& e) { + MaybeIssues = TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; + TransitToErrorState(); + } + } + + void HandleGetNextBatchForEmptyState(TEvS3Provider::TEvGetNextBatch::TPtr& ev) { + LOG_T( + "TS3FileQueueActor", + "HandleGetNextBatchForEmptyState Giving away rest of Objects"); + TrySendObjects(ev->Sender, ev->Get()->Record.GetTransportMeta()); + } + + STATEFN(AnErrorOccurredState) { + try { + switch (const auto etype = ev->GetTypeRewrite()) { + hFunc(TEvS3Provider::TEvUpdateConsumersCount, HandleUpdateConsumersCount); + hFunc(TEvS3Provider::TEvGetNextBatch, HandleGetNextBatchForErrorState); + cFunc(TEvPrivatePrivate::EvRoundRobinStageTimeout, HandleRoundRobinStageTimeout); + cFunc(NActors::TEvents::TSystem::Poison, HandlePoison); + default: + MaybeIssues = TIssues{TIssue{TStringBuilder() << "An event with unknown type has been received: '" << etype << "'"}}; + break; + } + } catch (const std::exception& e) { + MaybeIssues = TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; + } + } + + void HandleGetNextBatchForErrorState(TEvS3Provider::TEvGetNextBatch::TPtr& ev) { + LOG_D( + "TS3FileQueueActor", + "HandleGetNextBatchForErrorState Giving away rest of Objects"); + Send(ev->Sender, new TEvS3Provider::TEvObjectPathReadError(*MaybeIssues, ev->Get()->Record.GetTransportMeta())); + TryFinish(ev->Sender, ev->Get()->Record.GetTransportMeta().GetSeqNo()); + } + + void HandleUpdateConsumersCount(TEvS3Provider::TEvUpdateConsumersCount::TPtr& ev) { + if (!UpdatedConsumers.contains(ev->Sender)) { + LOG_D( + "TS3FileQueueActor", + "HandleUpdateConsumersCount Reducing ConsumersCount by " << ev->Get()->Record.GetConsumersCountDelta() << ", recieved from " << ev->Sender); + UpdatedConsumers.insert(ev->Sender); + ConsumersCount -= ev->Get()->Record.GetConsumersCountDelta(); + } + Send(ev->Sender, new TEvS3Provider::TEvAck(ev->Get()->Record.GetTransportMeta())); + } + + void HandleRoundRobinStageTimeout() { + LOG_T("TS3FileQueueActor","Handle start stage timeout"); + if (!RoundRobinStageFinished) { + RoundRobinStageFinished = true; + AnswerPendingRequests(); + } + } + + void HandlePoison() { + AnswerPendingRequests(); + PassAway(); + } + + void PassAway() override { + LOG_D("TS3FileQueueActor", "PassAway"); + TBase::PassAway(); + } + +private: + void TrySendObjects(const NActors::TActorId& consumer, const NDqProto::TMessageTransportMeta& transportMeta) { + if (CanSendToConsumer(consumer)) { + SendObjects(consumer, transportMeta); + } else { + ScheduleRequest(consumer, transportMeta); + } + } + + void SendObjects(const NActors::TActorId& consumer, const NDqProto::TMessageTransportMeta& transportMeta) { + Y_ENSURE(!MaybeIssues.Defined()); + std::vector result; + if (Objects.size() > 0) { + size_t totalSize = 0; + do { + result.push_back(Objects.back()); + Objects.pop_back(); + totalSize += result.back().GetSize(); + } while (Objects.size() > 0 && result.size() < BatchObjectCountLimit && totalSize < BatchSizeLimit); + ObjectsTotalSize -= totalSize; + } + + LOG_T("TS3FileQueueActor", "SendObjects Sending " << result.size() << " objects to consumer with id " << consumer); + Send(consumer, new TEvS3Provider::TEvObjectPathBatch(std::move(result), HasNoMoreItems(), transportMeta)); + + if (HasNoMoreItems()) { + TryFinish(consumer, transportMeta.GetSeqNo()); + } + + if (!RoundRobinStageFinished) { + if (StartedConsumers.empty()) { + Schedule(RoundRobinStageTimeout, new TEvPrivatePrivate::TEvRoundRobinStageTimeout()); + } + StartedConsumers.insert(consumer); + if ((StartedConsumers.size() == ConsumersCount || HasNoMoreItems()) && !IsRoundRobinFinishScheduled) { + IsRoundRobinFinishScheduled = true; + Send(SelfId(), new TEvPrivatePrivate::TEvRoundRobinStageTimeout()); + } + } + } + + bool HasEnoughToSend() { + return Objects.size() >= BatchObjectCountLimit || ObjectsTotalSize >= BatchSizeLimit; + } + + bool CanSendToConsumer(const NActors::TActorId& consumer) { + return !UseRuntimeListing || RoundRobinStageFinished || + (StartedConsumers.size() < ConsumersCount && !StartedConsumers.contains(consumer)); + } + + bool HasNoMoreItems() const { + return !(MaybeLister.Defined() && (*MaybeLister)->HasNext()) && + Directories.empty() && Objects.empty(); + } + + bool TryPreFetch() { + if (Objects.size() < PrefetchSize) { + return TryFetch(); + } + return false; + } + + bool TryFetch() { + if (FetchingInProgress()) { + LOG_D("TS3FileQueueActor", "TryFetch fetching already in progress"); + return true; + } + + if (MaybeLister.Defined() && (*MaybeLister)->HasNext()) { + LOG_D("TS3FileQueueActor", "TryFetch fetching from current lister"); + Fetch(); + return true; + } + + if (!Directories.empty()) { + LOG_D("TS3FileQueueActor", "TryFetch fetching from new lister"); + + auto object = Directories.back(); + Directories.pop_back(); + CurrentDirectoryPathIndex = object.GetPathIndex(); + MaybeLister = NS3Lister::MakeS3Lister( + Gateway, + NS3Lister::TListingRequest{ + Url, + AuthInfo, + PatternVariant == NS3Lister::ES3PatternVariant::PathPattern + ? Pattern + : TStringBuilder{} << object.GetPath() << Pattern, + PatternType, + object.GetPath()}, + Nothing(), + false); + Fetch(); + return true; + } + + LOG_D("TS3FileQueueActor", "TryFetch couldn't start fetching"); + MaybeLister = Nothing(); + TransitToNoMoreDirectoriesToListState(); + return false; + } + + void Fetch() { + Y_ENSURE(!ListingFuture.Defined()); + Y_ENSURE(MaybeLister.Defined()); + NActors::TActorSystem* actorSystem = NActors::TActivationContext::ActorSystem(); + ListingFuture = + (*MaybeLister) + ->Next() + .Subscribe([actorSystem, selfId = SelfId()]( + const NThreading::TFuture& future) { + try { + actorSystem->Send( + selfId, + new TEvPrivatePrivate::TEvNextListingChunkReceived( + future.GetValue())); + } catch (const std::exception& e) { + actorSystem->Send( + selfId, + new TEvPrivatePrivate::TEvTransitToErrorState( + TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}})); + } + }); + } + + void ScheduleRequest(const NActors::TActorId& consumer, const NDqProto::TMessageTransportMeta& transportMeta) { + PendingRequests[consumer].push_back(transportMeta); + HasPendingRequests = true; + } + + void AnswerPendingRequests(bool earlyStop = false) { + bool handledRequest = true; + while (HasPendingRequests && handledRequest) { + bool isEmpty = true; + handledRequest = false; + for (auto& [consumer, requests] : PendingRequests) { + if (!CanSendToConsumer(consumer) || (earlyStop && !HasEnoughToSend())) { + if (!requests.empty()) { + isEmpty = false; + } + continue; + } + if (!requests.empty()) { + if (!MaybeIssues.Defined()) { + SendObjects(consumer, requests.front()); + } else { + Send(consumer, new TEvS3Provider::TEvObjectPathReadError(*MaybeIssues, requests.front())); + TryFinish(consumer, requests.front().GetSeqNo()); + } + requests.pop_front(); + handledRequest = true; + } + if (!requests.empty()) { + isEmpty = false; + } + } + if (isEmpty) { + HasPendingRequests = false; + } + } + } + + void TryFinish(const NActors::TActorId& consumer, ui64 seqNo) { + LOG_T("TS3FileQueueActor", "TryFinish from consumer " << consumer << ", " << FinishedConsumers.size() << " consumers already finished, seqNo=" << seqNo); + if (FinishingConsumerToLastSeqNo.contains(consumer)) { + LOG_T("TS3FileQueueActor", "TryFinish FinishingConsumerToLastSeqNo=" << FinishingConsumerToLastSeqNo[consumer]); + if (FinishingConsumerToLastSeqNo[consumer] < seqNo || SelfId().NodeId() == consumer.NodeId()) { + FinishedConsumers.insert(consumer); + if (FinishedConsumers.size() == ConsumersCount) { + PassAway(); + } + } + } else { + FinishingConsumerToLastSeqNo[consumer] = seqNo; + } + } + +private: + const TTxId TxId; + + std::vector Objects; + std::vector Directories; + + size_t PrefetchSize; + ui64 FileSizeLimit; + ui64 ReadLimit; + TMaybe MaybeLister = Nothing(); + TMaybe> ListingFuture; + size_t CurrentDirectoryPathIndex = 0; + THashMap> PendingRequests; + TMaybe MaybeIssues; + bool UseRuntimeListing; + ui64 ConsumersCount; + ui64 BatchSizeLimit; + ui64 BatchObjectCountLimit; + ui64 ObjectsTotalSize = 0; + THashMap FinishingConsumerToLastSeqNo; + THashSet FinishedConsumers; + bool RoundRobinStageFinished = false; + bool IsRoundRobinFinishScheduled = false; + bool HasPendingRequests = false; + THashSet StartedConsumers; + THashSet UpdatedConsumers; + + const IHTTPGateway::TPtr Gateway; + const TString Url; + const TS3Credentials::TAuthInfo AuthInfo; + const TString Pattern; + const NS3Lister::ES3PatternVariant PatternVariant; + const NS3Lister::ES3PatternType PatternType; + + static constexpr TDuration PoisonTimeout = TDuration::Hours(3); + static constexpr TDuration RoundRobinStageTimeout = TDuration::Seconds(3); +}; + +NActors::IActor* CreateS3FileQueueActor( + TTxId txId, + NS3Details::TPathList paths, + size_t prefetchSize, + ui64 fileSizeLimit, + ui64 readLimit, + bool useRuntimeListing, + ui64 consumersCount, + ui64 batchSizeLimit, + ui64 batchObjectCountLimit, + IHTTPGateway::TPtr gateway, + TString url, + TS3Credentials::TAuthInfo authInfo, + TString pattern, + NS3Lister::ES3PatternVariant patternVariant, + NS3Lister::ES3PatternType patternType) { + return new TS3FileQueueActor( + txId, + paths, + prefetchSize, + fileSizeLimit, + readLimit, + useRuntimeListing, + consumersCount, + batchSizeLimit, + batchObjectCountLimit, + gateway, + url, + authInfo, + pattern, + patternVariant, + patternType + ); +} + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/source_queue.h b/ydb/library/yql/providers/s3/actors/source_queue.h new file mode 100644 index 000000000000..e2b98a98429d --- /dev/null +++ b/ydb/library/yql/providers/s3/actors/source_queue.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include "ydb/library/yql/providers/s3/object_listers/yql_s3_list.h" +#include +#include +#include "ydb/library/yql/providers/s3/range_helpers/path_list_reader.h" +#include +#include + +namespace NYql::NDq { + +NActors::IActor* CreateS3FileQueueActor( + TTxId txId, + NS3Details::TPathList paths, + size_t prefetchSize, + ui64 fileSizeLimit, + ui64 readLimit, + bool useRuntimeListing, + ui64 consumersCount, + ui64 batchSizeLimit, + ui64 batchObjectCountLimit, + IHTTPGateway::TPtr gateway, + TString url, + TS3Credentials::TAuthInfo authInfo, + TString pattern, + NYql::NS3Lister::ES3PatternVariant patternVariant, + NS3Lister::ES3PatternType patternType); + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/ya.make b/ydb/library/yql/providers/s3/actors/ya.make index 8ffe41cf0ddf..479bf2cabe47 100644 --- a/ydb/library/yql/providers/s3/actors/ya.make +++ b/ydb/library/yql/providers/s3/actors/ya.make @@ -9,6 +9,10 @@ ADDINCL( YQL_LAST_ABI_VERSION() SRCS( + parquet_cache.cpp + raw_read_actor.cpp + read_json_each_row.cpp + read_parquet.cpp yql_s3_actors_util.cpp yql_s3_applicator_actor.cpp yql_s3_sink_factory.cpp @@ -18,6 +22,7 @@ SRCS( PEERDIR( contrib/libs/fmt + contrib/libs/apache/arrow contrib/libs/poco/Util ydb/library/actors/http library/cpp/protobuf/util @@ -36,6 +41,8 @@ PEERDIR( ydb/library/yql/providers/s3/common ydb/library/yql/providers/s3/compressors ydb/library/yql/providers/s3/credentials + ydb/library/yql/providers/s3/events + ydb/library/yql/providers/s3/json ydb/library/yql/providers/s3/object_listers ydb/library/yql/providers/s3/proto ydb/library/yql/providers/s3/range_helpers @@ -47,6 +54,7 @@ PEERDIR( IF (CLANG AND NOT WITH_VALGRIND) SRCS( + source_queue.cpp yql_arrow_column_converters.cpp yql_s3_read_actor.cpp ) diff --git a/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.cpp b/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.cpp index 5fdc08f10f99..0c379dfbc91b 100644 --- a/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.cpp +++ b/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.cpp @@ -14,6 +14,8 @@ #include #include +#include + #ifdef THROW #undef THROW #endif @@ -534,4 +536,52 @@ TColumnConverter BuildColumnConverter(const std::string& columnName, const std:: }; } +void BuildColumnConverters(std::shared_ptr outputSchema, std::shared_ptr dataSchema, + std::vector& columnIndices, std::vector& columnConverters, + std::unordered_map> rowTypes, const NDB::FormatSettings& settings) { + + for (int i = 0; i < dataSchema->num_fields(); ++i) { + switch (dataSchema->field(i)->type()->id()) { + case arrow::Type::LIST: + throw parquet::ParquetException(TStringBuilder() << "File contains LIST field " + << dataSchema->field(i)->name() << " and can't be parsed"); + case arrow::Type::STRUCT: + throw parquet::ParquetException(TStringBuilder() << "File contains STRUCT field " + << dataSchema->field(i)->name() << " and can't be parsed"); + default: + ; + } + } + + columnConverters.reserve(outputSchema->num_fields()); + for (int i = 0; i < outputSchema->num_fields(); ++i) { + const auto& targetField = outputSchema->field(i); + auto srcFieldIndex = dataSchema->GetFieldIndex(targetField->name()); + if (srcFieldIndex == -1) { + throw parquet::ParquetException(TStringBuilder() << "Missing field: " << targetField->name()); + }; + auto targetType = targetField->type(); + auto originalType = dataSchema->field(srcFieldIndex)->type(); + if (originalType->layout().has_dictionary) { + throw parquet::ParquetException(TStringBuilder() << "Unsupported dictionary encoding is used for field: " + << targetField->name() << ", type: " << originalType->ToString()); + } + columnIndices.push_back(srcFieldIndex); + auto rowSpecColumnIt = rowTypes.find(targetField->name()); + YQL_ENSURE(rowSpecColumnIt != rowTypes.end(), "Column " << targetField->name() << " not found in row spec"); + columnConverters.emplace_back(BuildColumnConverter(targetField->name(), originalType, targetType, rowSpecColumnIt->second, settings)); + } +} + +std::shared_ptr ConvertArrowColumns(std::shared_ptr batch, std::vector& columnConverters) { + auto columns = batch->columns(); + for (size_t i = 0; i < columnConverters.size(); ++i) { + auto converter = columnConverters[i]; + if (converter) { + columns[i] = converter(columns[i]); + } + } + return arrow::RecordBatch::Make(batch->schema(), batch->num_rows(), columns); +} + } // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.h b/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.h index 4b85bf7ea2f2..770ba0db1773 100644 --- a/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.h +++ b/ydb/library/yql/providers/s3/actors/yql_arrow_column_converters.h @@ -1,6 +1,7 @@ #pragma once #include + #include namespace NYql::NDq { @@ -12,4 +13,16 @@ TColumnConverter BuildColumnConverter( NKikimr::NMiniKQL::TType* yqlType, const NDB::FormatSettings& formatSettings); +void BuildColumnConverters( + std::shared_ptr outputSchema, + std::shared_ptr dataSchema, + std::vector& columnIndices, + std::vector& columnConverters, + std::unordered_map> rowTypes, + const NDB::FormatSettings& settings); + +std::shared_ptr ConvertArrowColumns( + std::shared_ptr batch, + std::vector& columnConverters); + } // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp index d9590bdade64..d07d83aa437a 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp +++ b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.cpp @@ -36,6 +36,11 @@ #endif +#include "parquet_cache.h" +#include "raw_read_actor.h" +#include "read_json_each_row.h" +#include "read_parquet.h" +#include "source_queue.h" #include "yql_arrow_column_converters.h" #include "yql_s3_actors_util.h" #include "yql_s3_read_actor.h" @@ -64,8 +69,10 @@ #include #include +#include #include #include +#include #include #include #include @@ -145,1173 +152,11 @@ struct TS3ReadError : public yexception { using yexception::yexception; }; -using NS3::FileQueue::TObjectPath; -using NDqProto::TMessageTransportMeta; - -struct TEvS3FileQueue { - enum EEv : ui32 { - EvBegin = EventSpaceBegin(NKikimr::TKikimrEvents::ES_S3_FILE_QUEUE), - - EvUpdateConsumersCount = EvBegin, - EvAck, - EvGetNextBatch, - EvObjectPathBatch, - EvObjectPathReadError, - - EvEnd - }; - static_assert(EvEnd < EventSpaceEnd(NKikimr::TKikimrEvents::ES_S3_FILE_QUEUE), - "expect EvEnd < EventSpaceEnd(TEvents::ES_S3_FILE_QUEUE)"); - - struct TEvUpdateConsumersCount : - public TEventPB { - - explicit TEvUpdateConsumersCount(ui64 consumersCountDelta = 0) { - Record.SetConsumersCountDelta(consumersCountDelta); - } - }; - - struct TEvAck : - public TEventPB { - - TEvAck() = default; - - explicit TEvAck(const TMessageTransportMeta& transportMeta) { - Record.MutableTransportMeta()->CopyFrom(transportMeta); - } - }; - - struct TEvGetNextBatch : - public TEventPB { - }; - - struct TEvObjectPathBatch : - public NActors::TEventPB { - - TEvObjectPathBatch() { - Record.SetNoMoreFiles(false); - } - - TEvObjectPathBatch(std::vector objectPaths, bool noMoreFiles, const TMessageTransportMeta& transportMeta) { - Record.MutableObjectPaths()->Assign( - std::make_move_iterator(objectPaths.begin()), - std::make_move_iterator(objectPaths.end())); - Record.SetNoMoreFiles(noMoreFiles); - Record.MutableTransportMeta()->CopyFrom(transportMeta); - } - }; - - struct TEvObjectPathReadError : - public NActors::TEventPB { - - TEvObjectPathReadError() = default; - - TEvObjectPathReadError(TIssues issues, const TMessageTransportMeta& transportMeta) { - IssuesToMessage(issues, Record.MutableIssues()); - Record.MutableTransportMeta()->CopyFrom(transportMeta); - } - }; -}; - -struct TEvPrivate { - // Event ids - enum EEv : ui32 { - EvBegin = TEvRetryQueuePrivate::EvEnd, // Leave space for RetryQueue events - - EvReadResult = EvBegin, - EvDataPart, - EvReadStarted, - EvReadFinished, - EvReadError, - EvRetry, - EvNextBlock, - EvNextRecordBatch, - EvFileFinished, - EvContinue, - EvReadResult2, - - EvEnd - }; - - static_assert(EvEnd < EventSpaceEnd(TEvents::ES_PRIVATE), "expect EvEnd < EventSpaceEnd(TEvents::ES_PRIVATE)"); - - // Events - struct TEvReadResult : public TEventLocal { - TEvReadResult( - IHTTPGateway::TContent&& result, - const TString& requestId, - size_t pathInd, - TString path) - : Result(std::move(result)) - , RequestId(requestId) - , PathIndex(pathInd) - , Path(std::move(path)) { } - - IHTTPGateway::TContent Result; - const TString RequestId; - const size_t PathIndex; - const TString Path; - }; - - struct TEvDataPart : public TEventLocal { - TEvDataPart(IHTTPGateway::TCountedContent&& data) : Result(std::move(data)) {} - IHTTPGateway::TCountedContent Result; - }; - - struct TEvReadStarted : public TEventLocal { - TEvReadStarted(CURLcode curlResponseCode, long httpResponseCode) - : CurlResponseCode(curlResponseCode), HttpResponseCode(httpResponseCode) {} - const CURLcode CurlResponseCode; - const long HttpResponseCode; - }; - - struct TEvReadFinished : public TEventLocal { - TEvReadFinished(size_t pathIndex, CURLcode curlResponseCode, TIssues&& issues) - : PathIndex(pathIndex), CurlResponseCode(curlResponseCode), Issues(std::move(issues)) { - } - const size_t PathIndex; - const CURLcode CurlResponseCode; - TIssues Issues; - }; - - struct TEvFileFinished : public TEventLocal { - TEvFileFinished(size_t pathIndex, ui64 ingressDelta, TDuration cpuTimeDelta) - : PathIndex(pathIndex), IngressDelta(ingressDelta), CpuTimeDelta(cpuTimeDelta) { - } - const size_t PathIndex; - const ui64 IngressDelta; - const TDuration CpuTimeDelta; - }; - - struct TEvReadError : public TEventLocal { - TEvReadError( - TIssues&& error, - const TString& requestId, - size_t pathInd, - TString path) - : Error(std::move(error)) - , RequestId(requestId) - , PathIndex(pathInd) - , Path(std::move(path)) { } - - const TIssues Error; - const TString RequestId; - const size_t PathIndex; - const TString Path; - }; - - struct TEvRetryEventFunc : public NActors::TEventLocal { - explicit TEvRetryEventFunc(std::function functor) : Functor(std::move(functor)) {} - const std::function Functor; - }; - - struct TEvNextBlock : public NActors::TEventLocal { - TEvNextBlock(NDB::Block& block, size_t pathInd, ui64 ingressDelta, TDuration cpuTimeDelta) - : PathIndex(pathInd), IngressDelta(ingressDelta), CpuTimeDelta(cpuTimeDelta) { - Block.swap(block); - } - NDB::Block Block; - const size_t PathIndex; - const ui64 IngressDelta; - const TDuration CpuTimeDelta; - }; - - struct TEvNextRecordBatch : public NActors::TEventLocal { - TEvNextRecordBatch(const std::shared_ptr& batch, size_t pathInd, ui64 ingressDelta, TDuration cpuTimeDelta) - : Batch(batch), PathIndex(pathInd), IngressDelta(ingressDelta), CpuTimeDelta(cpuTimeDelta) { - } - std::shared_ptr Batch; - const size_t PathIndex; - const ui64 IngressDelta; - const TDuration CpuTimeDelta; - }; - - struct TEvContinue : public NActors::TEventLocal { - }; - - struct TReadRange { - int64_t Offset; - int64_t Length; - }; - - struct TEvReadResult2 : public TEventLocal { - TEvReadResult2(TReadRange readRange, IHTTPGateway::TContent&& result) : ReadRange(readRange), Failure(false), Result(std::move(result)) { } - TEvReadResult2(TReadRange readRange, TIssues&& issues) : ReadRange(readRange), Failure(true), Result(""), Issues(std::move(issues)) { } - const TReadRange ReadRange; - const bool Failure; - IHTTPGateway::TContent Result; - const TIssues Issues; - }; - -}; - -using namespace NKikimr::NMiniKQL; - -class TS3FileQueueActor : public TActorBootstrapped { -public: - static constexpr char ActorName[] = "YQ_S3_FILE_QUEUE_ACTOR"; - - struct TEvPrivatePrivate { - enum { - EvBegin = TEvRetryQueuePrivate::EvEnd, // Leave space for RetryQueue events - - EvNextListingChunkReceived = EvBegin, - EvRoundRobinStageTimeout, - EvTransitToErrorState, - - EvEnd - }; - static_assert( - EvEnd <= EventSpaceEnd(TEvents::ES_PRIVATE), - "expected EvEnd <= EventSpaceEnd(TEvents::ES_PRIVATE)"); - - struct TEvNextListingChunkReceived : public TEventLocal { - NS3Lister::TListResult ListingResult; - TEvNextListingChunkReceived(NS3Lister::TListResult listingResult) - : ListingResult(std::move(listingResult)){}; - }; - - struct TEvRoundRobinStageTimeout : public TEventLocal { - }; - - struct TEvTransitToErrorState : public TEventLocal { - explicit TEvTransitToErrorState(TIssues&& issues) - : Issues(issues) { - } - TIssues Issues; - }; - }; - using TBase = TActorBootstrapped; - - TS3FileQueueActor( - TTxId txId, - TPathList paths, - size_t prefetchSize, - ui64 fileSizeLimit, - ui64 readLimit, - bool useRuntimeListing, - ui64 consumersCount, - ui64 batchSizeLimit, - ui64 batchObjectCountLimit, - IHTTPGateway::TPtr gateway, - TString url, - TS3Credentials::TAuthInfo authInfo, - TString pattern, - ES3PatternVariant patternVariant, - ES3PatternType patternType) - : TxId(std::move(txId)) - , PrefetchSize(prefetchSize) - , FileSizeLimit(fileSizeLimit) - , ReadLimit(readLimit) - , MaybeIssues(Nothing()) - , UseRuntimeListing(useRuntimeListing) - , ConsumersCount(consumersCount) - , BatchSizeLimit(batchSizeLimit) - , BatchObjectCountLimit(batchObjectCountLimit) - , Gateway(std::move(gateway)) - , Url(std::move(url)) - , AuthInfo(std::move(authInfo)) - , Pattern(std::move(pattern)) - , PatternVariant(patternVariant) - , PatternType(patternType) { - for (size_t i = 0; i < paths.size(); ++i) { - TObjectPath object; - object.SetPath(paths[i].Path); - object.SetPathIndex(paths[i].PathIndex); - if (paths[i].IsDirectory) { - LOG_T("TS3FileQueueActor", "TS3FileQueueActor adding dir: " << paths[i].Path); - object.SetSize(0); - Directories.emplace_back(std::move(object)); - } else { - LOG_T("TS3FileQueueActor", "TS3FileQueueActor adding path: " << paths[i].Path << " of size " << paths[i].Size); - object.SetSize(paths[i].Size); - Objects.emplace_back(std::move(object)); - ObjectsTotalSize += paths[i].Size; - } - } - } - - void Bootstrap() { - if (UseRuntimeListing) { - Schedule(PoisonTimeout, new TEvents::TEvPoison()); - } - if (Directories.empty()) { - LOG_I("TS3FileQueueActor", "Bootstrap there is no directories to list, consumersCount=" << ConsumersCount); - Become(&TS3FileQueueActor::NoMoreDirectoriesState); - } else { - LOG_I("TS3FileQueueActor", "Bootstrap there are directories to list, consumersCount=" << ConsumersCount); - TryPreFetch(); - Become(&TS3FileQueueActor::ThereAreDirectoriesToListState); - } - } - - STATEFN(ThereAreDirectoriesToListState) { - try { - switch (const auto etype = ev->GetTypeRewrite()) { - hFunc(TEvS3FileQueue::TEvUpdateConsumersCount, HandleUpdateConsumersCount); - hFunc(TEvS3FileQueue::TEvGetNextBatch, HandleGetNextBatch); - hFunc(TEvPrivatePrivate::TEvNextListingChunkReceived, HandleNextListingChunkReceived); - cFunc(TEvPrivatePrivate::EvRoundRobinStageTimeout, HandleRoundRobinStageTimeout); - hFunc(TEvPrivatePrivate::TEvTransitToErrorState, HandleTransitToErrorState); - cFunc(TEvents::TSystem::Poison, HandlePoison); - default: - MaybeIssues = TIssues{TIssue{TStringBuilder() << "An event with unknown type has been received: '" << etype << "'"}}; - TransitToErrorState(); - break; - } - } catch (const std::exception& e) { - MaybeIssues = TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; - TransitToErrorState(); - } - } - - void HandleGetNextBatch(TEvS3FileQueue::TEvGetNextBatch::TPtr& ev) { - if (HasEnoughToSend()) { - LOG_D("TS3FileQueueActor", "HandleGetNextBatch sending right away"); - TrySendObjects(ev->Sender, ev->Get()->Record.GetTransportMeta()); - TryPreFetch(); - } else { - LOG_D("TS3FileQueueActor", "HandleGetNextBatch have not enough objects cached. Start fetching"); - ScheduleRequest(ev->Sender, ev->Get()->Record.GetTransportMeta()); - TryFetch(); - } - } - - void HandleNextListingChunkReceived(TEvPrivatePrivate::TEvNextListingChunkReceived::TPtr& ev) { - Y_ENSURE(FetchingInProgress()); - ListingFuture = Nothing(); - LOG_D("TS3FileQueueActor", "HandleNextListingChunkReceived"); - if (SaveRetrievedResults(ev->Get()->ListingResult)) { - AnswerPendingRequests(true); - if (!HasPendingRequests) { - LOG_D("TS3FileQueueActor", "HandleNextListingChunkReceived no pending requests. Trying to prefetch"); - TryPreFetch(); - } else { - LOG_D("TS3FileQueueActor", "HandleNextListingChunkReceived there are pending requests. Fetching more objects"); - TryFetch(); - } - } else { - TransitToErrorState(); - } - } - - void HandleTransitToErrorState(TEvPrivatePrivate::TEvTransitToErrorState::TPtr& ev) { - MaybeIssues = ev->Get()->Issues; - TransitToErrorState(); - } - - bool SaveRetrievedResults(const NS3Lister::TListResult& listingResult) { - if (std::holds_alternative(listingResult)) { - MaybeIssues = std::get(listingResult).Issues; - LOG_E("TS3FileQueueActor", "SaveRetrievedResults error: [" << (MaybeIssues ? MaybeIssues->ToOneLineString() : "") << "]"); - return false; - } - - auto listingChunk = std::get(listingResult); - LOG_D("TS3FileQueueActor", "SaveRetrievedResults saving: " << listingChunk.Objects.size() << " entries"); - Y_ENSURE(listingChunk.Directories.empty()); - for (auto& object: listingChunk.Objects) { - if (object.Path.EndsWith('/')) { - // skip 'directories' - continue; - } - - const ui64 bytesUsed = std::min(object.Size, ReadLimit); - if (bytesUsed > FileSizeLimit) { - auto errorMessage = TStringBuilder() - << "Size of object " << object.Path << " = " - << object.Size - << " and exceeds limit = " << FileSizeLimit; - LOG_E("TS3FileQueueActor", errorMessage); - MaybeIssues = TIssues{TIssue{errorMessage}}; - return false; - } - LOG_T("TS3FileQueueActor", "SaveRetrievedResults adding path: " << object.Path << " of size " << object.Size); - TObjectPath objectPath; - objectPath.SetPath(object.Path); - objectPath.SetSize(bytesUsed); - objectPath.SetPathIndex(CurrentDirectoryPathIndex); - Objects.emplace_back(std::move(objectPath)); - ObjectsTotalSize += bytesUsed; - } - return true; - } - - bool FetchingInProgress() const { return ListingFuture.Defined(); } - - void TransitToNoMoreDirectoriesToListState() { - LOG_I("TS3FileQueueActor", "TransitToNoMoreDirectoriesToListState no more directories to list"); - AnswerPendingRequests(); - Become(&TS3FileQueueActor::NoMoreDirectoriesState); - } - - void TransitToErrorState() { - Y_ENSURE(MaybeIssues.Defined()); - LOG_I("TS3FileQueueActor", "TransitToErrorState an error occurred sending "); - AnswerPendingRequests(); - Objects.clear(); - Directories.clear(); - Become(&TS3FileQueueActor::AnErrorOccurredState); - } - - STATEFN(NoMoreDirectoriesState) { - try { - switch (const auto etype = ev->GetTypeRewrite()) { - hFunc(TEvS3FileQueue::TEvUpdateConsumersCount, HandleUpdateConsumersCount); - hFunc(TEvS3FileQueue::TEvGetNextBatch, HandleGetNextBatchForEmptyState); - cFunc(TEvPrivatePrivate::EvRoundRobinStageTimeout, HandleRoundRobinStageTimeout); - cFunc(TEvents::TSystem::Poison, HandlePoison); - default: - MaybeIssues = TIssues{TIssue{TStringBuilder() << "An event with unknown type has been received: '" << etype << "'"}}; - TransitToErrorState(); - break; - } - } catch (const std::exception& e) { - MaybeIssues = TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; - TransitToErrorState(); - } - } - - void HandleGetNextBatchForEmptyState(TEvS3FileQueue::TEvGetNextBatch::TPtr& ev) { - LOG_T( - "TS3FileQueueActor", - "HandleGetNextBatchForEmptyState Giving away rest of Objects"); - TrySendObjects(ev->Sender, ev->Get()->Record.GetTransportMeta()); - } - - STATEFN(AnErrorOccurredState) { - try { - switch (const auto etype = ev->GetTypeRewrite()) { - hFunc(TEvS3FileQueue::TEvUpdateConsumersCount, HandleUpdateConsumersCount); - hFunc(TEvS3FileQueue::TEvGetNextBatch, HandleGetNextBatchForErrorState); - cFunc(TEvPrivatePrivate::TEvRoundRobinStageTimeout::EventType, HandleRoundRobinStageTimeout); - cFunc(TEvents::TSystem::Poison, HandlePoison); - default: - MaybeIssues = TIssues{TIssue{TStringBuilder() << "An event with unknown type has been received: '" << etype << "'"}}; - break; - } - } catch (const std::exception& e) { - MaybeIssues = TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; - } - } - - void HandleGetNextBatchForErrorState(TEvS3FileQueue::TEvGetNextBatch::TPtr& ev) { - LOG_D( - "TS3FileQueueActor", - "HandleGetNextBatchForErrorState Giving away rest of Objects"); - Send(ev->Sender, new TEvS3FileQueue::TEvObjectPathReadError(*MaybeIssues, ev->Get()->Record.GetTransportMeta())); - TryFinish(ev->Sender, ev->Get()->Record.GetTransportMeta().GetSeqNo()); - } - - void HandleUpdateConsumersCount(TEvS3FileQueue::TEvUpdateConsumersCount::TPtr& ev) { - if (!UpdatedConsumers.contains(ev->Sender)) { - UpdatedConsumers.insert(ev->Sender); - ConsumersCount -= ev->Get()->Record.GetConsumersCountDelta(); - LOG_D( - "TS3FileQueueActor", - "HandleUpdateConsumersCount Reducing ConsumersCount by " << ev->Get()->Record.GetConsumersCountDelta() - << " to " << ConsumersCount << ", received from " << ev->Sender); - } - Send(ev->Sender, new TEvS3FileQueue::TEvAck(ev->Get()->Record.GetTransportMeta())); - } - - void HandleRoundRobinStageTimeout() { - LOG_D("TS3FileQueueActor","Handle start stage timeout"); - if (!RoundRobinStageFinished) { - RoundRobinStageFinished = true; - AnswerPendingRequests(); - } - } - - void HandlePoison() { - AnswerPendingRequests(); - PassAway(); - } - - void PassAway() override { - LOG_D("TS3FileQueueActor", "PassAway"); - TBase::PassAway(); - } - -private: - void TrySendObjects(const TActorId& consumer, const NDqProto::TMessageTransportMeta& transportMeta) { - if (CanSendToConsumer(consumer)) { - SendObjects(consumer, transportMeta); - } else { - ScheduleRequest(consumer, transportMeta); - } - } - - void SendObjects(const TActorId& consumer, const NDqProto::TMessageTransportMeta& transportMeta) { - Y_ENSURE(!MaybeIssues.Defined()); - std::vector result; - if (Objects.size() > 0) { - size_t totalSize = 0; - do { - result.push_back(Objects.back()); - Objects.pop_back(); - totalSize += result.back().GetSize(); - } while (Objects.size() > 0 && result.size() < BatchObjectCountLimit && totalSize < BatchSizeLimit); - ObjectsTotalSize -= totalSize; - } - - LOG_D("TS3FileQueueActor", "SendObjects Sending " << result.size() << " objects to consumer with id " << consumer << ", " << ObjectsTotalSize << " bytes left"); - Send(consumer, new TEvS3FileQueue::TEvObjectPathBatch(std::move(result), HasNoMoreItems(), transportMeta)); - - if (HasNoMoreItems()) { - TryFinish(consumer, transportMeta.GetSeqNo()); - } - - if (!RoundRobinStageFinished) { - if (StartedConsumers.empty()) { - Schedule(RoundRobinStageTimeout, new TEvPrivatePrivate::TEvRoundRobinStageTimeout()); - } - StartedConsumers.insert(consumer); - if ((StartedConsumers.size() == ConsumersCount || HasNoMoreItems()) && !IsRoundRobinFinishScheduled) { - IsRoundRobinFinishScheduled = true; - Send(SelfId(), new TEvPrivatePrivate::TEvRoundRobinStageTimeout()); - } - } - } - - bool HasEnoughToSend() { - return Objects.size() >= BatchObjectCountLimit || ObjectsTotalSize >= BatchSizeLimit; - } - - bool CanSendToConsumer(const TActorId& consumer) { - return !UseRuntimeListing || RoundRobinStageFinished || - (StartedConsumers.size() < ConsumersCount && !StartedConsumers.contains(consumer)); - } - - bool HasNoMoreItems() const { - return !(MaybeLister.Defined() && (*MaybeLister)->HasNext()) && - Directories.empty() && Objects.empty(); - } - - bool TryPreFetch() { - if (Objects.size() < PrefetchSize) { - return TryFetch(); - } - return false; - } - - bool TryFetch() { - if (FetchingInProgress()) { - LOG_D("TS3FileQueueActor", "TryFetch fetching already in progress"); - return true; - } - - if (MaybeLister.Defined() && (*MaybeLister)->HasNext()) { - LOG_D("TS3FileQueueActor", "TryFetch fetching from current lister"); - Fetch(); - return true; - } - - if (!Directories.empty()) { - LOG_D("TS3FileQueueActor", "TryFetch fetching from new lister"); - - auto object = Directories.back(); - Directories.pop_back(); - CurrentDirectoryPathIndex = object.GetPathIndex(); - MaybeLister = NS3Lister::MakeS3Lister( - Gateway, - NS3Lister::TListingRequest{ - Url, - AuthInfo, - PatternVariant == ES3PatternVariant::PathPattern - ? Pattern - : TStringBuilder{} << object.GetPath() << Pattern, - PatternType, - object.GetPath()}, - Nothing(), - false); - Fetch(); - return true; - } - - LOG_D("TS3FileQueueActor", "TryFetch couldn't start fetching"); - MaybeLister = Nothing(); - TransitToNoMoreDirectoriesToListState(); - return false; - } - - void Fetch() { - Y_ENSURE(!ListingFuture.Defined()); - Y_ENSURE(MaybeLister.Defined()); - NActors::TActorSystem* actorSystem = NActors::TActivationContext::ActorSystem(); - ListingFuture = - (*MaybeLister) - ->Next() - .Subscribe([actorSystem, selfId = SelfId()]( - const NThreading::TFuture& future) { - try { - actorSystem->Send( - selfId, - new TEvPrivatePrivate::TEvNextListingChunkReceived( - future.GetValue())); - } catch (const std::exception& e) { - actorSystem->Send( - selfId, - new TEvPrivatePrivate::TEvTransitToErrorState( - TIssues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}})); - } - }); - } - - void ScheduleRequest(const TActorId& consumer, const TMessageTransportMeta& transportMeta) { - PendingRequests[consumer].push_back(transportMeta); - HasPendingRequests = true; - } - - void AnswerPendingRequests(bool earlyStop = false) { - bool handledRequest = true; - while (HasPendingRequests && handledRequest) { - bool isEmpty = true; - handledRequest = false; - for (auto& [consumer, requests] : PendingRequests) { - if (!CanSendToConsumer(consumer) || (earlyStop && !HasEnoughToSend())) { - if (!requests.empty()) { - isEmpty = false; - } - continue; - } - if (!requests.empty()) { - if (!MaybeIssues.Defined()) { - SendObjects(consumer, requests.front()); - } else { - Send(consumer, new TEvS3FileQueue::TEvObjectPathReadError(*MaybeIssues, requests.front())); - TryFinish(consumer, requests.front().GetSeqNo()); - } - requests.pop_front(); - handledRequest = true; - } - if (!requests.empty()) { - isEmpty = false; - } - } - if (isEmpty) { - HasPendingRequests = false; - } - } - } - - void TryFinish(const TActorId& consumer, ui64 seqNo) { - LOG_T("TS3FileQueueActor", "TryFinish from consumer " << consumer << ", " << FinishedConsumers.size() << " consumers already finished, seqNo=" << seqNo); - if (FinishingConsumerToLastSeqNo.contains(consumer)) { - LOG_T("TS3FileQueueActor", "TryFinish FinishingConsumerToLastSeqNo=" << FinishingConsumerToLastSeqNo[consumer]); - if (FinishingConsumerToLastSeqNo[consumer] < seqNo || SelfId().NodeId() == consumer.NodeId()) { - FinishedConsumers.insert(consumer); - if (FinishedConsumers.size() == ConsumersCount) { - PassAway(); - } - } - } else { - FinishingConsumerToLastSeqNo[consumer] = seqNo; - } - } - -private: - const TTxId TxId; - - std::vector Objects; - std::vector Directories; - - size_t PrefetchSize; - ui64 FileSizeLimit; - ui64 ReadLimit; - TMaybe MaybeLister = Nothing(); - TMaybe> ListingFuture; - size_t CurrentDirectoryPathIndex = 0; - THashMap> PendingRequests; - TMaybe MaybeIssues; - bool UseRuntimeListing; - ui64 ConsumersCount; - ui64 BatchSizeLimit; - ui64 BatchObjectCountLimit; - ui64 ObjectsTotalSize = 0; - THashMap FinishingConsumerToLastSeqNo; - THashSet FinishedConsumers; - bool RoundRobinStageFinished = false; - bool IsRoundRobinFinishScheduled = false; - bool HasPendingRequests = false; - THashSet StartedConsumers; - THashSet UpdatedConsumers; - - const IHTTPGateway::TPtr Gateway; - const TString Url; - const TS3Credentials::TAuthInfo AuthInfo; - const TString Pattern; - const ES3PatternVariant PatternVariant; - const ES3PatternType PatternType; - - static constexpr TDuration PoisonTimeout = TDuration::Hours(3); - static constexpr TDuration RoundRobinStageTimeout = TDuration::Seconds(3); -}; - -ui64 SubtractSaturating(ui64 lhs, ui64 rhs) { - return (lhs > rhs) ? lhs - rhs : 0; -} - -class TS3ReadActor : public TActorBootstrapped, public IDqComputeActorAsyncInput { -public: - TS3ReadActor(ui64 inputIndex, - TCollectStatsLevel statsLevel, - const TTxId& txId, - IHTTPGateway::TPtr gateway, - const THolderFactory& holderFactory, - const TString& url, - const TS3Credentials::TAuthInfo& authInfo, - const TString& pattern, - ES3PatternVariant patternVariant, - TPathList&& paths, - bool addPathIndex, - const NActors::TActorId& computeActorId, - ui64 sizeLimit, - const IHTTPGateway::TRetryPolicy::TPtr& retryPolicy, - const TS3ReadActorFactoryConfig& readActorFactoryCfg, - ::NMonitoring::TDynamicCounterPtr counters, - ::NMonitoring::TDynamicCounterPtr taskCounters, - ui64 fileSizeLimit, - std::optional rowsLimitHint, - bool useRuntimeListing, - TActorId fileQueueActor, - ui64 fileQueueBatchSizeLimit, - ui64 fileQueueBatchObjectCountLimit, - ui64 fileQueueConsumersCountDelta) - : ReadActorFactoryCfg(readActorFactoryCfg) - , Gateway(std::move(gateway)) - , HolderFactory(holderFactory) - , InputIndex(inputIndex) - , TxId(txId) - , ComputeActorId(computeActorId) - , RetryPolicy(retryPolicy) - , ActorSystem(TActivationContext::ActorSystem()) - , Url(url) - , AuthInfo(authInfo) - , Pattern(pattern) - , PatternVariant(patternVariant) - , Paths(std::move(paths)) - , FileQueueActor(fileQueueActor) - , AddPathIndex(addPathIndex) - , SizeLimit(sizeLimit) - , Counters(counters) - , TaskCounters(taskCounters) - , FileSizeLimit(fileSizeLimit) - , FilesRemained(rowsLimitHint) - , UseRuntimeListing(useRuntimeListing) - , FileQueueBatchSizeLimit(fileQueueBatchSizeLimit) - , FileQueueBatchObjectCountLimit(fileQueueBatchObjectCountLimit) - , FileQueueConsumersCountDelta(fileQueueConsumersCountDelta) { - if (Counters) { - QueueDataSize = Counters->GetCounter("QueueDataSize"); - QueueDataLimit = Counters->GetCounter("QueueDataLimit"); - QueueBlockCount = Counters->GetCounter("QueueBlockCount"); - QueueDataLimit->Add(ReadActorFactoryCfg.DataInflight); - } - if (TaskCounters) { - TaskQueueDataSize = TaskCounters->GetCounter("QueueDataSize"); - TaskQueueDataLimit = TaskCounters->GetCounter("QueueDataLimit"); - TaskQueueDataLimit->Add(ReadActorFactoryCfg.DataInflight); - } - IngressStats.Level = statsLevel; - } - - void Bootstrap() { - if (!UseRuntimeListing) { - FileQueueActor = RegisterWithSameMailbox(new TS3FileQueueActor{ - TxId, - std::move(Paths), - ReadActorFactoryCfg.MaxInflight * 2, - FileSizeLimit, - SizeLimit, - false, - 1, - FileQueueBatchSizeLimit, - FileQueueBatchObjectCountLimit, - Gateway, - Url, - AuthInfo, - Pattern, - PatternVariant, - ES3PatternType::Wildcard}); - } - - LOG_D("TS3ReadActor", "Bootstrap" << ", InputIndex: " << InputIndex << ", FileQueue: " << FileQueueActor << (UseRuntimeListing ? " (remote)" : " (local")); - - FileQueueEvents.Init(TxId, SelfId(), SelfId()); - FileQueueEvents.OnNewRecipientId(FileQueueActor); - if (UseRuntimeListing && FileQueueConsumersCountDelta > 0) { - FileQueueEvents.Send(new TEvS3FileQueue::TEvUpdateConsumersCount(FileQueueConsumersCountDelta)); - } - SendPathBatchRequest(); - - Become(&TS3ReadActor::StateFunc); - } - - bool TryStartDownload() { - TrySendPathBatchRequest(); - if (PathBatchQueue.empty()) { - // no path is pending - return false; - } - if (IsCurrentBatchEmpty) { - // waiting for batch to finish - return false; - } - if (QueueTotalDataSize > ReadActorFactoryCfg.DataInflight) { - // too large data inflight - return false; - } - if (DownloadInflight >= ReadActorFactoryCfg.MaxInflight) { - // too large download inflight - return false; - } - if (ConsumedEnoughFiles()) { - // started enough downloads - return false; - } - - StartDownload(); - return true; - } - - void StartDownload() { - DownloadInflight++; - const auto& object = ReadPathFromCache(); - auto url = Url + object.GetPath(); - auto id = object.GetPathIndex(); - const TString requestId = CreateGuidAsString(); - LOG_D("TS3ReadActor", "Download: " << url << ", ID: " << id << ", request id: [" << requestId << "]"); - Gateway->Download( - UrlEscapeRet(url, true), - IHTTPGateway::MakeYcHeaders(requestId, AuthInfo.GetToken(), {}, AuthInfo.GetAwsUserPwd(), AuthInfo.GetAwsSigV4()), - 0U, - std::min(object.GetSize(), SizeLimit), - std::bind(&TS3ReadActor::OnDownloadFinished, ActorSystem, SelfId(), requestId, std::placeholders::_1, id, object.GetPath()), - {}, - RetryPolicy); - } - - TObjectPath ReadPathFromCache() { - Y_ENSURE(!PathBatchQueue.empty()); - auto& currentBatch = PathBatchQueue.front(); - Y_ENSURE(!currentBatch.empty()); - auto object = currentBatch.back(); - currentBatch.pop_back(); - if (currentBatch.empty()) { - PathBatchQueue.pop_front(); - IsCurrentBatchEmpty = true; - } - TrySendPathBatchRequest(); - return object; - } - void TrySendPathBatchRequest() { - if (PathBatchQueue.size() < 2 && !IsFileQueueEmpty && !ConsumedEnoughFiles() && !IsWaitingFileQueueResponse) { - SendPathBatchRequest(); - } - } - void SendPathBatchRequest() { - FileQueueEvents.Send(new TEvS3FileQueue::TEvGetNextBatch()); - IsWaitingFileQueueResponse = true; - } - - static constexpr char ActorName[] = "S3_READ_ACTOR"; - -private: - void SaveState(const NDqProto::TCheckpoint&, TSourceState&) final {} - void LoadState(const TSourceState&) final {} - void CommitState(const NDqProto::TCheckpoint&) final {} - - ui64 GetInputIndex() const final { - return InputIndex; - } - - const TDqAsyncStats& GetIngressStats() const override { - return IngressStats; - } - - TDuration GetCpuTime() override { - return CpuTime; - } - - bool ConsumedEnoughFiles() const { - return FilesRemained && (*FilesRemained == 0); - } - - STRICT_STFUNC_EXC(StateFunc, - hFunc(TEvPrivate::TEvReadResult, Handle); - hFunc(TEvPrivate::TEvReadError, Handle); - hFunc(TEvS3FileQueue::TEvObjectPathBatch, HandleObjectPathBatch); - hFunc(TEvS3FileQueue::TEvObjectPathReadError, HandleObjectPathReadError); - hFunc(TEvS3FileQueue::TEvAck, HandleAck); - hFunc(NYql::NDq::TEvRetryQueuePrivate::TEvRetry, Handle); - hFunc(NActors::TEvInterconnect::TEvNodeDisconnected, Handle); - hFunc(NActors::TEvInterconnect::TEvNodeConnected, Handle); - hFunc(NActors::TEvents::TEvUndelivered, Handle); - , catch (const std::exception& e) { - TIssues issues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; - Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); - } - ) - - void HandleObjectPathBatch(TEvS3FileQueue::TEvObjectPathBatch::TPtr& objectPathBatch) { - if (!FileQueueEvents.OnEventReceived(objectPathBatch)) { - LOG_W("TS3ReadActor", "Duplicated TEvObjectPathBatch (likely resent) from " << FileQueueActor); - return; - } - - Y_ENSURE(IsWaitingFileQueueResponse); - IsWaitingFileQueueResponse = false; - auto& objectBatch = objectPathBatch->Get()->Record; - ListedFiles += objectBatch.GetObjectPaths().size(); - IsFileQueueEmpty = objectBatch.GetNoMoreFiles(); - if (IsFileQueueEmpty && !IsConfirmedFileQueueFinish) { - LOG_D("TS3ReadActor", "Confirm finish to " << FileQueueActor); - SendPathBatchRequest(); - IsConfirmedFileQueueFinish = true; - } - if (!objectBatch.GetObjectPaths().empty()) { - PathBatchQueue.emplace_back( - std::make_move_iterator(objectBatch.MutableObjectPaths()->begin()), - std::make_move_iterator(objectBatch.MutableObjectPaths()->end())); - } - while (TryStartDownload()) {} - - if (LastFileWasProcessed()) { - Send(ComputeActorId, new TEvNewAsyncInputDataArrived(InputIndex)); - } - } - void HandleObjectPathReadError(TEvS3FileQueue::TEvObjectPathReadError::TPtr& result) { - if (!FileQueueEvents.OnEventReceived(result)) { - LOG_W("TS3ReadActor", "Duplicated TEvObjectPathReadError (likely resent) from " << FileQueueActor); - return; - } - - IsFileQueueEmpty = true; - if (!IsConfirmedFileQueueFinish) { - LOG_D("TS3ReadActor", "Confirm finish (with errors) to " << FileQueueActor); - SendPathBatchRequest(); - IsConfirmedFileQueueFinish = true; - } - TIssues issues; - IssuesFromMessage(result->Get()->Record.GetIssues(), issues); - LOG_E("TS3ReadActor", "Error while object listing, details: TEvObjectPathReadError: " << issues.ToOneLineString()); - issues = NS3Util::AddParentIssue(TStringBuilder{} << "Error while object listing", std::move(issues)); - Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); - } - - void HandleAck(TEvS3FileQueue::TEvAck::TPtr& ev) { - FileQueueEvents.OnEventReceived(ev); - } - - static void OnDownloadFinished(TActorSystem* actorSystem, TActorId selfId, const TString& requestId, IHTTPGateway::TResult&& result, size_t pathInd, const TString path) { - if (!result.Issues) { - actorSystem->Send(new IEventHandle(selfId, TActorId(), new TEvPrivate::TEvReadResult(std::move(result.Content), requestId, pathInd, path))); - } else { - actorSystem->Send(new IEventHandle(selfId, TActorId(), new TEvPrivate::TEvReadError(std::move(result.Issues), requestId, pathInd, path))); - } - } - - i64 GetAsyncInputData(TUnboxedValueBatch& buffer, TMaybe&, bool& finished, i64 freeSpace) final { - i64 total = 0LL; - if (!Blocks.empty()) { - do { - auto& content = std::get(Blocks.front()); - const auto size = content.size(); - auto value = MakeString(std::string_view(content)); - if (AddPathIndex) { - NUdf::TUnboxedValue* tupleItems = nullptr; - auto tuple = ContainerCache.NewArray(HolderFactory, 2, tupleItems); - *tupleItems++ = value; - *tupleItems++ = NUdf::TUnboxedValuePod(std::get(Blocks.front())); - value = tuple; - } - - buffer.emplace_back(std::move(value)); - Blocks.pop(); - total += size; - freeSpace -= size; - - QueueTotalDataSize -= size; - if (Counters) { - QueueDataSize->Sub(size); - QueueBlockCount->Dec(); - } - if (TaskCounters) { - TaskQueueDataSize->Sub(size); - } - TryStartDownload(); - } while (!Blocks.empty() && freeSpace > 0LL); - } - - if ((LastFileWasProcessed() || ConsumedEnoughFiles()) && !FileQueueEvents.RemoveConfirmedEvents()) { - finished = true; - ContainerCache.Clear(); - } - - if (!total) { - IngressStats.TryPause(); - } - - return total; - } - bool LastFileWasProcessed() const { - return Blocks.empty() && (ListedFiles == CompletedFiles) && IsFileQueueEmpty; - } - - void Handle(TEvPrivate::TEvReadResult::TPtr& result) { - ++CompletedFiles; - const auto id = result->Get()->PathIndex; - const auto path = result->Get()->Path; - const auto httpCode = result->Get()->Result.HttpResponseCode; - const auto requestId = result->Get()->RequestId; - LOG_D("TS3ReadActor", "ID: " << id << ", Path: " << path << ", read size: " << result->Get()->Result.size() << ", HTTP response code: " << httpCode << ", request id: [" << requestId << "]"); - if (200 == httpCode || 206 == httpCode) { - auto size = result->Get()->Result.size(); - - // in TS3ReadActor all files (aka Splits) are loaded in single Chunks - IngressStats.Bytes += size; - IngressStats.Rows++; - IngressStats.Chunks++; - IngressStats.Splits++; - IngressStats.Resume(); - - QueueTotalDataSize += size; - if (Counters) { - QueueBlockCount->Inc(); - QueueDataSize->Add(size); - } - if (TaskCounters) { - TaskQueueDataSize->Add(size); - } - Blocks.emplace(std::make_tuple(std::move(result->Get()->Result), id)); - DownloadInflight--; - if (IsCurrentBatchEmpty && DownloadInflight == 0) { - IsCurrentBatchEmpty = false; - } - if (FilesRemained) { - *FilesRemained = SubtractSaturating(*FilesRemained, 1); - } - TryStartDownload(); - Send(ComputeActorId, new TEvNewAsyncInputDataArrived(InputIndex)); - } else { - TString errorText = result->Get()->Result.Extract(); - TString errorCode; - TString message; - if (!ParseS3ErrorResponse(errorText, errorCode, message)) { - message = errorText; - } - message = TStringBuilder{} << "Error while reading file " << path << ", details: " << message << ", request id: [" << requestId << "]"; - Send(ComputeActorId, new TEvAsyncInputError(InputIndex, BuildIssues(httpCode, errorCode, message), NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); - } - } - - void Handle(TEvPrivate::TEvReadError::TPtr& result) { - ++CompletedFiles; - auto id = result->Get()->PathIndex; - const auto requestId = result->Get()->RequestId; - const auto path = result->Get()->Path; - LOG_W("TS3ReadActor", "Error while reading file " << path << ", details: ID: " << id << ", TEvReadError: " << result->Get()->Error.ToOneLineString() << ", request id: [" << requestId << "]"); - auto issues = NS3Util::AddParentIssue(TStringBuilder{} << "Error while reading file " << path << " with request id [" << requestId << "]", TIssues{result->Get()->Error}); - Send(ComputeActorId, new TEvAsyncInputError(InputIndex, std::move(issues), NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); - } - - void Handle(const NYql::NDq::TEvRetryQueuePrivate::TEvRetry::TPtr&) { - FileQueueEvents.Retry(); - } - - void Handle(NActors::TEvInterconnect::TEvNodeDisconnected::TPtr& ev) { - LOG_T("TS3ReadActor", "Handle disconnected FileQueue " << ev->Get()->NodeId); - FileQueueEvents.HandleNodeDisconnected(ev->Get()->NodeId); - } - - void Handle(NActors::TEvInterconnect::TEvNodeConnected::TPtr& ev) { - LOG_T("TS3ReadActor", "Handle connected FileQueue " << ev->Get()->NodeId); - FileQueueEvents.HandleNodeConnected(ev->Get()->NodeId); - } - - void Handle(NActors::TEvents::TEvUndelivered::TPtr& ev) { - LOG_T("TS3ReadActor", "Handle undelivered FileQueue "); - if (!FileQueueEvents.HandleUndelivered(ev)) { - TIssues issues{TIssue{TStringBuilder() << "FileQueue was lost"}}; - Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::UNAVAILABLE)); - } - } - - // IActor & IDqComputeActorAsyncInput - void PassAway() override { // Is called from Compute Actor - LOG_D("TS3ReadActor", "PassAway"); - - if (Counters) { - QueueDataSize->Sub(QueueTotalDataSize); - QueueBlockCount->Sub(Blocks.size()); - QueueDataLimit->Sub(ReadActorFactoryCfg.DataInflight); - } - if (TaskCounters) { - TaskQueueDataSize->Sub(QueueTotalDataSize); - TaskQueueDataLimit->Sub(ReadActorFactoryCfg.DataInflight); - } - QueueTotalDataSize = 0; - - ContainerCache.Clear(); - FileQueueEvents.Unsubscribe(); - TActorBootstrapped::PassAway(); - } - -private: - const TS3ReadActorFactoryConfig ReadActorFactoryCfg; - const IHTTPGateway::TPtr Gateway; - const THolderFactory& HolderFactory; - TPlainContainerCache ContainerCache; - - const ui64 InputIndex; - TDqAsyncStats IngressStats; - const TTxId TxId; - const NActors::TActorId ComputeActorId; - const IHTTPGateway::TRetryPolicy::TPtr RetryPolicy; - - TActorSystem* const ActorSystem; - - const TString Url; - const TS3Credentials::TAuthInfo AuthInfo; - const TString Pattern; - const ES3PatternVariant PatternVariant; - TPathList Paths; - size_t ListedFiles = 0; - size_t CompletedFiles = 0; - NActors::TActorId FileQueueActor; - const bool AddPathIndex; - const ui64 SizeLimit; - TDuration CpuTime; - - std::queue> Blocks; - - ::NMonitoring::TDynamicCounters::TCounterPtr QueueDataSize; - ::NMonitoring::TDynamicCounters::TCounterPtr QueueDataLimit; - ::NMonitoring::TDynamicCounters::TCounterPtr QueueBlockCount; - ::NMonitoring::TDynamicCounters::TCounterPtr DownloadPaused; - ::NMonitoring::TDynamicCounters::TCounterPtr DeferredQueueSize; - ::NMonitoring::TDynamicCounters::TCounterPtr TaskDownloadPaused; - ::NMonitoring::TDynamicCounters::TCounterPtr TaskQueueDataSize; - ::NMonitoring::TDynamicCounters::TCounterPtr TaskQueueDataLimit; - ::NMonitoring::TDynamicCounterPtr Counters; - ::NMonitoring::TDynamicCounterPtr TaskCounters; - ui64 QueueTotalDataSize = 0; - ui64 DownloadInflight = 0; - const ui64 FileSizeLimit; - std::optional FilesRemained; +using namespace NKikimr::NMiniKQL; - bool UseRuntimeListing; - ui64 FileQueueBatchSizeLimit; - ui64 FileQueueBatchObjectCountLimit; - ui64 FileQueueConsumersCountDelta; - bool IsFileQueueEmpty = false; - bool IsCurrentBatchEmpty = false; - bool IsWaitingFileQueueResponse = false; - bool IsConfirmedFileQueueFinish = false; - TRetryEventsQueue FileQueueEvents; - TDeque> PathBatchQueue; -}; +ui64 SubtractSaturating(ui64 lhs, ui64 rhs) { + return (lhs > rhs) ? lhs - rhs : 0; +} struct TReadSpec { using TPtr = std::shared_ptr; @@ -1385,15 +230,15 @@ struct TRetryStuff { }; void OnDownloadStart(TActorSystem* actorSystem, const TActorId& self, const TActorId& parent, CURLcode curlResponseCode, long httpResponseCode) { - actorSystem->Send(new IEventHandle(self, parent, new TEvPrivate::TEvReadStarted(curlResponseCode, httpResponseCode))); + actorSystem->Send(new IEventHandle(self, parent, new TEvS3Provider::TEvDownloadStart(curlResponseCode, httpResponseCode))); } void OnNewData(TActorSystem* actorSystem, const TActorId& self, const TActorId& parent, IHTTPGateway::TCountedContent&& data) { - actorSystem->Send(new IEventHandle(self, parent, new TEvPrivate::TEvDataPart(std::move(data)))); + actorSystem->Send(new IEventHandle(self, parent, new TEvS3Provider::TEvDownloadData(std::move(data)))); } void OnDownloadFinished(TActorSystem* actorSystem, const TActorId& self, const TActorId& parent, size_t pathIndex, CURLcode curlResponseCode, TIssues issues) { - actorSystem->Send(new IEventHandle(self, parent, new TEvPrivate::TEvReadFinished(pathIndex, curlResponseCode, std::move(issues)))); + actorSystem->Send(new IEventHandle(self, parent, new TEvS3Provider::TEvDownloadFinish(pathIndex, curlResponseCode, std::move(issues)))); } void DownloadStart(const TRetryStuff::TPtr& retryStuff, TActorSystem* actorSystem, const TActorId& self, const TActorId& parent, size_t pathIndex, const ::NMonitoring::TDynamicCounters::TCounterPtr& inflightCounter) { @@ -1408,156 +253,6 @@ void DownloadStart(const TRetryStuff::TPtr& retryStuff, TActorSystem* actorSyste inflightCounter); } -std::shared_ptr ConvertArrowColumns(std::shared_ptr batch, std::vector& columnConverters) { - auto columns = batch->columns(); - for (size_t i = 0; i < columnConverters.size(); ++i) { - auto converter = columnConverters[i]; - if (converter) { - columns[i] = converter(columns[i]); - } - } - return arrow::RecordBatch::Make(batch->schema(), batch->num_rows(), columns); -} - -struct TReadBufferCounter { - using TPtr = std::shared_ptr; - - TReadBufferCounter(ui64 limit, - TActorSystem* actorSystem, - NMonitoring::TDynamicCounters::TCounterPtr queueDataSize, - NMonitoring::TDynamicCounters::TCounterPtr taskQueueDataSize, - NMonitoring::TDynamicCounters::TCounterPtr downloadPaused, - NMonitoring::TDynamicCounters::TCounterPtr taskDownloadPaused, - NMonitoring::TDynamicCounters::TCounterPtr taskChunkDownloadCount, - NMonitoring::THistogramPtr decodedChunkSizeHist) - : Limit(limit) - , ActorSystem(actorSystem) - , QueueDataSize(queueDataSize) - , TaskQueueDataSize(taskQueueDataSize) - , DownloadPaused(downloadPaused) - , TaskDownloadPaused(taskDownloadPaused) - , TaskChunkDownloadCount(taskChunkDownloadCount) - , DecodedChunkSizeHist(decodedChunkSizeHist) - { - } - - ~TReadBufferCounter() { - Notify(); - if (Value) { - if (QueueDataSize) { - QueueDataSize->Sub(Value); - } - if (TaskQueueDataSize) { - TaskQueueDataSize->Sub(Value); - } - Value = 0; - } - } - - bool IsFull() const { - return Value >= Limit; - } - - double Ratio() const { - return DownloadedBytes ? static_cast(DecodedBytes) / DownloadedBytes : 1.0; - } - - ui64 FairShare() { - return CoroCount ? Limit / CoroCount : Limit; - } - - void IncChunk() { - ChunkCount++; - if (TaskChunkDownloadCount) { - TaskChunkDownloadCount->Inc(); - } - } - - void DecChunk() { - ChunkCount--; - if (TaskChunkDownloadCount) { - TaskChunkDownloadCount->Dec(); - } - } - - bool Add(ui64 delta, NActors::TActorId producer, bool paused = false) { - if (DecodedChunkSizeHist) { - DecodedChunkSizeHist->Collect(delta); - } - Value += delta; - if (QueueDataSize) { - QueueDataSize->Add(delta); - } - if (TaskQueueDataSize) { - TaskQueueDataSize->Add(delta); - } - if ((Value + delta / 2) >= Limit) { - if (!paused) { - if (DownloadPaused) { - DownloadPaused->Inc(); - } - if (TaskDownloadPaused) { - TaskDownloadPaused->Inc(); - } - Producers.push_back(producer); - paused = true; - } - } - return paused; - } - - void Sub(ui64 delta) { - Y_ASSERT(Value >= delta); - Value -= delta; - if (QueueDataSize) { - QueueDataSize->Sub(delta); - } - if (TaskQueueDataSize) { - TaskQueueDataSize->Sub(delta); - } - if (Value * 4 < Limit * 3) { // l.eq.t 75% - Notify(); - } - } - - void Notify() { - if (!Producers.empty()) { - if (DownloadPaused) { - DownloadPaused->Sub(Producers.size()); - } - if (TaskDownloadPaused) { - TaskDownloadPaused->Sub(Producers.size()); - } - for (auto producer : Producers) { - ActorSystem->Send(new IEventHandle(producer, TActorId{}, new TEvPrivate::TEvContinue())); - } - Producers.clear(); - } - } - - void UpdateProgress(ui64 deltaDownloadedBytes, ui64 deltaDecodedBytes, ui64 deltaDecodedRows) { - DownloadedBytes += deltaDownloadedBytes; - DecodedBytes += deltaDecodedBytes; - DecodedRows += deltaDecodedRows; - } - - ui64 Value = 0; - const ui64 Limit; - ui64 CoroCount = 0; - ui64 ChunkCount = 0; - ui64 DownloadedBytes = 0; - ui64 DecodedBytes = 0; - ui64 DecodedRows = 0; - std::vector Producers; - TActorSystem* ActorSystem = nullptr; - NMonitoring::TDynamicCounters::TCounterPtr QueueDataSize; - NMonitoring::TDynamicCounters::TCounterPtr TaskQueueDataSize; - NMonitoring::TDynamicCounters::TCounterPtr DownloadPaused; - NMonitoring::TDynamicCounters::TCounterPtr TaskDownloadPaused; - NMonitoring::TDynamicCounters::TCounterPtr TaskChunkDownloadCount; - NMonitoring::THistogramPtr DecodedChunkSizeHist; -}; - struct TParquetFileInfo { ui64 RowCount = 0; ui64 CompressedSize = 0; @@ -1695,16 +390,16 @@ class TS3ReadCoroImpl : public TActorCoroImpl { ); while (NDB::Block batch = stream->read()) { - Paused = QueueBufferCounter->Add(batch.bytes(), SelfActorId); - const bool isStopped = StopIfConsumedEnough(batch.rows()); - Send(ParentActorId, new TEvPrivate::TEvNextBlock(batch, PathIndex, TakeIngressDelta(), TakeCpuTimeDelta())); + Paused = SourceContext->Add(batch.bytes(), SelfActorId); + const bool isCancelled = StopIfConsumedEnough(batch.rows()); + Send(ParentActorId, new TEvS3Provider::TEvNextBlock(batch, PathIndex, TakeIngressDelta(), TakeCpuTimeDelta())); if (Paused) { CpuTime += GetCpuTimeDelta(); - auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); + auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); HandleEvent(*ev); StartCycleCount = GetCycleCountFast(); } - if (isStopped) { + if (isCancelled) { LOG_CORO_D("RunClickHouseParserOverHttp - STOPPED ON SATURATION"); break; } @@ -1736,12 +431,12 @@ class TS3ReadCoroImpl : public TActorCoroImpl { ); while (NDB::Block batch = stream->read()) { - Paused = QueueBufferCounter->Add(batch.bytes(), SelfActorId); + Paused = SourceContext->Add(batch.bytes(), SelfActorId); const bool isCancelled = StopIfConsumedEnough(batch.rows()); - Send(ParentActorId, new TEvPrivate::TEvNextBlock(batch, PathIndex, TakeIngressDelta(), TakeCpuTimeDelta())); + Send(ParentActorId, new TEvS3Provider::TEvNextBlock(batch, PathIndex, TakeIngressDelta(), TakeCpuTimeDelta())); if (Paused) { CpuTime += GetCpuTimeDelta(); - auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); + auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); HandleEvent(*ev); StartCycleCount = GetCycleCountFast(); } @@ -1755,42 +450,6 @@ class TS3ReadCoroImpl : public TActorCoroImpl { LOG_CORO_D("RunClickHouseParserOverFile FINISHED"); } - void BuildColumnConverters(std::shared_ptr outputSchema, std::shared_ptr dataSchema, - std::vector& columnIndices, std::vector& columnConverters) { - - for (int i = 0; i < dataSchema->num_fields(); ++i) { - switch (dataSchema->field(i)->type()->id()) { - case arrow::Type::LIST: - throw parquet::ParquetException(TStringBuilder() << "File contains LIST field " - << dataSchema->field(i)->name() << " and can't be parsed"); - case arrow::Type::STRUCT: - throw parquet::ParquetException(TStringBuilder() << "File contains STRUCT field " - << dataSchema->field(i)->name() << " and can't be parsed"); - default: - ; - } - } - - columnConverters.reserve(outputSchema->num_fields()); - for (int i = 0; i < outputSchema->num_fields(); ++i) { - const auto& targetField = outputSchema->field(i); - auto srcFieldIndex = dataSchema->GetFieldIndex(targetField->name()); - if (srcFieldIndex == -1) { - throw parquet::ParquetException(TStringBuilder() << "Missing field: " << targetField->name()); - }; - auto targetType = targetField->type(); - auto originalType = dataSchema->field(srcFieldIndex)->type(); - if (originalType->layout().has_dictionary) { - throw parquet::ParquetException(TStringBuilder() << "Unsupported dictionary encoding is used for field: " - << targetField->name() << ", type: " << originalType->ToString()); - } - columnIndices.push_back(srcFieldIndex); - auto rowSpecColumnIt = ReadSpec->RowSpec.find(targetField->name()); - YQL_ENSURE(rowSpecColumnIt != ReadSpec->RowSpec.end(), "Column " << targetField->name() << " not found in row spec"); - columnConverters.emplace_back(BuildColumnConverter(targetField->name(), originalType, targetType, rowSpecColumnIt->second, ReadSpec->Settings)); - } - } - struct TReadCache { ui64 Cookie = 0; TString Data; @@ -1800,25 +459,25 @@ class TS3ReadCoroImpl : public TActorCoroImpl { struct TReadRangeCompare { - bool operator() (const TEvPrivate::TReadRange& lhs, const TEvPrivate::TReadRange& rhs) const + bool operator() (const TEvS3Provider::TReadRange& lhs, const TEvS3Provider::TReadRange& rhs) const { return (lhs.Offset < rhs.Offset) || (lhs.Offset == rhs.Offset && lhs.Length < rhs.Length); } }; ui64 RangeCookie = 0; - std::map RangeCache; + std::map RangeCache; std::map ReadInflightSize; std::optional CurrentRowGroupIndex; std::map RowGroupRangeInflight; std::priority_queue, std::greater> ReadyRowGroups; std::map RowGroupReaderIndex; - static void OnResult(TActorSystem* actorSystem, TActorId selfId, TEvPrivate::TReadRange range, ui64 cookie, IHTTPGateway::TResult&& result) { + static void OnResult(TActorSystem* actorSystem, TActorId selfId, TEvS3Provider::TReadRange range, ui64 cookie, IHTTPGateway::TResult&& result) { if (!result.Issues) { - actorSystem->Send(new IEventHandle(selfId, TActorId{}, new TEvPrivate::TEvReadResult2(range, std::move(result.Content)), 0, cookie)); + actorSystem->Send(new IEventHandle(selfId, TActorId{}, new TEvS3Provider::TEvReadResult2(range, std::move(result.Content)), 0, cookie)); } else { - actorSystem->Send(new IEventHandle(selfId, TActorId{}, new TEvPrivate::TEvReadResult2(range, std::move(result.Issues)), 0, cookie)); + actorSystem->Send(new IEventHandle(selfId, TActorId{}, new TEvS3Provider::TEvReadResult2(range, std::move(result.Issues)), 0, cookie)); } } @@ -1834,7 +493,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } - TReadCache& GetOrCreate(TEvPrivate::TReadRange range) { + TReadCache& GetOrCreate(TEvS3Provider::TReadRange range) { auto it = RangeCache.find(range); if (it != RangeCache.end()) { return it->second; @@ -1869,13 +528,13 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } } else { for (auto& range : readRanges) { - GetOrCreate(TEvPrivate::TReadRange{ .Offset = range.offset, .Length = range.length }); + GetOrCreate(TEvS3Provider::TReadRange{ .Offset = range.offset, .Length = range.length }); } } return {}; } - void HandleEvent(TEvPrivate::TEvReadResult2::THandle& event) { + void HandleEvent(TEvS3Provider::TEvReadResult2::THandle& event) { if (event.Get()->Failure) { throw yexception() << event.Get()->Issues.ToOneLineString(); @@ -1914,13 +573,13 @@ class TS3ReadCoroImpl : public TActorCoroImpl { arrow::Result> ReadAt(int64_t position, int64_t nbytes) { LOG_CORO_D("ReadAt STARTED [" << position << "-" << nbytes << "]"); - TEvPrivate::TReadRange range { .Offset = position, .Length = nbytes }; + TEvS3Provider::TReadRange range { .Offset = position, .Length = nbytes }; auto& cache = GetOrCreate(range); CpuTime += GetCpuTimeDelta(); while (!cache.Ready) { - auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); + auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); HandleEvent(*ev); } @@ -1963,7 +622,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { std::vector columnIndices; std::vector columnConverters; - BuildColumnConverters(ReadSpec->ArrowSchema, schema, columnIndices, columnConverters); + BuildColumnConverters(ReadSpec->ArrowSchema, schema, columnIndices, columnConverters, ReadSpec->RowSpec, ReadSpec->Settings); // select count(*) case - single reader is enough if (!columnIndices.empty()) { @@ -1980,7 +639,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } } // count = (fair_share / 2) / (compressed_size / num_group) - auto desiredReaderCount = (QueueBufferCounter->FairShare() * numGroups) / (compressedSize * 2); + auto desiredReaderCount = (SourceContext->FairShare() * numGroups) / (compressedSize * 2); // min is 1 // max is 5 (should be also tuned probably) if (desiredReaderCount) { @@ -2007,7 +666,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { if (!columnIndices.empty()) { CurrentRowGroupIndex = i; THROW_ARROW_NOT_OK(readers[i]->WillNeedRowGroups({ static_cast(i) }, columnIndices)); - QueueBufferCounter->IncChunk(); + SourceContext->IncChunkCount(); } RowGroupReaderIndex[i] = i; } @@ -2018,7 +677,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { while (readyGroupCount < numGroups) { if (Paused) { CpuTime += GetCpuTimeDelta(); - auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); + auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); HandleEvent(*ev); StartCycleCount = GetCycleCountFast(); } @@ -2041,7 +700,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { // select count(*) case - no columns, no download, just fetch meta info instantly readyGroupIndex = readyGroupCount; } - QueueBufferCounter->DecChunk(); + SourceContext->DecChunkCount(); auto readyReaderIndex = RowGroupReaderIndex[readyGroupIndex]; RowGroupReaderIndex.erase(readyGroupIndex); @@ -2060,23 +719,24 @@ class TS3ReadCoroImpl : public TActorCoroImpl { std::shared_ptr batch; arrow::Status status; bool isCancelled = false; + ui64 numRows = 0; while (status = reader->ReadNext(&batch), status.ok() && batch) { auto convertedBatch = ConvertArrowColumns(batch, columnConverters); auto size = NUdf::GetSizeOfArrowBatchInBytes(*convertedBatch); decodedBytes += size; - Paused = QueueBufferCounter->Add(size, SelfActorId, Paused); - Send(ParentActorId, new TEvPrivate::TEvNextRecordBatch( + Paused = SourceContext->Add(size, SelfActorId, Paused); + Send(ParentActorId, new TEvS3Provider::TEvNextRecordBatch( convertedBatch, PathIndex, TakeIngressDelta(), TakeCpuTimeDelta() )); - if (StopIfConsumedEnough(convertedBatch->num_rows())) { - isCancelled = true; - break; - } + numRows += convertedBatch->num_rows(); + } + if (StopIfConsumedEnough(numRows)) { + isCancelled = true; } if (!status.ok()) { throw yexception() << status.ToString(); } - QueueBufferCounter->UpdateProgress(downloadedBytes, decodedBytes, table->num_rows()); + SourceContext->UpdateProgress(downloadedBytes, decodedBytes, table->num_rows()); if (RawInflightSize) { RawInflightSize->Sub(downloadedBytes); } @@ -2084,7 +744,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { if (!columnIndices.empty()) { CurrentRowGroupIndex = nextGroup; THROW_ARROW_NOT_OK(readers[readyReaderIndex]->WillNeedRowGroups({ static_cast(nextGroup) }, columnIndices)); - QueueBufferCounter->IncChunk(); + SourceContext->IncChunkCount(); } RowGroupReaderIndex[nextGroup] = readyReaderIndex; nextGroup++; @@ -2093,8 +753,8 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } if (isCancelled) { LOG_CORO_D("RunCoroBlockArrowParserOverHttp - STOPPED ON SATURATION, downloaded " << - QueueBufferCounter->DownloadedBytes << " bytes"); - break; + SourceContext->GetDownloadedBytes() << " bytes"); + break; } } } @@ -2125,16 +785,16 @@ class TS3ReadCoroImpl : public TActorCoroImpl { std::vector columnIndices; std::vector columnConverters; - BuildColumnConverters(ReadSpec->ArrowSchema, schema, columnIndices, columnConverters); + BuildColumnConverters(ReadSpec->ArrowSchema, schema, columnIndices, columnConverters, ReadSpec->RowSpec, ReadSpec->Settings); for (int group = 0; group < fileReader->num_row_groups(); group++) { if (Paused) { CpuTime += GetCpuTimeDelta(); - LOG_CORO_D("RunCoroBlockArrowParserOverFile - PAUSED " << QueueBufferCounter->Value); - auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); + LOG_CORO_D("RunCoroBlockArrowParserOverFile - PAUSED " << SourceContext->GetValue()); + auto ev = WaitForSpecificEvent(&TS3ReadCoroImpl::ProcessUnexpectedEvent); HandleEvent(*ev); - LOG_CORO_D("RunCoroBlockArrowParserOverFile - CONTINUE " << QueueBufferCounter->Value); + LOG_CORO_D("RunCoroBlockArrowParserOverFile - CONTINUE " << SourceContext->GetValue()); StartCycleCount = GetCycleCountFast(); } @@ -2148,23 +808,24 @@ class TS3ReadCoroImpl : public TActorCoroImpl { std::shared_ptr batch; ::arrow::Status status; bool isCancelled = false; + ui64 numRows = 0; while (status = reader->ReadNext(&batch), status.ok() && batch) { auto convertedBatch = ConvertArrowColumns(batch, columnConverters); auto size = NUdf::GetSizeOfArrowBatchInBytes(*convertedBatch); decodedBytes += size; - Paused = QueueBufferCounter->Add(size, SelfActorId, Paused); - Send(ParentActorId, new TEvPrivate::TEvNextRecordBatch( + Paused = SourceContext->Add(size, SelfActorId, Paused); + Send(ParentActorId, new TEvS3Provider::TEvNextRecordBatch( convertedBatch, PathIndex, TakeIngressDelta(), TakeCpuTimeDelta() )); - if (StopIfConsumedEnough(batch->num_rows())) { - isCancelled = true; - break; - } + numRows += batch->num_rows(); + } + if (StopIfConsumedEnough(numRows)) { + isCancelled = true; } if (!status.ok()) { throw yexception() << status.ToString(); } - QueueBufferCounter->UpdateProgress(downloadedBytes, decodedBytes, table->num_rows()); + SourceContext->UpdateProgress(downloadedBytes, decodedBytes, table->num_rows()); if (isCancelled) { LOG_CORO_D("RunCoroBlockArrowParserOverFile - STOPPED ON SATURATION"); break; @@ -2175,11 +836,11 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } STRICT_STFUNC(StateFunc, - hFunc(TEvPrivate::TEvReadStarted, Handle); - hFunc(TEvPrivate::TEvDataPart, Handle); - hFunc(TEvPrivate::TEvReadFinished, Handle); - hFunc(TEvPrivate::TEvContinue, Handle); - hFunc(TEvPrivate::TEvReadResult2, Handle); + hFunc(TEvS3Provider::TEvDownloadStart, Handle); + hFunc(TEvS3Provider::TEvDownloadData, Handle); + hFunc(TEvS3Provider::TEvDownloadFinish, Handle); + hFunc(TEvS3Provider::TEvContinue, Handle); + hFunc(TEvS3Provider::TEvReadResult2, Handle); hFunc(NActors::TEvents::TEvPoison, Handle); ) @@ -2196,24 +857,24 @@ class TS3ReadCoroImpl : public TActorCoroImpl { StateFunc(ev); } - void ExtractDataPart(TEvPrivate::TEvDataPart& event, bool deferred = false) { + void ExtractDataPart(TEvS3Provider::TEvDownloadData& event, bool deferred = false) { InputBuffer = event.Result.Extract(); IngressBytes += InputBuffer.size(); RetryStuff->Offset += InputBuffer.size(); RetryStuff->SizeLimit -= InputBuffer.size(); LastOffset = RetryStuff->Offset; LastData = InputBuffer; - LOG_CORO_T("TEvDataPart (" << (deferred ? "deferred" : "instant") << "), size: " << InputBuffer.size()); + LOG_CORO_T("TEvDownloadData (" << (deferred ? "deferred" : "instant") << "), size: " << InputBuffer.size()); Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvNewAsyncInputDataArrived(InputIndex)); } - void Handle(TEvPrivate::TEvReadStarted::TPtr& ev) { + void Handle(TEvS3Provider::TEvDownloadStart::TPtr& ev) { HttpResponseCode = ev->Get()->HttpResponseCode; CurlResponseCode = ev->Get()->CurlResponseCode; - LOG_CORO_D("TEvReadStarted, Http code: " << HttpResponseCode); + LOG_CORO_D("TEvDownloadStart, Http code: " << HttpResponseCode); } - void Handle(TEvPrivate::TEvDataPart::TPtr& ev) { + void Handle(TEvS3Provider::TEvDownloadData::TPtr& ev) { if (HttpDataRps) { HttpDataRps->Inc(); } @@ -2232,11 +893,11 @@ class TS3ReadCoroImpl : public TActorCoroImpl { ErrorText.append(ev->Get()->Result.Extract()); else if (!ErrorText.EndsWith(TruncatedSuffix)) ErrorText.append(TruncatedSuffix); - LOG_CORO_W("TEvDataPart, ERROR: " << ErrorText << ", LastOffset: " << LastOffset << ", LastData: " << GetLastDataAsText()); + LOG_CORO_W("TEvDownloadData, ERROR: " << ErrorText << ", LastOffset: " << LastOffset << ", LastData: " << GetLastDataAsText()); } } - void Handle(TEvPrivate::TEvReadFinished::TPtr& ev) { + void Handle(TEvS3Provider::TEvDownloadFinish::TPtr& ev) { if (CurlResponseCode == CURLE_OK) { CurlResponseCode = ev->Get()->CurlResponseCode; @@ -2246,7 +907,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { if (!ErrorText.empty()) { TString errorCode; TString message; - if (!ParseS3ErrorResponse(ErrorText, errorCode, message)) { + if (ParseS3ErrorResponse(ErrorText, errorCode, message)) { message = ErrorText; } Issues.AddIssues(BuildIssues(HttpResponseCode, errorCode, message)); @@ -2263,7 +924,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { if (Issues) { RetryStuff->NextRetryDelay = RetryStuff->GetRetryState()->GetNextRetryDelay(CurlResponseCode, HttpResponseCode); - LOG_CORO_D("TEvReadFinished with Issues (try to retry): " << Issues.ToOneLineString()); + LOG_CORO_D("TEvDownloadFinish with Issues (try to retry): " << Issues.ToOneLineString()); if (RetryStuff->NextRetryDelay) { // inplace retry: report problem to TransientIssues and repeat Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvAsyncInputError(InputIndex, Issues, NYql::NDqProto::StatusIds::UNSPECIFIED)); @@ -2277,17 +938,17 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } if (!RetryStuff->IsCancelled() && RetryStuff->NextRetryDelay && RetryStuff->SizeLimit > 0ULL) { - GetActorSystem()->Schedule(*RetryStuff->NextRetryDelay, new IEventHandle(ParentActorId, SelfActorId, new TEvPrivate::TEvRetryEventFunc(std::bind(&DownloadStart, RetryStuff, GetActorSystem(), SelfActorId, ParentActorId, PathIndex, HttpInflightSize)))); + GetActorSystem()->Schedule(*RetryStuff->NextRetryDelay, new IEventHandle(ParentActorId, SelfActorId, new TEvS3Provider::TEvRetryEventFunc(std::bind(&DownloadStart, RetryStuff, GetActorSystem(), SelfActorId, ParentActorId, PathIndex, HttpInflightSize)))); InputBuffer.clear(); if (DeferredDataParts.size()) { if (DeferredQueueSize) { DeferredQueueSize->Sub(DeferredDataParts.size()); } - std::queue> tmp; + std::queue> tmp; DeferredDataParts.swap(tmp); } } else { - LOG_CORO_D("TEvReadFinished, LastOffset: " << LastOffset << ", Error: " << ServerReturnedError); + LOG_CORO_D("TEvDownloadFinish, LastOffset: " << LastOffset << ", Error: " << ServerReturnedError); InputFinished = true; if (ServerReturnedError) { throw TS3ReadError(); // Don't pass control to data parsing, because it may validate eof and show wrong issues about incorrect data format @@ -2295,16 +956,16 @@ class TS3ReadCoroImpl : public TActorCoroImpl { } } - void HandleEvent(TEvPrivate::TEvContinue::THandle&) { + void HandleEvent(TEvS3Provider::TEvContinue::THandle&) { LOG_CORO_D("TEvContinue"); Paused = false; } - void Handle(TEvPrivate::TEvContinue::TPtr& ev) { + void Handle(TEvS3Provider::TEvContinue::TPtr& ev) { HandleEvent(*ev); } - void Handle(TEvPrivate::TEvReadResult2::TPtr& ev) { + void Handle(TEvS3Provider::TEvReadResult2::TPtr& ev) { HandleEvent(*ev); } @@ -2321,7 +982,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { const TRetryStuff::TPtr& retryStuff, const TReadSpec::TPtr& readSpec, size_t pathIndex, const TString& path, const TString& url, std::optional maxRows, const TS3ReadActorFactoryConfig& readActorFactoryCfg, - TReadBufferCounter::TPtr queueBufferCounter, + TSourceContext::TPtr queueBufferCounter, const ::NMonitoring::TDynamicCounters::TCounterPtr& deferredQueueSize, const ::NMonitoring::TDynamicCounters::TCounterPtr& httpInflightSize, const ::NMonitoring::TDynamicCounters::TCounterPtr& httpDataRps, @@ -2329,7 +990,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { : TActorCoroImpl(256_KB), ReadActorFactoryCfg(readActorFactoryCfg), InputIndex(inputIndex), TxId(txId), RetryStuff(retryStuff), ReadSpec(readSpec), ComputeActorId(computeActorId), PathIndex(pathIndex), Path(path), Url(url), RowsRemained(maxRows), - QueueBufferCounter(queueBufferCounter), + SourceContext(queueBufferCounter), DeferredQueueSize(deferredQueueSize), HttpInflightSize(httpInflightSize), HttpDataRps(httpDataRps), RawInflightSize(rawInflightSize) { } @@ -2438,7 +1099,7 @@ class TS3ReadCoroImpl : public TActorCoroImpl { if (issues) Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvAsyncInputError(InputIndex, std::move(issues), fatalCode)); else - Send(ParentActorId, new TEvPrivate::TEvFileFinished(PathIndex, TakeIngressDelta(), TakeCpuTimeDelta())); + Send(ParentActorId, new TEvS3Provider::TEvFileFinished(PathIndex, TakeIngressDelta(), TakeCpuTimeDelta(), RetryStuff->SizeLimit)); } void ProcessUnexpectedEvent(TAutoPtr ev) { @@ -2504,8 +1165,8 @@ class TS3ReadCoroImpl : public TActorCoroImpl { TString InputBuffer; std::optional RowsRemained; bool Paused = false; - std::queue> DeferredDataParts; - TReadBufferCounter::TPtr QueueBufferCounter; + std::queue> DeferredDataParts; + TSourceContext::TPtr SourceContext; const ::NMonitoring::TDynamicCounters::TCounterPtr DeferredQueueSize; const ::NMonitoring::TDynamicCounters::TCounterPtr HttpInflightSize; const ::NMonitoring::TDynamicCounters::TCounterPtr HttpDataRps; @@ -2551,7 +1212,9 @@ class TS3StreamReadActor : public TActorBootstrapped, public TActorId fileQueueActor, ui64 fileQueueBatchSizeLimit, ui64 fileQueueBatchObjectCountLimit, - ui64 fileQueueConsumersCountDelta + ui64 fileQueueConsumersCountDelta, + bool asyncDecoding, + bool sourceCoroActor ) : ReadActorFactoryCfg(readActorFactoryCfg) , Gateway(std::move(gateway)) , HolderFactory(holderFactory) @@ -2576,7 +1239,9 @@ class TS3StreamReadActor : public TActorBootstrapped, public , UseRuntimeListing(useRuntimeListing) , FileQueueBatchSizeLimit(fileQueueBatchSizeLimit) , FileQueueBatchObjectCountLimit(fileQueueBatchObjectCountLimit) - , FileQueueConsumersCountDelta(fileQueueConsumersCountDelta) { + , FileQueueConsumersCountDelta(fileQueueConsumersCountDelta) + , AsyncDecoding(asyncDecoding) + , SourceCoroActor(sourceCoroActor) { if (Counters) { QueueDataSize = Counters->GetCounter("QueueDataSize"); QueueDataLimit = Counters->GetCounter("QueueDataLimit"); @@ -2614,18 +1279,38 @@ class TS3StreamReadActor : public TActorBootstrapped, public return; } - QueueBufferCounter = std::make_shared( + SourceContext = std::make_shared( + SelfId(), ReadActorFactoryCfg.DataInflight, TActivationContext::ActorSystem(), QueueDataSize, TaskQueueDataSize, + DownloadCount, DownloadPaused, + TaskDownloadCount, TaskDownloadPaused, TaskChunkDownloadCount, - DecodedChunkSizeHist); + DecodedChunkSizeHist, + HttpInflightSize, + HttpDataRps, + DeferredQueueSize, + ReadSpec->Format, + ReadSpec->Compression, + ReadSpec->ArrowSchema, + ReadSpec->RowSpec, + ReadSpec->Settings + ); + + UseParquetCache = (ReadSpec->Format == "parquet") && !SourceCoroActor; + + if (UseParquetCache) { + Send(ParquetCacheActorId(), new TEvS3Provider::TEvCacheSourceStart( + SourceContext->SourceId, TxId, SourceContext->Schema + )); + } if (!UseRuntimeListing) { - FileQueueActor = RegisterWithSameMailbox(new TS3FileQueueActor{ + FileQueueActor = RegisterWithSameMailbox(CreateS3FileQueueActor( TxId, std::move(Paths), ReadActorFactoryCfg.MaxInflight * 2, @@ -2640,12 +1325,12 @@ class TS3StreamReadActor : public TActorBootstrapped, public AuthInfo, Pattern, PatternVariant, - ES3PatternType::Wildcard}); + ES3PatternType::Wildcard)); } FileQueueEvents.Init(TxId, SelfId(), SelfId()); FileQueueEvents.OnNewRecipientId(FileQueueActor); if (UseRuntimeListing && FileQueueConsumersCountDelta > 0) { - FileQueueEvents.Send(new TEvS3FileQueue::TEvUpdateConsumersCount(FileQueueConsumersCountDelta)); + FileQueueEvents.Send(new TEvS3Provider::TEvUpdateConsumersCount(FileQueueConsumersCountDelta)); } SendPathBatchRequest(); @@ -2663,48 +1348,34 @@ class TS3StreamReadActor : public TActorBootstrapped, public // waiting for batch to finish return false; } - if (QueueBufferCounter->IsFull()) { + if (SourceContext->IsFull()) { // too large data inflight return false; } - if (QueueBufferCounter->CoroCount >= ReadActorFactoryCfg.MaxInflight) { + + auto splitCount = SourceContext->GetSplitCount(); + + if (splitCount >= ReadActorFactoryCfg.MaxInflight) { // hard limit return false; } - if (ReadSpec->ParallelDownloadCount) { - if (QueueBufferCounter->CoroCount >= ReadSpec->ParallelDownloadCount) { - // explicit limit - return false; - } - } else { - if (QueueBufferCounter->CoroCount && DownloadSize * QueueBufferCounter->Ratio() > ReadActorFactoryCfg.DataInflight * 2) { - // dynamic limit - return false; - } + if (ReadSpec->ParallelDownloadCount && splitCount >= ReadSpec->ParallelDownloadCount) { + // explicit limit + return false; + } + if (splitCount && DownloadSize * SourceContext->Ratio() > ReadActorFactoryCfg.DataInflight * 2) { + // dynamic limit + return false; } RegisterCoro(); return true; } void RegisterCoro() { - QueueBufferCounter->CoroCount++; - if (Counters) { - DownloadCount->Inc(); - } - if (TaskCounters) { - TaskDownloadCount->Inc(); - } + SourceContext->IncSplitCount(); const auto& object = ReadPathFromCache(); DownloadSize += object.GetSize(); const TString requestId = CreateGuidAsString(); - auto stuff = std::make_shared( - Gateway, - Url + object.GetPath(), - IHTTPGateway::MakeYcHeaders(requestId, AuthInfo.GetToken(), {}, AuthInfo.GetAwsUserPwd(), AuthInfo.GetAwsSigV4()), - object.GetSize(), - TxId, - requestId, - RetryPolicy); auto pathIndex = object.GetPathIndex(); if (TaskCounters) { HttpInflightLimit->Add(Gateway->GetBuffersSizePerStream()); @@ -2713,28 +1384,79 @@ class TS3StreamReadActor : public TActorBootstrapped, public "TS3StreamReadActor", "RegisterCoro with path " << object.GetPath() << " with pathIndex " << pathIndex); - auto impl = MakeHolder( - InputIndex, - TxId, - ComputeActorId, - std::move(stuff), - ReadSpec, - pathIndex, - object.GetPath(), - Url, - RowsRemained, - ReadActorFactoryCfg, - QueueBufferCounter, - DeferredQueueSize, - HttpInflightSize, - HttpDataRps, - RawInflightSize); - auto coroActorId = RegisterWithSameMailbox(new TS3ReadCoroActor(std::move(impl))); - CoroActors.insert(coroActorId); - RetryStuffForFile.emplace(coroActorId, stuff); + + TActorId actorId; + if (ReadSpec->Format == "json_each_row" && ReadSpec->Arrow) { + auto splitContext = std::make_shared( + SourceContext, + Gateway, + Url + object.GetPath(), + 0, object.GetSize(), object.GetSize(), // complete + object.GetPathIndex(), + IHTTPGateway::MakeYcHeaders(requestId, AuthInfo.GetToken(), {}, AuthInfo.GetAwsUserPwd(), AuthInfo.GetAwsSigV4()), + RetryPolicy, + TxId, + requestId + ); + if (AsyncDecoding) { + actorId = Register(CreateS3ReadJsonEachRowActor(splitContext)); + } else { + actorId = RegisterWithSameMailbox(CreateS3ReadJsonEachRowActor(splitContext)); + } + } else if (ReadSpec->Format == "parquet" && !SourceCoroActor) { + Y_ASSERT(ReadSpec->Arrow); + auto splitContext = std::make_shared( + SourceContext, + Gateway, + Url + object.GetPath(), + 0, object.GetSize(), object.GetSize(), + object.GetPathIndex(), + IHTTPGateway::MakeYcHeaders(requestId, AuthInfo.GetToken(), {}, AuthInfo.GetAwsUserPwd(), AuthInfo.GetAwsSigV4()), + RetryPolicy, + TxId, + requestId + ); + if (AsyncDecoding) { + actorId = Register(CreateS3ReadParquetActor(splitContext, ReadSpec->ParallelRowGroupCount, ReadSpec->RowGroupReordering, UseParquetCache)); + } else { + actorId = RegisterWithSameMailbox(CreateS3ReadParquetActor(splitContext, ReadSpec->ParallelRowGroupCount, ReadSpec->RowGroupReordering, UseParquetCache)); + } + } else { + auto stuff = std::make_shared( + Gateway, + Url + object.GetPath(), + IHTTPGateway::MakeYcHeaders(requestId, AuthInfo.GetToken(), {}, AuthInfo.GetAwsUserPwd(), AuthInfo.GetAwsSigV4()), + object.GetSize(), + TxId, + requestId, + RetryPolicy); + auto impl = MakeHolder( + InputIndex, + TxId, + ComputeActorId, + std::move(stuff), + ReadSpec, + pathIndex, + object.GetPath(), + Url, + RowsRemained, + ReadActorFactoryCfg, + SourceContext, + DeferredQueueSize, + HttpInflightSize, + HttpDataRps, + RawInflightSize + ); + if (AsyncDecoding) { + actorId = Register(new TS3ReadCoroActor(std::move(impl))); + } else { + actorId = RegisterWithSameMailbox(new TS3ReadCoroActor(std::move(impl))); + } + } + CoroActors.insert(actorId); } - TObjectPath ReadPathFromCache() { + NS3::FileQueue::TObjectPath ReadPathFromCache() { Y_ENSURE(!PathBatchQueue.empty()); auto& currentBatch = PathBatchQueue.front(); Y_ENSURE(!currentBatch.empty()); @@ -2753,7 +1475,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public } } void SendPathBatchRequest() { - FileQueueEvents.Send(new TEvS3FileQueue::TEvGetNextBatch()); + FileQueueEvents.Send(new TEvS3Provider::TEvGetNextBatch()); IsWaitingFileQueueResponse = true; } @@ -2781,8 +1503,8 @@ class TS3StreamReadActor : public TActorBootstrapped, public class TReadyBlock { public: - TReadyBlock(TEvPrivate::TEvNextBlock::TPtr& event) : PathInd(event->Get()->PathIndex) { Block.swap(event->Get()->Block); } - TReadyBlock(TEvPrivate::TEvNextRecordBatch::TPtr& event) : Batch(event->Get()->Batch), PathInd(event->Get()->PathIndex) {} + TReadyBlock(TEvS3Provider::TEvNextBlock::TPtr& event) : PathInd(event->Get()->PathIndex) { Block.swap(event->Get()->Block); } + TReadyBlock(TEvS3Provider::TEvNextRecordBatch::TPtr& event) : Batch(event->Get()->Batch), PathInd(event->Get()->PathIndex) {} NDB::Block Block; std::shared_ptr Batch; size_t PathInd; @@ -2816,7 +1538,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public NUdf::TUnboxedValue value; if (ReadSpec->Arrow) { const auto& batch = *Blocks.front().Batch; - +// Cerr << "ASYNC batch with COLS=" << batch.num_columns() << " and ROWS=" << batch.num_rows() << Endl; NUdf::TUnboxedValue* structItems = nullptr; auto structObj = ArrowRowContainerCache.NewArray(HolderFactory, 1 + batch.num_columns(), structItems); for (int i = 0; i < batch.num_columns(); ++i) { @@ -2841,7 +1563,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public total += s; output.emplace_back(std::move(value)); Blocks.pop_front(); - QueueBufferCounter->Sub(s); + SourceContext->Sub(s); if (Counters) { QueueBlockCount->Dec(); } @@ -2853,6 +1575,12 @@ class TS3StreamReadActor : public TActorBootstrapped, public ContainerCache.Clear(); ArrowTupleContainerCache.Clear(); ArrowRowContainerCache.Clear(); + if (UseParquetCache) { + Send(ParquetCacheActorId(), new TEvS3Provider::TEvCacheSourceFinish( + SourceContext->SourceId + )); + } + } else if(!total) { IngressStats.TryPause(); } @@ -2867,15 +1595,12 @@ class TS3StreamReadActor : public TActorBootstrapped, public if (Counters) { QueueBlockCount->Sub(Blocks.size()); QueueDataLimit->Sub(ReadActorFactoryCfg.DataInflight); - DownloadCount->Sub(QueueBufferCounter->CoroCount); } if (TaskCounters) { TaskQueueDataLimit->Sub(ReadActorFactoryCfg.DataInflight); HttpInflightLimit->Sub(Gateway->GetBuffersSizePerStream() * CoroActors.size()); - TaskDownloadCount->Sub(QueueBufferCounter->CoroCount); - TaskChunkDownloadCount->Sub(QueueBufferCounter->ChunkCount); } - QueueBufferCounter.reset(); + SourceContext.reset(); for (const auto actorId : CoroActors) { Send(actorId, new NActors::TEvents::TEvPoison()); @@ -2897,24 +1622,25 @@ class TS3StreamReadActor : public TActorBootstrapped, public } STRICT_STFUNC_EXC(StateFunc, - hFunc(TEvPrivate::TEvRetryEventFunc, HandleRetry); - hFunc(TEvPrivate::TEvNextBlock, HandleNextBlock); - hFunc(TEvPrivate::TEvNextRecordBatch, HandleNextRecordBatch); - hFunc(TEvPrivate::TEvFileFinished, HandleFileFinished); - hFunc(TEvS3FileQueue::TEvAck, Handle); - hFunc(TEvS3FileQueue::TEvObjectPathBatch, HandleObjectPathBatch); - hFunc(TEvS3FileQueue::TEvObjectPathReadError, HandleObjectPathReadError); + hFunc(TEvS3Provider::TEvRetryEventFunc, HandleRetry); + hFunc(TEvS3Provider::TEvNextBlock, HandleNextBlock); + hFunc(TEvS3Provider::TEvNextRecordBatch, HandleNextRecordBatch); + hFunc(TEvS3Provider::TEvFileFinished, HandleFileFinished); + hFunc(TEvS3Provider::TEvAck, Handle); + hFunc(TEvS3Provider::TEvObjectPathBatch, HandleObjectPathBatch); + hFunc(TEvS3Provider::TEvObjectPathReadError, HandleObjectPathReadError); hFunc(NYql::NDq::TEvRetryQueuePrivate::TEvRetry, Handle); hFunc(NActors::TEvInterconnect::TEvNodeDisconnected, Handle); hFunc(NActors::TEvInterconnect::TEvNodeConnected, Handle); hFunc(NActors::TEvents::TEvUndelivered, Handle); + hFunc(IDqComputeActorAsyncInput::TEvAsyncInputError, Handle); , catch (const std::exception& e) { TIssues issues{TIssue{TStringBuilder() << "An unknown exception has occurred: '" << e.what() << "'"}}; Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::INTERNAL_ERROR)); } ) - void HandleObjectPathBatch(TEvS3FileQueue::TEvObjectPathBatch::TPtr& objectPathBatch) { + void HandleObjectPathBatch(TEvS3Provider::TEvObjectPathBatch::TPtr& objectPathBatch) { if (!FileQueueEvents.OnEventReceived(objectPathBatch)) { return; } @@ -2944,7 +1670,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public } } - void HandleObjectPathReadError(TEvS3FileQueue::TEvObjectPathReadError::TPtr& result) { + void HandleObjectPathReadError(TEvS3Provider::TEvObjectPathReadError::TPtr& result) { if (!FileQueueEvents.OnEventReceived(result)) { return; } @@ -2962,11 +1688,11 @@ class TS3StreamReadActor : public TActorBootstrapped, public Send(ComputeActorId, new TEvAsyncInputError(InputIndex, std::move(issues), NYql::NDqProto::StatusIds::EXTERNAL_ERROR)); } - void HandleRetry(TEvPrivate::TEvRetryEventFunc::TPtr& retry) { + void HandleRetry(TEvS3Provider::TEvRetryEventFunc::TPtr& retry) { return retry->Get()->Functor(); } - void HandleNextBlock(TEvPrivate::TEvNextBlock::TPtr& next) { + void HandleNextBlock(TEvS3Provider::TEvNextBlock::TPtr& next) { YQL_ENSURE(!ReadSpec->Arrow); auto rows = next->Get()->Block.rows(); IngressStats.Bytes += next->Get()->IngressDelta; @@ -2982,7 +1708,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvNewAsyncInputDataArrived(InputIndex)); } - void HandleNextRecordBatch(TEvPrivate::TEvNextRecordBatch::TPtr& next) { + void HandleNextRecordBatch(TEvS3Provider::TEvNextRecordBatch::TPtr& next) { YQL_ENSURE(ReadSpec->Arrow); auto rows = next->Get()->Batch->num_rows(); IngressStats.Bytes += next->Get()->IngressDelta; @@ -2998,7 +1724,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public Send(ComputeActorId, new IDqComputeActorAsyncInput::TEvNewAsyncInputDataArrived(InputIndex)); } - void HandleFileFinished(TEvPrivate::TEvFileFinished::TPtr& ev) { + void HandleFileFinished(TEvS3Provider::TEvFileFinished::TPtr& ev) { CoroActors.erase(ev->Sender); if (IsCurrentBatchEmpty && CoroActors.size() == 0) { IsCurrentBatchEmpty = false; @@ -3009,13 +1735,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public IngressStats.Resume(); } CpuTime += ev->Get()->CpuTimeDelta; - - auto it = RetryStuffForFile.find(ev->Sender); - if (it == RetryStuffForFile.end()) { - return; - } - auto size = it->second->SizeLimit; - RetryStuffForFile.erase(it); + auto size = ev->Get()->SplitSize; if (DownloadSize < size) { DownloadSize = 0; } else { @@ -3025,13 +1745,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public if (TaskCounters) { HttpInflightLimit->Sub(Gateway->GetBuffersSizePerStream()); } - QueueBufferCounter->CoroCount--; - if (Counters) { - DownloadCount->Dec(); - } - if (TaskCounters) { - TaskDownloadCount->Dec(); - } + SourceContext->DecSplitCount(); CompletedFiles++; IngressStats.Splits++; if (!PathBatchQueue.empty()) { @@ -3047,8 +1761,8 @@ class TS3StreamReadActor : public TActorBootstrapped, public } } } - - void Handle(TEvS3FileQueue::TEvAck::TPtr& ev) { + + void Handle(TEvS3Provider::TEvAck::TPtr& ev) { FileQueueEvents.OnEventReceived(ev); } @@ -3073,6 +1787,11 @@ class TS3StreamReadActor : public TActorBootstrapped, public Send(ComputeActorId, new TEvAsyncInputError(InputIndex, issues, NYql::NDqProto::StatusIds::UNAVAILABLE)); } } + + void Handle(IDqComputeActorAsyncInput::TEvAsyncInputError::TPtr& ev) { + Send(ev->Forward(ComputeActorId)); + } + bool LastFileWasProcessed() const { return Blocks.empty() && (ListedFiles == CompletedFiles) && IsFileQueueEmpty; } @@ -3097,7 +1816,6 @@ class TS3StreamReadActor : public TActorBootstrapped, public const TS3ReadActorFactoryConfig ReadActorFactoryCfg; const IHTTPGateway::TPtr Gateway; - THashMap RetryStuffForFile; const THolderFactory& HolderFactory; TPlainContainerCache ContainerCache; TPlainContainerCache ArrowTupleContainerCache; @@ -3122,7 +1840,7 @@ class TS3StreamReadActor : public TActorBootstrapped, public std::deque Blocks; TDuration CpuTime; mutable TInstant LastMemoryReport = TInstant::Now(); - TReadBufferCounter::TPtr QueueBufferCounter; + TSourceContext::TPtr SourceContext; ::NMonitoring::TDynamicCounters::TCounterPtr QueueDataSize; ::NMonitoring::TDynamicCounters::TCounterPtr QueueDataLimit; ::NMonitoring::TDynamicCounters::TCounterPtr QueueBlockCount; @@ -3152,12 +1870,15 @@ class TS3StreamReadActor : public TActorBootstrapped, public ui64 FileQueueBatchSizeLimit; ui64 FileQueueBatchObjectCountLimit; ui64 FileQueueConsumersCountDelta; + const bool AsyncDecoding; + const bool SourceCoroActor; bool IsCurrentBatchEmpty = false; bool IsFileQueueEmpty = false; bool IsWaitingFileQueueResponse = false; bool IsConfirmedFileQueueFinish = false; TRetryEventsQueue FileQueueEvents; - TDeque> PathBatchQueue; + TDeque> PathBatchQueue; + bool UseParquetCache = false; }; using namespace NKikimr::NMiniKQL; @@ -3298,43 +2019,6 @@ NDB::FormatSettings::TimestampFormat ToTimestampFormat(const TString& formatName } // namespace -using namespace NKikimr::NMiniKQL; - -IActor* CreateS3FileQueueActor( - TTxId txId, - TPathList paths, - size_t prefetchSize, - ui64 fileSizeLimit, - ui64 readLimit, - bool useRuntimeListing, - ui64 consumersCount, - ui64 batchSizeLimit, - ui64 batchObjectCountLimit, - IHTTPGateway::TPtr gateway, - TString url, - TS3Credentials::TAuthInfo authInfo, - TString pattern, - ES3PatternVariant patternVariant, - ES3PatternType patternType) { - return new TS3FileQueueActor( - txId, - paths, - prefetchSize, - fileSizeLimit, - readLimit, - useRuntimeListing, - consumersCount, - batchSizeLimit, - batchObjectCountLimit, - gateway, - url, - authInfo, - pattern, - patternVariant, - patternType - ); -} - std::pair CreateS3ReadActor( const TTypeEnvironment& typeEnv, const THolderFactory& holderFactory, @@ -3436,8 +2120,14 @@ std::pair CreateS3ReadActor( YQL_ENSURE(outputItemType->IsStruct(), "Row type is not struct"); const auto structType = static_cast(outputItemType); + if (params.GetFormat() == "parquet") { + YQL_ENSURE(params.GetArrow(), "Only arrow blocks format is supported for \"parquet\""); + } else if (params.GetFormat() != "json_each_row") { + YQL_ENSURE(!params.GetArrow(), "Arrow blocks format is supported for \"" << params.GetFormat() << '\"'); + } + const auto readSpec = std::make_shared(); - readSpec->Arrow = params.GetFormat() == "parquet"; + readSpec->Arrow = params.GetArrow(); readSpec->ParallelRowGroupCount = params.GetParallelRowGroupCount(); readSpec->RowGroupReordering = params.GetRowGroupReordering(); readSpec->ParallelDownloadCount = params.GetParallelDownloadCount(); @@ -3540,7 +2230,8 @@ std::pair CreateS3ReadActor( const auto actor = new TS3StreamReadActor(inputIndex, statsLevel, txId, std::move(gateway), holderFactory, params.GetUrl(), authInfo, pathPattern, pathPatternVariant, std::move(paths), addPathIndex, readSpec, computeActorId, retryPolicy, cfg, counters, taskCounters, fileSizeLimit, sizeLimit, rowsLimitHint, memoryQuotaManager, - params.GetUseRuntimeListing(), fileQueueActor, fileQueueBatchSizeLimit, fileQueueBatchObjectCountLimit, fileQueueConsumersCountDelta); + params.GetUseRuntimeListing(), fileQueueActor, fileQueueBatchSizeLimit, fileQueueBatchObjectCountLimit, fileQueueConsumersCountDelta, + params.GetAsyncDecoding(), params.GetSourceCoroActor()); return {actor, actor}; } else { @@ -3548,11 +2239,10 @@ std::pair CreateS3ReadActor( if (const auto it = settings.find("sizeLimit"); settings.cend() != it) sizeLimit = FromString(it->second); - const auto actor = new TS3ReadActor(inputIndex, statsLevel, txId, std::move(gateway), holderFactory, params.GetUrl(), authInfo, pathPattern, pathPatternVariant, + return CreateRawReadActor(inputIndex, statsLevel, txId, std::move(gateway), holderFactory, params.GetUrl(), authInfo, pathPattern, pathPatternVariant, std::move(paths), addPathIndex, computeActorId, sizeLimit, retryPolicy, cfg, counters, taskCounters, fileSizeLimit, rowsLimitHint, params.GetUseRuntimeListing(), fileQueueActor, fileQueueBatchSizeLimit, fileQueueBatchObjectCountLimit, fileQueueConsumersCountDelta); - return {actor, actor}; } } diff --git a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h index 17ddc2ba26fc..93d12aa709b6 100644 --- a/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h +++ b/ydb/library/yql/providers/s3/actors/yql_s3_read_actor.h @@ -14,7 +14,7 @@ namespace NYql::NDq { struct TS3ReadActorFactoryConfig; NActors::IActor* CreateS3FileQueueActor( - TTxId txId, + TTxId txId, NS3Details::TPathList paths, size_t prefetchSize, ui64 fileSizeLimit, diff --git a/ydb/library/yql/providers/s3/common/source_context.cpp b/ydb/library/yql/providers/s3/common/source_context.cpp new file mode 100644 index 000000000000..e4c5bfefa909 --- /dev/null +++ b/ydb/library/yql/providers/s3/common/source_context.cpp @@ -0,0 +1,133 @@ +#include "source_context.h" + +namespace NYql::NDq { + +TSourceContext::~TSourceContext() { + Notify(); + if (Value) { + if (QueueDataSize) { + QueueDataSize->Sub(Value); + } + if (TaskQueueDataSize) { + TaskQueueDataSize->Sub(Value); + } + Value = 0; + } + if (DownloadCount) { + DownloadCount->Sub(GetSplitCount()); + } + if (TaskDownloadCount) { + TaskDownloadCount->Sub(GetSplitCount()); + } + if (TaskChunkDownloadCount) { + TaskChunkDownloadCount->Sub(GetChunkCount()); + } +} + +void TSourceContext::IncChunkCount() { + ChunkCount++; + if (TaskChunkDownloadCount) { + TaskChunkDownloadCount->Inc(); + } +} + +void TSourceContext::DecChunkCount() { + ChunkCount--; + if (TaskChunkDownloadCount) { + TaskChunkDownloadCount->Dec(); + } +} + +bool TSourceContext::Add(ui64 delta, NActors::TActorId producer, bool paused) { + if (DecodedChunkSizeHist) { + DecodedChunkSizeHist->Collect(delta); + } + Value += delta; + if (QueueDataSize) { + QueueDataSize->Add(delta); + } + if (TaskQueueDataSize) { + TaskQueueDataSize->Add(delta); + } + if ((Value.load() + delta / 2) >= Limit) { + if (!paused) { + { + std::lock_guard guard(Mutex); + Producers.push_back(producer); + } + if (DownloadPaused) { + DownloadPaused->Inc(); + } + if (TaskDownloadPaused) { + TaskDownloadPaused->Inc(); + } + paused = true; + } + } + return paused; +} + +void TSourceContext::Sub(ui64 delta) { + auto prev = Value.fetch_sub(delta); + Y_ASSERT(prev >= delta); + if (QueueDataSize) { + QueueDataSize->Sub(delta); + } + if (TaskQueueDataSize) { + TaskQueueDataSize->Sub(delta); + } + if (Value.load() * 4 < Limit * 3) { // l.eq.t 75% + Notify(); + } +} + +void TSourceContext::Notify() { + std::size_t size = 0; + { + std::lock_guard guard(Mutex); + if (!Producers.empty()) { + size = Producers.size(); + for (auto producer : Producers) { + ActorSystem->Send(new NActors::IEventHandle(producer, NActors::TActorId{}, new TEvS3Provider::TEvContinue())); + } + Producers.clear(); + } + } + + if (size) { + if (DownloadPaused) { + DownloadPaused->Sub(size); + } + if (TaskDownloadPaused) { + TaskDownloadPaused->Sub(size); + } + } +} + +void TSourceContext::IncSplitCount() { + SplitCount++; + if (DownloadCount) { + DownloadCount->Inc(); + } + if (TaskDownloadCount) { + TaskDownloadCount->Inc(); + } +} + +void TSourceContext::DecSplitCount() { + SplitCount--; + if (DownloadCount) { + DownloadCount->Dec(); + } + if (TaskDownloadCount) { + TaskDownloadCount->Dec(); + } +} + +void TSourceContext::UpdateProgress(ui64 deltaDownloadedBytes, ui64 deltaDecodedBytes, ui64 deltaDecodedRows) { + DownloadedBytes += deltaDownloadedBytes; + DecodedBytes += deltaDecodedBytes; + DecodedRows += deltaDecodedRows; +} + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/common/source_context.h b/ydb/library/yql/providers/s3/common/source_context.h new file mode 100644 index 000000000000..f418540d2dbd --- /dev/null +++ b/ydb/library/yql/providers/s3/common/source_context.h @@ -0,0 +1,188 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#include + +#include + +#include +#include + +namespace NYql::NDq { + +// common context for the Source (reads multiple splits) + +struct TSourceContext { + using TPtr = std::shared_ptr; + + TSourceContext(NActors::TActorId sourceId + , ui64 limit + , NActors::TActorSystem* actorSystem + , NMonitoring::TDynamicCounters::TCounterPtr queueDataSize + , NMonitoring::TDynamicCounters::TCounterPtr taskQueueDataSize + , NMonitoring::TDynamicCounters::TCounterPtr downloadCount + , NMonitoring::TDynamicCounters::TCounterPtr downloadPaused + , NMonitoring::TDynamicCounters::TCounterPtr taskDownloadCount + , NMonitoring::TDynamicCounters::TCounterPtr taskDownloadPaused + , NMonitoring::TDynamicCounters::TCounterPtr taskChunkDownloadCount + , NMonitoring::THistogramPtr decodedChunkSizeHist + , NMonitoring::TDynamicCounters::TCounterPtr httpInflightSize + , NMonitoring::TDynamicCounters::TCounterPtr httpDataRps + , NMonitoring::TDynamicCounters::TCounterPtr deferredQueueSize + , const TString format + , const TString compression + , std::shared_ptr schema + , std::unordered_map> rowTypes + , NDB::FormatSettings settings) + : SourceId(sourceId) + , Limit(limit) + , ActorSystem(actorSystem) + , QueueDataSize(queueDataSize) + , TaskQueueDataSize(taskQueueDataSize) + , DownloadCount(downloadCount) + , DownloadPaused(downloadPaused) + , TaskDownloadCount(taskDownloadCount) + , TaskDownloadPaused(taskDownloadPaused) + , TaskChunkDownloadCount(taskChunkDownloadCount) + , DecodedChunkSizeHist(decodedChunkSizeHist) + , HttpInflightSize(httpInflightSize) + , HttpDataRps(httpDataRps) + , DeferredQueueSize(deferredQueueSize) + , Format(format) + , Compression(compression) + , Schema(schema) + , RowTypes(rowTypes) + , Settings(settings) + { + } + + ~TSourceContext(); + + bool IsFull() const { + return Value.load() >= Limit; + } + + double Ratio() const { + auto downloadedBytes = DownloadedBytes.load(); + return downloadedBytes ? static_cast(downloadedBytes) / DownloadedBytes.load() : 1.0; + } + + ui64 FairShare() { + auto splitCount = GetSplitCount(); + return splitCount ? Limit / splitCount : Limit; + } + + ui64 GetValue() { return Value.load(); } + ui32 GetChunkCount() { return ChunkCount.load(); } + void IncChunkCount(); + void DecChunkCount(); + bool Add(ui64 delta, NActors::TActorId producer, bool paused = false); + void Sub(ui64 delta); + void Notify(); + void UpdateProgress(ui64 deltaDownloadedBytes, ui64 deltaDecodedBytes, ui64 deltaDecodedRows); + ui32 GetSplitCount() { return SplitCount.load(); } + void IncSplitCount(); + void DecSplitCount(); + ui64 GetDownloadedBytes() { return DownloadedBytes.load(); } + + const NActors::TActorId SourceId; + const ui64 Limit; + NActors::TActorSystem* ActorSystem; + NMonitoring::TDynamicCounters::TCounterPtr QueueDataSize; + NMonitoring::TDynamicCounters::TCounterPtr TaskQueueDataSize; + NMonitoring::TDynamicCounters::TCounterPtr DownloadCount; + NMonitoring::TDynamicCounters::TCounterPtr DownloadPaused; + NMonitoring::TDynamicCounters::TCounterPtr TaskDownloadCount; + NMonitoring::TDynamicCounters::TCounterPtr TaskDownloadPaused; + NMonitoring::TDynamicCounters::TCounterPtr TaskChunkDownloadCount; + NMonitoring::THistogramPtr DecodedChunkSizeHist; + NMonitoring::TDynamicCounters::TCounterPtr HttpInflightSize; + NMonitoring::TDynamicCounters::TCounterPtr HttpDataRps; + NMonitoring::TDynamicCounters::TCounterPtr DeferredQueueSize; + const TString Format; + const TString Compression; + std::shared_ptr Schema; + std::unordered_map> RowTypes; + NDB::FormatSettings Settings; +private: + std::atomic_uint64_t Value; + std::mutex Mutex; + std::vector Producers; + std::atomic_uint32_t ChunkCount; + std::atomic_uint32_t SplitCount; + std::atomic_uint64_t DownloadedBytes; + std::atomic_uint64_t DecodedBytes; + std::atomic_uint64_t DecodedRows; +}; + +// per split context to pass params to load/decoding implementation + +struct TSplitReadContext { + using TPtr = std::shared_ptr; + + TSplitReadContext( + TSourceContext::TPtr sourceContext, + IHTTPGateway::TPtr gateway, + TString url, + std::size_t splitOffset, std::size_t splitSize, std::size_t fileSize, + std::size_t pathIndex, + IHTTPGateway::THeaders headers, + IHTTPGateway::TRetryPolicy::TPtr retryPolicy, + TTxId txId, TString requestId) + : SourceContext(sourceContext) + , Gateway(gateway) + , Url(url) + , SplitOffset(splitOffset), SplitSize(splitSize), FileSize(fileSize) + , PathIndex(pathIndex) + , Headers(headers) + , RetryPolicy(retryPolicy) + , TxId(txId), RequestId(requestId) + { + + } + + TSourceContext::TPtr SourceContext; + + const IHTTPGateway::TPtr Gateway; + const TString Url; + const std::size_t SplitOffset; + const std::size_t SplitSize; + const std::size_t FileSize; + const std::size_t PathIndex; + const IHTTPGateway::THeaders Headers; + IHTTPGateway::TRetryPolicy::TPtr RetryPolicy; + IHTTPGateway::TRetryPolicy::IRetryState::TPtr RetryState; + IHTTPGateway::TCancelHook CancelHook; + TMaybe NextRetryDelay; + std::atomic_bool Cancelled = false; + + const TTxId TxId; + const TString RequestId; + + const IHTTPGateway::TRetryPolicy::IRetryState::TPtr& GetRetryState() { + if (!RetryState) { + RetryState = RetryPolicy->CreateRetryState(); + } + return RetryState; + } + + void Cancel() { + Cancelled.store(true); + if (const auto cancelHook = std::move(CancelHook)) { + CancelHook = {}; + cancelHook(TIssue("Request cancelled.")); + } + } + + bool IsCancelled() { + return Cancelled.load(); + } +}; + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/common/ya.make b/ydb/library/yql/providers/s3/common/ya.make index f083f80fa6f9..fe29e92127fb 100644 --- a/ydb/library/yql/providers/s3/common/ya.make +++ b/ydb/library/yql/providers/s3/common/ya.make @@ -1,10 +1,23 @@ LIBRARY() +ADDINCL( + contrib/libs/poco/Foundation/include + ydb/library/yql/udfs/common/clickhouse/client/base + ydb/library/yql/udfs/common/clickhouse/client/base/pcg-random + ydb/library/yql/udfs/common/clickhouse/client/src +) + +YQL_LAST_ABI_VERSION() + SRCS( + source_context.cpp util.cpp ) PEERDIR( + ydb/core/kqp/common + ydb/library/yql/providers/common/http_gateway + ydb/library/yql/providers/s3/events ydb/library/yql/public/issue ydb/library/yql/public/issue/protos ) diff --git a/ydb/library/yql/providers/s3/events/events.cpp b/ydb/library/yql/providers/s3/events/events.cpp new file mode 100644 index 000000000000..b4d595d7a63b --- /dev/null +++ b/ydb/library/yql/providers/s3/events/events.cpp @@ -0,0 +1 @@ +#include "events.h" \ No newline at end of file diff --git a/ydb/library/yql/providers/s3/events/events.h b/ydb/library/yql/providers/s3/events/events.h new file mode 100644 index 000000000000..eb9caaee9a05 --- /dev/null +++ b/ydb/library/yql/providers/s3/events/events.h @@ -0,0 +1,264 @@ +#pragma once + +#include + +#include +#include + +#include +#include +#include + +#include + +#include + +#include + +namespace NYql::NDq { + +struct TEvS3Provider { + + enum EEv : ui32 { + EvBegin = EventSpaceBegin(NKikimr::TKikimrEvents::ES_S3_PROVIDER), + // Lister events + EvUpdateConsumersCount = EvBegin, + EvAck, + EvGetNextBatch, + EvObjectPathBatch, + EvObjectPathReadError, + // Gateway events + EvReadResult, // non-streaming download result + EvDownloadStart, // streaming started + EvDownloadData, // streaming data (part of) + EvDownloadFinish, // streaming finished + // Reader events + EvReadError, + EvRetry, + EvNextBlock, + EvNextRecordBatch, + EvFileFinished, + EvContinue, + EvReadResult2, // merge with EvReadResult and rename to EvDownloadResult + // Writer events + // Cache events + EvCacheSourceStart, + EvCacheCheckRequest, + EvCacheCheckResult, + EvCachePutRequest, + EvCacheNotification, + EvCacheSourceFinish, + EvEnd + }; + static_assert(EvEnd < EventSpaceEnd(NKikimr::TKikimrEvents::ES_S3_PROVIDER), "expect EvEnd < EventSpaceEnd(TEvents::ES_S3_PROVIDER)"); + + struct TEvUpdateConsumersCount : + public NActors::TEventPB { + + explicit TEvUpdateConsumersCount(ui64 consumersCountDelta = 0) { + Record.SetConsumersCountDelta(consumersCountDelta); + } + }; + + struct TEvAck : + public NActors::TEventPB { + + TEvAck() = default; + + explicit TEvAck(const NDqProto::TMessageTransportMeta& transportMeta) { + Record.MutableTransportMeta()->CopyFrom(transportMeta); + } + }; + + struct TEvGetNextBatch : + public NActors::TEventPB { + }; + + struct TEvObjectPathBatch : + public NActors::TEventPB { + + TEvObjectPathBatch() { + Record.SetNoMoreFiles(false); + } + + TEvObjectPathBatch(std::vector objectPaths, bool noMoreFiles, const NDqProto::TMessageTransportMeta& transportMeta) { + Record.MutableObjectPaths()->Assign( + std::make_move_iterator(objectPaths.begin()), + std::make_move_iterator(objectPaths.end())); + Record.SetNoMoreFiles(noMoreFiles); + Record.MutableTransportMeta()->CopyFrom(transportMeta); + } + }; + + struct TEvObjectPathReadError : + public NActors::TEventPB { + + TEvObjectPathReadError() = default; + + TEvObjectPathReadError(TIssues issues, const NDqProto::TMessageTransportMeta& transportMeta) { + NYql::IssuesToMessage(issues, Record.MutableIssues()); + Record.MutableTransportMeta()->CopyFrom(transportMeta); + } + }; + + struct TEvReadResult : public NActors::TEventLocal { + TEvReadResult( + IHTTPGateway::TContent&& result, + const TString& requestId, + size_t pathInd, + TString path) + : Result(std::move(result)) + , RequestId(requestId) + , PathIndex(pathInd) + , Path(std::move(path)) { } + + IHTTPGateway::TContent Result; + const TString RequestId; + const size_t PathIndex; + const TString Path; + }; + + struct TEvDownloadStart : public NActors::TEventLocal { + TEvDownloadStart(CURLcode curlResponseCode, long httpResponseCode) + : CurlResponseCode(curlResponseCode), HttpResponseCode(httpResponseCode) {} + const CURLcode CurlResponseCode; + const long HttpResponseCode; + }; + + struct TEvDownloadData : public NActors::TEventLocal { + TEvDownloadData(IHTTPGateway::TCountedContent&& data) : Result(std::move(data)) {} + IHTTPGateway::TCountedContent Result; + }; + + struct TEvDownloadFinish : public NActors::TEventLocal { + TEvDownloadFinish(size_t pathIndex, CURLcode curlResponseCode, TIssues&& issues) + : PathIndex(pathIndex), CurlResponseCode(curlResponseCode), Issues(std::move(issues)) { + } + const size_t PathIndex; + const CURLcode CurlResponseCode; + TIssues Issues; + }; + + struct TEvFileFinished : public NActors::TEventLocal { + TEvFileFinished(size_t pathIndex, ui64 ingressDelta, TDuration cpuTimeDelta, ui64 splitSize) + : PathIndex(pathIndex), IngressDelta(ingressDelta), CpuTimeDelta(cpuTimeDelta), SplitSize(splitSize) { + } + const size_t PathIndex; + const ui64 IngressDelta; + const TDuration CpuTimeDelta; + const ui64 SplitSize; + }; + + struct TEvReadError : public NActors::TEventLocal { + TEvReadError( + TIssues&& error, + const TString& requestId, + size_t pathInd, + TString path) + : Error(std::move(error)) + , RequestId(requestId) + , PathIndex(pathInd) + , Path(std::move(path)) { } + + const TIssues Error; + const TString RequestId; + const size_t PathIndex; + const TString Path; + }; + + struct TEvRetryEventFunc : public NActors::TEventLocal { + explicit TEvRetryEventFunc(std::function functor) : Functor(std::move(functor)) {} + const std::function Functor; + }; + + struct TEvNextBlock : public NActors::TEventLocal { + TEvNextBlock(NDB::Block& block, size_t pathInd, ui64 ingressDelta, TDuration cpuTimeDelta) + : PathIndex(pathInd), IngressDelta(ingressDelta), CpuTimeDelta(cpuTimeDelta) { + Block.swap(block); + } + NDB::Block Block; + const size_t PathIndex; + const ui64 IngressDelta; + const TDuration CpuTimeDelta; + }; + + struct TEvNextRecordBatch : public NActors::TEventLocal { + TEvNextRecordBatch(const std::shared_ptr& batch, size_t pathInd, ui64 ingressDelta, TDuration cpuTimeDelta) + : Batch(batch), PathIndex(pathInd), IngressDelta(ingressDelta), CpuTimeDelta(cpuTimeDelta) { + } + std::shared_ptr Batch; + const size_t PathIndex; + const ui64 IngressDelta; + const TDuration CpuTimeDelta; + }; + + struct TEvContinue : public NActors::TEventLocal { + }; + + struct TReadRange { + int64_t Offset; + int64_t Length; + }; + + struct TEvReadResult2 : public NActors::TEventLocal { + TEvReadResult2(TReadRange readRange, IHTTPGateway::TContent&& result) : ReadRange(readRange), Failure(false), Result(std::move(result)) { } + TEvReadResult2(TReadRange readRange, TIssues&& issues) : ReadRange(readRange), Failure(true), Result(""), Issues(std::move(issues)) { } + const TReadRange ReadRange; + const bool Failure; + IHTTPGateway::TContent Result; + const TIssues Issues; + }; + + struct TEvCacheSourceStart : public NActors::TEventLocal { + + TEvCacheSourceStart(NActors::TActorId sourceId, const TTxId& txId, std::shared_ptr schema) + : SourceId(sourceId), TxId(txId), Schema(schema) { + } + + NActors::TActorId SourceId; + TTxId TxId; + std::shared_ptr Schema; + }; + + struct TEvCacheCheckRequest : public NActors::TEventLocal { + + TEvCacheCheckRequest(NActors::TActorId sourceId, const TString& path, ui64 rowGroup) + : SourceId(sourceId), Path(path), RowGroup(rowGroup) { + } + + NActors::TActorId SourceId; + TString Path; + const ui64 RowGroup; + }; + + struct TEvCacheCheckResult : public NActors::TEventLocal { + TString Path; + ui64 RowGroup; + std::vector> Batches; + bool Hit = false; + }; + + struct TEvCachePutRequest : public NActors::TEventLocal { + NActors::TActorId SourceId; + TString Path; + ui64 RowGroup = 0; + std::vector> Batches; + }; + + struct TEvCacheNotification : public NActors::TEventLocal { + TString Path; + ui64 RowGroup = 0; + std::vector> Batches; + }; + + struct TEvCacheSourceFinish : public NActors::TEventLocal { + + TEvCacheSourceFinish(NActors::TActorId sourceId) + : SourceId(sourceId) { + } + + NActors::TActorId SourceId; + }; +}; + +} // namespace NYql::NDq diff --git a/ydb/library/yql/providers/s3/events/ya.make b/ydb/library/yql/providers/s3/events/ya.make new file mode 100644 index 000000000000..3067b37e25cb --- /dev/null +++ b/ydb/library/yql/providers/s3/events/ya.make @@ -0,0 +1,26 @@ +LIBRARY() + +ADDINCL( + contrib/libs/poco/Foundation/include + ydb/library/yql/udfs/common/clickhouse/client/base + ydb/library/yql/udfs/common/clickhouse/client/base/pcg-random + ydb/library/yql/udfs/common/clickhouse/client/src +) + +YQL_LAST_ABI_VERSION() + +SRCS( + events.cpp +) + +PEERDIR( + ydb/core/base + ydb/core/kqp/common + ydb/library/yql/dq/actors/protos + ydb/library/yql/providers/s3/proto + ydb/library/yql/providers/common/http_gateway + ydb/library/yql/public/issue + ydb/library/yql/udfs/common/clickhouse/client +) + +END() diff --git a/ydb/library/yql/providers/s3/expr_nodes/yql_s3_expr_nodes.json b/ydb/library/yql/providers/s3/expr_nodes/yql_s3_expr_nodes.json index 90df231ec912..73843919d48a 100644 --- a/ydb/library/yql/providers/s3/expr_nodes/yql_s3_expr_nodes.json +++ b/ydb/library/yql/providers/s3/expr_nodes/yql_s3_expr_nodes.json @@ -65,8 +65,9 @@ "Match": {"Type": "Callable", "Name": "S3ParseSettings"}, "Children": [ {"Index": 3, "Name": "Format", "Type": "TCoAtom"}, - {"Index": 4, "Name": "RowType", "Type": "TExprBase"}, - {"Index": 5, "Name": "Settings", "Type": "TExprBase", "Optional": true} + {"Index": 4, "Name": "Arrow", "Type": "TCoAtom"}, + {"Index": 5, "Name": "RowType", "Type": "TExprBase"}, + {"Index": 6, "Name": "Settings", "Type": "TExprBase", "Optional": true} ] }, { diff --git a/ydb/library/yql/providers/s3/json/json_row_parser.cpp b/ydb/library/yql/providers/s3/json/json_row_parser.cpp new file mode 100644 index 000000000000..a6fc7c9d61e3 --- /dev/null +++ b/ydb/library/yql/providers/s3/json/json_row_parser.cpp @@ -0,0 +1,416 @@ +#include "json_row_parser.h" + +#include + +#include +#include + +#include + +namespace NYql::NJson { + +struct TDefaultIgnoreWrapper : public TArrowBuilderWrapper { + TDefaultIgnoreWrapper(TString& errorDetails) : TArrowBuilderWrapper(errorDetails) {} + bool AddBoolean(bool) override { return true; } + bool AddInteger(long long) override { return true; } + bool AddUInteger(unsigned long long) override { return true; } + bool AddDouble(double) override { return true; } + bool AddString(const TStringBuf&) override { return true; } + void AddNull() override { } + std::shared_ptr Build() override { return nullptr; }; +}; + +template +struct TArrowUintBuildWrapper : public TArrowBuilderWrapper { + + NYql::NUdf::TFixedSizeArrayBuilder builder; + + TArrowUintBuildWrapper(TString& errorDetails, const std::shared_ptr& type, ui32 length) + : TArrowBuilderWrapper(errorDetails) + , builder(NKikimr::NMiniKQL::TTypeInfoHelper(), type, *arrow::system_memory_pool(), length) { + } + + bool AddBoolean(bool value) override { + builder.Add(NUdf::TBlockItem(static_cast(value))); + return true; + } + + bool AddInteger(long long) override { + ErrorDetails = "out of range"; + return true; + } + + bool AddUInteger(unsigned long long value) override { + builder.Add(NUdf::TBlockItem(static_cast(value))); + return true; + } + + bool AddDouble(double) override { + ErrorDetails = "conversion floating point to integer is not supported"; + return false; + } + + bool AddString(const TStringBuf&) override { + ErrorDetails = "conversion string to integer is not supported"; + return false; + } + + void AddNull() override { + builder.Add(NUdf::TBlockItem{}); + } + + std::shared_ptr Build() override { + return builder.Build(false).make_array(); + } +}; + +template +struct TArrowIntBuildWrapper : public TArrowBuilderWrapper { + + NYql::NUdf::TFixedSizeArrayBuilder builder; + + TArrowIntBuildWrapper(TString& errorDetails, const std::shared_ptr& type, ui32 length) + : TArrowBuilderWrapper(errorDetails) + , builder(NKikimr::NMiniKQL::TTypeInfoHelper(), type, *arrow::system_memory_pool(), length) { + } + + bool AddBoolean(bool value) override { + builder.Add(NUdf::TBlockItem(static_cast(value))); + return true; + } + + bool AddInteger(long long value) override { + builder.Add(NUdf::TBlockItem(static_cast(value))); + return true; + } + + bool AddUInteger(unsigned long long value) override { + // std::numeric_limits::max(); + builder.Add(NUdf::TBlockItem(static_cast(value))); + return true; + } + + bool AddDouble(double) override { + ErrorDetails = "conversion floating point to integer is not supported"; + return false; + } + + bool AddString(const TStringBuf&) override { + ErrorDetails = "conversion string to integer is not supported"; + return false; + } + + void AddNull() override { + builder.Add(NUdf::TBlockItem{}); + } + + std::shared_ptr Build() override { + return builder.Build(false).make_array(); + } +}; + +template +struct TArrowStringBuildWrapper : public TArrowBuilderWrapper { + + NYql::NUdf::TStringArrayBuilder builder; + + TArrowStringBuildWrapper(TString& errorDetails, const std::shared_ptr& type, ui32 length) + : TArrowBuilderWrapper(errorDetails) + , builder(NKikimr::NMiniKQL::TTypeInfoHelper(), type, *arrow::system_memory_pool(), length) { + } + + bool AddBoolean(bool) override { + ErrorDetails = "conversion boolean to string is not supported"; + return false; + } + + bool AddInteger(long long) override { + ErrorDetails = "conversion integer to string is not supported"; + return false; + } + + bool AddUInteger(unsigned long long) override { + ErrorDetails = "conversion unsigned integer to string is not supported"; + return false; + } + + bool AddDouble(double) override { + ErrorDetails = "conversion floating point to string is not supported"; + return false; + } + + bool AddString(const TStringBuf& value) override { + builder.Add(NUdf::TBlockItem{value}); + return true; + } + + void AddNull() override { + builder.Add(NUdf::TBlockItem{}); + } + + std::shared_ptr Build() override { + return builder.Build(false).make_array(); + } +}; + +std::shared_ptr CreateBuilderWrapper(TString& errorDetails, const std::shared_ptr& type, bool nullable, ui32 length) { + switch (type->id()) { + case arrow::Type::UINT8: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::INT8: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::UINT16: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::INT16: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::UINT32: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::INT32: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::UINT64: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::INT64: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::STRING: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::BINARY: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::TIMESTAMP: + return nullable ? + std::static_pointer_cast(std::make_shared>(errorDetails, type, length)) + : std::static_pointer_cast(std::make_shared>(errorDetails, type, length)); + case arrow::Type::BOOL: + case arrow::Type::HALF_FLOAT: + case arrow::Type::FLOAT: + case arrow::Type::DOUBLE: + case arrow::Type::DATE32: + case arrow::Type::DATE64: + case arrow::Type::TIME32: + case arrow::Type::TIME64: + default: +// Cerr << "DEFAULT BUILDER" << Endl; + return std::make_shared(errorDetails); + } +} + +TJsonRowParser::TJsonRowParser(std::shared_ptr outputSchema, ui32 maxRowCount) + : OutputSchema(outputSchema) + , Fields(outputSchema->fields()) + , MaxRowCount(maxRowCount) { + + InputLayout.reserve(32); + OutputColumnCount = Fields.size(); + for (ui32 i = 0; i < OutputColumnCount; i++) { +// Cerr << "FIELD " << Fields[i]->name() << Endl; + Builders.push_back(CreateBuilderWrapper(ErrorDetails, Fields[i]->type(), Fields[i]->nullable(), MaxRowCount)); + OutputLayout.emplace(Fields[i]->name(), i); + } + AssignedFields.resize(OutputColumnCount); +} + +void TJsonRowParser::ParseNextRow(TStringBuf rowBuffer) { +// Cerr << "PARSE ROW: " << rowBuffer << Endl; + Stack.resize(0); + std::fill(AssignedFields.begin(), AssignedFields.end(), false); + TopLevelMap = false; + ObjectFound = false; + NextIndex = 0; + ErrorFound = false; + ErrorDetails = ""; + FieldIndex.reset(); + Finished = false; + + ::NJson::ReadJsonFast(rowBuffer, this); + + if (!ErrorFound) { + for (ui32 i = 0; i < OutputColumnCount; i++) { + if (!AssignedFields[i]) { + if (!Fields[i]->nullable()) { + ErrorFound = true; + ErrorMessage = TStringBuilder() << "missed mandatory field \"" << Fields[i]->name() << "\""; +// Cerr << "ERROR: " << ErrorMessage << Endl; + return; + } else { + Builders[i]->AddNull(); + } + } + } + CurrentRowCount++; + } +} + +std::shared_ptr TJsonRowParser::TakeBatch() { + std::vector> columns; + for (auto& builder : Builders) { + columns.push_back(builder->Build()); + } + auto batch = arrow::RecordBatch::Make(OutputSchema, CurrentRowCount, columns); + CurrentRowCount = 0; + return batch; +} + +bool TJsonRowParser::OnMapKey(const TStringBuf& key, bool) { + if (!TopLevelMap) { + if (Stack.empty()) { + ErrorDetails = "top level json object is expected, starting with '{'"; + return false; + } + FieldIndex.reset(); + } else if (auto it = OutputLayout.find(key); it != OutputLayout.end()) { + FieldIndex = it->second; + AssignedFields[it->second] = true; + } else { + FieldIndex.reset(); + } + return true; +} + + // virtual bool OnNull(); + +bool TJsonRowParser::OnValue() { + if (Stack.empty()) { + ErrorDetails = "top level json object is expected, starting with '{'"; + return false; + } + return true; +} + +bool TJsonRowParser::OnBoolean(bool value) { + if (!OnValue()) { + return false; + } + return FieldIndex ? Builders[*FieldIndex]->AddBoolean(value) : true; +} + +bool TJsonRowParser::OnInteger(long long value) { + if (!OnValue()) { + return false; + } + return FieldIndex ? Builders[*FieldIndex]->AddInteger(value) : true; +} + +bool TJsonRowParser::OnUInteger(unsigned long long value) { + if (!OnValue()) { + return false; + } + return FieldIndex ? Builders[*FieldIndex]->AddUInteger(value) : true; +} + +bool TJsonRowParser::OnDouble(double value) { + if (!OnValue()) { + return false; + } + return FieldIndex ? Builders[*FieldIndex]->AddDouble(value) : true; +} + +bool TJsonRowParser::OnString(const TStringBuf& value) { + if (!OnValue()) { + return false; + } + return FieldIndex ? Builders[*FieldIndex]->AddString(value) : true; +} + +bool TJsonRowParser::OnOpenMap() { + Stack.push_back(ComplextType::TypeMap); + TopLevelMap = Stack.size() == 1; + ObjectFound = true; + return true; +} + +bool TJsonRowParser::OnMapKey(const TStringBuf& key) { + return OnMapKey(key, true); +} + +bool TJsonRowParser::OnCloseMap() { + if (Stack.empty()) { + ErrorDetails = "unbalanced '}''"; + return false; + } + + if (Stack.back() != ComplextType::TypeMap) { + ErrorDetails = "expected ']'"; + return false; + } + + Stack.pop_back(); + + if (Stack.empty()) { + Finished = true; + return false; + } + + TopLevelMap = Stack.size() == 1 && Stack.back() == ComplextType::TypeMap; + + return true; +} + +bool TJsonRowParser::OnOpenArray() { + if (Stack.empty()) { + ErrorDetails = "top level json object is expected, starting with '{'"; + return false; + } + + Stack.push_back(ComplextType::TypeArray); + TopLevelMap = false; + return true; +} + +bool TJsonRowParser::OnCloseArray() { + if (Stack.empty()) { + ErrorDetails = "unbalanced ']'"; + return false; + } + + if (Stack.back() != ComplextType::TypeArray) { + ErrorDetails = "expected '}'"; + return false; + } + + Stack.pop_back(); + TopLevelMap = Stack.size() == 1 && Stack.back() == ComplextType::TypeMap; + + return true; +} + +bool TJsonRowParser::OnMapKeyNoCopy(const TStringBuf& key) { + return OnMapKey(key, false); +} + +bool TJsonRowParser::OnEnd() { + return ObjectFound; +} + +void TJsonRowParser::OnError(size_t off, TStringBuf reason) { + if (!Finished) { + ErrorFound = true; + if (ErrorDetails) { + ErrorMessage = TStringBuilder() << reason << ", " << ErrorDetails; + } else { + ErrorMessage = reason; + } + ErrorOffset = off; +// Cerr << "ERROR at " << off << ": " << ErrorMessage << Endl; + } +} + +} // namespace NYql::NJson diff --git a/ydb/library/yql/providers/s3/json/json_row_parser.h b/ydb/library/yql/providers/s3/json/json_row_parser.h new file mode 100644 index 000000000000..3e5f0c40551b --- /dev/null +++ b/ydb/library/yql/providers/s3/json/json_row_parser.h @@ -0,0 +1,89 @@ +#pragma once + +#include +#include + +#include + +#include + +#define THROW_ARROW_NOT_OK(status) \ + do \ + { \ + if (::arrow::Status _s = (status); !_s.ok()) \ + throw yexception() << _s.ToString(); \ + } while (false) + +namespace NYql::NJson { + +struct TArrowBuilderWrapper { + + TArrowBuilderWrapper(TString& errorDetails) : ErrorDetails(errorDetails) {} + virtual ~TArrowBuilderWrapper() {} + + TString& ErrorDetails; + + virtual bool AddBoolean(bool value) = 0; + virtual bool AddInteger(long long value) = 0; + virtual bool AddUInteger(unsigned long long value) = 0; + virtual bool AddDouble(double value) = 0; + virtual bool AddString(const TStringBuf& value) = 0; + virtual void AddNull() = 0; + + virtual std::shared_ptr Build() = 0; +}; + +struct TJsonRowParser : public ::NJson::TJsonCallbacks { + + enum ComplextType { + TypeMap, + TypeArray + }; + + std::vector Stack; + std::vector> InputLayout; + std::map OutputLayout; + std::vector AssignedFields; + bool TopLevelMap = false; + bool ObjectFound = false; + ui32 NextIndex = 0; + ui32 OutputColumnCount = 0; + std::optional FieldIndex; + TString ErrorDetails; + bool ErrorFound = false; + bool Finished = false; + TString ErrorMessage; + ui32 ErrorOffset; + ui32 CurrentRowCount = 0; + + std::shared_ptr OutputSchema; + const std::vector>& Fields; + const ui32 MaxRowCount; + std::vector> Builders; + + TJsonRowParser(std::shared_ptr outputSchema, ui32 maxRowCount = 4096); + + void ParseNextRow(TStringBuf rowBuffer); + std::shared_ptr TakeBatch(); + + // virtual bool OnNull(); + bool OnBoolean(bool value) override; + bool OnInteger(long long value) override; + bool OnUInteger(unsigned long long value) override; + bool OnDouble(double value) override; + bool OnString(const TStringBuf& value) override; + bool OnOpenMap() override; + bool OnMapKey(const TStringBuf& key) override; + bool OnCloseMap() override; + bool OnOpenArray() override; + bool OnCloseArray() override; + // bool OnStringNoCopy(const TStringBuf& value) override; + bool OnMapKeyNoCopy(const TStringBuf& key) override; + bool OnEnd() override; + void OnError(size_t off, TStringBuf reason) override; + + bool OnValue(); + bool OnMapKey(const TStringBuf& key, bool copyValue); +}; + +} // namespace NYql::NJson diff --git a/ydb/library/yql/providers/s3/json/json_row_parser_ut.cpp b/ydb/library/yql/providers/s3/json/json_row_parser_ut.cpp new file mode 100644 index 000000000000..c87f52832aa9 --- /dev/null +++ b/ydb/library/yql/providers/s3/json/json_row_parser_ut.cpp @@ -0,0 +1,139 @@ +#include "json_row_parser.h" + +#include + +#include + +namespace NYql::NJson { + +Y_UNIT_TEST_SUITE(TJsonRowParse) { + + Y_UNIT_TEST(SyntaxTopLevel) { + arrow::SchemaBuilder builder; + TJsonRowParser parser(builder.Finish().ValueOrDie()); + + // not well formed + parser.ParseNextRow("Blah blah blah"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: 'blah'"); + + // field at top level is not allowed + parser.ParseNextRow("some_field: 0"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: ':', top level json object is expected, starting with '{'"); + + // array at top level is not allowed + parser.ParseNextRow("[0, 1, 2]"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: '[', top level json object is expected, starting with '{'"); + + // value at top level is not allowed + parser.ParseNextRow("0"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: '0', top level json object is expected, starting with '{'"); + + // empty string is not allowed + parser.ParseNextRow(" "sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid or truncated"); + + // minimal correct and well formed json object + parser.ParseNextRow("{}"sv); + UNIT_ASSERT(!parser.ErrorFound); + + // allow comma after object + parser.ParseNextRow("{},"sv); + UNIT_ASSERT(!parser.ErrorFound); + + // allow (ignore) any trash after object + parser.ParseNextRow("{}; -- Blah blah blah"sv); + UNIT_ASSERT(!parser.ErrorFound); + + // allow unclosed object map + parser.ParseNextRow("{"sv); + UNIT_ASSERT(!parser.ErrorFound); + } + + Y_UNIT_TEST(SyntaxStructure) { + arrow::SchemaBuilder builder; + TJsonRowParser parser(builder.Finish().ValueOrDie()); + + parser.ParseNextRow("{]}"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: ']', expected '}'"); + + parser.ParseNextRow("{[}]"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: '}', expected ']'"); + + parser.ParseNextRow("{ blah blah }"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "invalid syntax at token: 'blah'"); + } + + Y_UNIT_TEST(Nullables) { + arrow::SchemaBuilder builder; + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field1"), arrow::uint32(), false))); + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field2"), arrow::uint32(), true))); + TJsonRowParser parser(builder.Finish().ValueOrDie()); + + parser.ParseNextRow("{ field1: 1 }"sv); + UNIT_ASSERT(!parser.ErrorFound); + + parser.ParseNextRow("{ field2: 2 }"sv); + UNIT_ASSERT(parser.ErrorFound); + UNIT_ASSERT_VALUES_EQUAL(parser.ErrorMessage, "missed mandatory field \"field1\""); + } + + Y_UNIT_TEST(MixedOrder) { + arrow::SchemaBuilder builder; + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field1"), arrow::uint32(), false))); + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field2"), arrow::uint32(), false))); + TJsonRowParser parser(builder.Finish().ValueOrDie()); + + parser.ParseNextRow("{ field1: 1, field2: 2 }"sv); + UNIT_ASSERT(!parser.ErrorFound); + parser.ParseNextRow("{ field2: 3, field1: 4 }"sv); + UNIT_ASSERT(!parser.ErrorFound); + + { + auto batch = parser.TakeBatch(); + ::NYql::NUdf::TFixedSizeBlockReader reader; + auto column1 = batch->column(0); + auto column2 = batch->column(1); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column1->data(), 0).As(), 1); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column1->data(), 1).As(), 4); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column2->data(), 0).As(), 2); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column2->data(), 1).As(), 3); + } + + parser.ParseNextRow("{ field1: 5, field2: 6 }"sv); + UNIT_ASSERT(!parser.ErrorFound); + parser.ParseNextRow("{ field2: 7, field1: 8 }"sv); + UNIT_ASSERT(!parser.ErrorFound); + + { + auto batch = parser.TakeBatch(); + ::NYql::NUdf::TFixedSizeBlockReader reader; + auto column1 = batch->column(0); + auto column2 = batch->column(1); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column1->data(), 0).As(), 5); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column1->data(), 1).As(), 8); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column2->data(), 0).As(), 6); + UNIT_ASSERT_VALUES_EQUAL(reader.GetItem(*column2->data(), 1).As(), 7); + } + } + + Y_UNIT_TEST(SingleRow) { + arrow::SchemaBuilder builder; + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field1"), arrow::uint32(), true))); + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field2"), arrow::float64(), true))); + THROW_ARROW_NOT_OK(builder.AddField(std::make_shared(std::string("field3"), arrow::utf8(), true))); + TJsonRowParser parser(builder.Finish().ValueOrDie()); + + TString json = "{ field1: 1, field2: 2.5, field3: \"text\", field4: [1, \"a\", -5], field5: { sub1: 1, sub2: 2 }, field6: \"\\\"QQ\\\"\" },"; + parser.ParseNextRow(json); + } +} + +} // namespace NYql::NJson diff --git a/ydb/library/yql/providers/s3/json/ut/ya.make b/ydb/library/yql/providers/s3/json/ut/ya.make new file mode 100644 index 000000000000..fd87cb7c873f --- /dev/null +++ b/ydb/library/yql/providers/s3/json/ut/ya.make @@ -0,0 +1,15 @@ +UNITTEST_FOR(ydb/library/yql/providers/s3/json) + +YQL_LAST_ABI_VERSION() + +SRCS( + json_row_parser_ut.cpp +) + +PEERDIR( + ydb/library/yql/public/udf/arrow + ydb/library/yql/public/udf/service/exception_policy + ydb/library/yql/sql/pg_dummy +) + +END() diff --git a/ydb/library/yql/providers/s3/json/ya.make b/ydb/library/yql/providers/s3/json/ya.make new file mode 100644 index 000000000000..796ddc37c71b --- /dev/null +++ b/ydb/library/yql/providers/s3/json/ya.make @@ -0,0 +1,20 @@ +LIBRARY() + +YQL_LAST_ABI_VERSION() + +SRCS( + json_row_parser.cpp +) + +PEERDIR( + contrib/libs/apache/arrow + library/cpp/json/common + library/cpp/json/fast_sax + ydb/library/yql/minikql +) + +END() + +RECURSE_FOR_TESTS( + ut +) diff --git a/ydb/library/yql/providers/s3/proto/source.proto b/ydb/library/yql/providers/s3/proto/source.proto index 1ac3fd840d12..03ef487db1ab 100644 --- a/ydb/library/yql/providers/s3/proto/source.proto +++ b/ydb/library/yql/providers/s3/proto/source.proto @@ -15,11 +15,13 @@ message TSource { optional string RowType = 4; optional string Format = 5; map Settings = 6; - reserved 7; + bool Arrow = 7; reserved 8; uint64 ParallelRowGroupCount = 9; bool RowGroupReordering = 10; uint64 ParallelDownloadCount = 11; uint64 RowsLimitHint = 12; bool UseRuntimeListing = 13; + bool AsyncDecoding = 14; + bool SourceCoroActor = 15; } diff --git a/ydb/library/yql/providers/s3/provider/yql_s3_datasource_type_ann.cpp b/ydb/library/yql/providers/s3/provider/yql_s3_datasource_type_ann.cpp index 1e96dd0c36b2..6dfff78ec378 100644 --- a/ydb/library/yql/providers/s3/provider/yql_s3_datasource_type_ann.cpp +++ b/ydb/library/yql/providers/s3/provider/yql_s3_datasource_type_ann.cpp @@ -337,7 +337,7 @@ class TS3DataSourceTypeAnnotationTransformer : public TVisitorTransformerBase { } TStatus HandleS3ParseSettings(const TExprNode::TPtr& input, TExprContext& ctx) { - if (!EnsureMinMaxArgsCount(*input, 5U, 6U, ctx)) { + if (!EnsureMinMaxArgsCount(*input, 6U, 7U, ctx)) { return TStatus::Error; } @@ -362,6 +362,11 @@ class TS3DataSourceTypeAnnotationTransformer : public TVisitorTransformerBase { return TStatus::Error; } + if (!EnsureAtom(*input->Child(TS3ParseSettings::idx_Arrow), ctx)) + { + return TStatus::Error; + } + const auto& rowTypeNode = *input->Child(TS3ParseSettings::idx_RowType); if (!EnsureType(rowTypeNode, ctx)) { return TStatus::Error; @@ -379,7 +384,7 @@ class TS3DataSourceTypeAnnotationTransformer : public TVisitorTransformerBase { } const TTypeAnnotationNode* itemType = nullptr; - if (input->Child(TS3ParseSettings::idx_Format)->Content() == "parquet") { + if (FromString(input->Child(TS3ParseSettings::idx_Arrow)->Content())) { std::unordered_set extraColumnNames(extraColumnsType->GetSize()); for (const auto& extraColumn : extraColumnsType->GetItems()) { extraColumnNames.insert(TString{extraColumn->GetName()}); diff --git a/ydb/library/yql/providers/s3/provider/yql_s3_dq_integration.cpp b/ydb/library/yql/providers/s3/provider/yql_s3_dq_integration.cpp index 099fa7b8d61b..d2854a873f30 100644 --- a/ydb/library/yql/providers/s3/provider/yql_s3_dq_integration.cpp +++ b/ydb/library/yql/providers/s3/provider/yql_s3_dq_integration.cpp @@ -276,7 +276,15 @@ class TS3DqIntegration: public TDqIntegrationBase { } auto format = s3ReadObject.Object().Format().Ref().Content(); - if (const auto useCoro = State_->Configuration->SourceCoroActor.Get(); (!useCoro || *useCoro) && format != "raw" && format != "json_list") { + if (format != "raw" && format != "json_list") { + + auto arrow = false; + if (format == "parquet") { + arrow = true; + } else if (format == "json_each_row") { + arrow = State_->Configuration->UseBlocksSource.Get().GetOrElse(false); + } + return Build(ctx, read->Pos()) .Input() .Paths(s3ReadObject.Object().Paths()) @@ -285,6 +293,7 @@ class TS3DqIntegration: public TDqIntegrationBase { .Build() .RowsLimitHint(ctx.NewAtom(read->Pos(), "")) .Format(s3ReadObject.Object().Format()) + .Arrow(ctx.NewAtom(read->Pos(), ToString(arrow))) .RowType(ExpandType(s3ReadObject.Pos(), *rowType, ctx)) .Settings(s3ReadObject.Object().Settings()) .Build() @@ -369,6 +378,7 @@ class TS3DqIntegration: public TDqIntegrationBase { if (const auto mayParseSettings = settings.Maybe()) { const auto parseSettings = mayParseSettings.Cast(); srcDesc.SetFormat(parseSettings.Format().StringValue().c_str()); + srcDesc.SetArrow(FromString(parseSettings.Arrow().StringValue())); srcDesc.SetParallelRowGroupCount(State_->Configuration->ArrowParallelRowGroupCount.Get().GetOrElse(0)); srcDesc.SetRowGroupReordering(State_->Configuration->ArrowRowGroupReordering.Get().GetOrElse(true)); srcDesc.SetParallelDownloadCount(State_->Configuration->ParallelDownloadCount.Get().GetOrElse(0)); @@ -418,6 +428,9 @@ class TS3DqIntegration: public TDqIntegrationBase { srcDesc.MutableSettings()->insert({"addPathIndex", "true"}); } + srcDesc.SetAsyncDecoding(State_->Configuration->AsyncDecoding.Get().GetOrElse(false)); + srcDesc.SetSourceCoroActor(State_->Configuration->SourceCoroActor.Get().GetOrElse(true)); + #if defined(_linux_) || defined(_darwin_) auto useRuntimeListing = State_->Configuration->UseRuntimeListing.Get().GetOrElse(false); diff --git a/ydb/library/yql/providers/s3/provider/yql_s3_logical_opt.cpp b/ydb/library/yql/providers/s3/provider/yql_s3_logical_opt.cpp index 429f94293e02..ba6b98d15f0c 100644 --- a/ydb/library/yql/providers/s3/provider/yql_s3_logical_opt.cpp +++ b/ydb/library/yql/providers/s3/provider/yql_s3_logical_opt.cpp @@ -518,7 +518,8 @@ class TS3LogicalOptProposalTransformer : public TOptimizeTransformerBase { } auto formatName = parseSettings.Format().StringValue(); - if (outputRowDataItems.size() == 0 && readRowDataItems.size() != 0 && formatName != "parquet") { + if (outputRowDataItems.size() == 0 && readRowDataItems.size() != 0 && formatName != "parquet" + && (formatName != "json_each_row" || !FromString(parseSettings.Arrow().StringValue())) ) { const TStructExprType* readRowDataType = ctx.MakeType(readRowDataItems); auto item = GetLightColumn(*readRowDataType); YQL_ENSURE(item); diff --git a/ydb/library/yql/providers/s3/provider/yql_s3_settings.cpp b/ydb/library/yql/providers/s3/provider/yql_s3_settings.cpp index a5c373ff677e..6cb064d6ac4a 100644 --- a/ydb/library/yql/providers/s3/provider/yql_s3_settings.cpp +++ b/ydb/library/yql/providers/s3/provider/yql_s3_settings.cpp @@ -26,6 +26,7 @@ TS3Configuration::TS3Configuration() REGISTER_SETTING(*this, FileQueueBatchSizeLimit); REGISTER_SETTING(*this, FileQueueBatchObjectCountLimit); REGISTER_SETTING(*this, FileQueuePrefetchSize); + REGISTER_SETTING(*this, AsyncDecoding); } TS3Settings::TConstPtr TS3Configuration::Snapshot() const { diff --git a/ydb/library/yql/providers/s3/provider/yql_s3_settings.h b/ydb/library/yql/providers/s3/provider/yql_s3_settings.h index ebf6851a6fd3..8b0ed08efc0f 100644 --- a/ydb/library/yql/providers/s3/provider/yql_s3_settings.h +++ b/ydb/library/yql/providers/s3/provider/yql_s3_settings.h @@ -20,7 +20,7 @@ struct TS3Settings { NCommon::TConfSetting ArrowParallelRowGroupCount; // Number of parquet row groups to read in parallel, min == 1 NCommon::TConfSetting ArrowRowGroupReordering; // Allow to push rows from file in any order, default false, but usually it is OK NCommon::TConfSetting ParallelDownloadCount; // Number of files to read in parallel, min == 1 - NCommon::TConfSetting UseBlocksSource; // Deprecated and has to be removed after config cleanup + NCommon::TConfSetting UseBlocksSource; // Define read mode for json_each_row format NCommon::TConfSetting AtomicUploadCommit; // Commit each file independently, w/o transaction semantic over all files NCommon::TConfSetting UseConcurrentDirectoryLister; NCommon::TConfSetting MaxDiscoveryFilesPerDirectory; @@ -28,6 +28,7 @@ struct TS3Settings { NCommon::TConfSetting FileQueueBatchSizeLimit; // Limits total size of files in one PathBatch from FileQueue NCommon::TConfSetting FileQueueBatchObjectCountLimit; // Limits count of files in one PathBatch from FileQueue NCommon::TConfSetting FileQueuePrefetchSize; + NCommon::TConfSetting AsyncDecoding; // Parse and decode input data at separate mailbox/thread of TaskRunner }; struct TS3ClusterSettings { diff --git a/ydb/library/yql/providers/s3/ya.make b/ydb/library/yql/providers/s3/ya.make index f5546c3dea1f..9448a7713873 100644 --- a/ydb/library/yql/providers/s3/ya.make +++ b/ydb/library/yql/providers/s3/ya.make @@ -3,7 +3,9 @@ RECURSE( common compressors credentials + events expr_nodes + json object_listers path_generator provider