Skip to content

Commit 335f07a

Browse files
authored
Apply arcadia changes (#12106)
1 parent 9abfdac commit 335f07a

File tree

15 files changed

+68
-63
lines changed

15 files changed

+68
-63
lines changed

ydb/core/kqp/opt/kqp_opt_kql.cpp

+1-1
Original file line numberDiff line numberDiff line change
@@ -1017,7 +1017,7 @@ TMaybe<TKqlQueryList> BuildKqlQuery(TKiDataQueryBlocks dataQueryBlocks, const TK
10171017
auto dataSource = typesCtx.DataSourceMap.FindPtr(dataSourceName);
10181018
YQL_ENSURE(dataSource);
10191019
if (auto dqIntegration = (*dataSource)->GetDqIntegration()) {
1020-
auto newRead = dqIntegration->WrapRead(NYql::TDqSettings(), input.Cast().Ptr(), ctx);
1020+
auto newRead = dqIntegration->WrapRead(input.Cast().Ptr(), ctx, {});
10211021
if (newRead.Get() != input.Raw()) {
10221022
return newRead;
10231023
}

ydb/core/kqp/query_compiler/kqp_query_compiler.cpp

+4-1
Original file line numberDiff line numberDiff line change
@@ -1092,7 +1092,10 @@ class TKqpQueryCompiler : public IKqpQueryCompiler {
10921092
// We prepare a lot of partitions and distribute them between these tasks
10931093
// Constraint of 1 task per partition is NOT valid anymore
10941094
auto maxTasksPerStage = Config->MaxTasksPerStage.Get().GetOrElse(TDqSettings::TDefault::MaxTasksPerStage);
1095-
dqIntegration->Partition(NYql::TDqSettings(), maxTasksPerStage, source.Ref(), partitionParams, &clusterName, ctx, false);
1095+
IDqIntegration::TPartitionSettings pSettings;
1096+
pSettings.MaxPartitions = maxTasksPerStage;
1097+
pSettings.CanFallback = false;
1098+
dqIntegration->Partition(source.Ref(), partitionParams, &clusterName, ctx, pSettings);
10961099
externalSource.SetTaskParamKey(TString(dataSourceCategory));
10971100
for (const TString& partitionParam : partitionParams) {
10981101
externalSource.AddPartitionedTaskParams(partitionParam);

ydb/library/yql/dq/opt/dq_opt_log.cpp

+2-2
Original file line numberDiff line numberDiff line change
@@ -336,7 +336,7 @@ NNodes::TExprBase DqReplicateFieldSubset(NNodes::TExprBase node, TExprContext& c
336336
return node;
337337
}
338338

339-
IGraphTransformer::TStatus DqWrapIO(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, TTypeAnnotationContext& typesCtx, const TDqSettings& config) {
339+
IGraphTransformer::TStatus DqWrapIO(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, TTypeAnnotationContext& typesCtx, const IDqIntegration::TWrapReadSettings& wrSettings) {
340340
TOptimizeExprSettings settings{&typesCtx};
341341
auto status = OptimizeExpr(input, output, [&](const TExprNode::TPtr& node, TExprContext& ctx) {
342342
if (auto maybeRead = TMaybeNode<TCoRight>(node).Input()) {
@@ -345,7 +345,7 @@ IGraphTransformer::TStatus DqWrapIO(const TExprNode::TPtr& input, TExprNode::TPt
345345
auto dataSource = typesCtx.DataSourceMap.FindPtr(dataSourceName);
346346
YQL_ENSURE(dataSource);
347347
if (auto dqIntegration = (*dataSource)->GetDqIntegration()) {
348-
auto newRead = dqIntegration->WrapRead(config, maybeRead.Cast().Ptr(), ctx);
348+
auto newRead = dqIntegration->WrapRead(maybeRead.Cast().Ptr(), ctx, wrSettings);
349349
if (newRead.Get() != maybeRead.Raw()) {
350350
return newRead;
351351
}

ydb/library/yql/dq/opt/dq_opt_log.h

+2-2
Original file line numberDiff line numberDiff line change
@@ -1,5 +1,6 @@
11
#pragma once
22

3+
#include <yql/essentials/core/dq_integration/yql_dq_integration.h>
34
#include <yql/essentials/ast/yql_expr.h>
45
#include <yql/essentials/core/expr_nodes/yql_expr_nodes.h>
56
#include <yql/essentials/core/expr_nodes_gen/yql_expr_nodes_gen.h>
@@ -11,7 +12,6 @@
1112
namespace NYql {
1213
class IOptimizationContext;
1314
struct TTypeAnnotationContext;
14-
struct TDqSettings;
1515
struct IProviderContext;
1616
struct TRelOptimizerNode;
1717
struct TOptimizerStatistics;
@@ -38,7 +38,7 @@ NNodes::TExprBase DqSqlInDropCompact(NNodes::TExprBase node, TExprContext& ctx);
3838

3939
NNodes::TExprBase DqReplicateFieldSubset(NNodes::TExprBase node, TExprContext& ctx, const TParentsMap& parents);
4040

41-
IGraphTransformer::TStatus DqWrapIO(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, TTypeAnnotationContext& typesCtx, const TDqSettings& config);
41+
IGraphTransformer::TStatus DqWrapIO(const TExprNode::TPtr& input, TExprNode::TPtr& output, TExprContext& ctx, TTypeAnnotationContext& typesCtx, const IDqIntegration::TWrapReadSettings& wrSettings);
4242

4343
NNodes::TExprBase DqExpandMatchRecognize(NNodes::TExprBase node, TExprContext& ctx, TTypeAnnotationContext& typeAnnCtx);
4444

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

+2-2
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@ class TClickHouseDqIntegration: public TDqIntegrationBase {
3333
return Nothing();
3434
}
3535

36-
TExprNode::TPtr WrapRead(const TDqSettings&, const TExprNode::TPtr& read, TExprContext& ctx) override {
36+
TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings& ) override {
3737
if (const auto maybeClReadTable = TMaybeNode<TClReadTable>(read)) {
3838
const auto clReadTable = maybeClReadTable.Cast();
3939
const auto token = TString("cluster:default_") += clReadTable.DataSource().Cluster().StringValue();
@@ -66,7 +66,7 @@ class TClickHouseDqIntegration: public TDqIntegrationBase {
6666
return read;
6767
}
6868

69-
ui64 Partition(const TDqSettings&, size_t, const TExprNode&, TVector<TString>& partitions, TString*, TExprContext&, bool) override {
69+
ui64 Partition(const TExprNode&, TVector<TString>& partitions, TString*, TExprContext&, const TPartitionSettings&) override {
7070
partitions.clear();
7171
NCH::TRange range;
7272
// range.SetRange("limit 42 offset 42 order by ...."); // Possible set range like this.

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

+8-2
Original file line numberDiff line numberDiff line change
@@ -544,7 +544,13 @@ namespace NYql::NDqs {
544544
TVector<TString> parts;
545545
if (auto dqIntegration = (*datasource)->GetDqIntegration()) {
546546
TString clusterName;
547-
_MaxDataSizePerJob = Max(_MaxDataSizePerJob, dqIntegration->Partition(*Settings, maxPartitions, *read, parts, &clusterName, ExprContext, canFallback));
547+
IDqIntegration::TPartitionSettings settings {
548+
.DataSizePerJob = Settings->DataSizePerJob.Get(),
549+
.MaxPartitions = maxPartitions,
550+
.EnableComputeActor = Settings->EnableComputeActor.Get(),
551+
.CanFallback = canFallback
552+
};
553+
_MaxDataSizePerJob = Max(_MaxDataSizePerJob, dqIntegration->Partition(*read, parts, &clusterName, ExprContext, settings));
548554
TMaybe<::google::protobuf::Any> sourceSettings;
549555
TString sourceType;
550556
if (dqSource) {
@@ -585,7 +591,7 @@ namespace NYql::NDqs {
585591
YQL_ENSURE(dataSource);
586592
auto dqIntegration = (*dataSource)->GetDqIntegration();
587593
YQL_ENSURE(dqIntegration);
588-
594+
589595
google::protobuf::Any providerSpecificLookupSourceSettings;
590596
TString sourceType;
591597
dqIntegration->FillLookupSourceSettings(*rightInput.Raw(), providerSpecificLookupSourceSettings, sourceType);

ydb/library/yql/providers/dq/provider/yql_dq_recapture.cpp

+7-1
Original file line numberDiff line numberDiff line change
@@ -97,7 +97,13 @@ class TDqsRecaptureTransformer : public TSyncTransformerBase {
9797

9898
State_->TypeCtx->DqFallbackPolicy = State_->Settings->FallbackPolicy.Get().GetOrElse(EFallbackPolicy::Default);
9999

100-
IGraphTransformer::TStatus status = NDq::DqWrapIO(input, output, ctx, *State_->TypeCtx, *State_->Settings);
100+
IDqIntegration::TWrapReadSettings wrSettings {
101+
.WatermarksMode = State_->Settings->WatermarksMode.Get(),
102+
.WatermarksGranularityMs = State_->Settings->WatermarksGranularityMs.Get(),
103+
.WatermarksLateArrivalDelayMs = State_->Settings->WatermarksLateArrivalDelayMs.Get(),
104+
.WatermarksEnableIdlePartitions = State_->Settings->WatermarksEnableIdlePartitions.Get()
105+
};
106+
IGraphTransformer::TStatus status = NDq::DqWrapIO(input, output, ctx, *State_->TypeCtx, wrSettings);
101107
if (input != output) {
102108
YQL_CLOG(INFO, ProviderDq) << "DqsRecapture";
103109
// TODO: Add before/after recapture transformers

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

+1-1
Original file line numberDiff line numberDiff line change
@@ -162,7 +162,7 @@ class TBuildDqSourceSettingsTransformer: public TOptimizeTransformerBase {
162162
UNIT_ASSERT(genericDataSource != Types->DataSourceMap.end());
163163
auto dqIntegration = genericDataSource->second->GetDqIntegration();
164164
UNIT_ASSERT(dqIntegration);
165-
auto newRead = dqIntegration->WrapRead(TDqSettings(), input.Ptr(), ctx);
165+
auto newRead = dqIntegration->WrapRead(input.Ptr(), ctx, IDqIntegration::TWrapReadSettings{});
166166
BuildSettings(newRead, dqIntegration, ctx);
167167
return newRead;
168168
}

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

+2-3
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,7 @@ namespace NYql {
6363
return Nothing();
6464
}
6565

66-
TExprNode::TPtr WrapRead(const TDqSettings&, const TExprNode::TPtr& read, TExprContext& ctx) override {
66+
TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings&) override {
6767
if (const auto maybeGenReadTable = TMaybeNode<TGenReadTable>(read)) {
6868
const auto genReadTable = maybeGenReadTable.Cast();
6969
YQL_ENSURE(genReadTable.Ref().GetTypeAnn(), "No type annotation for node " << genReadTable.Ref().Content());
@@ -106,8 +106,7 @@ namespace NYql {
106106
return read;
107107
}
108108

109-
ui64 Partition(const TDqSettings&, size_t, const TExprNode&, TVector<TString>& partitions, TString*, TExprContext&,
110-
bool) override {
109+
ui64 Partition(const TExprNode&, TVector<TString>& partitions, TString*, TExprContext&, const TPartitionSettings&) override {
111110
partitions.clear();
112111
Generic::TRange range;
113112
partitions.emplace_back();

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

+14-16
Original file line numberDiff line numberDiff line change
@@ -59,20 +59,20 @@ class TPqDqIntegration: public TDqIntegrationBase {
5959
return 0;
6060
}
6161

62-
ui64 Partition(const TDqSettings&, size_t maxPartitions, const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, bool) override {
62+
ui64 Partition(const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, const TPartitionSettings& settings) override {
6363
if (auto maybePqRead = TMaybeNode<TPqReadTopic>(&node)) {
64-
return PartitionTopicRead(maybePqRead.Cast().Topic(), maxPartitions, partitions);
64+
return PartitionTopicRead(maybePqRead.Cast().Topic(), settings.MaxPartitions, partitions);
6565
}
6666
if (auto maybeDqSource = TMaybeNode<TDqSource>(&node)) {
67-
auto settings = maybeDqSource.Cast().Settings();
68-
if (auto topicSource = TMaybeNode<TDqPqTopicSource>(settings.Raw())) {
69-
return PartitionTopicRead(topicSource.Cast().Topic(), maxPartitions, partitions);
67+
auto srcSettings = maybeDqSource.Cast().Settings();
68+
if (auto topicSource = TMaybeNode<TDqPqTopicSource>(srcSettings.Raw())) {
69+
return PartitionTopicRead(topicSource.Cast().Topic(), settings.MaxPartitions, partitions);
7070
}
7171
}
7272
return 0;
7373
}
7474

75-
TExprNode::TPtr WrapRead(const TDqSettings& dqSettings, const TExprNode::TPtr& read, TExprContext& ctx) override {
75+
TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings& wrSettings) override {
7676
if (const auto& maybePqReadTopic = TMaybeNode<TPqReadTopic>(read)) {
7777
const auto& pqReadTopic = maybePqReadTopic.Cast();
7878
YQL_ENSURE(pqReadTopic.Ref().GetTypeAnn(), "No type annotation for node " << pqReadTopic.Ref().Content());
@@ -127,7 +127,7 @@ class TPqDqIntegration: public TDqIntegrationBase {
127127

128128
const auto& typeItems = pqReadTopic.Topic().RowSpec().Ref().GetTypeAnn()->Cast<TTypeExprType>()->GetType()->Cast<TStructExprType>()->GetItems();
129129
const auto pos = read->Pos();
130-
130+
131131
TExprNode::TListType colNames;
132132
colNames.reserve(typeItems.size());
133133
std::transform(typeItems.cbegin(), typeItems.cend(), std::back_inserter(colNames),
@@ -146,7 +146,7 @@ class TPqDqIntegration: public TDqIntegrationBase {
146146
.World(pqReadTopic.World())
147147
.Topic(pqReadTopic.Topic())
148148
.Columns(std::move(columnNames))
149-
.Settings(BuildTopicReadSettings(clusterName, dqSettings, read->Pos(), format, ctx))
149+
.Settings(BuildTopicReadSettings(clusterName, wrSettings, read->Pos(), format, ctx))
150150
.Token<TCoSecureParam>()
151151
.Name().Build(token)
152152
.Build()
@@ -325,7 +325,7 @@ class TPqDqIntegration: public TDqIntegrationBase {
325325

326326
NNodes::TCoNameValueTupleList BuildTopicReadSettings(
327327
const TString& cluster,
328-
const TDqSettings& dqSettings,
328+
const IDqIntegration::TWrapReadSettings& wrSettings,
329329
TPositionHandle pos,
330330
std::string_view format,
331331
TExprContext& ctx) const
@@ -349,7 +349,7 @@ class TPqDqIntegration: public TDqIntegrationBase {
349349
Add(props, ReconnectPeriod, ToString(clusterConfiguration->ReconnectPeriod), pos, ctx);
350350
Add(props, Format, format, pos, ctx);
351351

352-
352+
353353
if (clusterConfiguration->UseSsl) {
354354
Add(props, UseSslSetting, "1", pos, ctx);
355355
}
@@ -358,23 +358,21 @@ class TPqDqIntegration: public TDqIntegrationBase {
358358
Add(props, AddBearerToTokenSetting, "1", pos, ctx);
359359
}
360360

361-
if (dqSettings.WatermarksMode.Get().GetOrElse("") == "default") {
361+
if (wrSettings.WatermarksMode.GetOrElse("") == "default") {
362362
Add(props, WatermarksEnableSetting, ToString(true), pos, ctx);
363363

364-
const auto granularity = TDuration::MilliSeconds(dqSettings
364+
const auto granularity = TDuration::MilliSeconds(wrSettings
365365
.WatermarksGranularityMs
366-
.Get()
367366
.GetOrElse(TDqSettings::TDefault::WatermarksGranularityMs));
368367
Add(props, WatermarksGranularityUsSetting, ToString(granularity.MicroSeconds()), pos, ctx);
369368

370-
const auto lateArrivalDelay = TDuration::MilliSeconds(dqSettings
369+
const auto lateArrivalDelay = TDuration::MilliSeconds(wrSettings
371370
.WatermarksLateArrivalDelayMs
372-
.Get()
373371
.GetOrElse(TDqSettings::TDefault::WatermarksLateArrivalDelayMs));
374372
Add(props, WatermarksLateArrivalDelayUsSetting, ToString(lateArrivalDelay.MicroSeconds()), pos, ctx);
375373
}
376374

377-
if (dqSettings.WatermarksEnableIdlePartitions.Get().GetOrElse(false)) {
375+
if (wrSettings.WatermarksEnableIdlePartitions.GetOrElse(false)) {
378376
Add(props, WatermarksIdlePartitionsSetting, ToString(true), pos, ctx);
379377
}
380378

ydb/library/yql/providers/s3/provider/yql_s3_dq_integration.cpp

+5-4
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ class TS3DqIntegration: public TDqIntegrationBase {
8282
{
8383
}
8484

85-
ui64 Partition(const TDqSettings&, size_t maxPartitions, const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, bool) override {
85+
ui64 Partition(const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, const TPartitionSettings& settings) override {
8686
std::vector<std::vector<TPath>> parts;
8787
std::optional<ui64> mbLimitHint;
8888
bool hasDirectories = false;
@@ -108,6 +108,7 @@ class TS3DqIntegration: public TDqIntegrationBase {
108108
}
109109

110110
constexpr ui64 maxTaskRatio = 20;
111+
auto maxPartitions = settings.MaxPartitions;
111112
if (!maxPartitions || (mbLimitHint && maxPartitions > *mbLimitHint / maxTaskRatio)) {
112113
maxPartitions = std::max(*mbLimitHint / maxTaskRatio, ui64{1});
113114
YQL_CLOG(TRACE, ProviderS3) << "limited max partitions to " << maxPartitions;
@@ -223,15 +224,15 @@ class TS3DqIntegration: public TDqIntegrationBase {
223224
}
224225

225226
rows = size / 1024; // magic estimate
226-
return primaryKey
227+
return primaryKey
227228
? TOptimizerStatistics(BaseTable, rows, cols, size, size, TIntrusivePtr<TOptimizerStatistics::TKeyColumns>(new TOptimizerStatistics::TKeyColumns(*primaryKey)))
228229
: TOptimizerStatistics(BaseTable, rows, cols, size, size);
229230
} else {
230231
return Nothing();
231232
}
232233
}
233234

234-
TExprNode::TPtr WrapRead(const TDqSettings&, const TExprNode::TPtr& read, TExprContext& ctx) override {
235+
TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings& ) override {
235236
if (const auto& maybeS3ReadObject = TMaybeNode<TS3ReadObject>(read)) {
236237
const auto& s3ReadObject = maybeS3ReadObject.Cast();
237238
YQL_ENSURE(s3ReadObject.Ref().GetTypeAnn(), "No type annotation for node " << s3ReadObject.Ref().Content());
@@ -394,7 +395,7 @@ class TS3DqIntegration: public TDqIntegrationBase {
394395
TExprContext ctx;
395396
srcDesc.SetRowType(NCommon::WriteTypeToYson(ctx.MakeType<TStructExprType>(rowTypeItems), NYT::NYson::EYsonFormat::Text));
396397
}
397-
398+
398399
if (auto predicate = parseSettings.FilterPredicate(); !IsEmptyFilterPredicate(predicate)) {
399400
TStringBuilder err;
400401
if (!SerializeFilterPredicate(predicate, srcDesc.mutable_predicate(), err)) {

ydb/library/yql/providers/solomon/provider/yql_solomon_dq_integration.cpp

+2-3
Original file line numberDiff line numberDiff line change
@@ -79,8 +79,7 @@ class TSolomonDqIntegration: public TDqIntegrationBase {
7979
{
8080
}
8181

82-
ui64 Partition(const TDqSettings&, size_t maxPartitions, const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, bool) override {
83-
Y_UNUSED(maxPartitions);
82+
ui64 Partition(const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, const TPartitionSettings&) override {
8483
Y_UNUSED(node);
8584
Y_UNUSED(partitions);
8685
partitions.push_back("zz_partition");
@@ -95,7 +94,7 @@ class TSolomonDqIntegration: public TDqIntegrationBase {
9594
YQL_ENSURE(false, "Unimplemented");
9695
}
9796

98-
TExprNode::TPtr WrapRead(const TDqSettings&, const TExprNode::TPtr& read, TExprContext& ctx) override {
97+
TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings&) override {
9998
if (const auto& maybeSoReadObject = TMaybeNode<TSoReadObject>(read)) {
10099
const auto& soReadObject = maybeSoReadObject.Cast();
101100
YQL_ENSURE(soReadObject.Ref().GetTypeAnn(), "No type annotation for node " << soReadObject.Ref().Content());

ydb/library/yql/providers/ydb/provider/yql_ydb_dq_integration.cpp

+4-4
Original file line numberDiff line numberDiff line change
@@ -23,8 +23,7 @@ class TYdbDqIntegration: public TDqIntegrationBase {
2323
{
2424
}
2525

26-
ui64 Partition(const TDqSettings& settings, size_t maxPartitions, const TExprNode& node,
27-
TVector<TString>& partitions, TString*, TExprContext&, bool) override {
26+
ui64 Partition(const TExprNode& node, TVector<TString>& partitions, TString*, TExprContext&, const TPartitionSettings& settings) override {
2827
TString cluster, table;
2928
if (const TMaybeNode<TDqSource> source = &node) {
3029
cluster = source.Cast().DataSource().Cast<TYdbDataSource>().Cluster().Value();
@@ -35,9 +34,10 @@ class TYdbDqIntegration: public TDqIntegrationBase {
3534
}
3635

3736
auto& meta = State_->Tables[std::make_pair(cluster, table)];
38-
meta.ReadAsync = settings.EnableComputeActor.Get().GetOrElse(false); // TODO: Use special method for get settings.
37+
meta.ReadAsync = settings.EnableComputeActor.GetOrElse(false); // TODO: Use special method for get settings.
3938
auto parts = meta.Partitions;
4039

40+
auto maxPartitions = settings.MaxPartitions;
4141
if (maxPartitions && parts.size() > maxPartitions) {
4242
if (const auto extraParts = parts.size() - maxPartitions; extraParts > maxPartitions) {
4343
const auto dropsPerTask = (parts.size() - 1ULL) / maxPartitions;
@@ -80,7 +80,7 @@ class TYdbDqIntegration: public TDqIntegrationBase {
8080
return Nothing();
8181
}
8282

83-
TExprNode::TPtr WrapRead(const TDqSettings&, const TExprNode::TPtr& read, TExprContext& ctx) override {
83+
TExprNode::TPtr WrapRead(const TExprNode::TPtr& read, TExprContext& ctx, const TWrapReadSettings&) override {
8484
if (const auto& maybeYdbReadTable = TMaybeNode<TYdbReadTable>(read)) {
8585
const auto& ydbReadTable = maybeYdbReadTable.Cast();
8686
YQL_ENSURE(ydbReadTable.Ref().GetTypeAnn(), "No type annotation for node " << ydbReadTable.Ref().Content());

0 commit comments

Comments
 (0)