diff options
author | Alexander Smirnov <[email protected]> | 2025-05-29 11:09:23 +0000 |
---|---|---|
committer | Alexander Smirnov <[email protected]> | 2025-05-29 11:09:23 +0000 |
commit | a34a6816abefdcfe2c00295edb510cc5c99ad52c (patch) | |
tree | a264baadccf7add09a1b285786307ddd774472a5 /yql/essentials/core | |
parent | 84ec9093e10073ab151bfe5f81037a0d017c2362 (diff) | |
parent | fdbc38349df2ee0ddc678fa2bffe84786f9639a3 (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.cpp | 4 | ||||
-rw-r--r-- | yql/essentials/core/common_opt/yql_co_flow2.cpp | 6 | ||||
-rw-r--r-- | yql/essentials/core/common_opt/yql_co_simple1.cpp | 6 | ||||
-rw-r--r-- | yql/essentials/core/common_opt/yql_co_simple2.cpp | 46 | ||||
-rw-r--r-- | yql/essentials/core/common_opt/yql_flatmap_over_join.cpp | 2 | ||||
-rw-r--r-- | yql/essentials/core/expr_nodes/yql_expr_nodes.json | 15 | ||||
-rw-r--r-- | yql/essentials/core/file_storage/file_storage.cpp | 4 | ||||
-rw-r--r-- | yql/essentials/core/file_storage/ya.make | 1 | ||||
-rw-r--r-- | yql/essentials/core/peephole_opt/yql_opt_peephole_physical.cpp | 29 | ||||
-rw-r--r-- | yql/essentials/core/type_ann/type_ann_core.cpp | 5 | ||||
-rw-r--r-- | yql/essentials/core/yql_opt_utils.cpp | 4 | ||||
-rw-r--r-- | yql/essentials/core/yql_opt_utils.h | 1 | ||||
-rw-r--r-- | yql/essentials/core/yql_type_annotation.h | 2 | ||||
-rw-r--r-- | yql/essentials/core/yql_udf_index.cpp | 3 | ||||
-rw-r--r-- | yql/essentials/core/yql_udf_index.h | 2 |
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 |