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 | |
parent | 84ec9093e10073ab151bfe5f81037a0d017c2362 (diff) | |
parent | fdbc38349df2ee0ddc678fa2bffe84786f9639a3 (diff) |
Merge branch 'rightlib' into merge-libs-250529-1108
Diffstat (limited to 'yql/essentials')
98 files changed, 1351 insertions, 314 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 diff --git a/yql/essentials/docs/en/changelog/2025.03.md b/yql/essentials/docs/en/changelog/2025.03.md new file mode 100644 index 00000000000..e85da1f9c28 --- /dev/null +++ b/yql/essentials/docs/en/changelog/2025.03.md @@ -0,0 +1,3 @@ +## Changes in RE2 module {#re2-module} + +* Queries are guaranteed to fail when invalid regular expressions are passed diff --git a/yql/essentials/docs/en/changelog/toc_i.yaml b/yql/essentials/docs/en/changelog/toc_i.yaml index 968c0814ab5..37879aedee0 100644 --- a/yql/essentials/docs/en/changelog/toc_i.yaml +++ b/yql/essentials/docs/en/changelog/toc_i.yaml @@ -5,3 +5,5 @@ items: href: 2025.01.md - name: 2025.02 href: 2025.02.md +- name: 2025.03 + href: 2025.03.md diff --git a/yql/essentials/docs/en/udf/list/re2.md b/yql/essentials/docs/en/udf/list/re2.md index ab527b012a0..f83dd976c13 100644 --- a/yql/essentials/docs/en/udf/list/re2.md +++ b/yql/essentials/docs/en/udf/list/re2.md @@ -16,6 +16,13 @@ The Re2 module supports regular expressions based on [google::RE2](https://githu By default, the UTF-8 mode is enabled automatically if the regular expression is a valid UTF-8-encoded string, but is not a valid ASCII string. You can manually control the settings of the re2 library, if you pass the result of the `Re2::Options` function as the second argument to other module functions, next to the regular expression. +{% note info "Note" %} + +All regular expressions passed to functions must be valid. Otherwise, your query may fail. +Starting from version [2025.03](../../changelog/2025.03.md#re2-module), such a query will definitely fail. + +{% endnote %} + {% note warning %} Make sure to double all the backslashes in your regular expressions (if they are within a quoted string): standard string literals are treated as C-escaped strings in SQL. You can also format regular expressions as raw strings `@@regexp@@`: double slashes are not needed in this case. @@ -88,7 +95,7 @@ Notes on Re2::Options from the official [repository](https://github.com/google/r | CaseSensitive:Bool? | true | match is case-sensitive (regexp can override with (?i) unless in posix_syntax mode) | | DotNl:Bool? | false | let `.` match `\n` (default ) | | Literal:Bool? | false | interpret string as literal, not regexp | -| LogErrors:Bool? | true | log syntax and execution errors to ERROR | +| LogErrors:Bool? | true | this option is ignored | | LongestMatch:Bool? | false | search for longest match, not first match | | MaxMem:Uint64? | - | (see below) approx. max memory footprint of RE2 | | NeverCapture:Bool? | false | parse all parents as non-capturing | @@ -119,4 +126,3 @@ SELECT ``` In both cases, the word FOO will be found. Using the raw string @@regexp@@ lets you avoid double slashes. - diff --git a/yql/essentials/docs/ru/changelog/2025.03.md b/yql/essentials/docs/ru/changelog/2025.03.md new file mode 100644 index 00000000000..0a25c97b15b --- /dev/null +++ b/yql/essentials/docs/ru/changelog/2025.03.md @@ -0,0 +1,3 @@ +## Изменения в модуле RE2 {#re2-module} + +* Гарантируется падение запросов при передаче невалидных регулярных выражений diff --git a/yql/essentials/docs/ru/changelog/toc_i.yaml b/yql/essentials/docs/ru/changelog/toc_i.yaml index 9a1dac426db..bd294b79bbc 100644 --- a/yql/essentials/docs/ru/changelog/toc_i.yaml +++ b/yql/essentials/docs/ru/changelog/toc_i.yaml @@ -5,3 +5,5 @@ items: href: 2025.01.md - name: 2025.02 href: 2025.02.md +- name: 2025.03 + href: 2025.03.md diff --git a/yql/essentials/docs/ru/udf/list/math.md b/yql/essentials/docs/ru/udf/list/math.md index 467a8d1704c..6c5c811a3b0 100644 --- a/yql/essentials/docs/ru/udf/list/math.md +++ b/yql/essentials/docs/ru/udf/list/math.md @@ -148,15 +148,15 @@ SELECT Math::Rem(-1, 7); -- -1 #### Список функций -* `Math::RoundDownward() -> Tagged<Uint32, MathRoundingMode>` -- rounding towards negative infinity -* `Math::RoundToNearest() -> Tagged<Uint32, MathRoundingMode>` -- rounding towards nearest representable value -* `Math::RoundTowardZero() -> Tagged<Uint32, MathRoundingMode>` -- rounding towards zero -* `Math::RoundUpward() -> Tagged<Uint32, MathRoundingMode>` -- rounding towards positive infinity -* `Math::NearbyInt(AutoMap<Double>, Tagged<Uint32, MathRoundingMode>) -> Optional<Int64>` +* `Math::RoundDownward() -> Tagged<Uint32, MathRoundingMode>` — округление в сторону отрицательной бесконечности +* `Math::RoundToNearest() -> Tagged<Uint32, MathRoundingMode>` — округление в сторону ближайшего представимого значения +* `Math::RoundTowardZero() -> Tagged<Uint32, MathRoundingMode>` — округление в сторону нуля +* `Math::RoundUpward() -> Tagged<Uint32, MathRoundingMode>` — округление в сторону положительной бесконечности +* `Math::NearbyInt(AutoMap<Double>, Tagged<Uint32, MathRoundingMode>) -> Optional<Int64>` — округление до ближайшего целого значения -Функция `Math::NearbyInt` округляет первый аргумент до целого числа в соответсвии с режимом, заданным вторым аргументом. +Функция `Math::NearbyInt` округляет первый аргумент до целого числа в соответствии с режимом, заданным вторым аргументом. -Если результат выходит за пределы 64-битного целого числа, возращается NULL. +Если результат выходит за пределы 64-битного целого числа, возвращается NULL. #### Примеры diff --git a/yql/essentials/docs/ru/udf/list/re2.md b/yql/essentials/docs/ru/udf/list/re2.md index bece3053081..3c0906f30e1 100644 --- a/yql/essentials/docs/ru/udf/list/re2.md +++ b/yql/essentials/docs/ru/udf/list/re2.md @@ -16,6 +16,13 @@ Re2::Options([CaseSensitive:Bool?,DotNl:Bool?,Literal:Bool?,LogErrors:Bool?,Long По умолчанию UTF-8 режим включается автоматически, если регулярное выражение является валидной строкой в кодировке UTF-8, но не является валидной ASCII-строкой. Вручную настройками библиотеки re2 можно управлять с помощью передачи результата функции `Re2::Options` вторым аргументом другим функциям модуля, рядом с регулярным выражением. +{% note info "Примечание" %} + +Все регулярные выражения, переданные в функции, должны быть валидными. Иначе ваш запрос может упасть. +Начиная с версии [2025.03](../../changelog/2025.03.md#re2-module) такой запрос гарантированно завершится с ошибкой. + +{% endnote %} + {% note warning %} Все обратные слеши в регулярных выражениях (если они записаны в строке с кавычками) нужно удваивать, так как стандартные строковые литералы в SQL рассматриваются как С-escaped строки. Также можно записывать регулярное выражение в форме raw строки `@@regexp@@` — в этом случае удвоение слешей не требуется. @@ -88,7 +95,7 @@ SELECT | CaseSensitive:Bool? | true | match is case-sensitive (regexp can override with (?i) unless in posix_syntax mode) | | DotNl:Bool? | false | let `.` match `\n` (default ) | | Literal:Bool? | false | interpret string as literal, not regexp | -| LogErrors:Bool? | true | log syntax and execution errors to ERROR | +| LogErrors:Bool? | true | this option is ignored | | LongestMatch:Bool? | false | search for longest match, not first match | | MaxMem:Uint64? | - | (see below) approx. max memory footprint of RE2 | | NeverCapture:Bool? | false | parse all parens as non-capturing | diff --git a/yql/essentials/minikql/comp_nodes/mkql_udf.cpp b/yql/essentials/minikql/comp_nodes/mkql_udf.cpp index 0036091ad77..8230a759d6b 100644 --- a/yql/essentials/minikql/comp_nodes/mkql_udf.cpp +++ b/yql/essentials/minikql/comp_nodes/mkql_udf.cpp @@ -37,12 +37,14 @@ public: TString&& typeConfig, NUdf::TSourcePosition pos, const TCallableType* callableType, + const TCallableType* functionType, TType* userType) : TBaseComputation(mutables, EValueRepresentation::Boxed) , FunctionName(std::move(functionName)) , TypeConfig(std::move(typeConfig)) , Pos(pos) , CallableType(callableType) + , FunctionType(functionType) , UserType(userType) { this->Stateless = false; @@ -65,16 +67,55 @@ public: } NUdf::TUnboxedValue udf(NUdf::TUnboxedValuePod(funcInfo.Implementation.Release())); - TValidate<TValidatePolicy,TValidateMode>::WrapCallable(CallableType, udf, TStringBuilder() << "FunctionWithConfig<" << FunctionName << ">"); + TValidate<TValidatePolicy,TValidateMode>::WrapCallable(FunctionType, udf, TStringBuilder() << "FunctionWithConfig<" << FunctionName << ">"); + ExtendArgs(udf, CallableType, funcInfo.FunctionType); return udf.Release(); } private: + // xXX: This class implements the wrapper to properly handle + // the case when the signature of the emitted callable (i.e. + // callable type) requires less arguments than the actual + // function (i.e. function type). It wraps the unboxed value + // with the resolved UDF to introduce the bridge in the + // Run chain, preparing the valid argument vector for the + // chosen UDF implementation. + class TExtendedArgsWrapper: public NUdf::TBoxedValue { + public: + TExtendedArgsWrapper(NUdf::TUnboxedValue&& callable, size_t usedArgs, size_t requiredArgs) + : Callable_(callable) + , UsedArgs_(usedArgs) + , RequiredArgs_(requiredArgs) + {}; + + private: + NUdf::TUnboxedValue Run(const NUdf::IValueBuilder* valueBuilder, const NUdf::TUnboxedValuePod* args) const final { + NStackArray::TStackArray<NUdf::TUnboxedValue> values(ALLOC_ON_STACK(NUdf::TUnboxedValue, RequiredArgs_)); + for (size_t i = 0; i < UsedArgs_; i++) { + values[i] = args[i]; + } + return Callable_.Run(valueBuilder, values.data()); + } + + const NUdf::TUnboxedValue Callable_; + const size_t UsedArgs_; + const size_t RequiredArgs_; + }; + + void ExtendArgs(NUdf::TUnboxedValue& callable, const TCallableType* callableType, const TCallableType* functionType) const { + const auto callableArgc = callableType->GetArgumentsCount(); + const auto functionArgc = functionType->GetArgumentsCount(); + if (callableArgc < functionArgc) { + callable = NUdf::TUnboxedValuePod(new TExtendedArgsWrapper(std::move(callable), callableArgc, functionArgc)); + } + } + void RegisterDependencies() const final {} const TString FunctionName; const TString TypeConfig; const NUdf::TSourcePosition Pos; const TCallableType *const CallableType; + const TCallableType *const FunctionType; TType *const UserType; }; @@ -90,12 +131,13 @@ public: TString&& typeConfig, NUdf::TSourcePosition pos, const TCallableType* callableType, + const TCallableType* functionType, TType* userType, TString&& moduleIRUniqID, TString&& moduleIR, TString&& fuctioNameIR, NUdf::TUniquePtr<NUdf::IBoxedValue>&& impl) - : TSimpleUdfWrapper(mutables, std::move(functionName), std::move(typeConfig), pos, callableType, userType) + : TSimpleUdfWrapper(mutables, std::move(functionName), std::move(typeConfig), pos, callableType, functionType, userType) , ModuleIRUniqID(std::move(moduleIRUniqID)) , ModuleIR(std::move(moduleIR)) , IRFunctionName(std::move(fuctioNameIR)) @@ -138,7 +180,7 @@ public: NUdf::TSourcePosition pos, IComputationNode* runConfigNode, ui32 runConfigArgs, - const TCallableType* callableType, + const TCallableType* functionType, TType* userType) : TBaseComputation(mutables, EValueRepresentation::Boxed) , FunctionName(std::move(functionName)) @@ -146,7 +188,7 @@ public: , Pos(pos) , RunConfigNode(runConfigNode) , RunConfigArgs(runConfigArgs) - , CallableType(callableType) + , FunctionType(functionType) , UserType(userType) , UdfIndex(mutables.CurValueIndex++) { @@ -238,7 +280,7 @@ private: } void Wrap(NUdf::TUnboxedValue& callable) const { - TValidate<TValidatePolicy,TValidateMode>::WrapCallable(CallableType, callable, TStringBuilder() << "FunctionWithConfig<" << FunctionName << ">"); + TValidate<TValidatePolicy,TValidateMode>::WrapCallable(FunctionType, callable, TStringBuilder() << "FunctionWithConfig<" << FunctionName << ">"); } void RegisterDependencies() const final { @@ -250,7 +292,7 @@ private: const NUdf::TSourcePosition Pos; IComputationNode* const RunConfigNode; const ui32 RunConfigArgs; - const TCallableType* CallableType; + const TCallableType* FunctionType; TType* const UserType; const ui32 UdfIndex; }; @@ -317,6 +359,8 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont << status.GetError()).c_str()); } + const auto callableFuncType = AS_TYPE(TCallableType, funcInfo.FunctionType); + const auto callableNodeType = AS_TYPE(TCallableType, callable.GetType()->GetReturnType()); const auto runConfigFuncType = funcInfo.RunConfigType; const auto runConfigNodeType = runCfgNode.GetStaticType(); @@ -324,7 +368,8 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont // It's only legal, when the compiled UDF declares its // signature using run config at compilation phase, but then // omits it in favor to function currying at execution phase. - if (!runConfigFuncType->IsVoid()) { + // And vice versa for the forward compatibility. + if (!runConfigNodeType->IsVoid() && !runConfigFuncType->IsVoid()) { TString diff = TStringBuilder() << "run config type mismatch, expected: " << PrintNode((runConfigNodeType), true) @@ -337,17 +382,31 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont << TruncateTypeDiff(diff)).c_str()); } + const auto callableType = runConfigNodeType->IsVoid() + ? callableNodeType : callableFuncType; + const auto runConfigType = runConfigNodeType->IsVoid() + ? runConfigFuncType : runConfigNodeType; + // If so, check the following invariants: // * The first argument of the head function in the sequence // of the curried functions has to be the same as the // run config type. + // * All other arguments of the head function in the sequence + // of the curried function have to be optional. // * The type of the resulting callable has to be the same // as the function type. - const auto firstArgType = funcInfo.FunctionType->GetArgumentType(0); - if (!runConfigNodeType->IsSameType(*firstArgType)) { + if (callableType->GetArgumentsCount() - callableType->GetOptionalArgumentsCount() != 1U) { + UdfTerminate((TStringBuilder() << pos + << " Udf Function '" + << funcName + << "' wrapper has more than one required argument: " + << PrintNode(callableType)).c_str()); + } + const auto firstArgType = callableType->GetArgumentType(0); + if (!runConfigType->IsSameType(*firstArgType)) { TString diff = TStringBuilder() << "type mismatch, expected run config type: " - << PrintNode(runConfigNodeType, true) + << PrintNode(runConfigType, true) << ", actual: " << PrintNode(firstArgType, true); UdfTerminate((TStringBuilder() << pos @@ -356,14 +415,18 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont << "' " << TruncateTypeDiff(diff)).c_str()); } - const auto callableFuncType = funcInfo.FunctionType->GetReturnType(); - const auto callableNodeType = callable.GetType()->GetReturnType(); - if (!callableNodeType->IsSameType(*callableFuncType)) { + const auto closureFuncType = runConfigNodeType->IsVoid() + ? callableFuncType + : AS_TYPE(TCallableType, callableFuncType)->GetReturnType(); + const auto closureNodeType = runConfigNodeType->IsVoid() + ? AS_TYPE(TCallableType, callableNodeType)->GetReturnType() + : callableNodeType; + if (!closureNodeType->IsSameType(*closureFuncType)) { TString diff = TStringBuilder() << "type mismatch, expected return type: " - << PrintNode(callableNodeType, true) + << PrintNode(closureNodeType, true) << ", actual: " - << PrintNode(callableFuncType, true); + << PrintNode(closureFuncType, true); UdfTerminate((TStringBuilder() << pos << " Udf Function '" << funcName @@ -373,12 +436,14 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont const auto runConfigCompNode = LocateNode(ctx.NodeLocator, *runCfgNode.GetNode()); const auto runConfigArgs = funcInfo.FunctionType->GetArgumentsCount(); - return CreateUdfWrapper<false>(ctx, std::move(funcName), std::move(typeConfig), pos, runConfigCompNode, runConfigArgs, funcInfo.FunctionType, userType); + return runConfigNodeType->IsVoid() + ? CreateUdfWrapper<true>(ctx, std::move(funcName), std::move(typeConfig), pos, callableNodeType, callableFuncType, userType) + : CreateUdfWrapper<false>(ctx, std::move(funcName), std::move(typeConfig), pos, runConfigCompNode, runConfigArgs, callableFuncType, userType); } - if (!funcInfo.FunctionType->IsConvertableTo(*callable.GetType()->GetReturnType(), true)) { - TString diff = TStringBuilder() << "type mismatch, expected return type: " << PrintNode(callable.GetType()->GetReturnType(), true) << - ", actual:" << PrintNode(funcInfo.FunctionType, true); + if (!callableFuncType->IsConvertableTo(*callableNodeType, true)) { + TString diff = TStringBuilder() << "type mismatch, expected return type: " << PrintNode(callableNodeType, true) << + ", actual:" << PrintNode(callableFuncType, true); UdfTerminate((TStringBuilder() << pos << " UDF Function '" << funcName << "' " << TruncateTypeDiff(diff)).c_str()); } @@ -389,15 +454,15 @@ IComputationNode* WrapUdf(TCallable& callable, const TComputationNodeFactoryCont if (runConfigFuncType->IsVoid()) { if (ctx.ValidateMode == NUdf::EValidateMode::None && funcInfo.ModuleIR && funcInfo.IRFunctionName) { return new TUdfRunCodegeneratorNode( - ctx.Mutables, std::move(funcName), std::move(typeConfig), pos, funcInfo.FunctionType, userType, + ctx.Mutables, std::move(funcName), std::move(typeConfig), pos, callableNodeType, callableFuncType, userType, std::move(funcInfo.ModuleIRUniqID), std::move(funcInfo.ModuleIR), std::move(funcInfo.IRFunctionName), std::move(funcInfo.Implementation) ); } - return CreateUdfWrapper<true>(ctx, std::move(funcName), std::move(typeConfig), pos, funcInfo.FunctionType, userType); + return CreateUdfWrapper<true>(ctx, std::move(funcName), std::move(typeConfig), pos, callableNodeType, callableFuncType, userType); } const auto runCfgCompNode = LocateNode(ctx.NodeLocator, *runCfgNode.GetNode()); - return CreateUdfWrapper<false>(ctx, std::move(funcName), std::move(typeConfig), pos, runCfgCompNode, 1U, funcInfo.FunctionType, userType); + return CreateUdfWrapper<false>(ctx, std::move(funcName), std::move(typeConfig), pos, runCfgCompNode, 1U, callableFuncType, userType); } IComputationNode* WrapScriptUdf(TCallable& callable, const TComputationNodeFactoryContext& ctx) { diff --git a/yql/essentials/minikql/comp_nodes/ut/mkql_udf_ut.cpp b/yql/essentials/minikql/comp_nodes/ut/mkql_udf_ut.cpp index f5b8b7892c1..265ad3bd059 100644 --- a/yql/essentials/minikql/comp_nodes/ut/mkql_udf_ut.cpp +++ b/yql/essentials/minikql/comp_nodes/ut/mkql_udf_ut.cpp @@ -5,6 +5,31 @@ namespace NKikimr { namespace NMiniKQL { +class TImpl : public NYql::NUdf::TBoxedValue { +public: + explicit TImpl(NYql::NUdf::TSourcePosition pos, + const std::string_view upvalue) + : Pos_(pos) + , Upvalue_(upvalue) + {} + + NYql::NUdf::TUnboxedValue Run(const NYql::NUdf::IValueBuilder* valueBuilder, + const NYql::NUdf::TUnboxedValuePod* args) + const override try { + TStringStream concat; + concat << Upvalue_ << " " << args[0].AsStringRef(); + return valueBuilder->NewString(NYql::NUdf::TStringRef(concat.Data(), + concat.Size())); + } catch (const std::exception& e) { + UdfTerminate((TStringBuilder() << Pos_ << " " << e.what()).data()); + } + + +private: + const NYql::NUdf::TSourcePosition Pos_; + const TString Upvalue_; +}; + // Class, implementing the closure with run config. class TRunConfig : public NYql::NUdf::TBoxedValue { public: @@ -35,6 +60,15 @@ public: return true; } + NYql::NUdf::TUnboxedValue Run(const NYql::NUdf::IValueBuilder*, + const NYql::NUdf::TUnboxedValuePod* args) + const final try { + const std::string_view upvalue(args[0].AsStringRef()); + return NYql::NUdf::TUnboxedValuePod(new TImpl(Pos_, upvalue)); + } catch (const std::exception& e) { + UdfTerminate((TStringBuilder() << Pos_ << " " << e.what()).data()); + } + private: const NYql::NUdf::TSourcePosition Pos_; }; @@ -81,31 +115,6 @@ public: } private: - class TImpl : public NYql::NUdf::TBoxedValue { - public: - explicit TImpl(NYql::NUdf::TSourcePosition pos, - const std::string_view upvalue) - : Pos_(pos) - , Upvalue_(upvalue) - {} - - NYql::NUdf::TUnboxedValue Run(const NYql::NUdf::IValueBuilder* valueBuilder, - const NYql::NUdf::TUnboxedValuePod* args) - const override try { - TStringStream concat; - concat << Upvalue_ << " " << args[0].AsStringRef(); - return valueBuilder->NewString(NYql::NUdf::TStringRef(concat.Data(), - concat.Size())); - } catch (const std::exception& e) { - UdfTerminate((TStringBuilder() << Pos_ << " " << e.what()).data()); - } - - - private: - const NYql::NUdf::TSourcePosition Pos_; - const TString Upvalue_; - }; - const NYql::NUdf::TSourcePosition Pos_; }; @@ -118,6 +127,79 @@ private: SIMPLE_MODULE(TRunConfigUTModule, TRunConfig) SIMPLE_MODULE(TCurryingUTModule, TCurrying) + +SIMPLE_STRICT_UDF(TTest, char*(char*, char*, char*)) { + TStringStream concat; + concat << args[0].AsStringRef() << " " + << args[1].AsStringRef() << " " + << args[2].AsStringRef(); + return valueBuilder->NewString(NYql::NUdf::TStringRef(concat.Data(), + concat.Size())); +} + +template<bool Old> +class TNewTest : public NYql::NUdf::TBoxedValue { +public: + explicit TNewTest(NYql::NUdf::TSourcePosition pos) + : Pos_(pos) + {} + + static const NYql::NUdf::TStringRef& Name() { + static auto name = NYql::NUdf::TStringRef::Of("Test"); + return name; + } + + static bool DeclareSignature(const NYql::NUdf::TStringRef& name, + NYql::NUdf::TType*, + NYql::NUdf::IFunctionTypeInfoBuilder& builder, + bool typesOnly) + { + if (Name() != name) { + return false; + } + + if (Old && typesOnly) { + builder.SimpleSignature<char*(char*, char*, char*)>(); + return true; + } + + builder.SimpleSignature<char*(char*, char*, char*, NYql::NUdf::TOptional<char*>)>() + .OptionalArgs(1); + if (!typesOnly) { + builder.Implementation(new TNewTest(builder.GetSourcePosition())); + } + + return true; + } + + NYql::NUdf::TUnboxedValue Run(const NYql::NUdf::IValueBuilder* valueBuilder, + const NYql::NUdf::TUnboxedValuePod* args) + const override try { + TStringStream concat; + concat << args[0].AsStringRef() << " " + << args[1].AsStringRef() << " "; + if (args[3]) { + concat << args[3].AsStringRef() << " "; + } + concat << args[2].AsStringRef(); + return valueBuilder->NewString(NYql::NUdf::TStringRef(concat.Data(), + concat.Size())); + } catch (const std::exception& e) { + UdfTerminate((TStringBuilder() << Pos_ << " " << e.what()).data()); + } + +private: + const NYql::NUdf::TSourcePosition Pos_; +}; + +// XXX: "Old" UDF is declared via SIMPLE_UDF helper, so it has to +// use the *actual* function name as a class name. Furthermore, +// the UDF, declared by SIMPLE_UDF has to provide the same +// semantics as TNewTest<true>. +SIMPLE_MODULE(TOldUTModule, TTest) +SIMPLE_MODULE(TIncrementalUTModule, TNewTest<true>) +SIMPLE_MODULE(TNewUTModule, TNewTest<false>) + Y_UNIT_TEST_SUITE(TMiniKQLUdfTest) { Y_UNIT_TEST_LLVM(RunconfigToCurrying) { // Create the test setup, using TRunConfig implementation @@ -166,6 +248,238 @@ Y_UNIT_TEST_SUITE(TMiniKQLUdfTest) { UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(result.AsStringRef()), "Canary is alive"); UNIT_ASSERT(!iterator.Next(result)); } + + Y_UNIT_TEST_LLVM(CurryingToRunconfig) { + // Create the test setup, using TCurrying implementation + // for TestModule.Test UDF. + TVector<TUdfModuleInfo> compileModules; + compileModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TCurryingUTModule()} + ); + TSetup<LLVM> compileSetup(GetTestFactory(), std::move(compileModules)); + TProgramBuilder& pb = *compileSetup.PgmBuilder; + + // Build the graph on the setup with TRunConfig implementation. + const auto strType = pb.NewDataType(NUdf::TDataType<char*>::Id); + const auto upvalue = pb.NewDataLiteral<NUdf::EDataSlot::String>("Canary"); + const auto optional = pb.NewOptional(pb.NewDataLiteral(true)); + const auto value = pb.NewDataLiteral<NUdf::EDataSlot::String>("is alive"); + const auto userType = pb.NewTupleType({ + pb.NewTupleType({strType}), + pb.NewEmptyStructType(), + pb.NewEmptyTupleType()}); + const auto udf = pb.Udf("TestModule.Test", pb.NewVoid(), userType); + const auto closure = pb.Apply(udf, {upvalue, optional}); + + const auto list = pb.NewList(strType, {value}); + const auto pgmReturn = pb.Map(list, [&pb, closure](const TRuntimeNode item) { + return pb.Apply(closure, {item}); + }); + + // Create the test setup, using TRunConfig implementation + // for TestModule.Test UDF. + TVector<TUdfModuleInfo> runModules; + runModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TRunConfigUTModule()} + ); + TSetup<LLVM> runSetup(GetTestFactory(), std::move(runModules)); + // Move the graph from the one setup to another as a + // serialized bytecode sequence. + const auto bytecode = SerializeRuntimeNode(pgmReturn, *compileSetup.Env); + const auto root = DeserializeRuntimeNode(bytecode, *runSetup.Env); + + // Run the graph on the setup with TCurrying implementation. + const auto graph = runSetup.BuildGraph(root); + const auto iterator = graph->GetValue().GetListIterator(); + + NUdf::TUnboxedValue result; + UNIT_ASSERT(iterator.Next(result)); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(result.AsStringRef()), "Canary is alive"); + UNIT_ASSERT(!iterator.Next(result)); + } + + Y_UNIT_TEST_LLVM(OldToIncremental) { + // Create the test setup, using the old implementation for + // TestModule.Test UDF. + TVector<TUdfModuleInfo> compileModules; + compileModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TOldUTModule()} + ); + TSetup<LLVM> compileSetup(GetTestFactory(), std::move(compileModules)); + TProgramBuilder& pb = *compileSetup.PgmBuilder; + + // Build the graph, using the old setup. + const auto strType = pb.NewDataType(NUdf::TDataType<char*>::Id); + const auto arg1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("Canary"); + const auto arg2 = pb.NewDataLiteral<NUdf::EDataSlot::String>("is"); + const auto arg3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("alive"); + + const auto udf = pb.Udf("TestModule.Test"); + const auto argsType = pb.NewTupleType({strType, strType, strType}); + const auto argList = pb.NewList(argsType, {pb.NewTuple({arg1, arg2, arg3})}); + const auto pgmReturn = pb.Map(argList, [&pb, udf](const TRuntimeNode args) { + return pb.Apply(udf, {pb.Nth(args, 0), pb.Nth(args, 1), pb.Nth(args, 2)}); + }); + + // Create the test setup, using the incremental + // implementation for TestModule.Test UDF. + TVector<TUdfModuleInfo> runModules; + runModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TIncrementalUTModule()} + ); + TSetup<LLVM> runSetup(GetTestFactory(), std::move(runModules)); + // Move the graph from the one setup to another as a + // serialized bytecode sequence. + const auto bytecode = SerializeRuntimeNode(pgmReturn, *compileSetup.Env); + const auto root = DeserializeRuntimeNode(bytecode, *runSetup.Env); + + // Run the graph, using the incremental setup. + const auto graph = runSetup.BuildGraph(root); + const auto iterator = graph->GetValue().GetListIterator(); + + NUdf::TUnboxedValue result; + UNIT_ASSERT(iterator.Next(result)); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(result.AsStringRef()), "Canary is alive"); + UNIT_ASSERT(!iterator.Next(result)); + } + + Y_UNIT_TEST_LLVM(IncrementalToOld) { + // Create the test setup, using the incremental + // implementation for TestModule.Test UDF. + TVector<TUdfModuleInfo> compileModules; + compileModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TIncrementalUTModule()} + ); + TSetup<LLVM> compileSetup(GetTestFactory(), std::move(compileModules)); + TProgramBuilder& pb = *compileSetup.PgmBuilder; + + // Build the graph, using the incremental setup. + const auto strType = pb.NewDataType(NUdf::TDataType<char*>::Id); + const auto arg1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("Canary"); + const auto arg2 = pb.NewDataLiteral<NUdf::EDataSlot::String>("is"); + const auto arg3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("alive"); + + const auto udf = pb.Udf("TestModule.Test"); + const auto argsType = pb.NewTupleType({strType, strType, strType}); + const auto argList = pb.NewList(argsType, {pb.NewTuple({arg1, arg2, arg3})}); + const auto pgmReturn = pb.Map(argList, [&pb, udf](const TRuntimeNode args) { + return pb.Apply(udf, {pb.Nth(args, 0), pb.Nth(args, 1), pb.Nth(args, 2)}); + }); + + // Create the test setup, using the old implementation for + // TestModule.Test UDF. + TVector<TUdfModuleInfo> runModules; + runModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TOldUTModule()} + ); + TSetup<LLVM> runSetup(GetTestFactory(), std::move(runModules)); + // Move the graph from the one setup to another as a + // serialized bytecode sequence. + const auto bytecode = SerializeRuntimeNode(pgmReturn, *compileSetup.Env); + const auto root = DeserializeRuntimeNode(bytecode, *runSetup.Env); + + // Run the graph, using the old setup. + const auto graph = runSetup.BuildGraph(root); + const auto iterator = graph->GetValue().GetListIterator(); + + NUdf::TUnboxedValue result; + UNIT_ASSERT(iterator.Next(result)); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(result.AsStringRef()), "Canary is alive"); + UNIT_ASSERT(!iterator.Next(result)); + } + + Y_UNIT_TEST_LLVM(IncrementalToNew) { + // Create the test setup, using the incremental + // implementation for TestModule.Test UDF. + TVector<TUdfModuleInfo> compileModules; + compileModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TIncrementalUTModule()} + ); + TSetup<LLVM> compileSetup(GetTestFactory(), std::move(compileModules)); + TProgramBuilder& pb = *compileSetup.PgmBuilder; + + // Build the graph, using the incremental setup. + const auto strType = pb.NewDataType(NUdf::TDataType<char*>::Id); + const auto arg1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("Canary"); + const auto arg2 = pb.NewDataLiteral<NUdf::EDataSlot::String>("is"); + const auto arg3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("alive"); + + const auto udf = pb.Udf("TestModule.Test"); + const auto argsType = pb.NewTupleType({strType, strType, strType}); + const auto argList = pb.NewList(argsType, {pb.NewTuple({arg1, arg2, arg3})}); + const auto pgmReturn = pb.Map(argList, [&pb, udf](const TRuntimeNode args) { + return pb.Apply(udf, {pb.Nth(args, 0), pb.Nth(args, 1), pb.Nth(args, 2)}); + }); + + // Create the test setup, using the new implementation for + // TestModule.Test UDF. + TVector<TUdfModuleInfo> runModules; + runModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TNewUTModule()} + ); + TSetup<LLVM> runSetup(GetTestFactory(), std::move(runModules)); + // Move the graph from the one setup to another as a + // serialized bytecode sequence. + const auto bytecode = SerializeRuntimeNode(pgmReturn, *compileSetup.Env); + const auto root = DeserializeRuntimeNode(bytecode, *runSetup.Env); + + // Run the graph, using the new setup. + const auto graph = runSetup.BuildGraph(root); + const auto iterator = graph->GetValue().GetListIterator(); + + NUdf::TUnboxedValue result; + UNIT_ASSERT(iterator.Next(result)); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(result.AsStringRef()), "Canary is alive"); + UNIT_ASSERT(!iterator.Next(result)); + } + + Y_UNIT_TEST_LLVM(NewToIncremental) { + // Create the test setup, using the new implementation for + // TestModule.Test UDF. + TVector<TUdfModuleInfo> compileModules; + compileModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TNewUTModule()} + ); + TSetup<LLVM> compileSetup(GetTestFactory(), std::move(compileModules)); + TProgramBuilder& pb = *compileSetup.PgmBuilder; + + // Build the graph, using the new setup. + const auto strType = pb.NewDataType(NUdf::TDataType<char*>::Id); + const auto optType = pb.NewOptionalType(strType); + const auto arg1 = pb.NewDataLiteral<NUdf::EDataSlot::String>("Canary"); + const auto arg2 = pb.NewDataLiteral<NUdf::EDataSlot::String>("is"); + const auto arg3 = pb.NewDataLiteral<NUdf::EDataSlot::String>("alive"); + const auto arg4 = pb.NewDataLiteral<NUdf::EDataSlot::String>("still"); + const auto opt4 = pb.NewOptional(arg4); + + const auto udf = pb.Udf("TestModule.Test"); + const auto argsType = pb.NewTupleType({strType, strType, strType, optType}); + const auto argList = pb.NewList(argsType, {pb.NewTuple({arg1, arg2, arg3, opt4})}); + const auto pgmReturn = pb.Map(argList, [&pb, udf](const TRuntimeNode args) { + return pb.Apply(udf, {pb.Nth(args, 0), pb.Nth(args, 1), pb.Nth(args, 2), pb.Nth(args, 3)}); + }); + + // Create the test setup, using the incremental + // implementation for TestModule.Test UDF. + TVector<TUdfModuleInfo> runModules; + runModules.emplace_back( + TUdfModuleInfo{"", "TestModule", new TIncrementalUTModule()} + ); + TSetup<LLVM> runSetup(GetTestFactory(), std::move(runModules)); + // Move the graph from the one setup to another as a + // serialized bytecode sequence. + const auto bytecode = SerializeRuntimeNode(pgmReturn, *compileSetup.Env); + const auto root = DeserializeRuntimeNode(bytecode, *runSetup.Env); + + // Run the graph, using the incremental setup. + const auto graph = runSetup.BuildGraph(root); + const auto iterator = graph->GetValue().GetListIterator(); + + NUdf::TUnboxedValue result; + UNIT_ASSERT(iterator.Next(result)); + UNIT_ASSERT_STRINGS_EQUAL(TStringBuf(result.AsStringRef()), "Canary is still alive"); + UNIT_ASSERT(!iterator.Next(result)); + } } // Y_UNIT_TEST_SUITE } // namespace NMiniKQL diff --git a/yql/essentials/minikql/computation/mkql_computation_node.cpp b/yql/essentials/minikql/computation/mkql_computation_node.cpp index 7fa810fb8cd..6625d3a6eef 100644 --- a/yql/essentials/minikql/computation/mkql_computation_node.cpp +++ b/yql/essentials/minikql/computation/mkql_computation_node.cpp @@ -60,6 +60,7 @@ TComputationContext::TComputationContext(const THolderFactory& holderFactory, , CountersProvider(opts.CountersProvider) , SecureParamsProvider(opts.SecureParamsProvider) , LogProvider(opts.LogProvider) + , LangVer(opts.LangVer) { std::fill_n(MutableValues.get(), mutables.CurValueIndex, NUdf::TUnboxedValue(NUdf::TUnboxedValuePod::Invalid())); diff --git a/yql/essentials/providers/common/mkql/yql_provider_mkql.cpp b/yql/essentials/providers/common/mkql/yql_provider_mkql.cpp index 848284e2b41..733bc5a3ad5 100644 --- a/yql/essentials/providers/common/mkql/yql_provider_mkql.cpp +++ b/yql/essentials/providers/common/mkql/yql_provider_mkql.cpp @@ -3016,7 +3016,7 @@ TMkqlCommonCallableCompiler::TShared::TShared() { return MkqlBuildExpr(node.Head(), ctx); }); - AddCallable({ "AssumeStrict", "AssumeNonStrict", "Likely" }, [](const TExprNode& node, TMkqlBuildContext& ctx) { + AddCallable({ "AssumeStrict", "AssumeNonStrict", "NoPush", "Likely" }, [](const TExprNode& node, TMkqlBuildContext& ctx) { return MkqlBuildExpr(node.Head(), ctx); }); diff --git a/yql/essentials/providers/common/udf_resolve/yql_udf_resolver_with_index.cpp b/yql/essentials/providers/common/udf_resolve/yql_udf_resolver_with_index.cpp index 3892de463e7..45844281c4f 100644 --- a/yql/essentials/providers/common/udf_resolve/yql_udf_resolver_with_index.cpp +++ b/yql/essentials/providers/common/udf_resolve/yql_udf_resolver_with_index.cpp @@ -205,6 +205,8 @@ private: function.IsStrict = info.IsStrict; function.SupportsBlocks = info.SupportsBlocks; function.Messages = info.Messages; + function.MinLangVer = info.MinLangVer; + function.MaxLangVer = info.MaxLangVer; return true; } diff --git a/yql/essentials/sql/settings/translation_settings.h b/yql/essentials/sql/settings/translation_settings.h index 053e09acbec..99f0fc149fc 100644 --- a/yql/essentials/sql/settings/translation_settings.h +++ b/yql/essentials/sql/settings/translation_settings.h @@ -77,7 +77,7 @@ namespace NSQLTranslation { TTranslationSettings(); google::protobuf::Arena* Arena = nullptr; - NYql::TLangVersion LangVer = NYql::UnknownLangVersion; + NYql::TLangVersion LangVer = NYql::MinLangVersion; THashMap<TString, TString> ClusterMapping; TString PathPrefix; // keys (cluster name) should be normalized diff --git a/yql/essentials/sql/v1/SQLv1.g.in b/yql/essentials/sql/v1/SQLv1.g.in index 6ba91358316..ed04c337507 100644 --- a/yql/essentials/sql/v1/SQLv1.g.in +++ b/yql/essentials/sql/v1/SQLv1.g.in @@ -371,7 +371,7 @@ select_unparenthesized_stmt: select_kind_partial (select_op select_kind_parenthe select_kind_parenthesis: select_kind_partial | LPAREN select_kind_partial RPAREN; -select_op: UNION (ALL)? | INTERSECT | EXCEPT; +select_op: (UNION | INTERSECT | EXCEPT) (DISTINCT | ALL)?; select_kind_partial: select_kind (LIMIT expr ((OFFSET | COMMA) expr)?)? diff --git a/yql/essentials/sql/v1/SQLv1Antlr4.g.in b/yql/essentials/sql/v1/SQLv1Antlr4.g.in index 66be65e2aee..6bfdf3820f7 100644 --- a/yql/essentials/sql/v1/SQLv1Antlr4.g.in +++ b/yql/essentials/sql/v1/SQLv1Antlr4.g.in @@ -370,7 +370,7 @@ select_unparenthesized_stmt: select_kind_partial (select_op select_kind_parenthe select_kind_parenthesis: select_kind_partial | LPAREN select_kind_partial RPAREN; -select_op: UNION (ALL)? | INTERSECT | EXCEPT; +select_op: (UNION | INTERSECT | EXCEPT) (DISTINCT | ALL)?; select_kind_partial: select_kind (LIMIT expr ((OFFSET | COMMA) expr)?)? diff --git a/yql/essentials/sql/v1/complete/analysis/global/evaluate.cpp b/yql/essentials/sql/v1/complete/analysis/global/evaluate.cpp new file mode 100644 index 00000000000..b938eaf0631 --- /dev/null +++ b/yql/essentials/sql/v1/complete/analysis/global/evaluate.cpp @@ -0,0 +1,52 @@ +#include "evaluate.h" + +namespace NSQLComplete { + + namespace { + + class TVisitor: public SQLv1Antlr4BaseVisitor { + public: + explicit TVisitor(const TEnvironment* env) + : Env_(env) + { + } + + std::any visitBind_parameter(SQLv1::Bind_parameterContext* ctx) override { + std::string id = GetId(ctx); + if (const NYT::TNode* node = Env_->Parameters.FindPtr(id)) { + return *node; + } + return defaultResult(); + } + + std::any defaultResult() override { + return NYT::TNode(); + } + + private: + std::string GetId(SQLv1::Bind_parameterContext* ctx) const { + if (auto* x = ctx->an_id_or_type()) { + return x->getText(); + } else if (auto* x = ctx->TOKEN_TRUE()) { + return x->getText(); + } else if (auto* x = ctx->TOKEN_FALSE()) { + return x->getText(); + } else { + Y_ABORT("You should change implementation according grammar changes"); + } + } + + const TEnvironment* Env_; + }; + + NYT::TNode EvaluateG(antlr4::ParserRuleContext* ctx, const TEnvironment& env) { + return std::any_cast<NYT::TNode>(TVisitor(&env).visit(ctx)); + } + + } // namespace + + NYT::TNode Evaluate(SQLv1::Bind_parameterContext* ctx, const TEnvironment& env) { + return EvaluateG(ctx, env); + } + +} // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/analysis/global/evaluate.h b/yql/essentials/sql/v1/complete/analysis/global/evaluate.h new file mode 100644 index 00000000000..03cbcdc798b --- /dev/null +++ b/yql/essentials/sql/v1/complete/analysis/global/evaluate.h @@ -0,0 +1,11 @@ +#pragma once + +#include "parse_tree.h" + +#include <yql/essentials/sql/v1/complete/core/environment.h> + +namespace NSQLComplete { + + NYT::TNode Evaluate(SQLv1::Bind_parameterContext* ctx, const TEnvironment& env); + +} // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/analysis/global/global.cpp b/yql/essentials/sql/v1/complete/analysis/global/global.cpp index a216b03de1a..9acb3310fac 100644 --- a/yql/essentials/sql/v1/complete/analysis/global/global.cpp +++ b/yql/essentials/sql/v1/complete/analysis/global/global.cpp @@ -42,13 +42,13 @@ namespace NSQLComplete { Parser_.setErrorHandler(std::make_shared<TErrorStrategy>()); } - TGlobalContext Analyze(TCompletionInput input) override { + TGlobalContext Analyze(TCompletionInput input, TEnvironment env) override { SQLv1::Sql_queryContext* sqlQuery = Parse(input.Text); Y_ENSURE(sqlQuery); TGlobalContext ctx; - ctx.Use = FindUseStatement(sqlQuery, &Tokens_, input.CursorPosition); + ctx.Use = FindUseStatement(sqlQuery, &Tokens_, input.CursorPosition, env); return ctx; } @@ -70,9 +70,9 @@ namespace NSQLComplete { class TGlobalAnalysis: public IGlobalAnalysis { public: - TGlobalContext Analyze(TCompletionInput input) override { + TGlobalContext Analyze(TCompletionInput input, TEnvironment env) override { const bool isAnsiLexer = IsAnsiQuery(TString(input.Text)); - return GetSpecialized(isAnsiLexer).Analyze(std::move(input)); + return GetSpecialized(isAnsiLexer).Analyze(std::move(input), std::move(env)); } private: diff --git a/yql/essentials/sql/v1/complete/analysis/global/global.h b/yql/essentials/sql/v1/complete/analysis/global/global.h index a5249bc3a48..97c55738e45 100644 --- a/yql/essentials/sql/v1/complete/analysis/global/global.h +++ b/yql/essentials/sql/v1/complete/analysis/global/global.h @@ -1,6 +1,7 @@ #pragma once #include <yql/essentials/sql/v1/complete/core/input.h> +#include <yql/essentials/sql/v1/complete/core/environment.h> #include <util/generic/ptr.h> #include <util/generic/maybe.h> @@ -22,7 +23,7 @@ namespace NSQLComplete { using TPtr = THolder<IGlobalAnalysis>; virtual ~IGlobalAnalysis() = default; - virtual TGlobalContext Analyze(TCompletionInput input) = 0; + virtual TGlobalContext Analyze(TCompletionInput input, TEnvironment env) = 0; }; IGlobalAnalysis::TPtr MakeGlobalAnalysis(); diff --git a/yql/essentials/sql/v1/complete/analysis/global/use.cpp b/yql/essentials/sql/v1/complete/analysis/global/use.cpp index da7dc6a5751..2fd7fab6614 100644 --- a/yql/essentials/sql/v1/complete/analysis/global/use.cpp +++ b/yql/essentials/sql/v1/complete/analysis/global/use.cpp @@ -1,14 +1,20 @@ #include "use.h" +#include "evaluate.h" + namespace NSQLComplete { namespace { class TVisitor: public SQLv1Antlr4BaseVisitor { public: - TVisitor(antlr4::TokenStream* tokens, size_t cursorPosition) + TVisitor( + antlr4::TokenStream* tokens, + size_t cursorPosition, + const TEnvironment* env) : Tokens_(tokens) , CursorPosition_(cursorPosition) + , Env_(env) { } @@ -33,7 +39,9 @@ namespace NSQLComplete { } if (SQLv1::Pure_column_or_namedContext* ctx = expr->pure_column_or_named()) { - cluster = ctx->getText(); + if (auto id = GetId(ctx)) { + cluster = std::move(*id); + } } if (cluster.empty()) { @@ -76,8 +84,27 @@ namespace NSQLComplete { return antlr4::misc::Interval(CursorPosition_, CursorPosition_); } + TMaybe<TString> GetId(SQLv1::Pure_column_or_namedContext* ctx) const { + if (auto* x = ctx->bind_parameter()) { + return GetId(x); + } else if (auto* x = ctx->an_id()) { + return x->getText(); + } else { + Y_ABORT("You should change implementation according grammar changes"); + } + } + + TMaybe<TString> GetId(SQLv1::Bind_parameterContext* ctx) const { + NYT::TNode node = Evaluate(ctx, *Env_); + if (!node.HasValue() || !node.IsString()) { + return Nothing(); + } + return node.AsString(); + } + antlr4::TokenStream* Tokens_; size_t CursorPosition_; + const TEnvironment* Env_; }; } // namespace @@ -85,8 +112,9 @@ namespace NSQLComplete { TMaybe<TUseContext> FindUseStatement( SQLv1::Sql_queryContext* ctx, antlr4::TokenStream* tokens, - size_t cursorPosition) { - std::any result = TVisitor(tokens, cursorPosition).visit(ctx); + size_t cursorPosition, + const TEnvironment& env) { + std::any result = TVisitor(tokens, cursorPosition, &env).visit(ctx); if (!result.has_value()) { return Nothing(); } diff --git a/yql/essentials/sql/v1/complete/analysis/global/use.h b/yql/essentials/sql/v1/complete/analysis/global/use.h index 54f3557fd62..0cdb9b15469 100644 --- a/yql/essentials/sql/v1/complete/analysis/global/use.h +++ b/yql/essentials/sql/v1/complete/analysis/global/use.h @@ -12,6 +12,7 @@ namespace NSQLComplete { TMaybe<TUseContext> FindUseStatement( SQLv1::Sql_queryContext* ctx, antlr4::TokenStream* tokens, - size_t cursorPosition); + size_t cursorPosition, + const TEnvironment& env); } // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/analysis/global/ya.make b/yql/essentials/sql/v1/complete/analysis/global/ya.make index a28d99f94c2..a8aa1eb7214 100644 --- a/yql/essentials/sql/v1/complete/analysis/global/ya.make +++ b/yql/essentials/sql/v1/complete/analysis/global/ya.make @@ -1,6 +1,7 @@ LIBRARY() SRCS( + evaluate.cpp global.cpp use.cpp ) diff --git a/yql/essentials/sql/v1/complete/antlr4/c3i.h b/yql/essentials/sql/v1/complete/antlr4/c3i.h index ed69f3cf29b..89201e12562 100644 --- a/yql/essentials/sql/v1/complete/antlr4/c3i.h +++ b/yql/essentials/sql/v1/complete/antlr4/c3i.h @@ -36,6 +36,7 @@ namespace NSQLComplete { struct TConfig { std::unordered_set<TTokenId> IgnoredTokens; std::unordered_set<TRuleId> PreferredRules; + std::unordered_set<TRuleId> IgnoredRules; }; virtual TC3Candidates Complete(TStringBuf text, size_t caretTokenIndex) = 0; diff --git a/yql/essentials/sql/v1/complete/antlr4/c3t.h b/yql/essentials/sql/v1/complete/antlr4/c3t.h index 1f71553e04f..2b49068ac8f 100644 --- a/yql/essentials/sql/v1/complete/antlr4/c3t.h +++ b/yql/essentials/sql/v1/complete/antlr4/c3t.h @@ -25,12 +25,17 @@ namespace NSQLComplete { , Tokens_(&Lexer_) , Parser_(&Tokens_) , CompletionCore_(&Parser_) + , IgnoredRules_(std::move(config.IgnoredRules)) { Lexer_.removeErrorListeners(); Parser_.removeErrorListeners(); CompletionCore_.ignoredTokens = std::move(config.IgnoredTokens); CompletionCore_.preferredRules = std::move(config.PreferredRules); + + for (TRuleId rule : IgnoredRules_) { + CompletionCore_.preferredRules.emplace(rule); + } } TC3Candidates Complete(TStringBuf text, size_t caretTokenIndex) override { @@ -47,23 +52,36 @@ namespace NSQLComplete { Tokens_.fill(); } - static TC3Candidates Converted(c3::CandidatesCollection candidates) { + TC3Candidates Converted(c3::CandidatesCollection candidates) const { TC3Candidates converted; + for (auto& [token, following] : candidates.tokens) { converted.Tokens.emplace_back(token, std::move(following)); } + for (auto& [rule, data] : candidates.rules) { + if (IsIgnored(rule, data.ruleList)) { + continue; + } + converted.Rules.emplace_back(rule, std::move(data.ruleList)); converted.Rules.back().ParserCallStack.emplace_back(rule); } + return converted; } + bool IsIgnored(TRuleId head, const std::vector<TRuleId> tail) const { + return IgnoredRules_.contains(head) || + AnyOf(tail, [this](TRuleId r) { return IgnoredRules_.contains(r); }); + } + antlr4::ANTLRInputStream Chars_; G::TLexer Lexer_; antlr4::BufferedTokenStream Tokens_; G::TParser Parser_; c3::CodeCompletionCore CompletionCore_; + std::unordered_set<TRuleId> IgnoredRules_; }; } // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/core/environment.cpp b/yql/essentials/sql/v1/complete/core/environment.cpp new file mode 100644 index 00000000000..89f179533be --- /dev/null +++ b/yql/essentials/sql/v1/complete/core/environment.cpp @@ -0,0 +1 @@ +#include "environment.h" diff --git a/yql/essentials/sql/v1/complete/core/environment.h b/yql/essentials/sql/v1/complete/core/environment.h new file mode 100644 index 00000000000..0f0299a9353 --- /dev/null +++ b/yql/essentials/sql/v1/complete/core/environment.h @@ -0,0 +1,16 @@ +#pragma once + +#include <library/cpp/yson/node/node.h> + +#include <util/generic/string.h> +#include <util/generic/hash.h> + +namespace NSQLComplete { + + struct TEnvironment { + // Given `{ "$x": "{ "Data": "foo" }" }`, + // it will contain `{ "$x": "foo" }` + THashMap<TString, NYT::TNode> Parameters; + }; + +} // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/core/ya.make b/yql/essentials/sql/v1/complete/core/ya.make index 8bc457f8f95..599810301ca 100644 --- a/yql/essentials/sql/v1/complete/core/ya.make +++ b/yql/essentials/sql/v1/complete/core/ya.make @@ -1,7 +1,12 @@ LIBRARY() SRCS( + environment.cpp input.cpp ) +PEERDIR( + library/cpp/yson/node +) + END() diff --git a/yql/essentials/sql/v1/complete/name/object/dispatch/schema.cpp b/yql/essentials/sql/v1/complete/name/object/dispatch/schema.cpp deleted file mode 100644 index f6d79b280a0..00000000000 --- a/yql/essentials/sql/v1/complete/name/object/dispatch/schema.cpp +++ /dev/null @@ -1,36 +0,0 @@ -#include "schema.h" - -namespace NSQLComplete { - - namespace { - - class TSchema: public ISchema { - public: - explicit TSchema(THashMap<TString, ISchema::TPtr> mapping) - : Mapping_(std::move(mapping)) - { - } - - NThreading::TFuture<TListResponse> List(const TListRequest& request) const override { - auto iter = Mapping_.find(request.Cluster); - if (iter == std::end(Mapping_)) { - yexception e; - e << "unknown cluster '" << request.Cluster << "'"; - std::exception_ptr p = std::make_exception_ptr(e); - return NThreading::MakeErrorFuture<TListResponse>(p); - } - - return iter->second->List(request); - } - - private: - THashMap<TString, ISchema::TPtr> Mapping_; - }; - - } // namespace - - ISchema::TPtr MakeDispatchSchema(THashMap<TString, ISchema::TPtr> mapping) { - return new TSchema(std::move(mapping)); - } - -} // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/name/object/dispatch/schema.h b/yql/essentials/sql/v1/complete/name/object/dispatch/schema.h deleted file mode 100644 index 517a3ad0af7..00000000000 --- a/yql/essentials/sql/v1/complete/name/object/dispatch/schema.h +++ /dev/null @@ -1,9 +0,0 @@ -#pragma once - -#include <yql/essentials/sql/v1/complete/name/object/schema.h> - -namespace NSQLComplete { - - ISchema::TPtr MakeDispatchSchema(THashMap<TString, ISchema::TPtr> mapping); - -} // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/name/object/dispatch/ya.make b/yql/essentials/sql/v1/complete/name/object/dispatch/ya.make deleted file mode 100644 index 071bf5dff7d..00000000000 --- a/yql/essentials/sql/v1/complete/name/object/dispatch/ya.make +++ /dev/null @@ -1,11 +0,0 @@ -LIBRARY() - -SRCS( - schema.cpp -) - -PEERDIR( - yql/essentials/sql/v1/complete/name/object -) - -END() diff --git a/yql/essentials/sql/v1/complete/name/object/simple/schema_ut.cpp b/yql/essentials/sql/v1/complete/name/object/simple/schema_ut.cpp index 954ecc4da75..7fdd5d66b21 100644 --- a/yql/essentials/sql/v1/complete/name/object/simple/schema_ut.cpp +++ b/yql/essentials/sql/v1/complete/name/object/simple/schema_ut.cpp @@ -9,16 +9,16 @@ using namespace NSQLComplete; Y_UNIT_TEST_SUITE(StaticSchemaTests) { ISchema::TPtr MakeStaticSchemaUT() { - THashMap<TString, TVector<TFolderEntry>> fs = { - {"/", {{"Folder", "local"}, - {"Folder", "test"}, - {"Folder", "prod"}}}, - {"/local/", {{"Table", "example"}, - {"Table", "account"}, - {"Table", "abacaba"}}}, - {"/test/", {{"Folder", "service"}, - {"Table", "meta"}}}, - {"/test/service/", {{"Table", "example"}}}, + THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> fs = { + {"", {{"/", {{"Folder", "local"}, + {"Folder", "test"}, + {"Folder", "prod"}}}, + {"/local/", {{"Table", "example"}, + {"Table", "account"}, + {"Table", "abacaba"}}}, + {"/test/", {{"Folder", "service"}, + {"Table", "meta"}}}, + {"/test/service/", {{"Table", "example"}}}}}, }; return MakeSimpleSchema( MakeStaticSimpleSchema(std::move(fs))); diff --git a/yql/essentials/sql/v1/complete/name/object/simple/static/schema.cpp b/yql/essentials/sql/v1/complete/name/object/simple/static/schema.cpp index 99d946a0d26..3482da8a332 100644 --- a/yql/essentials/sql/v1/complete/name/object/simple/static/schema.cpp +++ b/yql/essentials/sql/v1/complete/name/object/simple/static/schema.cpp @@ -6,12 +6,14 @@ namespace NSQLComplete { class TSimpleSchema: public ISimpleSchema { public: - explicit TSimpleSchema(THashMap<TString, TVector<TFolderEntry>> data) + explicit TSimpleSchema(THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> data) : Data_(std::move(data)) { - for (const auto& [k, _] : Data_) { - Y_ENSURE(k.StartsWith("/"), k << " must start with the '/'"); - Y_ENSURE(k.EndsWith("/"), k << " must end with the '/'"); + for (const auto& [_, tables] : Data_) { + for (const auto& [k, _] : tables) { + Y_ENSURE(k.StartsWith("/"), k << " must start with the '/'"); + Y_ENSURE(k.EndsWith("/"), k << " must end with the '/'"); + } } } @@ -26,25 +28,31 @@ namespace NSQLComplete { return {head, tail}; } - NThreading::TFuture<TVector<TFolderEntry>> List(TString folder) const override { + NThreading::TFuture<TVector<TFolderEntry>> List(TString cluster, TString folder) const override { if (!folder.StartsWith('/')) { folder.prepend('/'); } TVector<TFolderEntry> entries; - if (const auto* data = Data_.FindPtr(folder)) { - entries = *data; + + const THashMap<TString, TVector<TFolderEntry>>* tables = nullptr; + const TVector<TFolderEntry>* items = nullptr; + if ((tables = Data_.FindPtr(cluster)) && + (items = tables->FindPtr(folder))) { + entries = *items; } + return NThreading::MakeFuture(std::move(entries)); } private: - THashMap<TString, TVector<TFolderEntry>> Data_; + THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> Data_; }; } // namespace - ISimpleSchema::TPtr MakeStaticSimpleSchema(THashMap<TString, TVector<TFolderEntry>> fs) { + ISimpleSchema::TPtr MakeStaticSimpleSchema( + THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> fs) { return new TSimpleSchema(std::move(fs)); } diff --git a/yql/essentials/sql/v1/complete/name/object/simple/static/schema.h b/yql/essentials/sql/v1/complete/name/object/simple/static/schema.h index f04c89f0b23..009b433ee4a 100644 --- a/yql/essentials/sql/v1/complete/name/object/simple/static/schema.h +++ b/yql/essentials/sql/v1/complete/name/object/simple/static/schema.h @@ -5,6 +5,6 @@ namespace NSQLComplete { ISimpleSchema::TPtr MakeStaticSimpleSchema( - THashMap<TString, TVector<TFolderEntry>> fs); + THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> fs); } // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/name/object/ya.make b/yql/essentials/sql/v1/complete/name/object/ya.make index 2561c018292..dd5e5046315 100644 --- a/yql/essentials/sql/v1/complete/name/object/ya.make +++ b/yql/essentials/sql/v1/complete/name/object/ya.make @@ -11,6 +11,5 @@ PEERDIR( END() RECURSE( - dispatch simple ) diff --git a/yql/essentials/sql/v1/complete/name/service/schema/name_service.cpp b/yql/essentials/sql/v1/complete/name/service/schema/name_service.cpp index de8e8db65ac..b21ad828317 100644 --- a/yql/essentials/sql/v1/complete/name/service/schema/name_service.cpp +++ b/yql/essentials/sql/v1/complete/name/service/schema/name_service.cpp @@ -32,12 +32,7 @@ namespace NSQLComplete { } static TString ClusterName(const TObjectNameConstraints& constraints) { - TString name = constraints.Cluster; - if (!constraints.Provider.empty()) { - name.prepend(":"); - name.prepend(constraints.Provider); - } - return name; + return constraints.Cluster; } static TListFilter ToListFilter(const TNameConstraints& constraints) { diff --git a/yql/essentials/sql/v1/complete/sql_complete.cpp b/yql/essentials/sql/v1/complete/sql_complete.cpp index 00d346f0770..807786512e4 100644 --- a/yql/essentials/sql/v1/complete/sql_complete.cpp +++ b/yql/essentials/sql/v1/complete/sql_complete.cpp @@ -1,5 +1,6 @@ #include "sql_complete.h" +#include <yql/essentials/sql/v1/complete/syntax/grammar.h> #include <yql/essentials/sql/v1/complete/text/word.h> #include <yql/essentials/sql/v1/complete/name/service/static/name_service.h> #include <yql/essentials/sql/v1/complete/syntax/local.h> @@ -18,17 +19,17 @@ namespace NSQLComplete { INameService::TPtr names, ISqlCompletionEngine::TConfiguration configuration) : Configuration_(std::move(configuration)) - , SyntaxAnalysis_(MakeLocalSyntaxAnalysis(lexer)) + , SyntaxAnalysis_(MakeLocalSyntaxAnalysis(lexer, Configuration_.IgnoredRules)) , GlobalAnalysis_(MakeGlobalAnalysis()) , Names_(std::move(names)) { } TCompletion Complete(TCompletionInput input) override { - return CompleteAsync(std::move(input)).ExtractValueSync(); + return CompleteAsync(std::move(input), {}).ExtractValueSync(); } - virtual NThreading::TFuture<TCompletion> CompleteAsync(TCompletionInput input) override { + virtual NThreading::TFuture<TCompletion> CompleteAsync(TCompletionInput input, TEnvironment env) override { if ( input.CursorPosition < input.Text.length() && IsUTF8ContinuationByte(input.Text.at(input.CursorPosition)) || @@ -41,7 +42,7 @@ namespace NSQLComplete { TLocalSyntaxContext context = SyntaxAnalysis_->Analyze(input); auto keywords = context.Keywords; - TGlobalContext global = GlobalAnalysis_->Analyze(input); + TGlobalContext global = GlobalAnalysis_->Analyze(input, std::move(env)); TNameRequest request = NameRequestFrom(input, context, global); if (request.IsEmpty()) { @@ -118,7 +119,7 @@ namespace NSQLComplete { if (context.Cluster) { TClusterName::TConstraints constraints; - constraints.Namespace = context.Cluster->Provider; + constraints.Namespace = ""; // TODO(YQL-19747): filter by provider request.Constraints.Cluster = std::move(constraints); } @@ -185,14 +186,14 @@ namespace NSQLComplete { if constexpr (std::is_base_of_v<TFolderName, T>) { name.Indentifier.append('/'); if (!context.Object->IsQuoted) { - name.Indentifier = Quoted(std::move(name.Indentifier)); + name.Indentifier.prepend('`'); } return {ECandidateKind::FolderName, std::move(name.Indentifier)}; } if constexpr (std::is_base_of_v<TTableName, T>) { if (!context.Object->IsQuoted) { - name.Indentifier = Quoted(std::move(name.Indentifier)); + name.Indentifier.prepend('`'); } return {ECandidateKind::TableName, std::move(name.Indentifier)}; } @@ -213,6 +214,45 @@ namespace NSQLComplete { INameService::TPtr Names_; }; + ISqlCompletionEngine::TConfiguration MakeConfiguration(THashSet<TString> allowedStmts) { + allowedStmts.emplace("sql_stmt"); + + ISqlCompletionEngine::TConfiguration config; + for (const std::string& name : GetSqlGrammar().GetAllRules()) { + if (name.ends_with("_stmt") && !allowedStmts.contains(name)) { + config.IgnoredRules.emplace(name); + } + } + return config; + } + + ISqlCompletionEngine::TConfiguration MakeYDBConfiguration() { + return { + .IgnoredRules = {}, + }; + } + + ISqlCompletionEngine::TConfiguration MakeYQLConfiguration() { + return MakeConfiguration(/* allowedStmts = */ { + "lambda_stmt", + "pragma_stmt", + "select_stmt", + "named_nodes_stmt", + "drop_table_stmt", + "use_stmt", + "into_table_stmt", + "commit_stmt", + "declare_stmt", + "import_stmt", + "export_stmt", + "do_stmt", + "define_action_or_subquery_stmt", + "if_stmt", + "for_stmt", + "values_stmt", + }); + } + ISqlCompletionEngine::TPtr MakeSqlCompletionEngine( TLexerSupplier lexer, INameService::TPtr names, diff --git a/yql/essentials/sql/v1/complete/sql_complete.h b/yql/essentials/sql/v1/complete/sql_complete.h index 1bc2c0ecf4e..e5dffcb8d27 100644 --- a/yql/essentials/sql/v1/complete/sql_complete.h +++ b/yql/essentials/sql/v1/complete/sql_complete.h @@ -1,6 +1,7 @@ #pragma once #include <yql/essentials/sql/v1/complete/core/input.h> +#include <yql/essentials/sql/v1/complete/core/environment.h> #include <yql/essentials/sql/v1/complete/name/service/name_service.h> #include <yql/essentials/sql/v1/lexer/lexer.h> @@ -8,6 +9,7 @@ #include <util/generic/string.h> #include <util/generic/vector.h> +#include <util/generic/hash_set.h> namespace NSQLComplete { @@ -31,6 +33,7 @@ namespace NSQLComplete { struct TCandidate { ECandidateKind Kind; TString Content; + size_t CursorShift = 0; friend bool operator==(const TCandidate& lhs, const TCandidate& rhs) = default; }; @@ -46,15 +49,21 @@ namespace NSQLComplete { struct TConfiguration { size_t Limit = 256; + THashSet<TString> IgnoredRules; }; virtual ~ISqlCompletionEngine() = default; virtual TCompletion Complete(TCompletionInput input) = 0; // TODO(YQL-19747): migrate YDB CLI to CompleteAsync - virtual NThreading::TFuture<TCompletion> CompleteAsync(TCompletionInput input) = 0; + virtual NThreading::TFuture<TCompletion> + CompleteAsync(TCompletionInput input, TEnvironment env = {}) = 0; }; using TLexerSupplier = std::function<NSQLTranslation::ILexer::TPtr(bool ansi)>; + ISqlCompletionEngine::TConfiguration MakeYDBConfiguration(); + + ISqlCompletionEngine::TConfiguration MakeYQLConfiguration(); + ISqlCompletionEngine::TPtr MakeSqlCompletionEngine( TLexerSupplier lexer, INameService::TPtr names, diff --git a/yql/essentials/sql/v1/complete/sql_complete_ut.cpp b/yql/essentials/sql/v1/complete/sql_complete_ut.cpp index d3a39d1c503..7460924b48f 100644 --- a/yql/essentials/sql/v1/complete/sql_complete_ut.cpp +++ b/yql/essentials/sql/v1/complete/sql_complete_ut.cpp @@ -1,7 +1,7 @@ #include "sql_complete.h" +#include <yql/essentials/sql/v1/complete/syntax/grammar.h> #include <yql/essentials/sql/v1/complete/name/cluster/static/discovery.h> -#include <yql/essentials/sql/v1/complete/name/object/dispatch/schema.h> #include <yql/essentials/sql/v1/complete/name/object/simple/schema.h> #include <yql/essentials/sql/v1/complete/name/object/simple/static/schema.h> #include <yql/essentials/sql/v1/complete/name/service/ranking/frequency.h> @@ -16,6 +16,8 @@ #include <yql/essentials/sql/v1/lexer/antlr4_pure_ansi/lexer.h> #include <library/cpp/testing/unittest/registar.h> +#include <library/cpp/iterator/iterate_keys.h> +#include <library/cpp/iterator/functools.h> #include <util/charset/utf8.h> @@ -78,7 +80,7 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { }, }; - THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> fss = { + THashMap<TString, THashMap<TString, TVector<TFolderEntry>>> fs = { {"", {{"/", {{"Folder", "local"}, {"Folder", "test"}, {"Folder", "prod"}, @@ -94,40 +96,29 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { {{"/", {{"Table", "people"}, {"Folder", "yql"}}}, {"/yql/", {{"Table", "tutorial"}}}}}, - {"yt:saurus", + {"saurus", {{"/", {{"Table", "maxim"}}}}}, }; - TVector<TString> clusters; - for (const auto& [cluster, _] : fss) { - clusters.emplace_back(cluster); - } - EraseIf(clusters, [](const auto& s) { return s.empty(); }); + auto clustersIt = NFuncTools::Filter( + [](const auto& x) { return !x.empty(); }, IterateKeys(fs)); + TVector<TString> clusters(begin(clustersIt), end(clustersIt)); TFrequencyData frequency; - IRanking::TPtr ranking = MakeDefaultRanking(frequency); - - THashMap<TString, ISchema::TPtr> schemasByCluster; - for (auto& [cluster, fs] : fss) { - schemasByCluster[std::move(cluster)] = - MakeSimpleSchema( - MakeStaticSimpleSchema(std::move(fs))); - } - TVector<INameService::TPtr> children = { MakeStaticNameService(std::move(names), frequency), - MakeSchemaNameService(MakeDispatchSchema(std::move(schemasByCluster))), + MakeSchemaNameService(MakeSimpleSchema(MakeStaticSimpleSchema(std::move(fs)))), MakeClusterNameService(MakeStaticClusterDiscovery(std::move(clusters))), }; - - INameService::TPtr service = MakeUnionNameService(std::move(children), ranking); + INameService::TPtr service = MakeUnionNameService( + std::move(children), MakeDefaultRanking(frequency)); return MakeSqlCompletionEngine(std::move(lexer), std::move(service)); } - TVector<TCandidate> Complete(ISqlCompletionEngine::TPtr& engine, TString sharped) { - return engine->CompleteAsync(SharpedInput(sharped)).GetValueSync().Candidates; + TVector<TCandidate> Complete(ISqlCompletionEngine::TPtr& engine, TString sharped, TEnvironment env = {}) { + return engine->CompleteAsync(SharpedInput(sharped), std::move(env)).GetValueSync().Candidates; } TVector<TCandidate> CompleteTop(size_t limit, ISqlCompletionEngine::TPtr& engine, TString sharped) { @@ -189,12 +180,47 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { Y_UNIT_TEST(Use) { TVector<TCandidate> expected = { {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, }; auto engine = MakeSqlCompletionEngineUT(); UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "USE "), expected); } + Y_UNIT_TEST(UseClusterResultion) { + auto engine = MakeSqlCompletionEngineUT(); + { + TVector<TCandidate> expected = { + {TableName, "`maxim"}, + {ClusterName, "example"}, + {ClusterName, "saurus"}, + {Keyword, "ANY"}, + }; + UNIT_ASSERT_VALUES_EQUAL( + Complete( + engine, + "USE yt:$cluster_name; SELECT * FROM ", + {.Parameters = {{"cluster_name", "saurus"}}}), + expected); + } + { + TVector<TCandidate> expected = { + {FolderName, "`.sys/"}, + {FolderName, "`local/"}, + {FolderName, "`prod/"}, + {FolderName, "`test/"}, + {ClusterName, "example"}, + {ClusterName, "saurus"}, + {Keyword, "ANY"}, + }; + UNIT_ASSERT_VALUES_EQUAL( + Complete( + engine, + "USE yt:$cluster_name; SELECT * FROM ", + {.Parameters = {}}), + expected); + } + } + Y_UNIT_TEST(Alter) { TVector<TCandidate> expected = { {Keyword, "ASYNC REPLICATION"}, @@ -243,12 +269,12 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { auto engine = MakeSqlCompletionEngineUT(); { TVector<TCandidate> expected = { - {FolderName, "`.sys/`"}, - {FolderName, "`local/`"}, - {FolderName, "`prod/`"}, - {FolderName, "`test/`"}, + {FolderName, "`.sys/"}, + {FolderName, "`local/"}, + {FolderName, "`prod/"}, + {FolderName, "`test/"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "IF NOT EXISTS"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "CREATE TABLE #"), expected); @@ -292,12 +318,12 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { Y_UNIT_TEST(DropObject) { TVector<TCandidate> expected = { - {FolderName, "`.sys/`"}, - {FolderName, "`local/`"}, - {FolderName, "`prod/`"}, - {FolderName, "`test/`"}, + {FolderName, "`.sys/"}, + {FolderName, "`local/"}, + {FolderName, "`prod/"}, + {FolderName, "`test/"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "IF EXISTS"}, }; auto engine = MakeSqlCompletionEngineUT(); @@ -488,12 +514,12 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { auto engine = MakeSqlCompletionEngineUT(); { TVector<TCandidate> expected = { - {FolderName, "`.sys/`"}, - {FolderName, "`local/`"}, - {FolderName, "`prod/`"}, - {FolderName, "`test/`"}, + {FolderName, "`.sys/"}, + {FolderName, "`local/"}, + {FolderName, "`prod/"}, + {FolderName, "`test/"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "ANY"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "SELECT * FROM "), expected); @@ -501,7 +527,7 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { { TString input = "SELECT * FROM pr"; TVector<TCandidate> expected = { - {FolderName, "`prod/`"}, + {FolderName, "`prod/"}, }; TCompletion actual = engine->Complete(SharpedInput(input)); UNIT_ASSERT_VALUES_EQUAL(actual.Candidates, expected); @@ -593,12 +619,7 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { auto engine = MakeSqlCompletionEngineUT(); { TVector<TCandidate> expected = { - {ClusterName, "yt:saurus"}, - }; - UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "SELECT * FROM yt#"), expected); - } - { - TVector<TCandidate> expected = { + {ClusterName, "example"}, {ClusterName, "saurus"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "SELECT * FROM yt:"), expected); @@ -611,13 +632,13 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { } { TVector<TCandidate> expected = { - {TableName, "`maxim`"}, + {TableName, "`maxim"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "SELECT * FROM yt:saurus."), expected); } { TVector<TCandidate> expected = { - {TableName, "`people`"}, + {TableName, "`people"}, }; UNIT_ASSERT_VALUES_EQUAL(CompleteTop(1, engine, "SELECT * FROM example."), expected); } @@ -633,34 +654,34 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { auto engine = MakeSqlCompletionEngineUT(); { TVector<TCandidate> expected = { - {TableName, "`maxim`"}, + {TableName, "`maxim"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "ANY"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "USE yt:saurus; SELECT * FROM "), expected); } { TVector<TCandidate> expected = { - {TableName, "`people`"}, - {FolderName, "`yql/`"}, + {TableName, "`people"}, + {FolderName, "`yql/"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "USE yt:saurus; SELECT * FROM example."), expected); } { TVector<TCandidate> expected = { - {TableName, "`maxim`"}, + {TableName, "`maxim"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "ANY"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "USE example; USE yt:saurus; SELECT * FROM "), expected); } { TVector<TCandidate> expected = { - {TableName, "`maxim`"}, + {TableName, "`maxim"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "ANY"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, R"( @@ -673,10 +694,10 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { } { TVector<TCandidate> expected = { - {TableName, "`people`"}, - {FolderName, "`yql/`"}, + {TableName, "`people"}, + {FolderName, "`yql/"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, {Keyword, "ANY"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, R"( @@ -745,12 +766,12 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { auto engine = MakeSqlCompletionEngineUT(); { TVector<TCandidate> expected = { - {FolderName, "`.sys/`"}, - {FolderName, "`local/`"}, - {FolderName, "`prod/`"}, - {FolderName, "`test/`"}, + {FolderName, "`.sys/"}, + {FolderName, "`local/"}, + {FolderName, "`prod/"}, + {FolderName, "`test/"}, {ClusterName, "example"}, - {ClusterName, "yt:saurus"}, + {ClusterName, "saurus"}, }; UNIT_ASSERT_VALUES_EQUAL(Complete(engine, "UPSERT INTO "), expected); } @@ -980,12 +1001,17 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { } Y_UNIT_TEST(Tabbing) { - TString query = - "SELECT \n" - " 123467, \"Hello, {name}! 编码\"}, \n" - " (1 + (5 * 1 / 0)), MIN(identifier), \n" - " Bool(field), Math::Sin(var) \n" - "FROM `local/test/space/table` JOIN test;"; + TString query = R"( +USE example; + +SELECT + 123467, \"Hello, {name}! 编码\"}, + (1 + (5 * 1 / 0)), MIN(identifier), + Bool(field), Math::Sin(var) +FROM `local/test/space/table` +JOIN yt:$cluster_name.test; +)"; + query += query + ";"; query += query + ";"; @@ -1176,4 +1202,30 @@ Y_UNIT_TEST_SUITE(SqlCompleteTests) { } } + Y_UNIT_TEST(IgnoredRules) { + auto lexer = MakePureLexerSupplier(); + + TNameSet names; + TFrequencyData frequency; + auto service = MakeStaticNameService(names, MakeDefaultRanking(frequency)); + + { + auto engine = MakeSqlCompletionEngine(lexer, service); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"UPDA"}).size(), 0); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"DELE"}).size(), 0); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"ROLL"}).size(), 0); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"INSE"}).size(), 0); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"SELE"}).size(), 0); + } + + auto config = MakeYQLConfiguration(); + auto engine = MakeSqlCompletionEngine(lexer, std::move(service), config); + + UNIT_ASSERT_EQUAL(Complete(engine, {"UPDA"}).size(), 0); + UNIT_ASSERT_EQUAL(Complete(engine, {"DELE"}).size(), 0); + UNIT_ASSERT_EQUAL(Complete(engine, {"ROLL"}).size(), 0); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"INSE"}).size(), 0); + UNIT_ASSERT_UNEQUAL(Complete(engine, {"SELE"}).size(), 0); + } + } // Y_UNIT_TEST_SUITE(SqlCompleteTests) diff --git a/yql/essentials/sql/v1/complete/syntax/grammar.cpp b/yql/essentials/sql/v1/complete/syntax/grammar.cpp index c080fae5ae4..790f272db86 100644 --- a/yql/essentials/sql/v1/complete/syntax/grammar.cpp +++ b/yql/essentials/sql/v1/complete/syntax/grammar.cpp @@ -34,6 +34,18 @@ namespace NSQLComplete { return Parser_->getRuleNames().at(rule); } + TRuleId GetRuleId(std::string_view symbolized) const override { + TRuleId index = Parser_->getRuleIndex(std::string(symbolized)); + if (index == INVALID_INDEX) { + ythrow yexception() << "Rule \"" << symbolized << "\" not found"; + } + return index; + } + + const std::vector<std::string>& GetAllRules() const override { + return Parser_->getRuleNames(); + } + private: static THolder<antlr4::Parser> MakeDummyParser() { return MakeHolder<NALADefaultAntlr4::SQLv1Antlr4Parser>(nullptr); diff --git a/yql/essentials/sql/v1/complete/syntax/grammar.h b/yql/essentials/sql/v1/complete/syntax/grammar.h index b128129e95f..4a259493dbc 100644 --- a/yql/essentials/sql/v1/complete/syntax/grammar.h +++ b/yql/essentials/sql/v1/complete/syntax/grammar.h @@ -6,6 +6,7 @@ #include <unordered_set> #include <string> +#include <string_view> #ifdef TOKEN_QUERY // Conflict with the winnt.h #undef TOKEN_QUERY @@ -21,6 +22,8 @@ namespace NSQLComplete { public: virtual const antlr4::dfa::Vocabulary& GetVocabulary() const = 0; virtual const std::string& SymbolizedRule(TRuleId rule) const = 0; + virtual TRuleId GetRuleId(std::string_view symbolized) const = 0; + virtual const std::vector<std::string>& GetAllRules() const = 0; virtual const std::unordered_set<TTokenId>& GetAllTokens() const = 0; virtual const std::unordered_set<TTokenId>& GetKeywordTokens() const = 0; virtual const std::unordered_set<TTokenId>& GetPunctuationTokens() const = 0; diff --git a/yql/essentials/sql/v1/complete/syntax/local.cpp b/yql/essentials/sql/v1/complete/syntax/local.cpp index a9803feae85..772f5b78dd5 100644 --- a/yql/essentials/sql/v1/complete/syntax/local.cpp +++ b/yql/essentials/sql/v1/complete/syntax/local.cpp @@ -49,10 +49,11 @@ namespace NSQLComplete { TDefaultYQLGrammar>; public: - explicit TSpecializedLocalSyntaxAnalysis(TLexerSupplier lexer) + explicit TSpecializedLocalSyntaxAnalysis( + TLexerSupplier lexer, const THashSet<TString>& IgnoredRules) : Grammar_(&GetSqlGrammar()) , Lexer_(lexer(/* ansi = */ IsAnsiLexer)) - , C3_(ComputeC3Config()) + , C3_(ComputeC3Config(IgnoredRules)) { } @@ -105,10 +106,11 @@ namespace NSQLComplete { } private: - IC3Engine::TConfig ComputeC3Config() const { + IC3Engine::TConfig ComputeC3Config(const THashSet<TString>& IgnoredRules) const { return { .IgnoredTokens = ComputeIgnoredTokens(), .PreferredRules = ComputePreferredRules(), + .IgnoredRules = ComputeIgnoredRules(IgnoredRules), }; } @@ -127,6 +129,15 @@ namespace NSQLComplete { return GetC3PreferredRules(); } + std::unordered_set<TRuleId> ComputeIgnoredRules(const THashSet<TString>& IgnoredRules) const { + std::unordered_set<TRuleId> ignored; + ignored.reserve(IgnoredRules.size()); + for (const auto& ruleName : IgnoredRules) { + ignored.emplace(Grammar_->GetRuleId(ruleName)); + } + return ignored; + } + TC3Candidates C3Complete(TCompletionInput statement, const TCursorTokenContext& context) { auto enclosing = context.Enclosing(); @@ -311,9 +322,10 @@ namespace NSQLComplete { class TLocalSyntaxAnalysis: public ILocalSyntaxAnalysis { public: - explicit TLocalSyntaxAnalysis(TLexerSupplier lexer) - : DefaultEngine_(lexer) - , AnsiEngine_(lexer) + explicit TLocalSyntaxAnalysis( + TLexerSupplier lexer, const THashSet<TString>& IgnoredRules) + : DefaultEngine_(lexer, IgnoredRules) + , AnsiEngine_(lexer, IgnoredRules) { } @@ -335,8 +347,9 @@ namespace NSQLComplete { TSpecializedLocalSyntaxAnalysis</* IsAnsiLexer = */ true> AnsiEngine_; }; - ILocalSyntaxAnalysis::TPtr MakeLocalSyntaxAnalysis(TLexerSupplier lexer) { - return MakeHolder<TLocalSyntaxAnalysis>(lexer); + ILocalSyntaxAnalysis::TPtr MakeLocalSyntaxAnalysis( + TLexerSupplier lexer, const THashSet<TString>& IgnoredRules) { + return MakeHolder<TLocalSyntaxAnalysis>(lexer, IgnoredRules); } } // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/syntax/local.h b/yql/essentials/sql/v1/complete/syntax/local.h index 635485d2b7c..0017afa0684 100644 --- a/yql/essentials/sql/v1/complete/syntax/local.h +++ b/yql/essentials/sql/v1/complete/syntax/local.h @@ -66,6 +66,7 @@ namespace NSQLComplete { virtual ~ILocalSyntaxAnalysis() = default; }; - ILocalSyntaxAnalysis::TPtr MakeLocalSyntaxAnalysis(TLexerSupplier lexer); + ILocalSyntaxAnalysis::TPtr MakeLocalSyntaxAnalysis( + TLexerSupplier lexer, const THashSet<TString>& IgnoredRules); } // namespace NSQLComplete diff --git a/yql/essentials/sql/v1/complete/ut/ya.make b/yql/essentials/sql/v1/complete/ut/ya.make index c978e6e6048..d53220ff3b2 100644 --- a/yql/essentials/sql/v1/complete/ut/ya.make +++ b/yql/essentials/sql/v1/complete/ut/ya.make @@ -8,7 +8,6 @@ PEERDIR( yql/essentials/sql/v1/lexer/antlr4_pure yql/essentials/sql/v1/lexer/antlr4_pure_ansi yql/essentials/sql/v1/complete/name/cluster/static - yql/essentials/sql/v1/complete/name/object/dispatch yql/essentials/sql/v1/complete/name/object/simple yql/essentials/sql/v1/complete/name/object/simple/static yql/essentials/sql/v1/complete/name/service/cluster diff --git a/yql/essentials/sql/v1/format/sql_format_ut.h b/yql/essentials/sql/v1/format/sql_format_ut.h index 8bb2af27939..0a9726aa64d 100644 --- a/yql/essentials/sql/v1/format/sql_format_ut.h +++ b/yql/essentials/sql/v1/format/sql_format_ut.h @@ -1534,6 +1534,8 @@ Y_UNIT_TEST(Union) { "SELECT\n\t1\nUNION ALL\nSELECT\n\t2\nUNION\nSELECT\n\t3\nUNION ALL\nSELECT\n\t4\nUNION\nSELECT\n\t5\n;\n"}, {"select 1 union all (select 2)", "SELECT\n\t1\nUNION ALL\n(\n\tSELECT\n\t\t2\n);\n"}, + {"select 1 union distinct select 2 union select 3 union distinct select 4 union select 5", + "SELECT\n\t1\nUNION DISTINCT\nSELECT\n\t2\nUNION\nSELECT\n\t3\nUNION DISTINCT\nSELECT\n\t4\nUNION\nSELECT\n\t5\n;\n"}, }; TSetup setup; diff --git a/yql/essentials/sql/v1/sql_select.cpp b/yql/essentials/sql/v1/sql_select.cpp index 52dd399a990..8e11721c84f 100644 --- a/yql/essentials/sql/v1/sql_select.cpp +++ b/yql/essentials/sql/v1/sql_select.cpp @@ -1388,18 +1388,28 @@ TSourcePtr TSqlSelect::Build(const TRule& node, TPosition pos, TSelectKindResult outermostSettings.Label = next.Settings.Label; } - switch (b.GetRule_select_op1().Alt_case()) { - case TRule_select_op::kAltSelectOp1: - break; - case TRule_select_op::kAltSelectOp2: - case TRule_select_op::kAltSelectOp3: - Ctx.Error() << "INTERSECT and EXCEPT are not implemented yet"; - return nullptr; - case TRule_select_op::ALT_NOT_SET: - Y_ABORT("You should change implementation according to grammar changes"); + auto selectOp = b.GetRule_select_op1(); + const TString token = ToLowerUTF8(Token(selectOp.GetToken1())); + if (token == "union") { + // nothing + } else if (token == "intersect" || token == "except") { + Ctx.Error() << "INTERSECT and EXCEPT are not implemented yet"; + return nullptr; + } else { + Y_ABORT("You should change implementation according to grammar changes. Invalid token: %s", token.c_str()); } - const bool quantifier = b.GetRule_select_op1().GetAlt_select_op1().HasBlock2(); + bool quantifier = false; + if (selectOp.HasBlock2()) { + const TString token = ToLowerUTF8(Token(selectOp.GetBlock2().GetToken1())); + if (token == "all") { + quantifier = true; + } else if (token == "distinct") { + // nothing + } else { + Y_ABORT("You should change implementation according to grammar changes. Invalid token: %s", token.c_str()); + } + } if (!second && quantifier != currentQuantifier) { auto source = BuildUnion(pos, std::move(sources), currentQuantifier, {}); diff --git a/yql/essentials/sql/v1/sql_ut_common.h b/yql/essentials/sql/v1/sql_ut_common.h index 15ef58b8d86..cead4e2799e 100644 --- a/yql/essentials/sql/v1/sql_ut_common.h +++ b/yql/essentials/sql/v1/sql_ut_common.h @@ -1714,6 +1714,15 @@ Y_UNIT_TEST_SUITE(SqlParsingOnly) { UNIT_ASSERT_VALUES_EQUAL(1, elementStat["Union"]); } + Y_UNIT_TEST(UnionDistinctTest) { + NYql::TAstParseResult res = SqlToYql("SELECT key FROM plato.Input UNION DISTINCT select subkey FROM plato.Input;"); + UNIT_ASSERT(res.Root); + + TWordCountHive elementStat = {{TString("Union"), 0}}; + VerifyProgram(res, elementStat, {}); + UNIT_ASSERT_VALUES_EQUAL(1, elementStat["Union"]); + } + Y_UNIT_TEST(UnionAggregationTest) { NYql::TAstParseResult res = SqlToYql(R"( PRAGMA DisableEmitUnionMerge; diff --git a/yql/essentials/tests/common/test_framework/yql_utils.py b/yql/essentials/tests/common/test_framework/yql_utils.py index 214f963393f..9ecbe4a1cf7 100644 --- a/yql/essentials/tests/common/test_framework/yql_utils.py +++ b/yql/essentials/tests/common/test_framework/yql_utils.py @@ -503,6 +503,14 @@ def get_langver(cfg): return None +def get_envs(cfg): + envs = dict() + for item in cfg: + if item[0] == 'env': + envs[item[1]] = item[2] + return envs + + def is_skip_forceblocks(cfg): for item in cfg: if item[0] == 'skip_forceblocks': diff --git a/yql/essentials/tests/common/udf_test/test.py b/yql/essentials/tests/common/udf_test/test.py index 359987b2604..6936bde408e 100644 --- a/yql/essentials/tests/common/udf_test/test.py +++ b/yql/essentials/tests/common/udf_test/test.py @@ -78,6 +78,9 @@ def test(case): if yql_utils.get_param('TARGET_PLATFORM') and xfail: pytest.skip('xfail is not supported on non-default target platform') langver = yql_utils.get_langver(cfg) + envs = yql_utils.get_envs(cfg) + if not langver: + langver = "unknown" # no default version, because UDFs may have different release cycles extra_env = dict(os.environ) @@ -85,6 +88,7 @@ def test(case): extra_env["YQL_ARCADIA_BINARY_PATH"] = os.path.expandvars(yatest.common.build_path('.')) extra_env["YQL_ARCADIA_SOURCE_PATH"] = os.path.expandvars(yatest.common.source_path('.')) extra_env["Y_NO_AVX_IN_DOT_PRODUCT"] = "1" + extra_env.update(envs) # this breaks tests using V0 syntax if "YA_TEST_RUNNER" in extra_env: diff --git a/yql/essentials/tests/sql/minirun/part6/canondata/result.json b/yql/essentials/tests/sql/minirun/part6/canondata/result.json index 74fb98a7cc1..b31a0804b64 100644 --- a/yql/essentials/tests/sql/minirun/part6/canondata/result.json +++ b/yql/essentials/tests/sql/minirun/part6/canondata/result.json @@ -1449,6 +1449,20 @@ "uri": "https://{canondata_backend}/995452/27fc70b9589f65bcb911832f0d505bea9e66db7f/resource.tar.gz#test.test_udf-named_args--Results_/results.txt" } ], + "test.test[union-union_distinct-default.txt-Debug]": [ + { + "checksum": "4e68e4a9a59a12794bcae33aa5d639fa", + "size": 297, + "uri": "https://{canondata_backend}/1937492/e480305ecff3ccfe7cdc193e8c9dcc11c255e2c8/resource.tar.gz#test.test_union-union_distinct-default.txt-Debug_/opt.yql" + } + ], + "test.test[union-union_distinct-default.txt-Results]": [ + { + "checksum": "c5237daf2d6c78619bf209e9a39b7a12", + "size": 976, + "uri": "https://{canondata_backend}/1937492/e480305ecff3ccfe7cdc193e8c9dcc11c255e2c8/resource.tar.gz#test.test_union-union_distinct-default.txt-Results_/results.txt" + } + ], "test.test[window-rank/nulls_legacy-default.txt-Debug]": [ { "checksum": "fe3da55ac0b8ab732b49a5d8760d80db", diff --git a/yql/essentials/tests/sql/sql2yql/canondata/result.json b/yql/essentials/tests/sql/sql2yql/canondata/result.json index 920aafb870e..d13181fa778 100644 --- a/yql/essentials/tests/sql/sql2yql/canondata/result.json +++ b/yql/essentials/tests/sql/sql2yql/canondata/result.json @@ -7286,6 +7286,13 @@ "uri": "https://{canondata_backend}/1775319/f1fa0c55bf9f13cff57cf1c990c2330caed8eb1b/resource.tar.gz#test_sql2yql.test_union-union_column_extention_/sql.yql" } ], + "test_sql2yql.test[union-union_distinct]": [ + { + "checksum": "fc0bf00619910dc8863d1e4982c07065", + "size": 1676, + "uri": "https://{canondata_backend}/1925842/1b6f19c4d4916f84ec5d150703910a73500e178a/resource.tar.gz#test_sql2yql.test_union-union_distinct_/sql.yql" + } + ], "test_sql2yql.test[union-union_mix]": [ { "checksum": "a4681f5145adcca6d4a4af7c5e164d73", @@ -11253,6 +11260,11 @@ "uri": "file://test_sql_format.test_union-union_column_extention_/formatted.sql" } ], + "test_sql_format.test[union-union_distinct]": [ + { + "uri": "file://test_sql_format.test_union-union_distinct_/formatted.sql" + } + ], "test_sql_format.test[union-union_mix]": [ { "uri": "file://test_sql_format.test_union-union_mix_/formatted.sql" diff --git a/yql/essentials/tests/sql/sql2yql/canondata/test_sql_format.test_union-union_distinct_/formatted.sql b/yql/essentials/tests/sql/sql2yql/canondata/test_sql_format.test_union-union_distinct_/formatted.sql new file mode 100644 index 00000000000..c3d0605b489 --- /dev/null +++ b/yql/essentials/tests/sql/sql2yql/canondata/test_sql_format.test_union-union_distinct_/formatted.sql @@ -0,0 +1,8 @@ +SELECT + 1 AS x, + 2 AS y +UNION DISTINCT +SELECT + 1 AS x, + 2 AS y +; diff --git a/yql/essentials/tests/sql/suites/union/union_distinct.sql b/yql/essentials/tests/sql/suites/union/union_distinct.sql new file mode 100644 index 00000000000..595d54c7284 --- /dev/null +++ b/yql/essentials/tests/sql/suites/union/union_distinct.sql @@ -0,0 +1,7 @@ +SELECT + 1 as x, + 2 as y +UNION DISTINCT +SELECT + 1 as x, + 2 as y; diff --git a/yql/essentials/tools/udf_resolver/discover.cpp b/yql/essentials/tools/udf_resolver/discover.cpp index 530e7d96e98..240cd174b46 100644 --- a/yql/essentials/tools/udf_resolver/discover.cpp +++ b/yql/essentials/tools/udf_resolver/discover.cpp @@ -86,6 +86,8 @@ NYql::TResolveResult DoDiscover(const NYql::TResolve& inMsg, IMutableFunctionReg udfRes->SetSupportsBlocks(funcInfo.SupportsBlocks); udfRes->SetIsStrict(funcInfo.IsStrict); + udfRes->SetMinLangVer(funcInfo.MinLangVer); + udfRes->SetMaxLangVer(funcInfo.MaxLangVer); } } } diff --git a/yql/essentials/tools/udf_resolver/udf_resolver.cpp b/yql/essentials/tools/udf_resolver/udf_resolver.cpp index 03116858f7e..9db127d0864 100644 --- a/yql/essentials/tools/udf_resolver/udf_resolver.cpp +++ b/yql/essentials/tools/udf_resolver/udf_resolver.cpp @@ -194,6 +194,8 @@ void ResolveUDFs() { udfRes->SetSupportsBlocks(funcInfo.SupportsBlocks); udfRes->SetIsStrict(funcInfo.IsStrict); + udfRes->SetMinLangVer(funcInfo.MinLangVer); + udfRes->SetMaxLangVer(funcInfo.MaxLangVer); } catch (yexception& e) { udfRes->SetError(TStringBuilder() << "Internal error was found when udf metadata is loading for function: " << udf.GetName() diff --git a/yql/essentials/tools/yql_complete/yql_complete b/yql/essentials/tools/yql_complete/yql_complete deleted file mode 100644 index 7fc1116ee5d..00000000000 --- a/yql/essentials/tools/yql_complete/yql_complete +++ /dev/null @@ -1 +0,0 @@ -/home/vityaman/.ya/build/symres/bbe5c007c4bcc83d4396e13689e6b39b/yql_complete
\ No newline at end of file diff --git a/yql/essentials/tools/yql_facade_run/yql_facade_run.cpp b/yql/essentials/tools/yql_facade_run/yql_facade_run.cpp index ef74c02a1d3..d4a7121599c 100644 --- a/yql/essentials/tools/yql_facade_run/yql_facade_run.cpp +++ b/yql/essentials/tools/yql_facade_run/yql_facade_run.cpp @@ -443,7 +443,9 @@ void TFacadeRunOptions::Parse(int argc, const char *argv[]) { opts.AddLongOption("langver", "Set current language version").Optional().RequiredArgument("VER") .Handler1T<TString>([this](const TString& str) { - if (!ParseLangVersion(str, LangVer)) { + if (str == "unknown") { + LangVer = UnknownLangVersion; + } else if (!ParseLangVersion(str, LangVer)) { throw yexception() << "Failed to parse language version: " << str; } }); diff --git a/yql/essentials/tools/yql_facade_run/yql_facade_run.h b/yql/essentials/tools/yql_facade_run/yql_facade_run.h index a8661861073..1e6965cf888 100644 --- a/yql/essentials/tools/yql_facade_run/yql_facade_run.h +++ b/yql/essentials/tools/yql_facade_run/yql_facade_run.h @@ -70,7 +70,7 @@ public: ~TFacadeRunOptions(); EProgramType ProgramType = EProgramType::SExpr; - TLangVersion LangVer = UnknownLangVersion; + TLangVersion LangVer = MinLangVersion; TLangVersion MaxLangVer = GetMaxLangVersion(); NYson::EYsonFormat ResultsFormat = NYson::EYsonFormat::Text; ERunMode Mode = ERunMode::Run; diff --git a/yql/essentials/udfs/common/re2/re2_udf.cpp b/yql/essentials/udfs/common/re2/re2_udf.cpp index 965d24fe368..55c110b3458 100644 --- a/yql/essentials/udfs/common/re2/re2_udf.cpp +++ b/yql/essentials/udfs/common/re2/re2_udf.cpp @@ -1,8 +1,11 @@ +#include <yql/essentials/public/langver/yql_langver.h> #include <yql/essentials/public/udf/udf_helpers.h> +#include <yql/essentials/public/udf/udf_type_ops.h> #include <yql/essentials/public/udf/udf_value_builder.h> #include <contrib/libs/re2/re2/re2.h> +#include <util/system/env.h> #include <util/charset/utf8.h> #include <util/string/cast.h> @@ -36,7 +39,40 @@ namespace { xx(WordBoundary, 11, bool, false, set_word_boundary, Id) \ xx(OneLine, 12, bool, false, set_one_line, Id) - enum EOptionsField : ui32 { + ui64 GetFailProbability() { + auto envResult = TryGetEnv("YQL_RE2_REGEXP_PROBABILITY_FAIL"); + if (!envResult) { + return 0; + } + ui64 result; + bool isValid = TryIntFromString<10, ui64>(envResult->data(), envResult->size(), result); + Y_ENSURE(isValid, TStringBuilder() << "Error while parsing YQL_RE2_REGEXP_PROBABILITY_FAIL. Actual value is: " << *envResult); + return result; + } + + bool ShouldFailOnInvalidRegexp(const std::string_view regexp, NYql::TLangVersion currentLangVersion) { + if (currentLangVersion >= NYql::MakeLangVersion(2025, 3)) { + return true; + } + THashType hash = GetStringHash(regexp) % 100; + ui64 failProbability = GetFailProbability(); + return hash < failProbability; + } + + RE2::Options CreateDefaultOptions(){ + RE2::Options options; +#define FIELD_HANDLE(name, index, type, defVal, setter, conv) options.setter(conv(defVal)); + OPTIONS_MAP(FIELD_HANDLE) +#undef FIELD_HANDLE + options.set_log_errors(false); + return options; + } + + TString FormatRegexpError(const RE2& Regexp) { + return TStringBuilder() << "Regexp compilation failed. Regexp: \"" << Regexp.pattern() << "\". Original error is: \"" << Regexp.error() << "\""; + } + + enum EOptionsField: ui32 { OPTIONS_MAP(ENUM_VALUE_GEN) Count }; @@ -69,11 +105,13 @@ namespace { EMode mode, const TOptionsSchema& optionsSchema, TSourcePosition pos, + NYql::TLangVersion currentlangVersion, const TRegexpGroups& regexpGroups = TRegexpGroups()) : Mode(mode) , OptionsSchema(optionsSchema) , Pos_(pos) , RegexpGroups(regexpGroups) + , CurrentLangVersion(currentlangVersion) { } @@ -89,13 +127,15 @@ namespace { Mode, posix, OptionsSchema, - Pos_)); + Pos_, + CurrentLangVersion)); } EMode Mode; const TOptionsSchema OptionsSchema; TSourcePosition Pos_; const TRegexpGroups RegexpGroups; + NYql::TLangVersion CurrentLangVersion; }; static const TStringRef& Name(EMode mode) { @@ -130,18 +170,19 @@ namespace { EMode mode, bool posix, const TOptionsSchema& optionsSchema, - TSourcePosition pos) + TSourcePosition pos, + NYql::TLangVersion currentLangVersion) : RegexpGroups(regexpGroups) , Mode(mode) , Captured() , OptionsSchema(optionsSchema) , Pos_(pos) - { + , CurrentLangVersion(currentLangVersion) { try { auto patternValue = runConfig.GetElement(0); auto optionsValue = runConfig.GetElement(1); const std::string_view pattern(patternValue.AsStringRef()); - RE2::Options options; + RE2::Options options = CreateDefaultOptions(); options.set_posix_syntax(posix); bool needUtf8 = (UTF8Detect(pattern) == UTF8); @@ -154,10 +195,15 @@ namespace { #define FIELD_HANDLE(name, index, type, defVal, setter, conv) options.setter(conv(optionsValue.GetElement(OptionsSchema.Indices[index]).Get<type>())); OPTIONS_MAP(FIELD_HANDLE) #undef FIELD_HANDLE + options.set_log_errors(false); } Regexp = std::make_unique<RE2>(StringPiece(pattern.data(), pattern.size()), options); + if (!Regexp->ok() && ShouldFailOnInvalidRegexp(pattern, CurrentLangVersion)) { + throw yexception() << FormatRegexpError(*Regexp); + } + if (mode == EMode::CAPTURE) { Captured = std::make_unique<StringPiece[]>(Regexp->NumberOfCapturingGroups() + 1); } @@ -252,6 +298,7 @@ namespace { std::unique_ptr<StringPiece[]> Captured; const TOptionsSchema OptionsSchema; TSourcePosition Pos_; + NYql::TLangVersion CurrentLangVersion; TUnboxedValue BuildEmptyStruct(const IValueBuilder* valueBuilder) const { TUnboxedValue* items = nullptr; @@ -451,13 +498,18 @@ namespace { if (!typesOnly) { const auto mode = isMatch ? TRe2Udf::EMode::MATCH : TRe2Udf::EMode::GREP; - builder.Implementation(new TRe2Udf::TFactory<posix>(mode, optionsSchema, builder.GetSourcePosition())); + builder.Implementation(new TRe2Udf::TFactory<posix>(mode, optionsSchema, builder.GetSourcePosition(), builder.GetCurrentLangVer())); } } else if (isCapture) { TRegexpGroups groups; auto optionalStringType = builder.Optional()->Item<char*>().Build(); auto structBuilder = builder.Struct(); - RE2 regexp(StringPiece(typeConfig.Data(), typeConfig.Size())); + RE2::Options options = CreateDefaultOptions(); + RE2 regexp(StringPiece(typeConfig.Data(), typeConfig.Size()), options); + if (!regexp.ok()) { + builder.SetError(FormatRegexpError(regexp)); + return; + } const auto& groupNames = regexp.CapturingGroupNames(); int groupCount = regexp.NumberOfCapturingGroups(); if (groupCount >= 0) { @@ -487,35 +539,32 @@ namespace { if (!typesOnly) { builder.Implementation( - new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::CAPTURE, optionsSchema, builder.GetSourcePosition(), groups)); + new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::CAPTURE, optionsSchema, builder.GetSourcePosition(), builder.GetCurrentLangVer(), groups)); } } else { - if (regexp.ok()) { - builder.SetError("Regexp contains no capturing groups"); - } else { - builder.SetError(regexp.error()); - } + Y_ENSURE(regexp.ok()); + builder.SetError("Regexp contains no capturing groups"); } } else if (isReplace) { builder.SimpleSignature<TOptional<char*>(TOptional<char*>, char*)>() .RunConfig(MakeRunConfigType(builder, optOptionsStructType)); if (!typesOnly) { - builder.Implementation(new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::REPLACE, optionsSchema, builder.GetSourcePosition())); + builder.Implementation(new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::REPLACE, optionsSchema, builder.GetSourcePosition(), builder.GetCurrentLangVer())); } } else if (isCount) { builder.SimpleSignature<ui32(TOptional<char*>)>() .RunConfig(MakeRunConfigType(builder, optOptionsStructType)); if (!typesOnly) { - builder.Implementation(new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::COUNT, optionsSchema, builder.GetSourcePosition())); + builder.Implementation(new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::COUNT, optionsSchema, builder.GetSourcePosition(), builder.GetCurrentLangVer())); } } else if (isFindAndConsume) { builder.SimpleSignature<TListType<char*>(TOptional<char*>)>() .RunConfig(MakeRunConfigType(builder, optOptionsStructType)); if (!typesOnly) { - builder.Implementation(new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::FIND_AND_CONSUME, optionsSchema, builder.GetSourcePosition())); + builder.Implementation(new TRe2Udf::TFactory<posix>(TRe2Udf::EMode::FIND_AND_CONSUME, optionsSchema, builder.GetSourcePosition(), builder.GetCurrentLangVer())); } } else if (!( TEscape::DeclareSignature(name, userType, builder, typesOnly) || diff --git a/yql/essentials/udfs/common/re2/test/canondata/result.json b/yql/essentials/udfs/common/re2/test/canondata/result.json index b9a16f32d66..75db00ebaf8 100644 --- a/yql/essentials/udfs/common/re2/test/canondata/result.json +++ b/yql/essentials/udfs/common/re2/test/canondata/result.json @@ -24,6 +24,26 @@ "uri": "file://test.test_FindAndConsumeEmpty_/results.txt" } ], + "test.test[InvalidCaptureRegexFail]": [ + { + "uri": "file://test.test_InvalidCaptureRegexFail_/extracted" + } + ], + "test.test[InvalidRegexFail_2025.02]": [ + { + "uri": "file://test.test_InvalidRegexFail_2025.02_/extracted" + } + ], + "test.test[InvalidRegexFail_2025.03]": [ + { + "uri": "file://test.test_InvalidRegexFail_2025.03_/extracted" + } + ], + "test.test[InvalidRegexSuccess_2025.02]": [ + { + "uri": "file://test.test_InvalidRegexSuccess_2025.02_/results.txt" + } + ], "test.test[LikeEscape]": [ { "uri": "file://test.test_LikeEscape_/results.txt" diff --git a/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidCaptureRegexFail_/extracted b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidCaptureRegexFail_/extracted new file mode 100644 index 00000000000..f4d1c329f33 --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidCaptureRegexFail_/extracted @@ -0,0 +1,14 @@ +<tmp_path>/program.sql:<main>: Error: Type annotation + + <tmp_path>/program.sql:<main>:8:1: Error: At function: RemovePrefixMembers, At function: Unordered, At function: PersistableRepr, At function: OrderedSqlProject, At tuple, At function: SqlProjectItem, At lambda + select $invalidCaptureRegexp("abc"); + ^ + <tmp_path>/program.sql:<main>:8:8: Error: At function: Apply + select $invalidCaptureRegexp("abc"); + ^ + <tmp_path>/program.sql:<main>:4:30: Error: At function: Udf, At Re2.Capture + $invalidCaptureRegexp = Re2::Capture("["); + ^ + <tmp_path>/program.sql:<main>:4:30: Error: Failed to find UDF function: Re2.Capture, reason: Error: Module: Re2, function: Capture, error: Regexp compilation failed. Regexp: "[". Original error is: "missing ]: [" + $invalidCaptureRegexp = Re2::Capture("["); + ^
\ No newline at end of file diff --git a/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexFail_2025.02_/extracted b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexFail_2025.02_/extracted new file mode 100644 index 00000000000..f1fda30e02a --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexFail_2025.02_/extracted @@ -0,0 +1,8 @@ +<tmp_path>/program.sql:<main>: Error: Execution + + <tmp_path>/program.sql:<main>:8:1: Error: Execution of node: Result + SELECT $invalidRe("abaa"); + ^ + <tmp_path>/program.sql:<main>:4:19: Error: Regexp compilation failed. Regexp: "[". Original error is: "missing ]: [" + $invalidRe = Re2::FindAndConsume("["); + ^
\ No newline at end of file diff --git a/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexFail_2025.03_/extracted b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexFail_2025.03_/extracted new file mode 100644 index 00000000000..f1fda30e02a --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexFail_2025.03_/extracted @@ -0,0 +1,8 @@ +<tmp_path>/program.sql:<main>: Error: Execution + + <tmp_path>/program.sql:<main>:8:1: Error: Execution of node: Result + SELECT $invalidRe("abaa"); + ^ + <tmp_path>/program.sql:<main>:4:19: Error: Regexp compilation failed. Regexp: "[". Original error is: "missing ]: [" + $invalidRe = Re2::FindAndConsume("["); + ^
\ No newline at end of file diff --git a/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexSuccess_2025.02_/results.txt b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexSuccess_2025.02_/results.txt new file mode 100644 index 00000000000..c37c844a342 --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/canondata/test.test_InvalidRegexSuccess_2025.02_/results.txt @@ -0,0 +1,31 @@ +[ + { + "Write" = [ + { + "Type" = [ + "ListType"; + [ + "StructType"; + [ + [ + "column0"; + [ + "ListType"; + [ + "DataType"; + "String" + ] + ] + ] + ] + ] + ]; + "Data" = [ + [ + [] + ] + ] + } + ] + } +]
\ No newline at end of file diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidCaptureRegexFail.cfg b/yql/essentials/udfs/common/re2/test/cases/InvalidCaptureRegexFail.cfg new file mode 100644 index 00000000000..0704b3634e0 --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidCaptureRegexFail.cfg @@ -0,0 +1,2 @@ +xfail +env YQL_RE2_REGEXP_PROBABILITY_FAIL 0 // Test that failure does not depend on this variable. diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidCaptureRegexFail.sql b/yql/essentials/udfs/common/re2/test/cases/InvalidCaptureRegexFail.sql new file mode 100644 index 00000000000..09b17677508 --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidCaptureRegexFail.sql @@ -0,0 +1,4 @@ +/* syntax version 1 */ +$invalidCaptureRegexp = Re2::Capture("["); + +select $invalidCaptureRegexp("abc"); diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.02.cfg b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.02.cfg new file mode 100644 index 00000000000..c0ef97affef --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.02.cfg @@ -0,0 +1,3 @@ +xfail +env YQL_RE2_REGEXP_PROBABILITY_FAIL 100 +langver 2025.02 diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.02.sql b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.02.sql new file mode 100644 index 00000000000..087dcab2efd --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.02.sql @@ -0,0 +1,4 @@ +/* syntax version 1 */ +$invalidRe = Re2::FindAndConsume("["); + +SELECT $invalidRe("abaa"); diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.03.cfg b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.03.cfg new file mode 100644 index 00000000000..e37bb6b41a2 --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.03.cfg @@ -0,0 +1,2 @@ +langver 2025.03 +xfail diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.03.sql b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.03.sql new file mode 100644 index 00000000000..087dcab2efd --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexFail_2025.03.sql @@ -0,0 +1,4 @@ +/* syntax version 1 */ +$invalidRe = Re2::FindAndConsume("["); + +SELECT $invalidRe("abaa"); diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidRegexSuccess_2025.02.cfg b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexSuccess_2025.02.cfg new file mode 100644 index 00000000000..367bc6a9ec0 --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexSuccess_2025.02.cfg @@ -0,0 +1 @@ +langver 2025.02 diff --git a/yql/essentials/udfs/common/re2/test/cases/InvalidRegexSuccess_2025.02.sql b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexSuccess_2025.02.sql new file mode 100644 index 00000000000..087dcab2efd --- /dev/null +++ b/yql/essentials/udfs/common/re2/test/cases/InvalidRegexSuccess_2025.02.sql @@ -0,0 +1,4 @@ +/* syntax version 1 */ +$invalidRe = Re2::FindAndConsume("["); + +SELECT $invalidRe("abaa"); diff --git a/yql/essentials/udfs/common/re2/ya.make b/yql/essentials/udfs/common/re2/ya.make index 7e554133486..ca8be7370ba 100644 --- a/yql/essentials/udfs/common/re2/ya.make +++ b/yql/essentials/udfs/common/re2/ya.make @@ -2,7 +2,7 @@ YQL_UDF_CONTRIB(re2_udf) YQL_ABI_VERSION( 2 - 28 + 43 0 ) diff --git a/yql/essentials/udfs/common/string/test/canondata/result.json b/yql/essentials/udfs/common/string/test/canondata/result.json index aff883c405e..0feffac589c 100644 --- a/yql/essentials/udfs/common/string/test/canondata/result.json +++ b/yql/essentials/udfs/common/string/test/canondata/result.json @@ -14,6 +14,16 @@ "uri": "file://test.test_AsciiCmpIgnoreCase_2025_02_/extracted" } ], + "test.test[AsciiContainsIgnoreCase_2025_01]": [ + { + "uri": "file://test.test_AsciiContainsIgnoreCase_2025_01_/extracted" + } + ], + "test.test[AsciiContainsIgnoreCase_2025_01_scan]": [ + { + "uri": "file://test.test_AsciiContainsIgnoreCase_2025_01_scan_/extracted" + } + ], "test.test[Base32Decode]": [ { "uri": "file://test.test_Base32Decode_/results.txt" diff --git a/yql/essentials/udfs/common/string/test/canondata/test.test_AsciiContainsIgnoreCase_2025_01_/extracted b/yql/essentials/udfs/common/string/test/canondata/test.test_AsciiContainsIgnoreCase_2025_01_/extracted new file mode 100644 index 00000000000..6c66d69cd7f --- /dev/null +++ b/yql/essentials/udfs/common/string/test/canondata/test.test_AsciiContainsIgnoreCase_2025_01_/extracted @@ -0,0 +1,50 @@ +<tmp_path>/program.sql:<main>: Error: Type annotation + + <tmp_path>/program.sql:<main>:2:1: Error: At function: RemovePrefixMembers, At function: Unordered, At function: PersistableRepr, At function: OrderedSqlProject, At tuple + SELECT + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:6:13: Error: At function: Apply, At function: Udf + String::AsciiContainsIgnoreCase(value, "AS") AS iccontains, + ^ + <tmp_path>/program.sql:<main>:6:13: Error: UDF 'String.AsciiContainsIgnoreCase' is not available before version 2025.02 + String::AsciiContainsIgnoreCase(value, "AS") AS iccontains, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:8:13: Error: At function: Apply, At function: Udf + String::AsciiContainsIgnoreCase(value, "") AS icempty, + ^ + <tmp_path>/program.sql:<main>:8:13: Error: UDF 'String.AsciiContainsIgnoreCase' is not available before version 2025.02 + String::AsciiContainsIgnoreCase(value, "") AS icempty, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:10:13: Error: At function: Apply, At function: Udf + String::AsciiStartsWithIgnoreCase(value, "AS") AS icstarts, + ^ + <tmp_path>/program.sql:<main>:10:13: Error: UDF 'String.AsciiStartsWithIgnoreCase' is not available before version 2025.02 + String::AsciiStartsWithIgnoreCase(value, "AS") AS icstarts, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:12:13: Error: At function: Apply, At function: Udf + String::AsciiEndsWithIgnoreCase(value, "AS") AS icends, + ^ + <tmp_path>/program.sql:<main>:12:13: Error: UDF 'String.AsciiEndsWithIgnoreCase' is not available before version 2025.02 + String::AsciiEndsWithIgnoreCase(value, "AS") AS icends, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:14:13: Error: At function: Apply, At function: Udf + String::AsciiEqualsIgnoreCase(value, "FDSA") AS icequals, + ^ + <tmp_path>/program.sql:<main>:14:13: Error: UDF 'String.AsciiEqualsIgnoreCase' is not available before version 2025.02 + String::AsciiEqualsIgnoreCase(value, "FDSA") AS icequals, + ^
\ No newline at end of file diff --git a/yql/essentials/udfs/common/string/test/canondata/test.test_AsciiContainsIgnoreCase_2025_01_scan_/extracted b/yql/essentials/udfs/common/string/test/canondata/test.test_AsciiContainsIgnoreCase_2025_01_scan_/extracted new file mode 100644 index 00000000000..6c66d69cd7f --- /dev/null +++ b/yql/essentials/udfs/common/string/test/canondata/test.test_AsciiContainsIgnoreCase_2025_01_scan_/extracted @@ -0,0 +1,50 @@ +<tmp_path>/program.sql:<main>: Error: Type annotation + + <tmp_path>/program.sql:<main>:2:1: Error: At function: RemovePrefixMembers, At function: Unordered, At function: PersistableRepr, At function: OrderedSqlProject, At tuple + SELECT + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:6:13: Error: At function: Apply, At function: Udf + String::AsciiContainsIgnoreCase(value, "AS") AS iccontains, + ^ + <tmp_path>/program.sql:<main>:6:13: Error: UDF 'String.AsciiContainsIgnoreCase' is not available before version 2025.02 + String::AsciiContainsIgnoreCase(value, "AS") AS iccontains, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:8:13: Error: At function: Apply, At function: Udf + String::AsciiContainsIgnoreCase(value, "") AS icempty, + ^ + <tmp_path>/program.sql:<main>:8:13: Error: UDF 'String.AsciiContainsIgnoreCase' is not available before version 2025.02 + String::AsciiContainsIgnoreCase(value, "") AS icempty, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:10:13: Error: At function: Apply, At function: Udf + String::AsciiStartsWithIgnoreCase(value, "AS") AS icstarts, + ^ + <tmp_path>/program.sql:<main>:10:13: Error: UDF 'String.AsciiStartsWithIgnoreCase' is not available before version 2025.02 + String::AsciiStartsWithIgnoreCase(value, "AS") AS icstarts, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:12:13: Error: At function: Apply, At function: Udf + String::AsciiEndsWithIgnoreCase(value, "AS") AS icends, + ^ + <tmp_path>/program.sql:<main>:12:13: Error: UDF 'String.AsciiEndsWithIgnoreCase' is not available before version 2025.02 + String::AsciiEndsWithIgnoreCase(value, "AS") AS icends, + ^ + <tmp_path>/program.sql:<main>:2:1: Error: At function: SqlProjectItem, At lambda + SELECT + ^ + <tmp_path>/program.sql:<main>:14:13: Error: At function: Apply, At function: Udf + String::AsciiEqualsIgnoreCase(value, "FDSA") AS icequals, + ^ + <tmp_path>/program.sql:<main>:14:13: Error: UDF 'String.AsciiEqualsIgnoreCase' is not available before version 2025.02 + String::AsciiEqualsIgnoreCase(value, "FDSA") AS icequals, + ^
\ No newline at end of file diff --git a/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01.cfg b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01.cfg new file mode 100644 index 00000000000..63fef4f7b7c --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01.cfg @@ -0,0 +1,3 @@ +xfail +langver 2025.01 +in plato.Input default.in diff --git a/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01.sql b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01.sql new file mode 100644 index 00000000000..db2ebad20e0 --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01.sql @@ -0,0 +1,9 @@ +SELECT + value, + String::AsciiContainsIgnoreCase(value, "AS") AS iccontains, + String::AsciiContainsIgnoreCase(value, "") AS icempty, + String::AsciiStartsWithIgnoreCase(value, "AS") AS icstarts, + String::AsciiEndsWithIgnoreCase(value, "AS") AS icends, + String::AsciiEqualsIgnoreCase(value, "FDSA") AS icequals, +FROM Input; + diff --git a/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01_scan.cfg b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01_scan.cfg new file mode 100644 index 00000000000..ea25436b9bb --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01_scan.cfg @@ -0,0 +1,4 @@ +xfail +langver 2025.01 +scan_udfs +in plato.Input default.in diff --git a/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01_scan.sql b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01_scan.sql new file mode 100644 index 00000000000..db2ebad20e0 --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/AsciiContainsIgnoreCase_2025_01_scan.sql @@ -0,0 +1,9 @@ +SELECT + value, + String::AsciiContainsIgnoreCase(value, "AS") AS iccontains, + String::AsciiContainsIgnoreCase(value, "") AS icempty, + String::AsciiStartsWithIgnoreCase(value, "AS") AS icstarts, + String::AsciiEndsWithIgnoreCase(value, "AS") AS icends, + String::AsciiEqualsIgnoreCase(value, "FDSA") AS icequals, +FROM Input; + diff --git a/yql/essentials/udfs/common/string/test/cases/BlockReverseBits.cfg b/yql/essentials/udfs/common/string/test/cases/BlockReverseBits.cfg new file mode 100644 index 00000000000..12ab49f0ffc --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/BlockReverseBits.cfg @@ -0,0 +1,2 @@ +langver 2025.02 +in plato.Input BlockReverseBits.in diff --git a/yql/essentials/udfs/common/string/test/cases/BlockReverseBytes.cfg b/yql/essentials/udfs/common/string/test/cases/BlockReverseBytes.cfg new file mode 100644 index 00000000000..86f08753875 --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/BlockReverseBytes.cfg @@ -0,0 +1,2 @@ +langver 2025.02 +in plato.Input BlockReverseBytes.in diff --git a/yql/essentials/udfs/common/string/test/cases/ReverseBits.cfg b/yql/essentials/udfs/common/string/test/cases/ReverseBits.cfg new file mode 100644 index 00000000000..a431901cef4 --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/ReverseBits.cfg @@ -0,0 +1,2 @@ +langver 2025.02 +in plato.Input ReverseBits.in diff --git a/yql/essentials/udfs/common/string/test/cases/ReverseBytes.cfg b/yql/essentials/udfs/common/string/test/cases/ReverseBytes.cfg new file mode 100644 index 00000000000..f4b5316f29c --- /dev/null +++ b/yql/essentials/udfs/common/string/test/cases/ReverseBytes.cfg @@ -0,0 +1,2 @@ +langver 2025.02 +in plato.Input ReverseBytes.in |