Skip to content

Commit dd39aa2

Browse files
Hor911MetaGigachad
andauthored
Dynamic S3 Listing (ydb-platform#2398)
Co-authored-by: Yaroslav Plishan <[email protected]>
1 parent 3f96a63 commit dd39aa2

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

52 files changed

+1242
-396
lines changed

ydb/core/base/events.h

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -171,6 +171,9 @@ struct TKikimrEvents : TEvents {
171171
ES_TABLE_CREATOR,
172172
ES_PQ_PARTITION_CHOOSER,
173173
ES_GRAPH,
174+
ES_REPLICATION_SERVICE,
175+
ES_CHANGE_EXCHANGE_NEW, // rename ES_CHANGE_EXCHANGE to if path with rename of old ES_CHANGE_EXCHANGE to ES_CHANGE_EXCHANGE_DATASHARD is merged
176+
ES_S3_FILE_QUEUE,
174177
};
175178
};
176179

ydb/core/fq/libs/actors/proxy_private.h

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,6 @@ namespace NKikimr {
2222

2323
namespace NFq {
2424

25-
NActors::TActorId MakeYqPrivateProxyId();
26-
2725
NActors::IActor* CreateYqlAnalyticsPrivateProxy(
2826
const NConfig::TPrivateProxyConfig& privateProxyConfig,
2927
TIntrusivePtr<ITimeProvider> timeProvider,

ydb/core/fq/libs/checkpointing/checkpoint_coordinator.cpp

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -571,6 +571,16 @@ void TCheckpointCoordinator::Handle(NActors::TEvInterconnect::TEvNodeConnected::
571571
}
572572
}
573573

574+
void TCheckpointCoordinator::Handle(NActors::TEvents::TEvUndelivered::TPtr& ev) {
575+
CC_LOG_D("Handle undelivered");
576+
577+
if (const auto actorIt = AllActors.find(ev->Sender); actorIt != AllActors.end()) {
578+
actorIt->second->EventsQueue.HandleUndelivered(ev);
579+
}
580+
581+
NYql::TTaskControllerImpl<TCheckpointCoordinator>::OnUndelivered(ev);
582+
}
583+
574584
void TCheckpointCoordinator::Handle(NActors::TEvents::TEvPoison::TPtr& ev) {
575585
CC_LOG_D("Got TEvPoison");
576586
Send(ev->Sender, new NActors::TEvents::TEvPoisonTaken(), 0, ev->Cookie);

ydb/core/fq/libs/checkpointing/checkpoint_coordinator.h

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -60,6 +60,7 @@ class TCheckpointCoordinator : public NYql::TTaskControllerImpl<TCheckpointCoord
6060
void Handle(NActors::TEvents::TEvPoison::TPtr&);
6161
void Handle(NActors::TEvInterconnect::TEvNodeDisconnected::TPtr& ev);
6262
void Handle(NActors::TEvInterconnect::TEvNodeConnected::TPtr& ev);
63+
void Handle(NActors::TEvents::TEvUndelivered::TPtr& ev);
6364
void Handle(const TEvCheckpointCoordinator::TEvRunGraph::TPtr&);
6465
void HandleException(const std::exception& err);
6566

@@ -89,13 +90,13 @@ class TCheckpointCoordinator : public NYql::TTaskControllerImpl<TCheckpointCoord
8990
hFunc(NYql::NDq::TEvRetryQueuePrivate::TEvRetry, Handle)
9091

9192
hFunc(NActors::TEvents::TEvPoison, Handle)
92-
hFunc(NActors::TEvents::TEvUndelivered, NYql::TTaskControllerImpl<TCheckpointCoordinator>::OnUndelivered)
9393
hFunc(NActors::TEvents::TEvWakeup, NYql::TTaskControllerImpl<TCheckpointCoordinator>::OnWakeup)
9494

9595
hFunc(NActors::TEvInterconnect::TEvNodeDisconnected, Handle)
96-
hFunc(NActors::TEvInterconnect::TEvNodeConnected, Handle),
96+
hFunc(NActors::TEvInterconnect::TEvNodeConnected, Handle)
97+
hFunc(NActors::TEvents::TEvUndelivered, Handle)
9798

98-
ExceptionFunc(std::exception, HandleException)
99+
, ExceptionFunc(std::exception, HandleException)
99100
)
100101

101102
static constexpr char ActorName[] = "YQ_CHECKPOINT_COORDINATOR";

ydb/core/fq/libs/control_plane_proxy/control_plane_proxy.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,6 @@
11
#include "config.h"
22
#include "control_plane_proxy.h"
33
#include "probes.h"
4-
#include "utils.h"
54

65
#include <ydb/core/fq/libs/actors/logging/log.h>
76
#include <ydb/core/fq/libs/compute/ydb/control_plane/compute_database_control_plane_service.h>
@@ -23,6 +22,7 @@
2322
#include <ydb/core/fq/libs/control_plane_proxy/actors/utils.h>
2423
#include <ydb/core/fq/libs/control_plane_proxy/actors/ydb_schema_query_actor.h>
2524
#include <ydb/core/fq/libs/control_plane_proxy/events/events.h>
25+
#include <ydb/core/fq/libs/control_plane_proxy/utils/utils.h>
2626
#include <ydb/public/lib/fq/scope.h>
2727

2828
#include <ydb/library/actors/core/actor.h>

ydb/core/fq/libs/control_plane_proxy/events/events.h

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -394,4 +394,6 @@ struct TEvControlPlaneProxy {
394394
};
395395
};
396396

397+
NActors::TActorId ControlPlaneProxyActorId();
398+
397399
}
Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,9 @@
1+
LIBRARY()
2+
3+
PEERDIR(
4+
ydb/public/api/protos
5+
)
6+
7+
YQL_LAST_ABI_VERSION()
8+
9+
END()

ydb/core/fq/libs/control_plane_proxy/ya.make

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@ END()
3333
RECURSE(
3434
actors
3535
events
36+
utils
3637
)
3738

3839
RECURSE_FOR_TESTS(

ydb/core/fq/libs/events/events.h

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -250,6 +250,8 @@ struct TEvents {
250250
};
251251
};
252252

253+
NActors::TActorId MakeYqPrivateProxyId();
254+
253255
} // namespace NFq
254256

255257
template<>

ydb/core/grpc_services/rpc_fq.cpp

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -4,9 +4,8 @@
44
#include <ydb/core/grpc_services/service_fq.h>
55
#include <ydb/core/fq/libs/audit/events/events.h>
66
#include <ydb/core/fq/libs/audit/yq_audit_service.h>
7-
#include <ydb/core/fq/libs/control_plane_proxy/control_plane_proxy.h>
87
#include <ydb/core/fq/libs/control_plane_proxy/events/events.h>
9-
#include <ydb/core/fq/libs/control_plane_proxy/utils.h>
8+
#include <ydb/core/fq/libs/control_plane_proxy/utils/utils.h>
109
#include <ydb/public/api/protos/draft/fq.pb.h>
1110
#include <ydb/public/lib/fq/scope.h>
1211

ydb/core/grpc_services/rpc_fq_internal.cpp

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -3,7 +3,6 @@
33
#include "rpc_deferrable.h"
44

55
#include <ydb/core/fq/libs/events/events.h>
6-
#include <ydb/core/fq/libs/actors/proxy_private.h>
76
#include <ydb/core/fq/libs/protos/fq_private.pb.h>
87

98
#include <ydb/library/actors/core/hfunc.h>

ydb/core/grpc_services/service_fq.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@
44
#include <memory>
55

66
#include <ydb/core/base/ticket_parser.h>
7-
#include <ydb/core/fq/libs/control_plane_proxy/utils.h>
7+
#include <ydb/core/fq/libs/control_plane_proxy/utils/utils.h>
88

99
namespace NKikimr {
1010
namespace NGRpcService {

ydb/core/grpc_services/ya.make

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -95,8 +95,7 @@ PEERDIR(
9595
ydb/core/discovery
9696
ydb/core/engine
9797
ydb/core/formats
98-
ydb/core/fq/libs/actors
99-
ydb/core/fq/libs/control_plane_proxy
98+
ydb/core/fq/libs/events
10099
ydb/core/fq/libs/control_plane_proxy/events
101100
ydb/core/grpc_services/base
102101
ydb/core/grpc_services/counters

ydb/core/kqp/executer_actor/ya.make

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,6 +26,7 @@ PEERDIR(
2626
ydb/core/client/minikql_compile
2727
ydb/core/formats
2828
ydb/core/kqp/common
29+
ydb/core/kqp/compute_actor
2930
ydb/core/kqp/query_compiler
3031
ydb/core/kqp/rm_service
3132
ydb/core/kqp/topics

ydb/core/kqp/host/kqp_host.cpp

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1474,11 +1474,11 @@ class TKqpHost : public IKqpHost {
14741474
state->CredentialsFactory = FederatedQuerySetup->CredentialsFactory;
14751475
state->Configuration->WriteThroughDqIntegration = true;
14761476
state->Configuration->AllowAtomicUploadCommit = queryType == EKikimrQueryType::Script;
1477-
14781477
state->Configuration->Init(FederatedQuerySetup->S3GatewayConfig, TypesCtx);
1478+
state->Gateway = FederatedQuerySetup->HttpGateway;
14791479

1480-
auto dataSource = NYql::CreateS3DataSource(state, FederatedQuerySetup->HttpGateway);
1481-
auto dataSink = NYql::CreateS3DataSink(state, FederatedQuerySetup->HttpGateway);
1480+
auto dataSource = NYql::CreateS3DataSource(state);
1481+
auto dataSink = NYql::CreateS3DataSink(state);
14821482

14831483
TypesCtx->AddDataSource(NYql::S3ProviderName, std::move(dataSource));
14841484
TypesCtx->AddDataSink(NYql::S3ProviderName, std::move(dataSink));

ydb/core/kqp/node_service/ya.make

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ PEERDIR(
1010
ydb/core/base
1111
ydb/core/cms/console
1212
ydb/core/kqp/common
13+
ydb/core/kqp/compute_actor
1314
ydb/core/kqp/counters
1415
ydb/core/mind
1516
ydb/core/protos

ydb/core/kqp/query_compiler/kqp_query_compiler.cpp

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -944,11 +944,6 @@ class TKqpQueryCompiler : public IKqpQueryCompiler {
944944
NYql::IDqIntegration* dqIntegration = provider->second->GetDqIntegration();
945945
YQL_ENSURE(dqIntegration, "Unsupported dq source for provider: \"" << dataSourceCategory << "\"");
946946
auto& externalSource = *protoSource->MutableExternalSource();
947-
google::protobuf::Any& settings = *externalSource.MutableSettings();
948-
TString& sourceType = *externalSource.MutableType();
949-
dqIntegration->FillSourceSettings(source.Ref(), settings, sourceType);
950-
YQL_ENSURE(!settings.type_url().empty(), "Data source provider \"" << dataSourceCategory << "\" did't fill dq source settings for its dq source node");
951-
YQL_ENSURE(sourceType, "Data source provider \"" << dataSourceCategory << "\" did't fill dq source settings type for its dq source node");
952947

953948
// Partitioning
954949
TVector<TString> partitionParams;
@@ -973,6 +968,12 @@ class TKqpQueryCompiler : public IKqpQueryCompiler {
973968
externalSource.SetAuthInfo(CreateStructuredTokenParser(token).ToBuilder().RemoveSecrets().ToJson());
974969
CreateStructuredTokenParser(token).ListReferences(SecretNames);
975970
}
971+
972+
google::protobuf::Any& settings = *externalSource.MutableSettings();
973+
TString& sourceType = *externalSource.MutableType();
974+
dqIntegration->FillSourceSettings(source.Ref(), settings, sourceType, maxTasksPerStage);
975+
YQL_ENSURE(!settings.type_url().empty(), "Data source provider \"" << dataSourceCategory << "\" didn't fill dq source settings for its dq source node");
976+
YQL_ENSURE(sourceType, "Data source provider \"" << dataSourceCategory << "\" didn't fill dq source settings type for its dq source node");
976977
}
977978
}
978979

ydb/library/yql/dq/actors/compute/dq_compute_actor_checkpoints.cpp

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -151,6 +151,7 @@ STRICT_STFUNC_EXC(TDqComputeActorCheckpoints::StateFunc,
151151
hFunc(TEvDqCompute::TEvRun, Handle);
152152
hFunc(NActors::TEvInterconnect::TEvNodeDisconnected, Handle);
153153
hFunc(NActors::TEvInterconnect::TEvNodeConnected, Handle);
154+
hFunc(NActors::TEvents::TEvUndelivered, Handle);
154155
hFunc(TEvRetryQueuePrivate::TEvRetry, Handle);
155156
hFunc(TEvents::TEvWakeup, Handle);
156157
cFunc(TEvents::TEvPoisonPill::EventType, PassAway);,
@@ -393,6 +394,13 @@ void TDqComputeActorCheckpoints::Handle(NActors::TEvInterconnect::TEvNodeConnect
393394
EventsQueue.HandleNodeConnected(ev->Get()->NodeId);
394395
}
395396

397+
void TDqComputeActorCheckpoints::Handle(NActors::TEvents::TEvUndelivered::TPtr& ev) {
398+
LOG_D("Handle undelivered");
399+
if (!EventsQueue.HandleUndelivered(ev)) {
400+
LOG_E("TEvUndelivered: " << ev->Get()->SourceType);
401+
}
402+
}
403+
396404
void TDqComputeActorCheckpoints::Handle(TEvRetryQueuePrivate::TEvRetry::TPtr& ev) {
397405
Y_UNUSED(ev);
398406
EventsQueue.Retry();

ydb/library/yql/dq/actors/compute/dq_compute_actor_checkpoints.h

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -127,6 +127,7 @@ class TDqComputeActorCheckpoints : public NActors::TActor<TDqComputeActorCheckpo
127127
void Handle(NActors::TEvents::TEvPoison::TPtr&);
128128
void Handle(NActors::TEvInterconnect::TEvNodeDisconnected::TPtr& ev);
129129
void Handle(NActors::TEvInterconnect::TEvNodeConnected::TPtr& ev);
130+
void Handle(NActors::TEvents::TEvUndelivered::TPtr& ev);
130131
void Handle(TEvRetryQueuePrivate::TEvRetry::TPtr& ev);
131132
void Handle(NActors::TEvents::TEvWakeup::TPtr& ev);
132133
void HandleException(const std::exception& err);

ydb/library/yql/dq/actors/compute/retry_queue.cpp

Lines changed: 10 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,16 @@ void TRetryEventsQueue::HandleNodeConnected(ui32 nodeId) {
4747
}
4848
}
4949

50+
bool TRetryEventsQueue::HandleUndelivered(NActors::TEvents::TEvUndelivered::TPtr& ev) {
51+
if (ev->Sender == RecipientId && ev->Get()->Reason == NActors::TEvents::TEvUndelivered::Disconnected) {
52+
Connected = false;
53+
ScheduleRetry();
54+
return true;
55+
}
56+
57+
return false;
58+
}
59+
5060
void TRetryEventsQueue::Retry() {
5161
RetryScheduled = false;
5262
if (!Connected) {

ydb/library/yql/dq/actors/compute/retry_queue.h

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -120,10 +120,16 @@ class TRetryEventsQueue {
120120
}
121121
return false;
122122
}
123+
124+
bool RemoveConfirmedEvents() {
125+
RemoveConfirmedEvents(MyConfirmedSeqNo);
126+
return !Events.empty();
127+
}
123128

124129
void OnNewRecipientId(const NActors::TActorId& recipientId, bool unsubscribe = true);
125130
void HandleNodeConnected(ui32 nodeId);
126131
void HandleNodeDisconnected(ui32 nodeId);
132+
bool HandleUndelivered(NActors::TEvents::TEvUndelivered::TPtr& ev);
127133
void Retry();
128134
void Unsubscribe();
129135

@@ -160,7 +166,7 @@ class TRetryEventsQueue {
160166
THolder<T> ev = MakeHolder<T>();
161167
ev->Record = Event->Record;
162168
ev->Record.MutableTransportMeta()->SetConfirmedSeqNo(confirmedSeqNo);
163-
return MakeHolder<NActors::IEventHandle>(Recipient, Sender, ev.Release(), 0, Cookie);
169+
return MakeHolder<NActors::IEventHandle>(Recipient, Sender, ev.Release(), NActors::IEventHandle::FlagTrackDelivery, Cookie);
164170
}
165171

166172
private:

ydb/library/yql/dq/integration/yql_dq_integration.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -62,7 +62,7 @@ class IDqIntegration {
6262
virtual bool CanBlockRead(const NNodes::TExprBase& node, TExprContext& ctx, TTypeAnnotationContext& typesCtx) = 0;
6363
virtual void RegisterMkqlCompiler(NCommon::TMkqlCallableCompilerBase& compiler) = 0;
6464
virtual bool CanFallback() = 0;
65-
virtual void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sourceType) = 0;
65+
virtual void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sourceType, size_t maxPartitions) = 0;
6666
virtual void FillSinkSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sinkType) = 0;
6767
virtual void FillTransformSettings(const TExprNode& node, ::google::protobuf::Any& settings) = 0;
6868
virtual void Annotate(const TExprNode& node, THashMap<TString, TString>& params) = 0;

ydb/library/yql/providers/clickhouse/provider/yql_clickhouse_dq_integration.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -75,7 +75,7 @@ class TClickHouseDqIntegration: public TDqIntegrationBase {
7575
return 0ULL;
7676
}
7777

78-
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& protoSettings, TString& sourceType) override {
78+
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& protoSettings, TString& sourceType, size_t) override {
7979
const TDqSource source(&node);
8080
if (const auto maySettings = source.Settings().Maybe<TClSourceSettings>()) {
8181
const auto settings = maySettings.Cast();

ydb/library/yql/providers/common/dq/yql_dq_integration_impl.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -51,7 +51,7 @@ bool TDqIntegrationBase::CanFallback() {
5151
return false;
5252
}
5353

54-
void TDqIntegrationBase::FillSourceSettings(const TExprNode&, ::google::protobuf::Any&, TString&) {
54+
void TDqIntegrationBase::FillSourceSettings(const TExprNode&, ::google::protobuf::Any&, TString&, size_t) {
5555
}
5656

5757
void TDqIntegrationBase::FillSinkSettings(const TExprNode&, ::google::protobuf::Any&, TString&) {

ydb/library/yql/providers/common/dq/yql_dq_integration_impl.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,7 @@ class TDqIntegrationBase: public IDqIntegration {
1818
bool CanBlockRead(const NNodes::TExprBase& node, TExprContext& ctx, TTypeAnnotationContext& typesCtx) override;
1919
TExprNode::TPtr WrapWrite(const TExprNode::TPtr& write, TExprContext& ctx) override;
2020
bool CanFallback() override;
21-
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sourceType) override;
21+
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sourceType, size_t) override;
2222
void FillSinkSettings(const TExprNode& node, ::google::protobuf::Any& settings, TString& sinkType) override;
2323
void FillTransformSettings(const TExprNode& node, ::google::protobuf::Any& settings) override;
2424
void Annotate(const TExprNode& node, THashMap<TString, TString>& params) override;

ydb/library/yql/providers/dq/planner/execution_planner.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -554,7 +554,7 @@ namespace NYql::NDqs {
554554
TString sourceType;
555555
if (dqSource) {
556556
sourceSettings.ConstructInPlace();
557-
dqIntegration->FillSourceSettings(*read, *sourceSettings, sourceType);
557+
dqIntegration->FillSourceSettings(*read, *sourceSettings, sourceType, maxPartitions);
558558
YQL_ENSURE(!sourceSettings->type_url().empty(), "Data source provider \"" << dataSourceName << "\" did't fill dq source settings for its dq source node");
559559
YQL_ENSURE(sourceType, "Data source provider \"" << dataSourceName << "\" did't fill dq source settings type for its dq source node");
560560
}

ydb/library/yql/providers/generic/provider/ut/pushdown/pushdown_ut.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -180,7 +180,7 @@ class TBuildDqSourceSettingsTransformer: public TOptimizeTransformerBase {
180180
.Ptr();
181181
::google::protobuf::Any settings;
182182
TString sourceType;
183-
dqIntegration->FillSourceSettings(*dqSourceNode, settings, sourceType);
183+
dqIntegration->FillSourceSettings(*dqSourceNode, settings, sourceType, 1);
184184
UNIT_ASSERT_STRINGS_EQUAL(sourceType, "PostgreSqlGeneric");
185185
UNIT_ASSERT(settings.Is<Generic::TSource>());
186186
settings.UnpackTo(DqSourceSettings_);

ydb/library/yql/providers/generic/provider/yql_generic_dq_integration.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -93,7 +93,7 @@ namespace NYql {
9393
}
9494

9595
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& protoSettings,
96-
TString& sourceType) override {
96+
TString& sourceType, size_t) override {
9797
const TDqSource source(&node);
9898
if (const auto maybeSettings = source.Settings().Maybe<TGenSourceSettings>()) {
9999
const auto settings = maybeSettings.Cast();

ydb/library/yql/providers/pq/provider/yql_pq_dq_integration.cpp

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -179,7 +179,7 @@ class TPqDqIntegration: public TDqIntegrationBase {
179179
}
180180
}
181181

182-
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& protoSettings, TString& sourceType) override {
182+
void FillSourceSettings(const TExprNode& node, ::google::protobuf::Any& protoSettings, TString& sourceType, size_t) override {
183183
if (auto maybeDqSource = TMaybeNode<TDqSource>(&node)) {
184184
auto settings = maybeDqSource.Cast().Settings();
185185
if (auto maybeTopicSource = TMaybeNode<TDqPqTopicSource>(settings.Raw())) {

ydb/library/yql/providers/s3/actors/ya.make

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@ PEERDIR(
2424
library/cpp/string_utils/base64
2525
library/cpp/string_utils/quote
2626
library/cpp/xml/document
27+
ydb/core/base
2728
ydb/core/fq/libs/events
2829
ydb/library/yql/dq/actors/compute
2930
ydb/library/yql/minikql/computation
@@ -37,6 +38,8 @@ PEERDIR(
3738
ydb/library/yql/providers/s3/credentials
3839
ydb/library/yql/providers/s3/object_listers
3940
ydb/library/yql/providers/s3/proto
41+
ydb/library/yql/providers/s3/range_helpers
42+
ydb/library/yql/public/issue
4043
ydb/library/yql/public/types
4144
ydb/library/yql/udfs/common/clickhouse/client
4245
)

0 commit comments

Comments
 (0)