summaryrefslogtreecommitdiffstats
path: root/yql/essentials/core
diff options
context:
space:
mode:
authorAlexander Smirnov <[email protected]>2025-05-29 11:09:23 +0000
committerAlexander Smirnov <[email protected]>2025-05-29 11:09:23 +0000
commita34a6816abefdcfe2c00295edb510cc5c99ad52c (patch)
treea264baadccf7add09a1b285786307ddd774472a5 /yql/essentials/core
parent84ec9093e10073ab151bfe5f81037a0d017c2362 (diff)
parentfdbc38349df2ee0ddc678fa2bffe84786f9639a3 (diff)
Merge branch 'rightlib' into merge-libs-250529-1108
Diffstat (limited to 'yql/essentials/core')
-rw-r--r--yql/essentials/core/common_opt/yql_co_finalizers.cpp4
-rw-r--r--yql/essentials/core/common_opt/yql_co_flow2.cpp6
-rw-r--r--yql/essentials/core/common_opt/yql_co_simple1.cpp6
-rw-r--r--yql/essentials/core/common_opt/yql_co_simple2.cpp46
-rw-r--r--yql/essentials/core/common_opt/yql_flatmap_over_join.cpp2
-rw-r--r--yql/essentials/core/expr_nodes/yql_expr_nodes.json15
-rw-r--r--yql/essentials/core/file_storage/file_storage.cpp4
-rw-r--r--yql/essentials/core/file_storage/ya.make1
-rw-r--r--yql/essentials/core/peephole_opt/yql_opt_peephole_physical.cpp29
-rw-r--r--yql/essentials/core/type_ann/type_ann_core.cpp5
-rw-r--r--yql/essentials/core/yql_opt_utils.cpp4
-rw-r--r--yql/essentials/core/yql_opt_utils.h1
-rw-r--r--yql/essentials/core/yql_type_annotation.h2
-rw-r--r--yql/essentials/core/yql_udf_index.cpp3
-rw-r--r--yql/essentials/core/yql_udf_index.h2
15 files changed, 88 insertions, 42 deletions
diff --git a/yql/essentials/core/common_opt/yql_co_finalizers.cpp b/yql/essentials/core/common_opt/yql_co_finalizers.cpp
index 0e616abd9c1..933b0bb9310 100644
--- a/yql/essentials/core/common_opt/yql_co_finalizers.cpp
+++ b/yql/essentials/core/common_opt/yql_co_finalizers.cpp
@@ -112,8 +112,8 @@ void FilterPushdownWithMultiusage(const TExprNode::TPtr& node, TNodeOnNodeOwnedM
if (auto cond = parentFlatMap.Lambda().Body().Maybe<TCoConditionalValueBase>()) {
const TCoArgument lambdaArg = parentFlatMap.Lambda().Args().Arg(0);
auto pred = cond.Cast().Predicate();
- if (pred.Maybe<TCoLikely>() ||
- (pred.Maybe<TCoAnd>() && AnyOf(pred.Ref().ChildrenList(), [](const auto& p) { return p->IsCallable("Likely"); })) ||
+ if (pred.Maybe<TCoNoPushBase>() ||
+ (pred.Maybe<TCoAnd>() && AnyOf(pred.Ref().ChildrenList(), [](const auto& p) { return IsNoPush(*p); })) ||
!IsStrict(pred.Ptr()) ||
HasDependsOn(pred.Ptr(), lambdaArg.Ptr()) ||
IsDepended(parentFlatMap.Lambda().Ref(), *node))
diff --git a/yql/essentials/core/common_opt/yql_co_flow2.cpp b/yql/essentials/core/common_opt/yql_co_flow2.cpp
index 1bd95745579..7099574b572 100644
--- a/yql/essentials/core/common_opt/yql_co_flow2.cpp
+++ b/yql/essentials/core/common_opt/yql_co_flow2.cpp
@@ -1684,7 +1684,7 @@ TExprBase FilterOverAggregate(const TCoFlatMapBase& node, TExprContext& ctx, TOp
size_t separableComponents = 0;
for (auto& p : andComponents) {
TSet<TStringBuf> usedFields;
- if (p->IsCallable("Likely") ||
+ if (IsNoPush(*p) ||
HasDependsOn(p, arg.Ptr()) ||
!HaveFieldsSubset(p, arg.Ref(), usedFields, *optCtx.ParentsMap) ||
!AllOf(usedFields, [&](TStringBuf field) { return keyColumns.contains(field); }) ||
@@ -1701,7 +1701,7 @@ TExprBase FilterOverAggregate(const TCoFlatMapBase& node, TExprContext& ctx, TOp
size_t maxKeyPredicates = 0;
if (AllowComplexFiltersOverAggregatePushdown(optCtx)) {
for (auto& p : restComponents) {
- if (p->IsCallable("Likely")) {
+ if (IsNoPush(*p)) {
continue;
}
const TNodeMap<ESubgraphType> marked = MarkSubgraphForAggregate(p, arg, keyColumns);
@@ -1740,7 +1740,7 @@ TExprBase FilterOverAggregate(const TCoFlatMapBase& node, TExprContext& ctx, TOp
calculator->DropCache();
}
nonSeparableComponents += canPush;
- p = ctx.WrapByCallableIf(canPush, "Likely", std::move(p));
+ p = ctx.WrapByCallableIf(canPush, "NoPush", std::move(p));
}
}
diff --git a/yql/essentials/core/common_opt/yql_co_simple1.cpp b/yql/essentials/core/common_opt/yql_co_simple1.cpp
index f109c4fb289..99534916211 100644
--- a/yql/essentials/core/common_opt/yql_co_simple1.cpp
+++ b/yql/essentials/core/common_opt/yql_co_simple1.cpp
@@ -659,7 +659,7 @@ TExprNode::TPtr RemoveOptionalReduceOverData(const TExprNode::TPtr& node, TExprC
}
TExprNode::TPtr PropagateCoalesceWithConstIntoLogicalOps(const TExprNode::TPtr& node, TExprContext& ctx) {
- if (node->Head().IsCallable({"Likely", "AssumeStrict", "AssumeNonStrict"})) {
+ if (node->Head().IsCallable({"NoPush", "Likely", "AssumeStrict", "AssumeNonStrict"})) {
const auto value = FromString<bool>(node->Child(1)->Head().Content());
if (!value) {
YQL_CLOG(DEBUG, Core) << "PropagateCoalesceWithConst over " << node->Head().Content() << " (false)";
@@ -6950,8 +6950,8 @@ void RegisterCoSimpleCallables1(TCallableOptimizerMap& map) {
return node;
};
- map["Likely"] = [](const TExprNode::TPtr& node, TExprContext& /*ctx*/, TOptimizeContext& /*optCtx*/) {
- if (node->Head().IsCallable("Likely")) {
+ map["Likely"] = map["NoPush"] = [](const TExprNode::TPtr& node, TExprContext& /*ctx*/, TOptimizeContext& /*optCtx*/) {
+ if (IsNoPush(node->Head())) {
YQL_CLOG(DEBUG, Core) << node->Content() << " over " << node->Head().Content();
return node->HeadPtr();
}
diff --git a/yql/essentials/core/common_opt/yql_co_simple2.cpp b/yql/essentials/core/common_opt/yql_co_simple2.cpp
index 2adcdb6f22e..e7628c076cb 100644
--- a/yql/essentials/core/common_opt/yql_co_simple2.cpp
+++ b/yql/essentials/core/common_opt/yql_co_simple2.cpp
@@ -350,18 +350,18 @@ void DropDups(TExprNode::TListType& children) {
}
}
-void StripLikely(TExprNodeList& args, TNodeOnNodeOwnedMap& likelyArgs) {
+void StripNoPush(TExprNodeList& args, TNodeOnNodeOwnedMap& noPushArgs) {
for (auto& arg : args) {
- if (arg->IsCallable("Likely")) {
- likelyArgs[arg->Child(0)] = arg;
+ if (IsNoPush(*arg)) {
+ noPushArgs[arg->Child(0)] = arg;
arg = arg->HeadPtr();
}
}
}
-void UnstripLikely(TExprNodeList& args, const TNodeOnNodeOwnedMap& likelyArgs) {
+void UnstripNoPush(TExprNodeList& args, const TNodeOnNodeOwnedMap& noPushArgs) {
for (auto& arg : args) {
- if (auto it = likelyArgs.find(arg.Get()); it != likelyArgs.end()) {
+ if (auto it = noPushArgs.find(arg.Get()); it != noPushArgs.end()) {
arg = it->second;
}
}
@@ -369,11 +369,11 @@ void UnstripLikely(TExprNodeList& args, const TNodeOnNodeOwnedMap& likelyArgs) {
TExprNode::TPtr OptimizeDups(const TExprNode::TPtr& node, TExprContext& ctx) {
auto children = node->ChildrenList();
- TNodeOnNodeOwnedMap likelyArgs;
- StripLikely(children, likelyArgs);
+ TNodeOnNodeOwnedMap noPushArgs;
+ StripNoPush(children, noPushArgs);
DropDups(children);
if (children.size() < node->ChildrenSize()) {
- UnstripLikely(children, likelyArgs);
+ UnstripNoPush(children, noPushArgs);
YQL_CLOG(DEBUG, Core) << node->Content() << " with " << node->ChildrenSize() - children.size() << " dups";
return 1U == children.size() ? children.front() : ctx.ChangeChildren(*node, std::move(children));
}
@@ -517,7 +517,7 @@ bool AllOrNoneOr(const TExprNodeList& children) {
}
TExprNodeList GetOrAndChildren(TExprNode::TPtr node, bool visitOr) {
- if (node->IsCallable("Likely")) {
+ if (IsNoPush(*node)) {
node = node->HeadPtr();
}
if (visitOr && node->IsCallable("Or") || !visitOr && node->IsCallable("And")) {
@@ -573,8 +573,8 @@ TVector<TVector<size_t>> SplitToNonIntersectingGroups(const TExprNodeList& child
TExprNode::TPtr ApplyAndAbsorption(const TExprNode::TPtr& node, TExprContext& ctx) {
YQL_ENSURE(node->IsCallable("And"));
TExprNodeList children = node->ChildrenList();
- TNodeOnNodeOwnedMap likelyPreds;
- StripLikely(children, likelyPreds);
+ TNodeOnNodeOwnedMap noPushPreds;
+ StripNoPush(children, noPushPreds);
if (AllOrNoneOr(children)) {
return node;
}
@@ -608,7 +608,7 @@ TExprNode::TPtr ApplyAndAbsorption(const TExprNode::TPtr& node, TExprContext& ct
newChildren.push_back(children[i]);
}
}
- UnstripLikely(newChildren, likelyPreds);
+ UnstripNoPush(newChildren, noPushPreds);
bool addJust = node->GetTypeAnn()->GetKind() == ETypeAnnotationKind::Optional &&
AllOf(newChildren, [](const auto& node) {
YQL_ENSURE(node->GetTypeAnn());
@@ -661,7 +661,7 @@ TExprNode::TPtr ApplyOrAbsorption(const TExprNode::TPtr& node, TExprContext& ctx
for (auto& idx : andIndexes) {
TExprNodeList andChildren = children[idx]->ChildrenList();
bool haveCommonFactor = AnyOf(andChildren, [&](TExprNode::TPtr child) {
- if (child->IsCallable("Likely")) {
+ if (IsNoPush(*child)) {
child = child->HeadPtr();
}
TExprNodeList orList = GetOrChildren(child);
@@ -714,13 +714,13 @@ TExprNode::TPtr ApplyOrDistributive(const TExprNode::TPtr& node, TExprContext& c
}
TExprNodeList commonPreds = children[group.front()]->ChildrenList();
- TNodeOnNodeOwnedMap likelyPreds;
- StripLikely(commonPreds, likelyPreds);
+ TNodeOnNodeOwnedMap noPushPreds;
+ StripNoPush(commonPreds, noPushPreds);
Sort(commonPreds, ptrComparator);
for (size_t i = 1; i < group.size() && !commonPreds.empty(); ++i) {
TExprNodeList curr = children[group[i]]->ChildrenList();
- StripLikely(curr, likelyPreds);
+ StripNoPush(curr, noPushPreds);
Sort(curr, ptrComparator);
TExprNodeList intersected;
@@ -735,7 +735,7 @@ TExprNode::TPtr ApplyOrDistributive(const TExprNode::TPtr& node, TExprContext& c
for (const auto& c : commonPreds) {
commonSet.insert(c.Get());
}
- UnstripLikely(commonPreds, likelyPreds);
+ UnstripNoPush(commonPreds, noPushPreds);
// stabilize common predicate order
Sort(commonPreds, [](const auto& l, const auto& r) { return CompareNodes(*l, *r) < 0; });
@@ -743,7 +743,7 @@ TExprNode::TPtr ApplyOrDistributive(const TExprNode::TPtr& node, TExprContext& c
for (auto& idx : group) {
auto childAnd = children[idx];
TExprNodeList preds = childAnd->ChildrenList();
- EraseIf(preds, [&](const TExprNode::TPtr& p) { return commonSet.contains(p->IsCallable("Likely") ? p->Child(0) : p.Get()); });
+ EraseIf(preds, [&](const TExprNode::TPtr& p) { return commonSet.contains(IsNoPush(*p) ? p->Child(0) : p.Get()); });
if (!preds.empty()) {
newGroup.emplace_back(ctx.ChangeChildren(*childAnd, std::move(preds)));
}
@@ -772,13 +772,13 @@ TExprNode::TPtr OptimizeOr(const TExprNode::TPtr& node, TExprContext& ctx, TOpti
return opt;
}
- TNodeOnNodeOwnedMap likelyPreds;
+ TNodeOnNodeOwnedMap noPushPreds;
TExprNodeList children = node->ChildrenList();
- StripLikely(children, likelyPreds);
- if (!likelyPreds.empty()) {
+ StripNoPush(children, noPushPreds);
+ if (!noPushPreds.empty()) {
// Likely(A) OR B -> Likely(A OR B)
- YQL_CLOG(DEBUG, Core) << "Or with Likely argument";
- return ctx.NewCallable(node->Pos(), "Likely", { ctx.ChangeChildren(*node, std::move(children)) });
+ YQL_CLOG(DEBUG, Core) << "Or with NoPush argument";
+ return ctx.NewCallable(node->Pos(), "NoPush", { ctx.ChangeChildren(*node, std::move(children)) });
}
if (IsExtractCommonPredicatesFromLogicalOpsEnabled(optCtx)) {
diff --git a/yql/essentials/core/common_opt/yql_flatmap_over_join.cpp b/yql/essentials/core/common_opt/yql_flatmap_over_join.cpp
index fb9dbb9a934..1ebe47096e2 100644
--- a/yql/essentials/core/common_opt/yql_flatmap_over_join.cpp
+++ b/yql/essentials/core/common_opt/yql_flatmap_over_join.cpp
@@ -1524,7 +1524,7 @@ TExprBase FlatMapOverEquiJoin(
const bool skipNulls = NeedEmitSkipNullMembers(types);
for (const auto& andTerm : andTerms) {
- if (andTerm->IsCallable("Likely")) {
+ if (IsNoPush(*andTerm)) {
continue;
}
diff --git a/yql/essentials/core/expr_nodes/yql_expr_nodes.json b/yql/essentials/core/expr_nodes/yql_expr_nodes.json
index 740fae8474f..c62b447217e 100644
--- a/yql/essentials/core/expr_nodes/yql_expr_nodes.json
+++ b/yql/essentials/core/expr_nodes/yql_expr_nodes.json
@@ -2680,12 +2680,23 @@
"ListBase": "TCoReplicationTarget"
},
{
- "Name" : "TCoLikely",
+ "Name" : "TCoNoPushBase",
"Base" : "TCallable",
- "Match": {"Type": "Callable", "Name": "Likely"},
+ "Match": {"Type": "CallableBase"},
+ "Builder": {"Generate": "None"},
"Children": [
{"Index": 0, "Name": "Predicate", "Type": "TExprBase"}
]
+ },
+ {
+ "Name" : "TCoNoPush",
+ "Base" : "TCoNoPushBase",
+ "Match": {"Type": "Callable", "Name": "NoPush"}
+ },
+ {
+ "Name" : "TCoLikely",
+ "Base" : "TCoNoPushBase",
+ "Match": {"Type": "Callable", "Name": "Likely"}
}
]
}
diff --git a/yql/essentials/core/file_storage/file_storage.cpp b/yql/essentials/core/file_storage/file_storage.cpp
index 734226f7e59..de24524ac30 100644
--- a/yql/essentials/core/file_storage/file_storage.cpp
+++ b/yql/essentials/core/file_storage/file_storage.cpp
@@ -8,6 +8,8 @@
#include <yql/essentials/core/file_storage/http_download/http_download.h>
#include <yql/essentials/core/file_storage/defs/provider.h>
+#include <yql/essentials/public/issue/yql_issue.h>
+
#include <yql/essentials/utils/fetch/fetch.h>
#include <yql/essentials/utils/log/log.h>
#include <yql/essentials/utils/log/context.h>
@@ -153,7 +155,7 @@ public:
} catch (const std::exception& e) {
const TString msg = TStringBuilder() << "FileStorage: Failed to download file by URL \"" << urlStr << "\", details: " << e.what();
YQL_LOG(ERROR) << msg;
- YQL_LOG_CTX_THROW yexception() << msg;
+ YQL_LOG_CTX_THROW TErrorException(UNEXPECTED_ERROR) << msg;
}
}
diff --git a/yql/essentials/core/file_storage/ya.make b/yql/essentials/core/file_storage/ya.make
index 2fa051cf90d..0bc16b99d45 100644
--- a/yql/essentials/core/file_storage/ya.make
+++ b/yql/essentials/core/file_storage/ya.make
@@ -23,6 +23,7 @@ PEERDIR(
yql/essentials/core/file_storage/defs
yql/essentials/core/file_storage/download
yql/essentials/core/file_storage/http_download
+ yql/essentials/public/issue
yql/essentials/utils
yql/essentials/utils/log
yql/essentials/utils/fetch
diff --git a/yql/essentials/core/peephole_opt/yql_opt_peephole_physical.cpp b/yql/essentials/core/peephole_opt/yql_opt_peephole_physical.cpp
index 666f1c6183d..2759510b09a 100644
--- a/yql/essentials/core/peephole_opt/yql_opt_peephole_physical.cpp
+++ b/yql/essentials/core/peephole_opt/yql_opt_peephole_physical.cpp
@@ -32,6 +32,8 @@ namespace {
using namespace NNodes;
+constexpr size_t WideLimit = 101;
+
using TPeepHoleOptimizerPtr = TExprNode::TPtr (*const)(const TExprNode::TPtr&, TExprContext&);
using TPeepHoleOptimizerMap = std::unordered_map<std::string_view, TPeepHoleOptimizerPtr>;
@@ -2533,7 +2535,7 @@ TExprNode::TPtr ExpandFlatMap(const TExprNode::TPtr& node, TExprContext& ctx) {
}
if (const auto kind = node->Head().GetTypeAnn()->GetKind(); (kind == ETypeAnnotationKind::Flow || kind == ETypeAnnotationKind::List) &&
- body.IsCallable("AsList") && body.ChildrenSize() > 1U) {
+ body.IsCallable("AsList") && body.ChildrenSize() > 1U && body.ChildrenSize() <= WideLimit) {
constexpr auto multimap = Ordered ? "OrderedMultiMap" : "MultiMap";
YQL_CLOG(DEBUG, CorePeepHole) << "Expand " << node->Content() << " as " << multimap << " of size " << body.ChildrenSize();
return ctx.NewCallable(node->Pos(), multimap, {node->HeadPtr(), ctx.DeepCopyLambda(lambda, body.ChildrenList())});
@@ -3495,13 +3497,16 @@ TExprNode::TPtr OptimizeMap(const TExprNode::TPtr& node, TExprContext& ctx) {
}
TExprNode::TPtr MakeWideTableSource(const TExprNode& tableSource, TExprContext& ctx, TVector<TString>* narrowMapColumns = nullptr) {
- // TODO check wide limit
if (tableSource.GetTypeAnn()->GetKind() != ETypeAnnotationKind::List) {
return nullptr;
}
- YQL_CLOG(DEBUG, CorePeepHole) << "Generate WideTableSource";
auto structType = tableSource.GetTypeAnn()->Cast<TListExprType>()->GetItemType()->Cast<TStructExprType>();
+ if (structType->GetSize() > WideLimit) {
+ return nullptr;
+ }
+
+ YQL_CLOG(DEBUG, CorePeepHole) << "Generate WideTableSource";
TVector<TString> columns;
for (const auto& item : structType->GetItems()) {
columns.push_back(TString(item->GetName()));
@@ -5660,6 +5665,10 @@ TExprNode::TPtr OptimizeWideCombiner(const TExprNode::TPtr& node, TExprContext&
if (needKeyFlatten.front()) {
const auto flattenSize = *needKeyFlatten.front();
+ if (flattenSize > WideLimit) {
+ return node;
+ }
+
YQL_CLOG(DEBUG, CorePeepHole) << "Flatten key by tuple for " << node->Content() << " from " << originalKeySize << " to " << flattenSize;
auto children = node->ChildrenList();
@@ -5673,6 +5682,10 @@ TExprNode::TPtr OptimizeWideCombiner(const TExprNode::TPtr& node, TExprContext&
if (needKeyFlatten.back()) {
const auto flattenSize = *needKeyFlatten.back();
+ if (flattenSize > WideLimit) {
+ return node;
+ }
+
YQL_CLOG(DEBUG, CorePeepHole) << "Flatten key by struct for " << node->Content() << " from " << originalKeySize << " to " << flattenSize;
auto children = node->ChildrenList();
@@ -5694,6 +5707,10 @@ TExprNode::TPtr OptimizeWideCombiner(const TExprNode::TPtr& node, TExprContext&
if (needStateFlatten.front()) {
const auto flattenSize = *needStateFlatten.front();
+ if (flattenSize > WideLimit) {
+ return node;
+ }
+
YQL_CLOG(DEBUG, CorePeepHole) << "Flatten state by tuple for " << node->Content() << " from " << originalStateSize << " to " << flattenSize;
auto children = node->ChildrenList();
@@ -5707,6 +5724,10 @@ TExprNode::TPtr OptimizeWideCombiner(const TExprNode::TPtr& node, TExprContext&
if (needStateFlatten.back()) {
const auto flattenSize = *needStateFlatten.back();
+ if (flattenSize > WideLimit) {
+ return node;
+ }
+
YQL_CLOG(DEBUG, CorePeepHole) << "Flatten state by struct for " << node->Content() << " from " << originalStateSize << " to " << flattenSize;
auto children = node->ChildrenList();
@@ -6144,7 +6165,7 @@ bool CollectBlockRewrites(const TMultiExprType* multiInputType, bool keepInputCo
TExprNode::TListType funcArgs;
std::string_view arrowFunctionName;
- const bool rewriteAsIs = node->IsCallable({"AssumeStrict", "AssumeNonStrict", "Likely"});
+ const bool rewriteAsIs = node->IsCallable({"AssumeStrict", "AssumeNonStrict", "NoPush", "Likely"});
if (node->IsList() || rewriteAsIs ||
node->IsCallable({"DecimalMul", "DecimalDiv", "DecimalMod", "And", "Or", "Xor", "Not", "Coalesce", "Exists", "If", "Just", "AsStruct", "Member", "Nth", "ToPg", "FromPg", "PgResolvedCall", "PgResolvedOp"}))
{
diff --git a/yql/essentials/core/type_ann/type_ann_core.cpp b/yql/essentials/core/type_ann/type_ann_core.cpp
index b6a5cbfbb11..d8c9267479c 100644
--- a/yql/essentials/core/type_ann/type_ann_core.cpp
+++ b/yql/essentials/core/type_ann/type_ann_core.cpp
@@ -3733,7 +3733,7 @@ namespace NTypeAnnImpl {
return IGraphTransformer::TStatus::Ok;
}
- IGraphTransformer::TStatus LikelyWrapper(const TExprNode::TPtr& input, TExprNode::TPtr& output, TContext& ctx) {
+ IGraphTransformer::TStatus NoPushWrapper(const TExprNode::TPtr& input, TExprNode::TPtr& output, TContext& ctx) {
if (!EnsureArgsCount(*input, 1, ctx.Expr)) {
return IGraphTransformer::TStatus::Error;
}
@@ -12604,7 +12604,8 @@ template <NKikimr::NUdf::EDataSlot DataSlot>
Functions["Or"] = &LogicalWrapper<false>;
Functions["Xor"] = &LogicalWrapper<true>;
Functions["Not"] = &BoolOpt1Wrapper;
- Functions["Likely"] = &LikelyWrapper;
+ Functions["NoPush"] = &NoPushWrapper;
+ Functions["Likely"] = &NoPushWrapper;
Functions["Map"] = &MapWrapper;
Functions["OrderedMap"] = &MapWrapper;
Functions["MapNext"] = &MapNextWrapper;
diff --git a/yql/essentials/core/yql_opt_utils.cpp b/yql/essentials/core/yql_opt_utils.cpp
index 9a51b068010..0b956ea5de6 100644
--- a/yql/essentials/core/yql_opt_utils.cpp
+++ b/yql/essentials/core/yql_opt_utils.cpp
@@ -355,6 +355,10 @@ bool IsTablePropsDependent(const TExprNode& node) {
return found;
}
+bool IsNoPush(const TExprNode& node) {
+ return node.IsCallable({"NoPush", "Likely"});
+}
+
TExprNode::TPtr KeepColumnOrder(const TExprNode::TPtr& node, const TExprNode& src, TExprContext& ctx, const TTypeAnnotationContext& typeCtx) {
auto columnOrder = typeCtx.LookupColumnOrder(src);
if (!columnOrder) {
diff --git a/yql/essentials/core/yql_opt_utils.h b/yql/essentials/core/yql_opt_utils.h
index fd818dbd353..447d53737d1 100644
--- a/yql/essentials/core/yql_opt_utils.h
+++ b/yql/essentials/core/yql_opt_utils.h
@@ -26,6 +26,7 @@ bool IsRenameOrApplyFlatMapWithMapping(const NNodes::TCoFlatMapBase& node, TExpr
bool IsPassthroughFlatMap(const NNodes::TCoFlatMapBase& flatmap, TMaybe<THashSet<TStringBuf>>* passthroughFields, bool analyzeJustMember = false);
bool IsPassthroughLambda(const NNodes::TCoLambda& lambda, TMaybe<THashSet<TStringBuf>>* passthroughFields, bool analyzeJustMember = false);
bool IsTablePropsDependent(const TExprNode& node);
+bool IsNoPush(const TExprNode& node);
bool HasOnlyOneJoinType(const TExprNode& joinTree, TStringBuf joinType);
diff --git a/yql/essentials/core/yql_type_annotation.h b/yql/essentials/core/yql_type_annotation.h
index d07d5afea84..ce70891e2e0 100644
--- a/yql/essentials/core/yql_type_annotation.h
+++ b/yql/essentials/core/yql_type_annotation.h
@@ -384,7 +384,7 @@ inline TString GetRandomKey<TGUID>() {
struct TTypeAnnotationContext: public TThrRefBase {
TSimpleSharedPtr<NDq::TOrderingsStateMachine> OrderingsFSM;
- TLangVersion LangVer = UnknownLangVersion;
+ TLangVersion LangVer = MinLangVersion;
THashMap<TString, TIntrusivePtr<TOptimizerStatistics::TColumnStatMap>> ColumnStatisticsByTableName;
THashMap<ui64, std::shared_ptr<TOptimizerStatistics>> StatisticsMap;
TIntrusivePtr<ITimeProvider> TimeProvider;
diff --git a/yql/essentials/core/yql_udf_index.cpp b/yql/essentials/core/yql_udf_index.cpp
index f861e9eca65..c0b62596b07 100644
--- a/yql/essentials/core/yql_udf_index.cpp
+++ b/yql/essentials/core/yql_udf_index.cpp
@@ -53,6 +53,9 @@ TVector<TResourceInfo::TPtr> ConvertResolveResultToResources(const TResolveResul
newFunction.Messages.push_back(m);
}
+ newFunction.MinLangVer = udf.GetMinLangVer();
+ newFunction.MaxLangVer = udf.GetMaxLangVer();
+
functionIndex[package].push_back(newFunction);
}
diff --git a/yql/essentials/core/yql_udf_index.h b/yql/essentials/core/yql_udf_index.h
index 4f2f33404ca..315c21e4be6 100644
--- a/yql/essentials/core/yql_udf_index.h
+++ b/yql/essentials/core/yql_udf_index.h
@@ -20,6 +20,8 @@ struct TFunctionInfo {
bool IsStrict = false;
bool SupportsBlocks = false;
TVector<TString> Messages;
+ TLangVersion MinLangVer = UnknownLangVersion;
+ TLangVersion MaxLangVer = UnknownLangVersion;
};
// todo: specify whether path is frozen