node.cpp 98 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047104810491050105110521053105410551056105710581059106010611062106310641065106610671068106910701071107210731074107510761077107810791080108110821083108410851086108710881089109010911092109310941095109610971098109911001101110211031104110511061107110811091110111111121113111411151116111711181119112011211122112311241125112611271128112911301131113211331134113511361137113811391140114111421143114411451146114711481149115011511152115311541155115611571158115911601161116211631164116511661167116811691170117111721173117411751176117711781179118011811182118311841185118611871188118911901191119211931194119511961197119811991200120112021203120412051206120712081209121012111212121312141215121612171218121912201221122212231224122512261227122812291230123112321233123412351236123712381239124012411242124312441245124612471248124912501251125212531254125512561257125812591260126112621263126412651266126712681269127012711272127312741275127612771278127912801281128212831284128512861287128812891290129112921293129412951296129712981299130013011302130313041305130613071308130913101311131213131314131513161317131813191320132113221323132413251326132713281329133013311332133313341335133613371338133913401341134213431344134513461347134813491350135113521353135413551356135713581359136013611362136313641365136613671368136913701371137213731374137513761377137813791380138113821383138413851386138713881389139013911392139313941395139613971398139914001401140214031404140514061407140814091410141114121413141414151416141714181419142014211422142314241425142614271428142914301431143214331434143514361437143814391440144114421443144414451446144714481449145014511452145314541455145614571458145914601461146214631464146514661467146814691470147114721473147414751476147714781479148014811482148314841485148614871488148914901491149214931494149514961497149814991500150115021503150415051506150715081509151015111512151315141515151615171518151915201521152215231524152515261527152815291530153115321533153415351536153715381539154015411542154315441545154615471548154915501551155215531554155515561557155815591560156115621563156415651566156715681569157015711572157315741575157615771578157915801581158215831584158515861587158815891590159115921593159415951596159715981599160016011602160316041605160616071608160916101611161216131614161516161617161816191620162116221623162416251626162716281629163016311632163316341635163616371638163916401641164216431644164516461647164816491650165116521653165416551656165716581659166016611662166316641665166616671668166916701671167216731674167516761677167816791680168116821683168416851686168716881689169016911692169316941695169616971698169917001701170217031704170517061707170817091710171117121713171417151716171717181719172017211722172317241725172617271728172917301731173217331734173517361737173817391740174117421743174417451746174717481749175017511752175317541755175617571758175917601761176217631764176517661767176817691770177117721773177417751776177717781779178017811782178317841785178617871788178917901791179217931794179517961797179817991800180118021803180418051806180718081809181018111812181318141815181618171818181918201821182218231824182518261827182818291830183118321833183418351836183718381839184018411842184318441845184618471848184918501851185218531854185518561857185818591860186118621863186418651866186718681869187018711872187318741875187618771878187918801881188218831884188518861887188818891890189118921893189418951896189718981899190019011902190319041905190619071908190919101911191219131914191519161917191819191920192119221923192419251926192719281929193019311932193319341935193619371938193919401941194219431944194519461947194819491950195119521953195419551956195719581959196019611962196319641965196619671968196919701971197219731974197519761977197819791980198119821983198419851986198719881989199019911992199319941995199619971998199920002001200220032004200520062007200820092010201120122013201420152016201720182019202020212022202320242025202620272028202920302031203220332034203520362037203820392040204120422043204420452046204720482049205020512052205320542055205620572058205920602061206220632064206520662067206820692070207120722073207420752076207720782079208020812082208320842085208620872088208920902091209220932094209520962097209820992100210121022103210421052106210721082109211021112112211321142115211621172118211921202121212221232124212521262127212821292130213121322133213421352136213721382139214021412142214321442145214621472148214921502151215221532154215521562157215821592160216121622163216421652166216721682169217021712172217321742175217621772178217921802181218221832184218521862187218821892190219121922193219421952196219721982199220022012202220322042205220622072208220922102211221222132214221522162217221822192220222122222223222422252226222722282229223022312232223322342235223622372238223922402241224222432244224522462247224822492250225122522253225422552256225722582259226022612262226322642265226622672268226922702271227222732274227522762277227822792280228122822283228422852286228722882289229022912292229322942295229622972298229923002301230223032304230523062307230823092310231123122313231423152316231723182319232023212322232323242325232623272328232923302331233223332334233523362337233823392340234123422343234423452346234723482349235023512352235323542355235623572358235923602361236223632364236523662367236823692370237123722373237423752376237723782379238023812382238323842385238623872388238923902391239223932394239523962397239823992400240124022403240424052406240724082409241024112412241324142415241624172418241924202421242224232424242524262427242824292430243124322433243424352436243724382439244024412442244324442445244624472448244924502451245224532454245524562457245824592460246124622463246424652466246724682469247024712472247324742475247624772478247924802481248224832484248524862487248824892490249124922493249424952496249724982499250025012502250325042505250625072508250925102511251225132514251525162517251825192520252125222523252425252526252725282529253025312532253325342535253625372538253925402541254225432544254525462547254825492550255125522553255425552556255725582559256025612562256325642565256625672568256925702571257225732574257525762577257825792580258125822583258425852586258725882589259025912592259325942595259625972598259926002601260226032604260526062607260826092610261126122613261426152616261726182619262026212622262326242625262626272628262926302631263226332634263526362637263826392640264126422643264426452646264726482649265026512652265326542655265626572658265926602661266226632664266526662667266826692670267126722673267426752676267726782679268026812682268326842685268626872688268926902691269226932694269526962697269826992700270127022703270427052706270727082709271027112712271327142715271627172718271927202721272227232724272527262727272827292730273127322733273427352736273727382739274027412742274327442745274627472748274927502751275227532754275527562757275827592760276127622763276427652766276727682769277027712772277327742775277627772778277927802781278227832784278527862787278827892790279127922793279427952796279727982799280028012802280328042805280628072808280928102811281228132814281528162817281828192820282128222823282428252826282728282829283028312832283328342835283628372838283928402841284228432844284528462847284828492850285128522853285428552856285728582859286028612862286328642865286628672868286928702871287228732874287528762877287828792880288128822883288428852886288728882889289028912892289328942895289628972898289929002901290229032904290529062907290829092910291129122913291429152916291729182919292029212922292329242925292629272928292929302931293229332934293529362937293829392940294129422943294429452946294729482949295029512952295329542955295629572958295929602961296229632964296529662967296829692970297129722973297429752976297729782979298029812982298329842985298629872988298929902991299229932994299529962997299829993000300130023003300430053006300730083009301030113012301330143015301630173018301930203021302230233024302530263027302830293030303130323033303430353036303730383039304030413042304330443045304630473048304930503051305230533054305530563057305830593060306130623063306430653066306730683069307030713072307330743075307630773078307930803081308230833084308530863087308830893090309130923093309430953096309730983099310031013102310331043105310631073108310931103111311231133114311531163117311831193120312131223123312431253126312731283129313031313132313331343135313631373138313931403141314231433144314531463147314831493150315131523153315431553156315731583159316031613162316331643165316631673168316931703171317231733174317531763177317831793180318131823183318431853186318731883189319031913192319331943195319631973198319932003201320232033204320532063207320832093210321132123213321432153216321732183219322032213222322332243225322632273228322932303231323232333234323532363237323832393240324132423243324432453246324732483249325032513252325332543255325632573258325932603261326232633264326532663267326832693270327132723273327432753276327732783279328032813282328332843285328632873288328932903291329232933294329532963297329832993300330133023303330433053306330733083309331033113312331333143315331633173318331933203321332233233324332533263327332833293330333133323333333433353336333733383339334033413342334333443345334633473348334933503351335233533354335533563357335833593360336133623363336433653366336733683369337033713372337333743375337633773378337933803381338233833384338533863387338833893390339133923393339433953396339733983399340034013402340334043405340634073408340934103411341234133414341534163417341834193420342134223423342434253426342734283429343034313432343334343435343634373438343934403441344234433444344534463447344834493450345134523453345434553456345734583459346034613462346334643465346634673468346934703471347234733474347534763477347834793480348134823483348434853486348734883489349034913492349334943495349634973498349935003501350235033504350535063507350835093510
  1. #include "node.h"
  2. #include "source.h"
  3. #include "context.h"
  4. #include <yql/essentials/ast/yql_ast_escaping.h>
  5. #include <yql/essentials/ast/yql_expr.h>
  6. #include <yql/essentials/core/sql_types/simple_types.h>
  7. #include <yql/essentials/minikql/mkql_type_ops.h>
  8. #include <yql/essentials/parser/pg_catalog/catalog.h>
  9. #include <yql/essentials/utils/yql_panic.h>
  10. #include <library/cpp/containers/stack_vector/stack_vec.h>
  11. #include <library/cpp/charset/ci_string.h>
  12. #include <util/generic/hash_set.h>
  13. #include <util/stream/str.h>
  14. #include <util/string/cast.h>
  15. #include <util/string/escape.h>
  16. #include <util/string/subst.h>
  17. using namespace NYql;
  18. namespace NSQLTranslationV1 {
  19. TString ErrorDistinctWithoutCorrelation(const TString& column) {
  20. return TStringBuilder() << "DISTINCT columns for JOIN in SELECT should have table aliases (correlation name),"
  21. " add it if necessary to FROM section over 'AS <alias>' keyword and put it like '<alias>." << column << "'";
  22. }
  23. TString ErrorDistinctByGroupKey(const TString& column) {
  24. return TStringBuilder() << "Unable to use DISTINCT by grouping column: " << column << ". You should leave one of them.";
  25. }
  26. TTopicRef::TTopicRef(const TString& refName, const TDeferredAtom& cluster, TNodePtr keys)
  27. : RefName(refName)
  28. , Cluster(cluster)
  29. , Keys(keys)
  30. {
  31. }
  32. TColumnConstraints::TColumnConstraints(TNodePtr defaultExpr, bool nullable)
  33. : DefaultExpr(defaultExpr)
  34. , Nullable(nullable)
  35. {
  36. }
  37. TColumnSchema::TColumnSchema(TPosition pos, const TString& name, const TNodePtr& type, bool nullable,
  38. TVector<TIdentifier> families, bool serial, TNodePtr defaultExpr, ETypeOfChange typeOfChange)
  39. : Pos(pos)
  40. , Name(name)
  41. , Type(type)
  42. , Nullable(nullable)
  43. , Families(families)
  44. , Serial(serial)
  45. , DefaultExpr(defaultExpr)
  46. , TypeOfChange(typeOfChange)
  47. {
  48. }
  49. INode::INode(TPosition pos)
  50. : Pos(pos)
  51. {
  52. }
  53. INode::~INode()
  54. {
  55. }
  56. TPosition INode::GetPos() const {
  57. return Pos;
  58. }
  59. const TString& INode::GetLabel() const {
  60. return Label;
  61. }
  62. TMaybe<TPosition> INode::GetLabelPos() const {
  63. return LabelPos;
  64. }
  65. void INode::SetLabel(const TString& label, TMaybe<TPosition> pos) {
  66. Label = label;
  67. LabelPos = pos;
  68. }
  69. bool INode::IsImplicitLabel() const {
  70. return ImplicitLabel;
  71. }
  72. void INode::MarkImplicitLabel(bool isImplicitLabel) {
  73. ImplicitLabel = isImplicitLabel;
  74. }
  75. void INode::SetCountHint(bool isCount) {
  76. State.Set(ENodeState::CountHint, isCount);
  77. }
  78. bool INode::GetCountHint() const {
  79. return State.Test(ENodeState::CountHint);
  80. }
  81. bool INode::IsConstant() const {
  82. return HasState(ENodeState::Const);
  83. }
  84. bool INode::MaybeConstant() const {
  85. return HasState(ENodeState::MaybeConst);
  86. }
  87. bool INode::IsAggregated() const {
  88. return HasState(ENodeState::Aggregated);
  89. }
  90. bool INode::IsAggregationKey() const {
  91. return HasState(ENodeState::AggregationKey);
  92. }
  93. bool INode::IsOverWindow() const {
  94. return HasState(ENodeState::OverWindow);
  95. }
  96. bool INode::IsOverWindowDistinct() const {
  97. return HasState(ENodeState::OverWindowDistinct);
  98. }
  99. bool INode::IsNull() const {
  100. return false;
  101. }
  102. bool INode::IsLiteral() const {
  103. return false;
  104. }
  105. TString INode::GetLiteralType() const {
  106. return "";
  107. }
  108. TString INode::GetLiteralValue() const {
  109. return "";
  110. }
  111. bool INode::IsIntegerLiteral() const {
  112. return false;
  113. }
  114. INode::TPtr INode::ApplyUnaryOp(TContext& ctx, TPosition pos, const TString& opName) const {
  115. Y_UNUSED(ctx);
  116. if (IsNull()) {
  117. return BuildLiteralNull(pos);
  118. }
  119. return new TCallNodeImpl(pos, opName, { Clone() });
  120. }
  121. bool INode::IsAsterisk() const {
  122. return false;
  123. }
  124. const TString* INode::SubqueryAlias() const {
  125. return nullptr;
  126. }
  127. TString INode::GetOpName() const {
  128. return TString();
  129. }
  130. const TString* INode::GetLiteral(const TString& type) const {
  131. Y_UNUSED(type);
  132. return nullptr;
  133. }
  134. const TString* INode::GetColumnName() const {
  135. return nullptr;
  136. }
  137. bool INode::IsPlainColumn() const {
  138. return GetColumnName() != nullptr;
  139. }
  140. bool INode::IsTableRow() const {
  141. return false;
  142. }
  143. void INode::AssumeColumn() {
  144. }
  145. const TString* INode::GetSourceName() const {
  146. return nullptr;
  147. }
  148. const TString* INode::GetAtomContent() const {
  149. return nullptr;
  150. }
  151. bool INode::IsOptionalArg() const {
  152. return false;
  153. }
  154. size_t INode::GetTupleSize() const {
  155. return 0;
  156. }
  157. INode::TPtr INode::GetTupleElement(size_t index) const {
  158. Y_UNUSED(index);
  159. return nullptr;
  160. }
  161. ITableKeys* INode::GetTableKeys() {
  162. return nullptr;
  163. }
  164. ISource* INode::GetSource() {
  165. return nullptr;
  166. }
  167. TVector<TNodePtr>* INode::ContentListPtr() {
  168. return nullptr;
  169. }
  170. bool INode::Init(TContext& ctx, ISource* src) {
  171. if (State.Test(ENodeState::Failed)) {
  172. return false;
  173. }
  174. if (!State.Test(ENodeState::Initialized)) {
  175. if (!DoInit(ctx, src)) {
  176. State.Set(ENodeState::Failed);
  177. return false;
  178. }
  179. State.Set(ENodeState::Initialized);
  180. }
  181. return true;
  182. }
  183. bool INode::InitReference(TContext& ctx) {
  184. Y_UNUSED(ctx);
  185. return true;
  186. }
  187. bool INode::DoInit(TContext& ctx, ISource* src) {
  188. Y_UNUSED(ctx);
  189. Y_UNUSED(src);
  190. return true;
  191. }
  192. TNodePtr INode::AstNode() const {
  193. return new TAstListNodeImpl(Pos);
  194. }
  195. TNodePtr INode::AstNode(TNodePtr node) const {
  196. return node;
  197. }
  198. TNodePtr INode::AstNode(const TString& str) const {
  199. return new TAstAtomNodeImpl(Pos, str, TNodeFlags::Default);
  200. }
  201. TNodePtr INode::AstNode(TAstNode* node) const {
  202. return new TAstDirectNode(node);
  203. }
  204. TNodePtr INode::Clone() const {
  205. TNodePtr clone = DoClone();
  206. if (!clone) {
  207. clone = const_cast<INode*>(this);
  208. } else {
  209. YQL_ENSURE(!State.Test(ENodeState::Initialized), "Clone should be for uninitialized or persistent node");
  210. clone->SetLabel(Label, LabelPos);
  211. clone->MarkImplicitLabel(ImplicitLabel);
  212. }
  213. return clone;
  214. }
  215. TAggregationPtr INode::GetAggregation() const {
  216. return {};
  217. }
  218. void INode::CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) {
  219. Y_UNUSED(ctx);
  220. Y_UNUSED(src);
  221. Y_UNUSED(exprs);
  222. }
  223. INode::TPtr INode::WindowSpecFunc(const TPtr& type) const {
  224. Y_UNUSED(type);
  225. return {};
  226. }
  227. bool INode::SetViewName(TContext& ctx, TPosition pos, const TString& view) {
  228. Y_UNUSED(pos);
  229. Y_UNUSED(view);
  230. ctx.Error() << "Node not support views";
  231. return false;
  232. }
  233. bool INode::SetPrimaryView(TContext& ctx, TPosition pos) {
  234. Y_UNUSED(pos);
  235. ctx.Error() << "Node not support primary views";
  236. return false;
  237. }
  238. void INode::UseAsInner() {
  239. AsInner = true;
  240. }
  241. void INode::DisableSort() {
  242. DisableSort_ = true;
  243. }
  244. bool INode::UsedSubquery() const {
  245. return false;
  246. }
  247. bool INode::IsSelect() const {
  248. return false;
  249. }
  250. bool INode::HasSelectResult() const {
  251. return false;
  252. }
  253. const TString* INode::FuncName() const {
  254. return nullptr;
  255. }
  256. const TString* INode::ModuleName() const {
  257. return nullptr;
  258. }
  259. bool INode::HasSkip() const {
  260. return false;
  261. }
  262. TColumnNode* INode::GetColumnNode() {
  263. return nullptr;
  264. }
  265. const TColumnNode* INode::GetColumnNode() const {
  266. return nullptr;
  267. }
  268. TTupleNode* INode::GetTupleNode() {
  269. return nullptr;
  270. }
  271. const TTupleNode* INode::GetTupleNode() const {
  272. return nullptr;
  273. }
  274. TCallNode* INode::GetCallNode() {
  275. return nullptr;
  276. }
  277. const TCallNode* INode::GetCallNode() const {
  278. return nullptr;
  279. }
  280. TStructNode* INode::GetStructNode() {
  281. return nullptr;
  282. }
  283. const TStructNode* INode::GetStructNode() const {
  284. return nullptr;
  285. }
  286. TAccessNode* INode::GetAccessNode() {
  287. return nullptr;
  288. }
  289. const TAccessNode* INode::GetAccessNode() const {
  290. return nullptr;
  291. }
  292. TLambdaNode* INode::GetLambdaNode() {
  293. return nullptr;
  294. }
  295. const TLambdaNode* INode::GetLambdaNode() const {
  296. return nullptr;
  297. }
  298. TUdfNode* INode::GetUdfNode() {
  299. return nullptr;
  300. }
  301. const TUdfNode* INode::GetUdfNode() const {
  302. return nullptr;
  303. }
  304. void INode::VisitTree(const TVisitFunc& func) const {
  305. TVisitNodeSet visited;
  306. VisitTree(func, visited);
  307. }
  308. void INode::VisitTree(const TVisitFunc& func, TVisitNodeSet& visited) const {
  309. if (visited.emplace(this).second && HasState(ENodeState::Initialized) && func(*this)) {
  310. DoVisitChildren(func, visited);
  311. }
  312. }
  313. TNodePtr INode::ShallowCopy() const {
  314. Y_DEBUG_ABORT_UNLESS(false, "Node is not copyable");
  315. return nullptr;
  316. }
  317. void INode::DoUpdateState() const {
  318. }
  319. void INode::PrecacheState() const {
  320. if (State.Test(ENodeState::Failed)) {
  321. return;
  322. }
  323. /// Not work right now! It's better use Init at first, because some kind of update depend on it
  324. /// \todo turn on and remove all issues
  325. //Y_DEBUG_ABORT_UNLESS(State.Test(ENodeState::Initialized));
  326. if (State.Test(ENodeState::Precached)) {
  327. return;
  328. }
  329. DoUpdateState();
  330. State.Set(ENodeState::Precached);
  331. }
  332. void INode::DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const {
  333. Y_UNUSED(func);
  334. Y_UNUSED(visited);
  335. }
  336. void INode::DoAdd(TNodePtr node) {
  337. Y_UNUSED(node);
  338. Y_DEBUG_ABORT_UNLESS(false, "Node is not expandable");
  339. }
  340. bool IProxyNode::IsNull() const {
  341. return Inner->IsNull();
  342. }
  343. bool IProxyNode::IsLiteral() const {
  344. return Inner->IsNull();
  345. }
  346. TString IProxyNode::GetLiteralType() const {
  347. return Inner->GetLiteralType();
  348. }
  349. TString IProxyNode::GetLiteralValue() const {
  350. return Inner->GetLiteralValue();
  351. }
  352. bool IProxyNode::IsIntegerLiteral() const {
  353. return Inner->IsIntegerLiteral();
  354. }
  355. INode::TPtr IProxyNode::ApplyUnaryOp(TContext& ctx, TPosition pos, const TString& opName) const {
  356. return Inner->ApplyUnaryOp(ctx, pos, opName);
  357. }
  358. bool IProxyNode::IsAsterisk() const {
  359. return Inner->IsAsterisk();
  360. }
  361. const TString* IProxyNode::SubqueryAlias() const {
  362. return Inner->SubqueryAlias();
  363. }
  364. TString IProxyNode::GetOpName() const {
  365. return Inner->GetOpName();
  366. }
  367. const TString* IProxyNode::GetLiteral(const TString& type) const {
  368. return Inner->GetLiteral(type);
  369. }
  370. const TString* IProxyNode::GetColumnName() const {
  371. return Inner->GetColumnName();
  372. }
  373. bool IProxyNode::IsPlainColumn() const {
  374. return Inner->IsPlainColumn();
  375. }
  376. bool IProxyNode::IsTableRow() const {
  377. return Inner->IsTableRow();
  378. }
  379. void IProxyNode::AssumeColumn() {
  380. Inner->AssumeColumn();
  381. }
  382. const TString* IProxyNode::GetSourceName() const {
  383. return Inner->GetSourceName();
  384. }
  385. const TString* IProxyNode::GetAtomContent() const {
  386. return Inner->GetAtomContent();
  387. }
  388. bool IProxyNode::IsOptionalArg() const {
  389. return Inner->IsOptionalArg();
  390. }
  391. size_t IProxyNode::GetTupleSize() const {
  392. return Inner->GetTupleSize();
  393. }
  394. INode::TPtr IProxyNode::GetTupleElement(size_t index) const {
  395. return Inner->GetTupleElement(index);
  396. }
  397. ITableKeys* IProxyNode::GetTableKeys() {
  398. return Inner->GetTableKeys();
  399. }
  400. ISource* IProxyNode::GetSource() {
  401. return Inner->GetSource();
  402. }
  403. TVector<INode::TPtr>* IProxyNode::ContentListPtr() {
  404. return Inner->ContentListPtr();
  405. }
  406. TAggregationPtr IProxyNode::GetAggregation() const {
  407. return Inner->GetAggregation();
  408. }
  409. void IProxyNode::CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) {
  410. Inner->CollectPreaggregateExprs(ctx, src, exprs);
  411. }
  412. INode::TPtr IProxyNode::WindowSpecFunc(const TPtr& type) const {
  413. return Inner->WindowSpecFunc(type);
  414. }
  415. bool IProxyNode::SetViewName(TContext& ctx, TPosition pos, const TString& view) {
  416. return Inner->SetViewName(ctx, pos, view);
  417. }
  418. bool IProxyNode::SetPrimaryView(TContext& ctx, TPosition pos) {
  419. return Inner->SetPrimaryView(ctx, pos);
  420. }
  421. bool IProxyNode::UsedSubquery() const {
  422. return Inner->UsedSubquery();
  423. }
  424. bool IProxyNode::IsSelect() const {
  425. return Inner->IsSelect();
  426. }
  427. bool IProxyNode::HasSelectResult() const {
  428. return Inner->HasSelectResult();
  429. }
  430. const TString* IProxyNode::FuncName() const {
  431. return Inner->FuncName();
  432. }
  433. const TString* IProxyNode::ModuleName() const {
  434. return Inner->ModuleName();
  435. }
  436. bool IProxyNode::HasSkip() const {
  437. return Inner->HasSkip();
  438. }
  439. TColumnNode* IProxyNode::GetColumnNode() {
  440. return Inner->GetColumnNode();
  441. }
  442. const TColumnNode* IProxyNode::GetColumnNode() const {
  443. return static_cast<const INode*>(Inner.Get())->GetColumnNode();
  444. }
  445. TTupleNode* IProxyNode::GetTupleNode() {
  446. return Inner->GetTupleNode();
  447. }
  448. const TTupleNode* IProxyNode::GetTupleNode() const {
  449. return static_cast<const INode*>(Inner.Get())->GetTupleNode();
  450. }
  451. TCallNode* IProxyNode::GetCallNode() {
  452. return Inner->GetCallNode();
  453. }
  454. const TCallNode* IProxyNode::GetCallNode() const {
  455. return static_cast<const INode*>(Inner.Get())->GetCallNode();
  456. }
  457. TStructNode* IProxyNode::GetStructNode() {
  458. return Inner->GetStructNode();
  459. }
  460. const TStructNode* IProxyNode::GetStructNode() const {
  461. return static_cast<const INode*>(Inner.Get())->GetStructNode();
  462. }
  463. TAccessNode* IProxyNode::GetAccessNode() {
  464. return Inner->GetAccessNode();
  465. }
  466. const TAccessNode* IProxyNode::GetAccessNode() const {
  467. return static_cast<const INode*>(Inner.Get())->GetAccessNode();
  468. }
  469. TLambdaNode* IProxyNode::GetLambdaNode() {
  470. return Inner->GetLambdaNode();
  471. }
  472. const TLambdaNode* IProxyNode::GetLambdaNode() const {
  473. return static_cast<const INode*>(Inner.Get())->GetLambdaNode();
  474. }
  475. TUdfNode* IProxyNode::GetUdfNode() {
  476. return Inner->GetUdfNode();
  477. }
  478. const TUdfNode* IProxyNode::GetUdfNode() const {
  479. return static_cast<const INode*>(Inner.Get())->GetUdfNode();
  480. }
  481. void IProxyNode::DoUpdateState() const {
  482. static_assert(static_cast<int>(ENodeState::End) == 10, "Need to support new states here");
  483. State.Set(ENodeState::CountHint, Inner->GetCountHint());
  484. State.Set(ENodeState::Const, Inner->IsConstant());
  485. State.Set(ENodeState::MaybeConst, Inner->MaybeConstant());
  486. State.Set(ENodeState::Aggregated, Inner->IsAggregated());
  487. State.Set(ENodeState::AggregationKey, Inner->IsAggregationKey());
  488. State.Set(ENodeState::OverWindow, Inner->IsOverWindow());
  489. State.Set(ENodeState::OverWindowDistinct, Inner->IsOverWindowDistinct());
  490. }
  491. void IProxyNode::DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const {
  492. Inner->VisitTree(func, visited);
  493. }
  494. bool IProxyNode::InitReference(TContext& ctx) {
  495. return Inner->InitReference(ctx);
  496. }
  497. bool IProxyNode::DoInit(TContext& ctx, ISource* src) {
  498. return Inner->Init(ctx, src);
  499. }
  500. void IProxyNode::DoAdd(TPtr node) {
  501. Inner->Add(node);
  502. }
  503. void MergeHints(TTableHints& base, const TTableHints& overrides) {
  504. for (auto& i : overrides) {
  505. base[i.first] = i.second;
  506. }
  507. }
  508. TTableHints CloneContainer(const TTableHints& hints) {
  509. TTableHints result;
  510. for (auto& [name, nodes] : hints) {
  511. result.emplace(std::make_pair(name, CloneContainer(nodes)));
  512. }
  513. return result;
  514. }
  515. TAstAtomNode::TAstAtomNode(TPosition pos, const TString& content, ui32 flags, bool isOptionalArg)
  516. : INode(pos)
  517. , Content(content)
  518. , Flags(flags)
  519. , IsOptionalArg_(isOptionalArg)
  520. {
  521. }
  522. TAstAtomNode::~TAstAtomNode()
  523. {
  524. }
  525. void TAstAtomNode::DoUpdateState() const {
  526. State.Set(ENodeState::Const);
  527. }
  528. TAstNode* TAstAtomNode::Translate(TContext& ctx) const {
  529. return TAstNode::NewAtom(Pos, Content, *ctx.Pool, Flags);
  530. }
  531. const TString* TAstAtomNode::GetAtomContent() const {
  532. return &Content;
  533. }
  534. bool TAstAtomNode::IsOptionalArg() const {
  535. return IsOptionalArg_;
  536. }
  537. TAstDirectNode::TAstDirectNode(TAstNode* node)
  538. : INode(node->GetPosition())
  539. , Node(node)
  540. {
  541. }
  542. TAstNode* TAstDirectNode::Translate(TContext& ctx) const {
  543. Y_UNUSED(ctx);
  544. return Node;
  545. }
  546. TNodePtr BuildAtom(TPosition pos, const TString& content, ui32 flags, bool isOptionalArg) {
  547. return new TAstAtomNodeImpl(pos, content, flags, isOptionalArg);
  548. }
  549. TAstListNode::TAstListNode(TPosition pos)
  550. : INode(pos)
  551. {
  552. }
  553. TAstListNode::~TAstListNode()
  554. {
  555. }
  556. bool TAstListNode::DoInit(TContext& ctx, ISource* src) {
  557. for (auto& node: Nodes) {
  558. if (!node->Init(ctx, src)) {
  559. return false;
  560. }
  561. }
  562. return true;
  563. }
  564. TAstNode* TAstListNode::Translate(TContext& ctx) const {
  565. TSmallVec<TAstNode*> children;
  566. children.reserve(Nodes.size());
  567. auto listPos = Pos;
  568. for (auto& node: Nodes) {
  569. if (node) {
  570. auto astNode = node->Translate(ctx);
  571. if (!astNode) {
  572. return nullptr;
  573. }
  574. children.push_back(astNode);
  575. } else {
  576. ctx.Error(Pos) << "Translation error: encountered empty TNodePtr";
  577. return nullptr;
  578. }
  579. }
  580. return TAstNode::NewList(listPos, children.data(), children.size(), *ctx.Pool);
  581. }
  582. void TAstListNode::UpdateStateByListNodes(const TVector<TNodePtr>& nodes) const {
  583. bool isConst = true;
  584. struct TAttributesFlags {
  585. bool has = false;
  586. bool all = true;
  587. };
  588. std::array<ENodeState, 3> checkStates = {{ENodeState::Aggregated, ENodeState::AggregationKey, ENodeState::OverWindow}};
  589. std::map<ENodeState, TAttributesFlags> flags;
  590. for (auto& node: nodes) {
  591. const bool isNodeConst = node->IsConstant();
  592. const bool isNodeMaybeConst = node->MaybeConstant();
  593. for (auto state: checkStates) {
  594. if (node->HasState(state)) {
  595. flags[state].has = true;
  596. } else if (!isNodeConst && !isNodeMaybeConst) {
  597. flags[state].all = false;
  598. }
  599. if (!isNodeConst) {
  600. isConst = false;
  601. }
  602. }
  603. }
  604. State.Set(ENodeState::Const, isConst);
  605. for (auto& flag: flags) {
  606. State.Set(flag.first, flag.second.has && flag.second.all);
  607. }
  608. State.Set(ENodeState::MaybeConst, !isConst && AllOf(nodes, [](const auto& node) { return node->IsConstant() || node->MaybeConstant(); }));
  609. }
  610. void TAstListNode::DoUpdateState() const {
  611. UpdateStateByListNodes(Nodes);
  612. }
  613. void TAstListNode::DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const {
  614. for (auto& node : Nodes) {
  615. node->VisitTree(func, visited);
  616. }
  617. }
  618. TAstListNode::TAstListNode(const TAstListNode& node)
  619. : INode(node.Pos)
  620. , Nodes(node.Nodes)
  621. {
  622. Label = node.Label;
  623. State = node.State;
  624. }
  625. TAstListNode::TAstListNode(TPosition pos, TVector<TNodePtr>&& nodes)
  626. : INode(pos)
  627. , Nodes(std::move(nodes))
  628. {
  629. for (const auto& node: Nodes) {
  630. YQL_ENSURE(node, "Null ptr passed as list element");
  631. }
  632. }
  633. TNodePtr TAstListNode::ShallowCopy() const {
  634. return new TAstListNodeImpl(Pos, Nodes);
  635. }
  636. void TAstListNode::DoAdd(TNodePtr node) {
  637. Y_DEBUG_ABORT_UNLESS(node);
  638. Y_DEBUG_ABORT_UNLESS(node.Get() != this);
  639. Nodes.push_back(node);
  640. }
  641. TAstListNodeImpl::TAstListNodeImpl(TPosition pos)
  642. : TAstListNode(pos)
  643. {}
  644. TAstListNodeImpl::TAstListNodeImpl(TPosition pos, TVector<TNodePtr> nodes)
  645. : TAstListNode(pos)
  646. {
  647. for (const auto& node: nodes) {
  648. YQL_ENSURE(node, "Null ptr passed as list element");
  649. }
  650. Nodes.swap(nodes);
  651. }
  652. void TAstListNodeImpl::CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) {
  653. for (auto& node : Nodes) {
  654. node->CollectPreaggregateExprs(ctx, src, exprs);
  655. }
  656. }
  657. TNodePtr TAstListNodeImpl::DoClone() const {
  658. return new TAstListNodeImpl(Pos, CloneContainer(Nodes));
  659. }
  660. TCallNode::TCallNode(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  661. : TAstListNode(pos)
  662. , OpName(opName)
  663. , MinArgs(minArgs)
  664. , MaxArgs(maxArgs)
  665. , Args(args)
  666. {
  667. for (const auto& arg: Args) {
  668. YQL_ENSURE(arg, "Null ptr passed as call argument");
  669. }
  670. }
  671. TString TCallNode::GetOpName() const {
  672. return OpName;
  673. }
  674. const TString* DeriveCommonSourceName(const TVector<TNodePtr> &nodes) {
  675. const TString* name = nullptr;
  676. for (auto& node: nodes) {
  677. auto n = node->GetSourceName();
  678. if (!n) {
  679. continue;
  680. }
  681. if (name && *n != *name) {
  682. return nullptr;
  683. }
  684. name = n;
  685. }
  686. return name;
  687. }
  688. const TString* TCallNode::GetSourceName() const {
  689. return DeriveCommonSourceName(Args);
  690. }
  691. const TVector<TNodePtr>& TCallNode::GetArgs() const {
  692. return Args;
  693. }
  694. void TCallNode::DoUpdateState() const {
  695. UpdateStateByListNodes(Args);
  696. }
  697. TString TCallNode::GetCallExplain() const {
  698. auto derivedName = GetOpName();
  699. TStringBuilder sb;
  700. sb << derivedName << "()";
  701. if (derivedName != OpName) {
  702. sb << ", converted to " << OpName << "()";
  703. }
  704. return std::move(sb);
  705. }
  706. void TCallNode::CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) {
  707. for (auto& arg : Args) {
  708. arg->CollectPreaggregateExprs(ctx, src, exprs);
  709. }
  710. }
  711. bool TCallNode::ValidateArguments(TContext& ctx) const {
  712. const auto argsCount = static_cast<i32>(Args.size());
  713. if (MinArgs >= 0 && MaxArgs == MinArgs && argsCount != MinArgs) {
  714. ctx.Error(Pos) << GetCallExplain() << " requires exactly " << MinArgs << " arguments, given: " << Args.size();
  715. return false;
  716. }
  717. if (MinArgs >= 0 && argsCount < MinArgs) {
  718. ctx.Error(Pos) << GetCallExplain() << " requires at least " << MinArgs << " arguments, given: " << Args.size();
  719. return false;
  720. }
  721. if (MaxArgs >= 0 && argsCount > MaxArgs) {
  722. ctx.Error(Pos) << GetCallExplain() << " requires at most " << MaxArgs << " arguments, given: " << Args.size();
  723. return false;
  724. }
  725. return true;
  726. }
  727. bool TCallNode::DoInit(TContext& ctx, ISource* src) {
  728. if (!ValidateArguments(ctx)) {
  729. return false;
  730. }
  731. bool hasError = false;
  732. for (auto& arg: Args) {
  733. if (!arg->Init(ctx, src)) {
  734. hasError = true;
  735. continue;
  736. }
  737. }
  738. if (hasError) {
  739. return false;
  740. }
  741. Nodes.push_back(BuildAtom(Pos, OpName,
  742. OpName.cend() == std::find_if_not(OpName.cbegin(), OpName.cend(), [](char c) { return bool(std::isalnum(c)); }) ? TNodeFlags::Default : TNodeFlags::ArbitraryContent));
  743. Nodes.insert(Nodes.end(), Args.begin(), Args.end());
  744. return true;
  745. }
  746. TCallNode* TCallNode::GetCallNode() {
  747. return this;
  748. }
  749. const TCallNode* TCallNode::GetCallNode() const {
  750. return this;
  751. }
  752. TCallNodeImpl::TCallNodeImpl(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  753. : TCallNode(pos, opName, minArgs, maxArgs, args)
  754. {}
  755. TCallNodeImpl::TCallNodeImpl(TPosition pos, const TString& opName, const TVector<TNodePtr>& args)
  756. : TCallNode(pos, opName, args.size(), args.size(), args)
  757. {}
  758. TCallNode::TPtr TCallNodeImpl::DoClone() const {
  759. return new TCallNodeImpl(GetPos(), OpName, MinArgs, MaxArgs, CloneContainer(Args));
  760. }
  761. TFuncNodeImpl::TFuncNodeImpl(TPosition pos, const TString& opName)
  762. : TCallNode(pos, opName, 0, 0, {})
  763. {}
  764. TCallNode::TPtr TFuncNodeImpl::DoClone() const {
  765. return new TFuncNodeImpl(GetPos(), OpName);
  766. }
  767. const TString* TFuncNodeImpl::FuncName() const {
  768. return &OpName;
  769. }
  770. TCallNodeDepArgs::TCallNodeDepArgs(ui32 reqArgsCount, TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  771. : TCallNode(pos, opName, minArgs, maxArgs, args)
  772. , ReqArgsCount(reqArgsCount)
  773. {}
  774. TCallNodeDepArgs::TCallNodeDepArgs(ui32 reqArgsCount, TPosition pos, const TString& opName, const TVector<TNodePtr>& args)
  775. : TCallNode(pos, opName, args.size(), args.size(), args)
  776. , ReqArgsCount(reqArgsCount)
  777. {}
  778. TCallNode::TPtr TCallNodeDepArgs::DoClone() const {
  779. return new TCallNodeDepArgs(ReqArgsCount, GetPos(), OpName, MinArgs, MaxArgs, CloneContainer(Args));
  780. }
  781. bool TCallNodeDepArgs::DoInit(TContext& ctx, ISource* src) {
  782. if (!TCallNode::DoInit(ctx, src)) {
  783. return false;
  784. }
  785. for (ui32 i = 1 + ReqArgsCount; i < Nodes.size(); ++i) {
  786. Nodes[i] = Y("DependsOn", Nodes[i]);
  787. }
  788. return true;
  789. }
  790. TCallDirectRow::TPtr TCallDirectRow::DoClone() const {
  791. return new TCallDirectRow(Pos, OpName, CloneContainer(Args));
  792. }
  793. TCallDirectRow::TCallDirectRow(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  794. : TCallNode(pos, opName, minArgs, maxArgs, args)
  795. {}
  796. TCallDirectRow::TCallDirectRow(TPosition pos, const TString& opName, const TVector<TNodePtr>& args)
  797. : TCallNode(pos, opName, 0, 0, args)
  798. {}
  799. bool TCallDirectRow::DoInit(TContext& ctx, ISource* src) {
  800. if (!src || (ctx.CompactNamedExprs && src->IsFake())) {
  801. ctx.Error(Pos) << "Unable to use function: " << OpName << " without source";
  802. return false;
  803. }
  804. if (src->IsCompositeSource() || src->GetJoin() || src->HasAggregations() || src->IsFlattenByColumns() || src->IsOverWindowSource()) {
  805. ctx.Error(Pos) << "Failed to use function: " << OpName << " with aggregation, join, flatten by or window functions";
  806. return false;
  807. }
  808. if (!TCallNode::DoInit(ctx, src)) {
  809. return false;
  810. }
  811. Nodes.push_back(Y("DependsOn", "row"));
  812. return true;
  813. }
  814. void TCallDirectRow::DoUpdateState() const {
  815. State.Set(ENodeState::Const, false);
  816. }
  817. void TWinAggrEmulation::DoUpdateState() const {
  818. State.Set(ENodeState::OverWindow, true);
  819. }
  820. bool TWinAggrEmulation::DoInit(TContext& ctx, ISource* src) {
  821. if (!src) {
  822. ctx.Error(Pos) << "Unable to use window function " << OpName << " without source";
  823. return false;
  824. }
  825. if (!src->IsOverWindowSource()) {
  826. ctx.Error(Pos) << "Failed to use window function " << OpName << " without window specification";
  827. return false;
  828. }
  829. if (!src->AddFuncOverWindow(ctx, this)) {
  830. ctx.Error(Pos) << "Failed to use window function " << OpName << " without window specification or in wrong place";
  831. return false;
  832. }
  833. FuncAlias = "_yql_" + src->MakeLocalName(OpName);
  834. src->AddTmpWindowColumn(FuncAlias);
  835. if (!TCallNode::DoInit(ctx, src)) {
  836. return false;
  837. }
  838. Nodes.clear();
  839. Add("Member", "row", Q(FuncAlias));
  840. return true;
  841. }
  842. INode::TPtr TWinAggrEmulation::WindowSpecFunc(const TPtr& type) const {
  843. auto result = Y(OpName, type);
  844. for (const auto& arg: Args) {
  845. result = L(result, arg);
  846. }
  847. return Q(Y(Q(FuncAlias), result));
  848. }
  849. TWinAggrEmulation::TWinAggrEmulation(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  850. : TCallNode(pos, opName, minArgs, maxArgs, args)
  851. , FuncAlias(opName)
  852. {}
  853. TWinRowNumber::TWinRowNumber(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  854. : TWinAggrEmulation(pos, opName, minArgs, maxArgs, args)
  855. {}
  856. TWinCumeDist::TWinCumeDist(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  857. : TWinAggrEmulation(pos, opName, minArgs, maxArgs, args)
  858. {}
  859. bool TWinCumeDist::DoInit(TContext& ctx, ISource* src) {
  860. if (!ValidateArguments(ctx)) {
  861. return false;
  862. }
  863. YQL_ENSURE(Args.size() == 0);
  864. TVector<TNodePtr> optionsElements;
  865. if (ctx.AnsiCurrentRow) {
  866. optionsElements.push_back(BuildTuple(Pos, { BuildQuotedAtom(Pos, "ansi", NYql::TNodeFlags::Default) }));
  867. }
  868. Args.push_back(BuildTuple(Pos, optionsElements));
  869. MinArgs = MaxArgs = 1;
  870. if (!TWinAggrEmulation::DoInit(ctx, src)) {
  871. return false;
  872. }
  873. YQL_ENSURE(Args.size() == 1);
  874. return true;
  875. }
  876. TWinNTile::TWinNTile(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  877. : TWinAggrEmulation(pos, opName, minArgs, maxArgs, args)
  878. {
  879. FakeSource = BuildFakeSource(pos);
  880. }
  881. bool TWinNTile::DoInit(TContext& ctx, ISource* src) {
  882. if (Args.size() >= 1 && !Args[0]->Init(ctx, FakeSource.Get())) {
  883. return false;
  884. }
  885. if (!TWinAggrEmulation::DoInit(ctx, src)) {
  886. return false;
  887. }
  888. return true;
  889. }
  890. TWinLeadLag::TWinLeadLag(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  891. : TWinAggrEmulation(pos, opName, minArgs, maxArgs, args)
  892. {}
  893. bool TWinLeadLag::DoInit(TContext& ctx, ISource* src) {
  894. if (Args.size() >= 2) {
  895. if (!Args[1]->IsIntegerLiteral()) {
  896. ctx.Error(Args[1]->GetPos()) << "Expected integer literal as second parameter of " << OpName << "( ) function";
  897. return false;
  898. }
  899. }
  900. if (!TWinAggrEmulation::DoInit(ctx, src)) {
  901. return false;
  902. }
  903. if (Args.size() >= 1) {
  904. Args[0] = BuildLambda(Pos, Y("row"), Args[0]);
  905. }
  906. return true;
  907. }
  908. TWinRank::TWinRank(TPosition pos, const TString& opName, i32 minArgs, i32 maxArgs, const TVector<TNodePtr>& args)
  909. : TWinAggrEmulation(pos, opName, minArgs, maxArgs, args)
  910. {
  911. }
  912. bool TExternalFunctionConfig::DoInit(TContext& ctx, ISource* src) {
  913. for (auto& param: Config) {
  914. auto paramName = Y(BuildQuotedAtom(Pos, param.first));
  915. if (!param.second->Init(ctx, src)) {
  916. return false;
  917. }
  918. Nodes.push_back(Q(L(paramName, param.second)));
  919. }
  920. return true;
  921. }
  922. INode::TPtr TExternalFunctionConfig::DoClone() const {
  923. TFunctionConfig cloned;
  924. for (auto& [name, node] : Config) {
  925. cloned[name] = SafeClone(node);
  926. }
  927. return new TExternalFunctionConfig(GetPos(), cloned);
  928. }
  929. bool TWinRank::DoInit(TContext& ctx, ISource* src) {
  930. if (!ValidateArguments(ctx)) {
  931. return false;
  932. }
  933. if (!src) {
  934. ctx.Error(Pos) << "Unable to use window function: " << OpName << " without source";
  935. return false;
  936. }
  937. auto winNamePtr = src->GetWindowName();
  938. if (!winNamePtr) {
  939. ctx.Error(Pos) << "Failed to use window function: " << OpName << " without window";
  940. return false;
  941. }
  942. auto winSpecPtr = src->FindWindowSpecification(ctx, *winNamePtr);
  943. if (!winSpecPtr) {
  944. return false;
  945. }
  946. const auto& orderSpec = winSpecPtr->OrderBy;
  947. if (orderSpec.empty()) {
  948. if (Args.empty()) {
  949. ctx.Warning(GetPos(), TIssuesIds::YQL_RANK_WITHOUT_ORDER_BY) <<
  950. OpName << "() is used with unordered window - all rows will be considered equal to each other";
  951. } else {
  952. ctx.Warning(GetPos(), TIssuesIds::YQL_RANK_WITHOUT_ORDER_BY) <<
  953. OpName << "(<expression>) is used with unordered window - the result is likely to be undefined";
  954. }
  955. }
  956. if (Args.empty()) {
  957. for (const auto& spec: orderSpec) {
  958. Args.push_back(spec->Clone()->OrderExpr);
  959. }
  960. if (Args.size() != 1) {
  961. Args = {BuildTuple(GetPos(), Args)};
  962. }
  963. }
  964. YQL_ENSURE(Args.size() == 1);
  965. TVector<TNodePtr> optionsElements;
  966. if (!ctx.AnsiRankForNullableKeys.Defined()) {
  967. optionsElements.push_back(BuildTuple(Pos, { BuildQuotedAtom(Pos, "warnNoAnsi", NYql::TNodeFlags::Default) }));
  968. } else if (*ctx.AnsiRankForNullableKeys) {
  969. optionsElements.push_back(BuildTuple(Pos, { BuildQuotedAtom(Pos, "ansi", NYql::TNodeFlags::Default) }));
  970. }
  971. Args.push_back(BuildTuple(Pos, optionsElements));
  972. MinArgs = MaxArgs = 2;
  973. if (!TWinAggrEmulation::DoInit(ctx, src)) {
  974. return false;
  975. }
  976. YQL_ENSURE(Args.size() == 2);
  977. Args[0] = BuildLambda(Pos, Y("row"), Args[0]);
  978. return true;
  979. }
  980. class TQuotedAtomNode: public TAstListNode {
  981. public:
  982. TQuotedAtomNode(TPosition pos, const TString& content, ui32 flags)
  983. : TAstListNode(pos)
  984. {
  985. Add("quote", BuildAtom(pos, content, flags));
  986. }
  987. protected:
  988. TQuotedAtomNode(const TQuotedAtomNode& other)
  989. : TAstListNode(other.Pos)
  990. {
  991. Nodes = CloneContainer(other.Nodes);
  992. }
  993. TPtr DoClone() const final {
  994. return new TQuotedAtomNode(*this);
  995. }
  996. };
  997. TNodePtr BuildQuotedAtom(TPosition pos, const TString& content, ui32 flags) {
  998. return new TQuotedAtomNode(pos, content, flags);
  999. }
  1000. TNodePtr ITableKeys::AddView(TNodePtr key, const TViewDescription& view) {
  1001. if (view.PrimaryFlag) {
  1002. return L(key, Q(Y(Q("primary_view"))));
  1003. } else if (!view.empty()) {
  1004. return L(key, Q(Y(Q("view"), Y("String", BuildQuotedAtom(Pos, view.ViewName)))));
  1005. } else {
  1006. return key;
  1007. }
  1008. }
  1009. TString TColumns::AddUnnamed() {
  1010. TString desiredResult = TStringBuilder() << "column" << List.size();
  1011. if (!All) {
  1012. HasUnnamed = true;
  1013. List.emplace_back();
  1014. NamedColumns.push_back(false);
  1015. }
  1016. return desiredResult;
  1017. }
  1018. bool TColumns::Add(const TString* column, bool countHint, bool isArtificial, bool isReliable) {
  1019. if (!column || *column == "*") {
  1020. if (!countHint) {
  1021. SetAll();
  1022. }
  1023. } else if (!All) {
  1024. if (column->EndsWith('*')) {
  1025. QualifiedAll = true;
  1026. }
  1027. bool inserted = false;
  1028. if (isArtificial) {
  1029. inserted = Artificial.insert(*column).second;
  1030. } else {
  1031. inserted = Real.insert(*column).second;
  1032. }
  1033. if (!isReliable) {
  1034. HasUnreliable = true;
  1035. }
  1036. if (std::find(List.begin(), List.end(), *column) == List.end()) {
  1037. List.push_back(*column);
  1038. NamedColumns.push_back(true);
  1039. }
  1040. return inserted;
  1041. }
  1042. return All;
  1043. }
  1044. void TColumns::Merge(const TColumns& columns) {
  1045. if (columns.All) {
  1046. SetAll();
  1047. } else {
  1048. YQL_ENSURE(columns.List.size() == columns.NamedColumns.size());
  1049. size_t myUnnamed = NamedColumns.size() - std::accumulate(NamedColumns.begin(), NamedColumns.end(), 0);
  1050. size_t otherUnnamed = 0;
  1051. for (size_t i = 0; i < columns.List.size(); ++i) {
  1052. auto& c = columns.List[i];
  1053. if (!columns.NamedColumns[i]) {
  1054. if (++otherUnnamed > myUnnamed) {
  1055. AddUnnamed();
  1056. ++myUnnamed;
  1057. }
  1058. continue;
  1059. }
  1060. if (columns.Real.contains(c)) {
  1061. Add(&c, false, false);
  1062. }
  1063. if (columns.Artificial.contains(c)) {
  1064. Add(&c, false, true);
  1065. }
  1066. }
  1067. HasUnreliable |= columns.HasUnreliable;
  1068. HasUnnamed |= columns.HasUnnamed;
  1069. }
  1070. }
  1071. void TColumns::SetPrefix(const TString& prefix) {
  1072. Y_DEBUG_ABORT_UNLESS(!prefix.empty());
  1073. auto addPrefixFunc = [&prefix](const TString& str) {
  1074. return prefix + "." + str;
  1075. };
  1076. TSet<TString> newReal;
  1077. TSet<TString> newArtificial;
  1078. TVector<TString> newList;
  1079. std::transform(Real.begin(), Real.end(), std::inserter(newReal, newReal.begin()), addPrefixFunc);
  1080. std::transform(Artificial.begin(), Artificial.end(), std::inserter(newArtificial, newArtificial.begin()), addPrefixFunc);
  1081. std::transform(List.begin(), List.end(), std::back_inserter(newList), addPrefixFunc);
  1082. newReal.swap(Real);
  1083. newArtificial.swap(Artificial);
  1084. newList.swap(List);
  1085. }
  1086. void TColumns::SetAll() {
  1087. All = true;
  1088. QualifiedAll = false;
  1089. Real.clear();
  1090. List.clear();
  1091. Artificial.clear();
  1092. NamedColumns.clear();
  1093. HasUnnamed = HasUnreliable = false;
  1094. }
  1095. namespace {
  1096. bool MaybeAutogenerated(const TString& name) {
  1097. TStringBuf prefix = "column";
  1098. if (!name.StartsWith(prefix)) {
  1099. return false;
  1100. }
  1101. TString suffix = name.substr(prefix.size());
  1102. return !suffix.empty() && AllOf(suffix, [](const auto c) { return std::isdigit(c); });
  1103. }
  1104. bool MatchDotSuffix(const TSet<TString>& columns, const TString& column) {
  1105. for (const auto& col: columns) {
  1106. const auto pos = col.find_first_of(".");
  1107. if (pos == TString::npos) {
  1108. continue;
  1109. }
  1110. if (column == col.substr(pos + 1)) {
  1111. return true;
  1112. }
  1113. }
  1114. return false;
  1115. }
  1116. }
  1117. bool TColumns::IsColumnPossible(TContext& ctx, const TString& name) const {
  1118. if (All || Real.contains(name) || Artificial.contains(name)) {
  1119. return true;
  1120. }
  1121. if (ctx.SimpleColumns && !name.Contains('.') && (MatchDotSuffix(Real, name) || MatchDotSuffix(Artificial, name))) {
  1122. return true;
  1123. }
  1124. if (QualifiedAll) {
  1125. if (ctx.SimpleColumns) {
  1126. return true;
  1127. }
  1128. if (HasUnnamed) {
  1129. const auto dotPos = name.find_first_of(".");
  1130. TString suffix = (dotPos == TString::npos) ? name : name.substr(dotPos + 1);
  1131. if (MaybeAutogenerated(suffix)) {
  1132. return true;
  1133. }
  1134. }
  1135. for (const auto& real: Real) {
  1136. const auto pos = real.find_first_of("*");
  1137. if (pos == TString::npos) {
  1138. continue;
  1139. }
  1140. if (name.StartsWith(real.substr(0, pos))) {
  1141. return true;
  1142. }
  1143. }
  1144. } else if (HasUnnamed && MaybeAutogenerated(name)) {
  1145. return true;
  1146. }
  1147. return false;
  1148. }
  1149. TSortSpecification::TSortSpecification(const TNodePtr& orderExpr, bool ascending)
  1150. : OrderExpr(orderExpr->Clone())
  1151. , Ascending(ascending)
  1152. , CleanOrderExpr(orderExpr->Clone())
  1153. {
  1154. }
  1155. TSortSpecificationPtr TSortSpecification::Clone() const {
  1156. return MakeIntrusive<TSortSpecification>(CleanOrderExpr, Ascending);
  1157. }
  1158. TFrameBoundPtr TFrameBound::Clone() const {
  1159. auto res = MakeIntrusive<TFrameBound>();
  1160. res->Pos = Pos;
  1161. res->Bound = SafeClone(Bound);
  1162. res->Settings = Settings;
  1163. return res;
  1164. }
  1165. TFrameSpecificationPtr TFrameSpecification::Clone() const {
  1166. YQL_ENSURE(FrameBegin);
  1167. YQL_ENSURE(FrameEnd);
  1168. auto res = MakeIntrusive<TFrameSpecification>();
  1169. res->FrameType = FrameType;
  1170. res->FrameBegin = FrameBegin->Clone();
  1171. res->FrameEnd = FrameEnd->Clone();
  1172. res->FrameExclusion = FrameExclusion;
  1173. return res;
  1174. }
  1175. TWindowSpecificationPtr TWindowSpecification::Clone() const {
  1176. YQL_ENSURE(Frame);
  1177. auto res = MakeIntrusive<TWindowSpecification>();
  1178. res->ExistingWindowName = ExistingWindowName;
  1179. res->Partitions = CloneContainer(Partitions);
  1180. res->IsCompact = IsCompact;
  1181. res->OrderBy = CloneContainer(OrderBy);
  1182. res->Session = SafeClone(Session);
  1183. res->Frame = Frame->Clone();
  1184. return res;
  1185. }
  1186. TWinSpecs CloneContainer(const TWinSpecs& specs) {
  1187. TWinSpecs newSpecs;
  1188. for (auto cur: specs) {
  1189. newSpecs.emplace(cur.first, cur.second->Clone());
  1190. }
  1191. return newSpecs;
  1192. }
  1193. TLegacyHoppingWindowSpecPtr TLegacyHoppingWindowSpec::Clone() const {
  1194. auto res = MakeIntrusive<TLegacyHoppingWindowSpec>();
  1195. res->TimeExtractor = TimeExtractor->Clone();
  1196. res->Hop = Hop->Clone();
  1197. res->Interval = Interval->Clone();
  1198. res->Delay = Delay->Clone();
  1199. res->DataWatermarks = DataWatermarks;
  1200. return res;
  1201. }
  1202. TColumnNode::TColumnNode(TPosition pos, const TString& column, const TString& source, bool maybeType)
  1203. : INode(pos)
  1204. , ColumnName(column)
  1205. , Source(source)
  1206. , MaybeType(maybeType)
  1207. {
  1208. }
  1209. TColumnNode::TColumnNode(TPosition pos, const TNodePtr& column, const TString& source)
  1210. : INode(pos)
  1211. , ColumnExpr(column)
  1212. , Source(source)
  1213. {
  1214. }
  1215. TColumnNode::~TColumnNode()
  1216. {
  1217. }
  1218. bool TColumnNode::IsAsterisk() const {
  1219. return ColumnName == "*";
  1220. }
  1221. bool TColumnNode::IsArtificial() const {
  1222. return Artificial;
  1223. }
  1224. const TString* TColumnNode::GetColumnName() const {
  1225. return UseSourceAsColumn ? &Source : (ColumnExpr ? nullptr : &ColumnName);
  1226. }
  1227. const TString* TColumnNode::GetSourceName() const {
  1228. return UseSourceAsColumn ? &Empty : &Source;
  1229. }
  1230. TColumnNode* TColumnNode::GetColumnNode() {
  1231. return this;
  1232. }
  1233. const TColumnNode* TColumnNode::GetColumnNode () const {
  1234. return this;
  1235. }
  1236. bool TColumnNode::DoInit(TContext& ctx, ISource* src) {
  1237. if (src) {
  1238. YQL_ENSURE(!State.Test(ENodeState::Initialized)); /// should be not initialized or Aggregated already invalid
  1239. if (src->ShouldUseSourceAsColumn(*GetSourceName())) {
  1240. if (!IsAsterisk() && IsReliable()) {
  1241. SetUseSourceAsColumn();
  1242. }
  1243. }
  1244. if (GetColumnName()) {
  1245. auto fullName = Source ? DotJoin(Source, *GetColumnName()) : *GetColumnName();
  1246. auto alias = src->GetGroupByColumnAlias(fullName);
  1247. if (alias) {
  1248. ResetColumn(alias, {});
  1249. }
  1250. Artificial = !Source && src->IsExprAlias(*GetColumnName());
  1251. }
  1252. if (!src->AddColumn(ctx, *this)) {
  1253. return false;
  1254. }
  1255. if (GetColumnName()) {
  1256. if (src->GetJoin() && Source) {
  1257. GroupKey = src->IsGroupByColumn(DotJoin(Source, *GetColumnName()));
  1258. } else {
  1259. GroupKey = src->IsGroupByColumn(*GetColumnName()) || src->IsAlias(EExprSeat::GroupBy, *GetColumnName());
  1260. }
  1261. }
  1262. }
  1263. if (IsAsterisk()) {
  1264. Node = AstNode("row");
  1265. } else {
  1266. TString callable;
  1267. if (MaybeType) {
  1268. callable = Reliable && !UseSource ? "SqlPlainColumnOrType" : "SqlColumnOrType";
  1269. } else {
  1270. // TODO: consider replacing Member -> SqlPlainColumn
  1271. callable = Reliable && !UseSource ? "Member" : "SqlColumn";
  1272. }
  1273. Node = Y(callable, "row", ColumnExpr ? Y("EvaluateAtom", ColumnExpr) : BuildQuotedAtom(Pos, *GetColumnName()));
  1274. if (UseSource) {
  1275. YQL_ENSURE(Source);
  1276. Node = L(Node, BuildQuotedAtom(Pos, Source));
  1277. }
  1278. }
  1279. return Node->Init(ctx, src);
  1280. }
  1281. void TColumnNode::SetUseSourceAsColumn() {
  1282. YQL_ENSURE(!State.Test(ENodeState::Initialized)); /// should be not initialized or Aggregated already invalid
  1283. YQL_ENSURE(!IsAsterisk());
  1284. UseSourceAsColumn = true;
  1285. }
  1286. void TColumnNode::ResetAsReliable() {
  1287. Reliable = true;
  1288. }
  1289. void TColumnNode::SetAsNotReliable() {
  1290. Reliable = false;
  1291. }
  1292. void TColumnNode::SetUseSource() {
  1293. UseSource = true;
  1294. }
  1295. bool TColumnNode::IsUseSourceAsColumn() const {
  1296. return UseSourceAsColumn;
  1297. }
  1298. bool TColumnNode::IsUseSource() const {
  1299. return UseSource;
  1300. }
  1301. bool TColumnNode::IsReliable() const {
  1302. return Reliable;
  1303. }
  1304. bool TColumnNode::CanBeType() const {
  1305. return MaybeType;
  1306. }
  1307. TNodePtr TColumnNode::DoClone() const {
  1308. YQL_ENSURE(!Node, "TColumnNode::Clone: Node should not be initialized");
  1309. auto copy = ColumnExpr ? new TColumnNode(Pos, ColumnExpr, Source) : new TColumnNode(Pos, ColumnName, Source, MaybeType);
  1310. copy->GroupKey = GroupKey;
  1311. copy->Artificial = Artificial;
  1312. copy->Reliable = Reliable;
  1313. copy->UseSource = UseSource;
  1314. copy->UseSourceAsColumn = UseSourceAsColumn;
  1315. return copy;
  1316. }
  1317. void TColumnNode::DoUpdateState() const {
  1318. State.Set(ENodeState::Const, false);
  1319. State.Set(ENodeState::MaybeConst, MaybeType);
  1320. State.Set(ENodeState::Aggregated, GroupKey);
  1321. State.Set(ENodeState::AggregationKey, GroupKey);
  1322. }
  1323. TAstNode* TColumnNode::Translate(TContext& ctx) const {
  1324. return Node->Translate(ctx);
  1325. }
  1326. void TColumnNode::ResetColumn(const TString& column, const TString& source) {
  1327. YQL_ENSURE(!State.Test(ENodeState::Initialized)); /// should be not initialized
  1328. Reliable = true;
  1329. UseSource = false;
  1330. UseSourceAsColumn = false;
  1331. ColumnName = column;
  1332. ColumnExpr = nullptr;
  1333. Source = source;
  1334. }
  1335. void TColumnNode::ResetColumn(const TNodePtr& column, const TString& source) {
  1336. YQL_ENSURE(!State.Test(ENodeState::Initialized)); /// should be not initialized
  1337. Reliable = true;
  1338. UseSource = false;
  1339. UseSourceAsColumn = false;
  1340. ColumnName = "";
  1341. ColumnExpr = column;
  1342. Source = source;
  1343. }
  1344. const TString TColumnNode::Empty;
  1345. TNodePtr BuildColumn(TPosition pos, const TString& column, const TString& source) {
  1346. bool maybeType = false;
  1347. return new TColumnNode(pos, column, source, maybeType);
  1348. }
  1349. TNodePtr BuildColumn(TPosition pos, const TNodePtr& column, const TString& source) {
  1350. return new TColumnNode(pos, column, source);
  1351. }
  1352. TNodePtr BuildColumn(TPosition pos, const TDeferredAtom& column, const TString& source) {
  1353. return column.GetLiteral() ? BuildColumn(pos, *column.GetLiteral(), source) : BuildColumn(pos, column.Build(), source);
  1354. }
  1355. TNodePtr BuildColumnOrType(TPosition pos, const TString& column) {
  1356. TString source = "";
  1357. bool maybeType = true;
  1358. return new TColumnNode(pos, column, source, maybeType);
  1359. }
  1360. ITableKeys::ITableKeys(TPosition pos)
  1361. : INode(pos)
  1362. {
  1363. }
  1364. const TString* ITableKeys::GetTableName() const {
  1365. return nullptr;
  1366. }
  1367. ITableKeys* ITableKeys::GetTableKeys() {
  1368. return this;
  1369. }
  1370. TAstNode* ITableKeys::Translate(TContext& ctx) const {
  1371. Y_DEBUG_ABORT_UNLESS(false);
  1372. Y_UNUSED(ctx);
  1373. return nullptr;
  1374. }
  1375. bool IAggregation::IsDistinct() const {
  1376. return !DistinctKey.empty();
  1377. }
  1378. void IAggregation::DoUpdateState() const {
  1379. State.Set(ENodeState::Aggregated, AggMode == EAggregateMode::Normal);
  1380. State.Set(ENodeState::OverWindow, AggMode == EAggregateMode::OverWindow);
  1381. State.Set(ENodeState::OverWindowDistinct, AggMode == EAggregateMode::OverWindowDistinct);
  1382. }
  1383. const TString* IAggregation::GetGenericKey() const {
  1384. return nullptr;
  1385. }
  1386. void IAggregation::Join(IAggregation*) {
  1387. YQL_ENSURE(false, "Should not be called");
  1388. }
  1389. const TString& IAggregation::GetName() const {
  1390. return Name;
  1391. }
  1392. EAggregateMode IAggregation::GetAggregationMode() const {
  1393. return AggMode;
  1394. }
  1395. void IAggregation::MarkKeyColumnAsGenerated() {
  1396. IsGeneratedKeyColumn = true;
  1397. }
  1398. IAggregation::IAggregation(TPosition pos, const TString& name, const TString& func, EAggregateMode aggMode)
  1399. : INode(pos), Name(name), Func(func), AggMode(aggMode)
  1400. {}
  1401. TAstNode* IAggregation::Translate(TContext& ctx) const {
  1402. Y_DEBUG_ABORT_UNLESS(false);
  1403. Y_UNUSED(ctx);
  1404. return nullptr;
  1405. }
  1406. std::pair<TNodePtr, bool> IAggregation::AggregationTraits(const TNodePtr& type, bool overState, bool many, bool allowAggApply, TContext& ctx) const {
  1407. const bool distinct = AggMode == EAggregateMode::Distinct;
  1408. const auto listType = distinct ? Y("ListType", Y("StructMemberType", Y("ListItemType", type), BuildQuotedAtom(Pos, DistinctKey))) : type;
  1409. auto apply = GetApply(listType, many, allowAggApply, ctx);
  1410. if (!apply) {
  1411. return { nullptr, false };
  1412. }
  1413. auto wrapped = WrapIfOverState(apply, overState, many, ctx);
  1414. if (!wrapped) {
  1415. return { nullptr, false };
  1416. }
  1417. return { distinct ?
  1418. Q(Y(Q(Name), wrapped, BuildQuotedAtom(Pos, DistinctKey))) :
  1419. Q(Y(Q(Name), wrapped)), true };
  1420. }
  1421. TNodePtr IAggregation::WrapIfOverState(const TNodePtr& input, bool overState, bool many, TContext& ctx) const {
  1422. if (!overState) {
  1423. return input;
  1424. }
  1425. auto extractor = GetExtractor(many, ctx);
  1426. if (!extractor) {
  1427. return nullptr;
  1428. }
  1429. return Y(ToString("AggOverState"), extractor, BuildLambda(Pos, Y(), input));
  1430. }
  1431. void IAggregation::AddFactoryArguments(TNodePtr& apply) const {
  1432. Y_UNUSED(apply);
  1433. }
  1434. std::vector<ui32> IAggregation::GetFactoryColumnIndices() const {
  1435. return {0u};
  1436. }
  1437. TNodePtr IAggregation::WindowTraits(const TNodePtr& type, TContext& ctx) const {
  1438. YQL_ENSURE(AggMode == EAggregateMode::OverWindow || AggMode == EAggregateMode::OverWindowDistinct, "Windows traits is unavailable");
  1439. const bool distinct = AggMode == EAggregateMode::OverWindowDistinct;
  1440. const auto listType = distinct ? Y("ListType", Y("StructMemberType", Y("ListItemType", type), BuildQuotedAtom(Pos, DistinctKey))) : type;
  1441. auto traits = Y(Q(Name), GetApply(listType, false, false, ctx));
  1442. if (AggMode == EAggregateMode::OverWindowDistinct) {
  1443. traits->Add(BuildQuotedAtom(Pos, DistinctKey));
  1444. }
  1445. return Q(traits);
  1446. }
  1447. namespace {
  1448. bool UnescapeQuoted(const TString& str, TPosition& pos, char quoteChar, TString& result, TString& error, bool utf8Aware) {
  1449. result = error = {};
  1450. size_t readBytes = 0;
  1451. TStringBuf atom(str);
  1452. TStringOutput sout(result);
  1453. atom.Skip(1);
  1454. result.reserve(str.size());
  1455. auto unescapeResult = UnescapeArbitraryAtom(atom, quoteChar, &sout, &readBytes);
  1456. if (unescapeResult != EUnescapeResult::OK) {
  1457. TTextWalker walker(pos, utf8Aware);
  1458. walker.Advance(atom.Trunc(readBytes));
  1459. error = UnescapeResultToString(unescapeResult);
  1460. return false;
  1461. }
  1462. return true;
  1463. }
  1464. TString UnescapeAnsiQuoted(const TString& str) {
  1465. YQL_ENSURE(str.length() >= 2);
  1466. YQL_ENSURE(str[0] == str[str.length() - 1]);
  1467. YQL_ENSURE(str[0] == '\'' || str[0] == '"');
  1468. TString quote(1, str[0]);
  1469. TString replace(2, str[0]);
  1470. TString result = str.substr(1, str.length() - 2);
  1471. SubstGlobal(result, replace, quote);
  1472. return result;
  1473. }
  1474. enum class EStringContentMode : int {
  1475. Default = 0,
  1476. AnsiIdent,
  1477. TypedStringLiteral,
  1478. };
  1479. TMaybe<TStringContent>
  1480. StringContentInternal(TContext& ctx, TPosition pos, const TString& input, EStringContentMode mode) {
  1481. TStringContent result;
  1482. if (mode == EStringContentMode::AnsiIdent) {
  1483. if (!(input.size() >= 2 && input.StartsWith('"') && input.EndsWith('"'))) {
  1484. ctx.Error(pos) << "Expected double quoted identifier, got string literal";
  1485. return {};
  1486. }
  1487. result.Flags = NYql::TNodeFlags::ArbitraryContent;
  1488. result.Content = UnescapeAnsiQuoted(input);
  1489. return result;
  1490. }
  1491. TString str = input;
  1492. if (mode == EStringContentMode::TypedStringLiteral) {
  1493. auto lower = to_lower(str);
  1494. if (lower.EndsWith("y")) {
  1495. str = str.substr(0, str.size() - 1);
  1496. result.Type = NKikimr::NUdf::EDataSlot::Yson;
  1497. } else if (lower.EndsWith("j")) {
  1498. str = str.substr(0, str.size() - 1);
  1499. result.Type = NKikimr::NUdf::EDataSlot::Json;
  1500. } else if (lower.EndsWith("p")) {
  1501. str = str.substr(0, str.size() - 1);
  1502. result.PgType = "PgText";
  1503. } else if (lower.EndsWith("pt")) {
  1504. str = str.substr(0, str.size() - 2);
  1505. result.PgType = "PgText";
  1506. } else if (lower.EndsWith("pb")) {
  1507. str = str.substr(0, str.size() - 2);
  1508. result.PgType = "PgBytea";
  1509. } else if (lower.EndsWith("pv")) {
  1510. str = str.substr(0, str.size() - 2);
  1511. result.PgType = "PgVarchar";
  1512. } else if (lower.EndsWith("s")) {
  1513. str = str.substr(0, str.size() - 1);
  1514. result.Type = NKikimr::NUdf::EDataSlot::String;
  1515. } else if (lower.EndsWith("u")) {
  1516. str = str.substr(0, str.size() - 1);
  1517. result.Type = NKikimr::NUdf::EDataSlot::Utf8;
  1518. } else {
  1519. if (ctx.Scoped->WarnUntypedStringLiterals) {
  1520. ctx.Warning(pos, TIssuesIds::YQL_UNTYPED_STRING_LITERALS)
  1521. << "Please add suffix u for Utf8 strings or s for arbitrary binary strings";
  1522. }
  1523. if (ctx.Scoped->UnicodeLiterals) {
  1524. result.Type = NKikimr::NUdf::EDataSlot::Utf8;
  1525. }
  1526. }
  1527. }
  1528. if (mode == EStringContentMode::Default && (result.Type != NKikimr::NUdf::EDataSlot::String || result.PgType)) {
  1529. ctx.Error(pos) << "Type suffix is not allowed here";
  1530. return {};
  1531. }
  1532. bool doubleQuoted = (str.StartsWith('"') && str.EndsWith('"'));
  1533. bool singleQuoted = !doubleQuoted && (str.StartsWith('\'') && str.EndsWith('\''));
  1534. if (str.size() >= 2 && (doubleQuoted || singleQuoted)) {
  1535. result.Flags = NYql::TNodeFlags::ArbitraryContent;
  1536. if (ctx.Settings.AnsiLexer) {
  1537. YQL_ENSURE(singleQuoted);
  1538. result.Content = UnescapeAnsiQuoted(str);
  1539. } else {
  1540. TString error;
  1541. if (!UnescapeQuoted(str, pos, str[0], result.Content, error, ctx.Settings.Antlr4Parser)) {
  1542. ctx.Error(pos) << "Failed to parse string literal: " << error;
  1543. return {};
  1544. }
  1545. }
  1546. } else if (str.size() >= 4 && str.StartsWith("@@") && str.EndsWith("@@")) {
  1547. result.Flags = TNodeFlags::MultilineContent;
  1548. TString s = str.substr(2, str.length() - 4);
  1549. SubstGlobal(s, "@@@@", "@@");
  1550. result.Content.swap(s);
  1551. } else {
  1552. ctx.Error(pos) << "Invalid string literal: " << EscapeC(str);
  1553. return {};
  1554. }
  1555. if (!result.PgType.Defined() && !NKikimr::NMiniKQL::IsValidStringValue(result.Type, result.Content)) {
  1556. ctx.Error() << "Invalid value " << result.Content.Quote() << " for type " << result.Type;
  1557. return {};
  1558. }
  1559. return result;
  1560. }
  1561. } // namespace
  1562. TMaybe<TStringContent> StringContent(TContext& ctx, TPosition pos, const TString& input) {
  1563. if (ctx.AnsiQuotedIdentifiers && input.StartsWith('"')) {
  1564. ctx.Error() << "Expected string literal, got quoted identifier";
  1565. return {};
  1566. }
  1567. return StringContentInternal(ctx, pos, input, EStringContentMode::Default);
  1568. }
  1569. TMaybe<TStringContent> StringContentOrIdContent(TContext& ctx, TPosition pos, const TString& input) {
  1570. return StringContentInternal(ctx, pos, input,
  1571. (ctx.AnsiQuotedIdentifiers && input.StartsWith('"'))? EStringContentMode::AnsiIdent : EStringContentMode::Default);
  1572. }
  1573. TTtlSettings::TTierSettings::TTierSettings(const TNodePtr& evictionDelay, const std::optional<TIdentifier>& storageName)
  1574. : EvictionDelay(evictionDelay)
  1575. , StorageName(storageName) {
  1576. }
  1577. TTtlSettings::TTtlSettings(const TIdentifier& columnName, const std::vector<TTierSettings>& tiers, const TMaybe<EUnit>& columnUnit)
  1578. : ColumnName(columnName)
  1579. , Tiers(tiers)
  1580. , ColumnUnit(columnUnit)
  1581. {
  1582. }
  1583. TString IdContent(TContext& ctx, const TString& s) {
  1584. YQL_ENSURE(!s.empty(), "Empty identifier not expected");
  1585. if (!s.StartsWith('`')) {
  1586. return s;
  1587. }
  1588. auto endSym = '`';
  1589. if (s.size() < 2 || !s.EndsWith(endSym)) {
  1590. ctx.Error() << "The identifier that starts with: '" << s[0] << "' should ends with: '" << endSym << "'";
  1591. return {};
  1592. }
  1593. size_t skipSymbols = 1;
  1594. TStringBuf atom(s.data() + skipSymbols, s.size() - 2 * skipSymbols + 1);
  1595. TString unescapedStr;
  1596. TStringOutput sout(unescapedStr);
  1597. unescapedStr.reserve(s.size());
  1598. size_t readBytes = 0;
  1599. TPosition pos = ctx.Pos();
  1600. pos.Column += skipSymbols - 1;
  1601. auto unescapeResult = UnescapeArbitraryAtom(atom, endSym, &sout, &readBytes);
  1602. if (unescapeResult != EUnescapeResult::OK) {
  1603. TTextWalker walker(pos, ctx.Settings.Antlr4Parser);
  1604. walker.Advance(atom.Trunc(readBytes));
  1605. ctx.Error(pos) << "Cannot parse broken identifier: " << UnescapeResultToString(unescapeResult);
  1606. return {};
  1607. }
  1608. if (readBytes != atom.size()) {
  1609. ctx.Error() << "The identifier not parsed completely";
  1610. return {};
  1611. }
  1612. return unescapedStr;
  1613. }
  1614. TString IdContentFromString(TContext& ctx, const TString& str) {
  1615. if (!ctx.AnsiQuotedIdentifiers) {
  1616. ctx.Error() << "String literal can not be used here";
  1617. return {};
  1618. }
  1619. auto parsed = StringContentInternal(ctx, ctx.Pos(), str, EStringContentMode::AnsiIdent);
  1620. if (!parsed) {
  1621. return {};
  1622. }
  1623. return parsed->Content;
  1624. }
  1625. namespace {
  1626. class TInvalidLiteralNode final: public INode {
  1627. public:
  1628. TInvalidLiteralNode(TPosition pos)
  1629. : INode(pos)
  1630. {
  1631. }
  1632. bool DoInit(TContext& ctx, ISource* source) override {
  1633. Y_UNUSED(ctx);
  1634. Y_UNUSED(source);
  1635. return false;
  1636. }
  1637. TAstNode* Translate(TContext& ctx) const override {
  1638. Y_UNUSED(ctx);
  1639. return nullptr;
  1640. }
  1641. TPtr DoClone() const override {
  1642. return new TInvalidLiteralNode(GetPos());
  1643. }
  1644. };
  1645. }
  1646. TLiteralNode::TLiteralNode(TPosition pos, bool isNull)
  1647. : TAstListNode(pos)
  1648. , Null(isNull)
  1649. , Void(!isNull)
  1650. {
  1651. Add(isNull ? "Null" : "Void");
  1652. }
  1653. TLiteralNode::TLiteralNode(TPosition pos, const TString& type, const TString& value)
  1654. : TAstListNode(pos)
  1655. , Null(false)
  1656. , Void(false)
  1657. , Type(type)
  1658. , Value(value)
  1659. {
  1660. if (Type.StartsWith("Pg")) {
  1661. Add("PgConst", BuildQuotedAtom(Pos, Value), Y("PgType", Q(to_lower(Type.substr(2)))));
  1662. } else {
  1663. Add(Type, BuildQuotedAtom(Pos, Value));
  1664. }
  1665. }
  1666. TLiteralNode::TLiteralNode(TPosition pos, const TString& value, ui32 nodeFlags)
  1667. : TAstListNode(pos)
  1668. , Null(false)
  1669. , Void(false)
  1670. , Type("String")
  1671. , Value(value)
  1672. {
  1673. Add(Type, BuildQuotedAtom(pos, Value, nodeFlags));
  1674. }
  1675. TLiteralNode::TLiteralNode(TPosition pos, const TString& value, ui32 nodeFlags, const TString& type)
  1676. : TAstListNode(pos)
  1677. , Null(false)
  1678. , Void(false)
  1679. , Type(type)
  1680. , Value(value)
  1681. {
  1682. if (Type.StartsWith("Pg")) {
  1683. Add("PgConst", BuildQuotedAtom(Pos, Value, nodeFlags), Y("PgType", Q(to_lower(Type.substr(2)))));
  1684. } else {
  1685. Add(Type, BuildQuotedAtom(pos, Value, nodeFlags));
  1686. }
  1687. }
  1688. bool TLiteralNode::IsNull() const {
  1689. return Null;
  1690. }
  1691. const TString* TLiteralNode::GetLiteral(const TString& type) const {
  1692. return type == Type ? &Value : nullptr;
  1693. }
  1694. bool TLiteralNode::IsLiteral() const {
  1695. return true;
  1696. }
  1697. TString TLiteralNode::GetLiteralType() const {
  1698. return Type;
  1699. }
  1700. TString TLiteralNode::GetLiteralValue() const {
  1701. return Value;
  1702. }
  1703. void TLiteralNode::DoUpdateState() const {
  1704. State.Set(ENodeState::Const);
  1705. }
  1706. TNodePtr TLiteralNode::DoClone() const {
  1707. auto res = (Null || Void) ? MakeIntrusive<TLiteralNode>(Pos, Null) : MakeIntrusive<TLiteralNode>(Pos, Type, Value);
  1708. res->Nodes = Nodes;
  1709. return res;
  1710. }
  1711. template<typename T>
  1712. TLiteralNumberNode<T>::TLiteralNumberNode(TPosition pos, const TString& type, const TString& value, bool implicitType)
  1713. : TLiteralNode(pos, type, value)
  1714. , ImplicitType(implicitType)
  1715. {}
  1716. template<typename T>
  1717. TNodePtr TLiteralNumberNode<T>::DoClone() const {
  1718. return new TLiteralNumberNode<T>(Pos, Type, Value, ImplicitType);
  1719. }
  1720. template<typename T>
  1721. bool TLiteralNumberNode<T>::DoInit(TContext& ctx, ISource* src) {
  1722. Y_UNUSED(src);
  1723. T val;
  1724. if (!TryFromString(Value, val)) {
  1725. ctx.Error(Pos) << "Failed to parse " << Value << " as integer literal of " << Type << " type: value out of range for " << Type;
  1726. return false;
  1727. }
  1728. return true;
  1729. }
  1730. template<typename T>
  1731. bool TLiteralNumberNode<T>::IsIntegerLiteral() const {
  1732. return std::numeric_limits<T>::is_integer;
  1733. }
  1734. template<typename T>
  1735. TNodePtr TLiteralNumberNode<T>::ApplyUnaryOp(TContext& ctx, TPosition pos, const TString& opName) const {
  1736. YQL_ENSURE(!Value.empty());
  1737. if (opName == "Minus" && IsIntegerLiteral() && Value[0] != '-') {
  1738. if (ImplicitType) {
  1739. ui64 val = FromString<ui64>(Value);
  1740. TString negated = "-" + Value;
  1741. if (val <= ui64(std::numeric_limits<i32>::max()) + 1) {
  1742. // negated value fits in Int32
  1743. i32 v;
  1744. YQL_ENSURE(TryFromString(negated, v));
  1745. return new TLiteralNumberNode<i32>(pos, Type.StartsWith("Pg") ? "PgInt4" : "Int32", negated);
  1746. }
  1747. if (val <= ui64(std::numeric_limits<i64>::max()) + 1) {
  1748. // negated value fits in Int64
  1749. i64 v;
  1750. YQL_ENSURE(TryFromString(negated, v));
  1751. return new TLiteralNumberNode<i64>(pos, Type.StartsWith("Pg") ? "PgInt8" : "Int64", negated);
  1752. }
  1753. ctx.Error(pos) << "Failed to parse negative integer: " << negated << ", number limit overflow";
  1754. return {};
  1755. }
  1756. if (std::numeric_limits<T>::is_signed) {
  1757. return new TLiteralNumberNode<T>(pos, Type, "-" + Value);
  1758. }
  1759. }
  1760. return INode::ApplyUnaryOp(ctx, pos, opName);
  1761. }
  1762. template class TLiteralNumberNode<i32>;
  1763. template class TLiteralNumberNode<i64>;
  1764. template class TLiteralNumberNode<ui32>;
  1765. template class TLiteralNumberNode<ui64>;
  1766. template class TLiteralNumberNode<float>;
  1767. template class TLiteralNumberNode<double>;
  1768. template class TLiteralNumberNode<ui8>;
  1769. template class TLiteralNumberNode<i8>;
  1770. template class TLiteralNumberNode<ui16>;
  1771. template class TLiteralNumberNode<i16>;
  1772. TNodePtr BuildLiteralNull(TPosition pos) {
  1773. return new TLiteralNode(pos, true);
  1774. }
  1775. TNodePtr BuildLiteralVoid(TPosition pos) {
  1776. return new TLiteralNode(pos, false);
  1777. }
  1778. TNodePtr BuildLiteralSmartString(TContext& ctx, const TString& value) {
  1779. auto unescaped = StringContent(ctx, ctx.Pos(), value);
  1780. if (!unescaped) {
  1781. return new TInvalidLiteralNode(ctx.Pos());
  1782. }
  1783. YQL_ENSURE(unescaped->Type == NKikimr::NUdf::EDataSlot::String);
  1784. return new TLiteralNode(ctx.Pos(), unescaped->Content, unescaped->Flags, "String");
  1785. }
  1786. TMaybe<TExprOrIdent> BuildLiteralTypedSmartStringOrId(TContext& ctx, const TString& value) {
  1787. TExprOrIdent result;
  1788. if (ctx.AnsiQuotedIdentifiers && value.StartsWith('"')) {
  1789. auto unescaped = StringContentInternal(ctx, ctx.Pos(), value, EStringContentMode::AnsiIdent);
  1790. if (!unescaped) {
  1791. return {};
  1792. }
  1793. result.Ident = unescaped->Content;
  1794. return result;
  1795. }
  1796. auto unescaped = StringContentInternal(ctx, ctx.Pos(), value, EStringContentMode::TypedStringLiteral);
  1797. if (!unescaped) {
  1798. return {};
  1799. }
  1800. TString type = unescaped->PgType ? *unescaped->PgType : ToString(unescaped->Type);
  1801. result.Expr = new TLiteralNode(ctx.Pos(), unescaped->Content, unescaped->Flags, type);
  1802. return result;
  1803. }
  1804. TNodePtr BuildLiteralRawString(TPosition pos, const TString& value, bool isUtf8) {
  1805. return new TLiteralNode(pos, isUtf8 ? "Utf8" : "String", value);
  1806. }
  1807. TNodePtr BuildLiteralBool(TPosition pos, bool value) {
  1808. return new TLiteralNode(pos, "Bool", value ? "true" : "false");
  1809. }
  1810. TAsteriskNode::TAsteriskNode(TPosition pos)
  1811. : INode(pos)
  1812. {}
  1813. bool TAsteriskNode::IsAsterisk() const {
  1814. return true;
  1815. };
  1816. TNodePtr TAsteriskNode::DoClone() const {
  1817. return new TAsteriskNode(Pos);
  1818. }
  1819. TAstNode* TAsteriskNode::Translate(TContext& ctx) const {
  1820. ctx.Error(Pos) << "* is not allowed here";
  1821. return nullptr;
  1822. }
  1823. TNodePtr BuildEmptyAction(TPosition pos) {
  1824. TNodePtr params = new TAstListNodeImpl(pos);
  1825. TNodePtr arg = new TAstAtomNodeImpl(pos, "x", TNodeFlags::Default);
  1826. params->Add(arg);
  1827. return BuildLambda(pos, params, arg);
  1828. }
  1829. TDeferredAtom::TDeferredAtom()
  1830. {}
  1831. TDeferredAtom::TDeferredAtom(TPosition pos, const TString& str)
  1832. {
  1833. Node = BuildQuotedAtom(pos, str);
  1834. Explicit = str;
  1835. Repr = str;
  1836. }
  1837. TDeferredAtom::TDeferredAtom(TNodePtr node, TContext& ctx)
  1838. {
  1839. Node = node;
  1840. Repr = ctx.MakeName("DeferredAtom");
  1841. }
  1842. const TString* TDeferredAtom::GetLiteral() const {
  1843. return Explicit.Get();
  1844. }
  1845. bool TDeferredAtom::GetLiteral(TString& value, TContext& ctx) const {
  1846. if (Explicit) {
  1847. value = *Explicit;
  1848. return true;
  1849. }
  1850. ctx.Error(Node ? Node->GetPos() : ctx.Pos()) << "Expected literal value";
  1851. return false;
  1852. }
  1853. TNodePtr TDeferredAtom::Build() const {
  1854. return Node;
  1855. }
  1856. TString TDeferredAtom::GetRepr() const {
  1857. return Repr;
  1858. }
  1859. bool TDeferredAtom::Empty() const {
  1860. return !Node || Repr.empty();
  1861. }
  1862. bool TDeferredAtom::HasNode() const {
  1863. return !!Node;
  1864. }
  1865. TTupleNode::TTupleNode(TPosition pos, const TVector<TNodePtr>& exprs)
  1866. : TAstListNode(pos)
  1867. , Exprs(exprs)
  1868. {}
  1869. bool TTupleNode::IsEmpty() const {
  1870. return Exprs.empty();
  1871. }
  1872. const TVector<TNodePtr>& TTupleNode::Elements() const {
  1873. return Exprs;
  1874. }
  1875. TTupleNode* TTupleNode::GetTupleNode() {
  1876. return this;
  1877. }
  1878. const TTupleNode* TTupleNode::GetTupleNode() const {
  1879. return this;
  1880. }
  1881. bool TTupleNode::DoInit(TContext& ctx, ISource* src) {
  1882. auto node(Y());
  1883. for (auto& expr: Exprs) {
  1884. if (expr->GetLabel()) {
  1885. ctx.Error(expr->GetPos()) << "Tuple does not allow named members";
  1886. return false;
  1887. }
  1888. node = L(node, expr);
  1889. }
  1890. Add("quote", node);
  1891. return TAstListNode::DoInit(ctx, src);
  1892. }
  1893. size_t TTupleNode::GetTupleSize() const {
  1894. return Exprs.size();
  1895. }
  1896. TNodePtr TTupleNode::GetTupleElement(size_t index) const {
  1897. return Exprs[index];
  1898. }
  1899. TNodePtr TTupleNode::DoClone() const {
  1900. return new TTupleNode(Pos, CloneContainer(Exprs));
  1901. }
  1902. void TTupleNode::CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) {
  1903. for (auto& expr : Exprs) {
  1904. expr->CollectPreaggregateExprs(ctx, src, exprs);
  1905. }
  1906. }
  1907. const TString* TTupleNode::GetSourceName() const {
  1908. return DeriveCommonSourceName(Exprs);
  1909. }
  1910. TNodePtr BuildTuple(TPosition pos, const TVector<TNodePtr>& exprs) {
  1911. return new TTupleNode(pos, exprs);
  1912. }
  1913. TStructNode::TStructNode(TPosition pos, const TVector<TNodePtr>& exprs, const TVector<TNodePtr>& labels, bool ordered)
  1914. : TAstListNode(pos)
  1915. , Exprs(exprs)
  1916. , Labels(labels)
  1917. , Ordered(ordered)
  1918. {
  1919. YQL_ENSURE(Labels.empty() || Labels.size() == Exprs.size());
  1920. }
  1921. bool TStructNode::DoInit(TContext& ctx, ISource* src) {
  1922. Nodes.push_back(BuildAtom(Pos, (Ordered || Exprs.size() < 2) ? "AsStruct" : "AsStructUnordered", TNodeFlags::Default));
  1923. size_t i = 0;
  1924. for (const auto& expr : Exprs) {
  1925. TNodePtr label;
  1926. if (Labels.empty()) {
  1927. if (!expr->GetLabel()) {
  1928. ctx.Error(expr->GetPos()) << "Structure does not allow anonymous members";
  1929. return false;
  1930. }
  1931. label = BuildQuotedAtom(expr->GetPos(), expr->GetLabel());
  1932. } else {
  1933. label = Labels[i++];
  1934. }
  1935. Nodes.push_back(Q(Y(label, expr)));
  1936. }
  1937. return TAstListNode::DoInit(ctx, src);
  1938. }
  1939. TNodePtr TStructNode::DoClone() const {
  1940. return new TStructNode(Pos, CloneContainer(Exprs), CloneContainer(Labels), Ordered);
  1941. }
  1942. TStructNode* TStructNode::GetStructNode() {
  1943. return this;
  1944. }
  1945. const TStructNode* TStructNode::GetStructNode() const {
  1946. return this;
  1947. }
  1948. void TStructNode::CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) {
  1949. for (auto& expr : Exprs) {
  1950. expr->CollectPreaggregateExprs(ctx, src, exprs);
  1951. }
  1952. }
  1953. const TString* TStructNode::GetSourceName() const {
  1954. return DeriveCommonSourceName(Exprs);
  1955. }
  1956. TNodePtr BuildStructure(TPosition pos, const TVector<TNodePtr>& exprs) {
  1957. bool ordered = false;
  1958. return new TStructNode(pos, exprs, {}, ordered);
  1959. }
  1960. TNodePtr BuildStructure(TPosition pos, const TVector<TNodePtr>& exprsUnlabeled, const TVector<TNodePtr>& labels) {
  1961. bool ordered = false;
  1962. return new TStructNode(pos, exprsUnlabeled, labels, ordered);
  1963. }
  1964. TNodePtr BuildOrderedStructure(TPosition pos, const TVector<TNodePtr>& exprsUnlabeled, const TVector<TNodePtr>& labels) {
  1965. bool ordered = true;
  1966. return new TStructNode(pos, exprsUnlabeled, labels, ordered);
  1967. }
  1968. TListOfNamedNodes::TListOfNamedNodes(TPosition pos, TVector<TNodePtr>&& exprs)
  1969. : INode(pos)
  1970. , Exprs(std::move(exprs))
  1971. {}
  1972. TVector<TNodePtr>* TListOfNamedNodes::ContentListPtr() {
  1973. return &Exprs;
  1974. }
  1975. TAstNode* TListOfNamedNodes::Translate(TContext& ctx) const {
  1976. YQL_ENSURE(!"Unexpected usage");
  1977. Y_UNUSED(ctx);
  1978. return nullptr;
  1979. }
  1980. TNodePtr TListOfNamedNodes::DoClone() const {
  1981. return new TListOfNamedNodes(GetPos(), CloneContainer(Exprs));
  1982. }
  1983. void TListOfNamedNodes::DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const {
  1984. for (auto& expr : Exprs) {
  1985. expr->VisitTree(func, visited);
  1986. }
  1987. }
  1988. TNodePtr BuildListOfNamedNodes(TPosition pos, TVector<TNodePtr>&& exprs) {
  1989. return new TListOfNamedNodes(pos, std::move(exprs));
  1990. }
  1991. TArgPlaceholderNode::TArgPlaceholderNode(TPosition pos, const TString &name) :
  1992. INode(pos),
  1993. Name(name)
  1994. {
  1995. }
  1996. bool TArgPlaceholderNode::DoInit(TContext& ctx, ISource* src) {
  1997. Y_UNUSED(src);
  1998. ctx.Error(Pos) << Name << " can't be used as a part of expression.";
  1999. return false;
  2000. }
  2001. TAstNode* TArgPlaceholderNode::Translate(TContext& ctx) const {
  2002. Y_UNUSED(ctx);
  2003. return nullptr;
  2004. }
  2005. TString TArgPlaceholderNode::GetName() const {
  2006. return Name;
  2007. }
  2008. TNodePtr TArgPlaceholderNode::DoClone() const {
  2009. return new TArgPlaceholderNode(GetPos(), Name);
  2010. }
  2011. TNodePtr BuildArgPlaceholder(TPosition pos, const TString& name) {
  2012. return new TArgPlaceholderNode(pos, name);
  2013. }
  2014. class TAccessNode: public INode {
  2015. public:
  2016. TAccessNode(TPosition pos, const TVector<TIdPart>& ids, bool isLookup)
  2017. : INode(pos)
  2018. , Ids(ids)
  2019. , IsLookup(isLookup)
  2020. , ColumnOnly(false)
  2021. , IsColumnRequired(false)
  2022. , AccessOpName("AccessNode")
  2023. {
  2024. Y_DEBUG_ABORT_UNLESS(Ids.size() > 1);
  2025. Y_DEBUG_ABORT_UNLESS(Ids[0].Expr);
  2026. auto column = Ids[0].Expr->GetColumnNode();
  2027. if (column) {
  2028. ui32 idx = 1;
  2029. TString source;
  2030. if (Ids.size() > 2) {
  2031. source = Ids[idx].Name;
  2032. ++idx;
  2033. }
  2034. ColumnOnly = !IsLookup && Ids.size() < 4;
  2035. if (ColumnOnly && Ids[idx].Expr) {
  2036. column->ResetColumn(Ids[idx].Expr, source);
  2037. } else {
  2038. column->ResetColumn(Ids[idx].Name, source);
  2039. }
  2040. }
  2041. }
  2042. void AssumeColumn() override {
  2043. IsColumnRequired = true;
  2044. }
  2045. TMaybe<TString> TryMakeTable() {
  2046. if (!ColumnOnly) {
  2047. return Nothing();
  2048. }
  2049. ui32 idx = 1;
  2050. if (Ids.size() > 2) {
  2051. return Nothing();
  2052. }
  2053. return Ids[idx].Name;
  2054. }
  2055. const TString* GetColumnName() const override {
  2056. return ColumnOnly ? Ids[0].Expr->GetColumnName() : nullptr;
  2057. }
  2058. bool IsPlainColumn() const override {
  2059. if (GetColumnName()) {
  2060. return true;
  2061. }
  2062. if (Ids[0].Expr->IsTableRow()) {
  2063. return true;
  2064. }
  2065. return false;
  2066. }
  2067. const TString* GetSourceName() const override {
  2068. return Ids[0].Expr->GetSourceName();
  2069. }
  2070. TAccessNode* GetAccessNode() override {
  2071. return this;
  2072. }
  2073. const TAccessNode* GetAccessNode() const override {
  2074. return this;
  2075. }
  2076. bool DoInit(TContext& ctx, ISource* src) override {
  2077. auto expr = Ids[0].Expr;
  2078. const TPosition pos(expr->GetPos());
  2079. if (expr->IsAsterisk()) {
  2080. ctx.Error(pos) << "Asterisk column does not allow any access";
  2081. return false;
  2082. }
  2083. if (!expr->Init(ctx, src)) {
  2084. return false;
  2085. }
  2086. for (auto& id: Ids) {
  2087. if (id.Expr && !id.Expr->Init(ctx, src)) {
  2088. return false;
  2089. }
  2090. }
  2091. ui32 idx = 1;
  2092. auto column = expr->GetColumnNode();
  2093. if (column) {
  2094. const bool useSourceAsColumn = column->IsUseSourceAsColumn();
  2095. ColumnOnly &= !useSourceAsColumn;
  2096. if (IsColumnRequired && !ColumnOnly) {
  2097. ctx.Error(pos) << "Please use a full form (corellation.struct.field) or an alias (struct.field as alias) to access struct's field in the GROUP BY";
  2098. return false;
  2099. }
  2100. if (Ids.size() > 2) {
  2101. if (!CheckColumnId(pos, ctx, Ids[idx], ColumnOnly ? "Correlation" : "Column", true)) {
  2102. return false;
  2103. }
  2104. ++idx;
  2105. }
  2106. if (!useSourceAsColumn) {
  2107. if (!IsLookup && !CheckColumnId(pos, ctx, Ids[idx], ColumnOnly ? "Column" : "Member", false)) {
  2108. return false;
  2109. }
  2110. ++idx;
  2111. }
  2112. }
  2113. for (; idx < Ids.size(); ++idx) {
  2114. const auto& id = Ids[idx];
  2115. if (!id.Name.empty()) {
  2116. expr = Y("SqlAccess", Q("struct"), expr, id.Expr ? Y("EvaluateAtom", id.Expr) : BuildQuotedAtom(Pos, id.Name));
  2117. AccessOpName = "AccessStructMember";
  2118. } else if (id.Expr) {
  2119. expr = Y("SqlAccess", Q("dict"), expr, id.Expr);
  2120. AccessOpName = "AccessDictMember";
  2121. } else {
  2122. continue;
  2123. }
  2124. if (ctx.PragmaYsonAutoConvert || ctx.PragmaYsonStrict || ctx.PragmaYsonFast) {
  2125. auto ysonOptions = Y();
  2126. if (ctx.PragmaYsonAutoConvert) {
  2127. ysonOptions->Add(BuildQuotedAtom(Pos, "yson_auto_convert"));
  2128. }
  2129. if (ctx.PragmaYsonStrict) {
  2130. ysonOptions->Add(BuildQuotedAtom(Pos, "yson_strict"));
  2131. }
  2132. if (ctx.PragmaYsonFast) {
  2133. ysonOptions->Add(BuildQuotedAtom(Pos, "yson_fast"));
  2134. }
  2135. expr->Add(Q(ysonOptions));
  2136. }
  2137. }
  2138. Node = expr;
  2139. return true;
  2140. }
  2141. TAstNode* Translate(TContext& ctx) const override {
  2142. Y_DEBUG_ABORT_UNLESS(Node);
  2143. return Node->Translate(ctx);
  2144. }
  2145. TPtr DoClone() const override {
  2146. YQL_ENSURE(!Node, "TAccessNode::Clone: Node should not be initialized");
  2147. TVector<TIdPart> cloneIds;
  2148. cloneIds.reserve(Ids.size());
  2149. for (const auto& id: Ids) {
  2150. cloneIds.emplace_back(id.Clone());
  2151. }
  2152. auto copy = new TAccessNode(Pos, cloneIds, IsLookup);
  2153. copy->ColumnOnly = ColumnOnly;
  2154. return copy;
  2155. }
  2156. const TVector<TIdPart>& GetParts() const {
  2157. return Ids;
  2158. }
  2159. protected:
  2160. void DoUpdateState() const override {
  2161. YQL_ENSURE(Node);
  2162. State.Set(ENodeState::Const, Node->IsConstant());
  2163. State.Set(ENodeState::MaybeConst, Node->MaybeConstant());
  2164. State.Set(ENodeState::Aggregated, Node->IsAggregated());
  2165. State.Set(ENodeState::AggregationKey, Node->HasState(ENodeState::AggregationKey));
  2166. State.Set(ENodeState::OverWindow, Node->IsOverWindow());
  2167. }
  2168. void DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const final {
  2169. Y_DEBUG_ABORT_UNLESS(Node);
  2170. Node->VisitTree(func, visited);
  2171. }
  2172. bool CheckColumnId(TPosition pos, TContext& ctx, const TIdPart& id, const TString& where, bool checkLookup) {
  2173. if (id.Name.empty()) {
  2174. ctx.Error(pos) << where << " name can not be empty";
  2175. return false;
  2176. }
  2177. if (checkLookup && id.Expr) {
  2178. ctx.Error(pos) << where << " name does not allow dict lookup";
  2179. return false;
  2180. }
  2181. return true;
  2182. }
  2183. TString GetOpName() const override {
  2184. return AccessOpName;
  2185. }
  2186. void CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) override {
  2187. for (auto& id : Ids) {
  2188. if (id.Expr) {
  2189. id.Expr->CollectPreaggregateExprs(ctx, src, exprs);
  2190. }
  2191. }
  2192. }
  2193. private:
  2194. TNodePtr Node;
  2195. TVector<TIdPart> Ids;
  2196. bool IsLookup;
  2197. bool ColumnOnly;
  2198. bool IsColumnRequired;
  2199. TString AccessOpName;
  2200. };
  2201. TNodePtr BuildAccess(TPosition pos, const TVector<INode::TIdPart>& ids, bool isLookup) {
  2202. return new TAccessNode(pos, ids, isLookup);
  2203. }
  2204. TNodePtr BuildMatchRecognizeVarAccess(TPosition pos, const TString& var, const TString& column, bool theSameVar) {
  2205. return new TMatchRecognizeVarAccessNode(pos, var, column, theSameVar);
  2206. }
  2207. void WarnIfAliasFromSelectIsUsedInGroupBy(TContext& ctx, const TVector<TNodePtr>& selectTerms, const TVector<TNodePtr>& groupByTerms,
  2208. const TVector<TNodePtr>& groupByExprTerms)
  2209. {
  2210. THashMap<TString, TNodePtr> termsByLabel;
  2211. for (auto& term : selectTerms) {
  2212. auto label = term->GetLabel();
  2213. if (!label || term->IsOverWindow()) {
  2214. continue;
  2215. }
  2216. auto column = term->GetColumnName();
  2217. // do not warn for trivial renaming such as '[X.]foo AS foo'
  2218. if (column && *column == label) {
  2219. continue;
  2220. }
  2221. // skip terms with aggregation functions inside
  2222. bool hasAggregationFunction = false;
  2223. auto visitor = [&](const INode& current) {
  2224. hasAggregationFunction = hasAggregationFunction || current.GetAggregation();
  2225. return !hasAggregationFunction;
  2226. };
  2227. term->VisitTree(visitor);
  2228. if (!hasAggregationFunction) {
  2229. termsByLabel[label] = term;
  2230. }
  2231. }
  2232. if (termsByLabel.empty()) {
  2233. return;
  2234. }
  2235. bool found = false;
  2236. auto visitor = [&](const INode& current) {
  2237. if (found) {
  2238. return false;
  2239. }
  2240. if (auto columnName = current.GetColumnName()) {
  2241. // do not warn if source name is set
  2242. auto src = current.GetSourceName();
  2243. if (src && *src) {
  2244. return true;
  2245. }
  2246. auto it = termsByLabel.find(*columnName);
  2247. if (it != termsByLabel.end()) {
  2248. found = true;
  2249. ctx.Warning(current.GetPos(), TIssuesIds::YQL_PROJECTION_ALIAS_IS_REFERENCED_IN_GROUP_BY)
  2250. << "GROUP BY will aggregate by column `" << *columnName << "` instead of aggregating by SELECT expression with same alias";
  2251. ctx.Warning(it->second->GetPos(), TIssuesIds::YQL_PROJECTION_ALIAS_IS_REFERENCED_IN_GROUP_BY)
  2252. << "You should probably use alias in GROUP BY instead of using it here. Please consult documentation for more details";
  2253. return false;
  2254. }
  2255. }
  2256. return true;
  2257. };
  2258. TVector<TNodePtr> originalGroupBy;
  2259. {
  2260. THashSet<TString> groupByExprLabels;
  2261. for (auto& expr : groupByExprTerms) {
  2262. auto label = expr->GetLabel();
  2263. YQL_ENSURE(label);
  2264. groupByExprLabels.insert(label);
  2265. }
  2266. originalGroupBy = groupByTerms;
  2267. EraseIf(originalGroupBy, [&](const TNodePtr& node) {
  2268. auto column = node->GetColumnName();
  2269. auto src = node->GetSourceName();
  2270. return (!src || src->empty()) && column && groupByExprLabels.contains(*column);
  2271. });
  2272. originalGroupBy.insert(originalGroupBy.end(), groupByExprTerms.begin(), groupByExprTerms.end());
  2273. }
  2274. for (auto& groupByTerm : originalGroupBy) {
  2275. groupByTerm->VisitTree(visitor);
  2276. if (found) {
  2277. return;
  2278. }
  2279. }
  2280. }
  2281. bool ValidateAllNodesForAggregation(TContext& ctx, const TVector<TNodePtr>& nodes) {
  2282. for (auto& node: nodes) {
  2283. if (!node->HasState(ENodeState::Initialized) || node->IsConstant() || node->MaybeConstant()) {
  2284. continue;
  2285. }
  2286. // TODO: "!node->IsOverWindow()" doesn't look right here
  2287. if (!node->IsAggregated() && !node->IsOverWindow() && !node->IsOverWindowDistinct()) {
  2288. // locate column which is not a key column and not aggregated
  2289. const INode* found = nullptr;
  2290. auto visitor = [&found](const INode& current) {
  2291. if (found || current.IsAggregated() || current.IsOverWindow() || current.IsOverWindowDistinct()) {
  2292. return false;
  2293. }
  2294. if (current.GetColumnNode() || current.GetAccessNode()) {
  2295. found = &current;
  2296. return false;
  2297. }
  2298. return true;
  2299. };
  2300. node->VisitTree(visitor);
  2301. if (found) {
  2302. TString columnName;
  2303. if (auto col = found->GetColumnName(); col && *col) {
  2304. columnName = "`";
  2305. if (auto src = found->GetSourceName(); src && *src) {
  2306. columnName += DotJoin(*src, *col);
  2307. } else {
  2308. columnName += *col;
  2309. }
  2310. columnName += "` ";
  2311. }
  2312. ctx.Error(found->GetPos()) << "Column " << columnName << "must either be a key column in GROUP BY or it should be used in aggregation function";
  2313. } else {
  2314. ctx.Error(node->GetPos()) << "Expression has to be an aggregation function or key column, because aggregation is used elsewhere in this subquery";
  2315. }
  2316. return false;
  2317. }
  2318. }
  2319. return true;
  2320. }
  2321. class TBindNode: public TAstListNode {
  2322. public:
  2323. TBindNode(TPosition pos, const TString& module, const TString& alias)
  2324. : TAstListNode(pos)
  2325. {
  2326. Add("bind", AstNode(module), BuildQuotedAtom(pos, alias));
  2327. }
  2328. private:
  2329. TBindNode(const TBindNode& other)
  2330. : TAstListNode(other.GetPos())
  2331. {
  2332. Nodes = CloneContainer(other.Nodes);
  2333. }
  2334. TPtr DoClone() const final {
  2335. return new TBindNode(*this);
  2336. }
  2337. };
  2338. TNodePtr BuildBind(TPosition pos, const TString& module, const TString& alias) {
  2339. return new TBindNode(pos, module, alias);
  2340. }
  2341. class TLambdaNode: public TAstListNode {
  2342. public:
  2343. TLambdaNode(TPosition pos, TNodePtr params, TNodePtr body, const TString& resName)
  2344. : TAstListNode(pos)
  2345. {
  2346. if (!resName.empty()) {
  2347. body = Y("block", Q(L(body, Y("return", resName))));
  2348. }
  2349. Add("lambda", Q(params), body);
  2350. }
  2351. TLambdaNode(TPosition pos, TNodePtr params, TVector<TNodePtr> bodies)
  2352. : TAstListNode(pos)
  2353. {
  2354. Add("lambda", Q(params));
  2355. for (const auto& b : bodies) {
  2356. Add(b);
  2357. }
  2358. }
  2359. TLambdaNode* GetLambdaNode() override {
  2360. return this;
  2361. }
  2362. const TLambdaNode* GetLambdaNode() const override {
  2363. return this;
  2364. }
  2365. private:
  2366. TLambdaNode(const TLambdaNode& other)
  2367. : TAstListNode(other.GetPos())
  2368. {
  2369. Nodes = CloneContainer(other.Nodes);
  2370. }
  2371. TPtr DoClone() const final {
  2372. return new TLambdaNode(*this);
  2373. }
  2374. void DoUpdateState() const final {
  2375. State.Set(ENodeState::Const);
  2376. }
  2377. };
  2378. TNodePtr BuildLambda(TPosition pos, TNodePtr params, TNodePtr body, const TString& resName) {
  2379. return new TLambdaNode(pos, params, body, resName);
  2380. }
  2381. TNodePtr BuildLambda(TPosition pos, TNodePtr params, const TVector<TNodePtr>& bodies) {
  2382. return new TLambdaNode(pos, params, bodies);
  2383. }
  2384. TNodePtr BuildDataType(TPosition pos, const TString& typeName) {
  2385. return new TCallNodeImpl(pos, "DataType", {BuildQuotedAtom(pos, typeName, TNodeFlags::Default)});
  2386. }
  2387. TMaybe<TString> LookupSimpleType(const TStringBuf& alias, bool flexibleTypes, bool isPgType) {
  2388. TString normalized = to_lower(TString(alias));
  2389. if (isPgType) {
  2390. // expecting original pg type (like _int4 or varchar) with optional pg suffix (i.e. _pgint4, pgvarchar)
  2391. if (normalized.StartsWith("pg")) {
  2392. normalized = normalized.substr(2);
  2393. } else if (normalized.StartsWith("_pg")) {
  2394. normalized = "_" + normalized.substr(3);
  2395. }
  2396. if (!NPg::HasType(normalized)) {
  2397. return {};
  2398. }
  2399. if (normalized.StartsWith("_")) {
  2400. return "_pg" + normalized.substr(1);
  2401. }
  2402. return "pg" + normalized;
  2403. }
  2404. if (auto sqlAlias = LookupSimpleTypeBySqlAlias(alias, flexibleTypes)) {
  2405. return TString(*sqlAlias);
  2406. }
  2407. TString pgType;
  2408. if (normalized.StartsWith("_pg")) {
  2409. pgType = normalized.substr(3);
  2410. } else if (normalized.StartsWith("pg")) {
  2411. pgType = normalized.substr(2);
  2412. } else {
  2413. return {};
  2414. }
  2415. if (NPg::HasType(pgType)) {
  2416. return normalized;
  2417. }
  2418. return {};
  2419. }
  2420. TNodePtr BuildSimpleType(TContext& ctx, TPosition pos, const TString& typeName, bool dataOnly) {
  2421. bool explicitPgType = ctx.GetColumnReferenceState() == EColumnRefState::AsPgType;
  2422. auto found = LookupSimpleType(typeName, ctx.FlexibleTypes, explicitPgType);
  2423. if (!found) {
  2424. ctx.Error(pos) << "Unknown " << (explicitPgType ? "pg" : "simple") << " type '" << typeName << "'";
  2425. return {};
  2426. }
  2427. auto type = *found;
  2428. if (type == "Void" || type == "Unit" || type == "Generic" || type == "EmptyList" || type == "EmptyDict") {
  2429. if (dataOnly) {
  2430. ctx.Error(pos) << "Only data types are allowed here, but got: '" << typeName << "'";
  2431. return {};
  2432. }
  2433. type += "Type";
  2434. return new TCallNodeImpl(pos, type, {});
  2435. }
  2436. if (type.StartsWith("_pg") || type.StartsWith("pg")) {
  2437. TString pgType;
  2438. if (type.StartsWith("_pg")) {
  2439. pgType = "_" + type.substr(3);
  2440. } else {
  2441. pgType = type.substr(2);
  2442. }
  2443. return new TCallNodeImpl(pos, "PgType", { BuildQuotedAtom(pos, pgType, TNodeFlags::Default) });
  2444. }
  2445. return new TCallNodeImpl(pos, "DataType", { BuildQuotedAtom(pos, type, TNodeFlags::Default) });
  2446. }
  2447. TString TypeByAlias(const TString& alias, bool normalize) {
  2448. TString type(alias);
  2449. TCiString typeAlias(alias);
  2450. if (typeAlias.StartsWith("varchar")) {
  2451. type = "String";
  2452. } else if (typeAlias == "tinyint") {
  2453. type = "Int8";
  2454. } else if (typeAlias == "byte") {
  2455. type = "Uint8";
  2456. } else if (typeAlias == "smallint") {
  2457. type = "Int16";
  2458. } else if (typeAlias == "int" || typeAlias == "integer") {
  2459. type = "Int32";
  2460. } else if (typeAlias == "bigint") {
  2461. type = "Int64";
  2462. }
  2463. return normalize ? NormalizeTypeString(type) : type;
  2464. }
  2465. TNodePtr BuildIsNullOp(TPosition pos, TNodePtr a) {
  2466. if (!a) {
  2467. return nullptr;
  2468. }
  2469. if (a->IsNull()) {
  2470. return BuildLiteralBool(pos, true);
  2471. }
  2472. return new TCallNodeImpl(pos, "Not", {new TCallNodeImpl(pos, "Exists", {a})});
  2473. }
  2474. TUdfNode::TUdfNode(TPosition pos, const TVector<TNodePtr>& args)
  2475. : INode(pos)
  2476. , Args(args)
  2477. {
  2478. if (Args.size()) {
  2479. // If there aren't any named args, args are passed as vector of positional args,
  2480. // else Args has length 2: tuple for positional args and struct for named args,
  2481. // so let's construct tuple of args there. Other type checks will within DoInit call.
  2482. if (!Args[0]->GetTupleNode()) {
  2483. Args = {BuildTuple(pos, args)};
  2484. }
  2485. }
  2486. }
  2487. bool TUdfNode::DoInit(TContext& ctx, ISource* src) {
  2488. Y_UNUSED(src);
  2489. if (Args.size() < 1) {
  2490. ctx.Error(Pos) << "Udf: expected at least one argument";
  2491. return false;
  2492. }
  2493. TTupleNode* as_tuple = Args[0]->GetTupleNode();
  2494. if (!as_tuple || as_tuple->GetTupleSize() < 1) {
  2495. ctx.Error(Pos) << "Udf: first argument must be a callable, like Foo::Bar";
  2496. return false;
  2497. }
  2498. TNodePtr function = as_tuple->GetTupleElement(0);
  2499. if (!function || !function->FuncName()) {
  2500. ctx.Error(Pos) << "Udf: first argument must be a callable, like Foo::Bar";
  2501. return false;
  2502. }
  2503. FunctionName = function->FuncName();
  2504. ModuleName = function->ModuleName();
  2505. TVector<TNodePtr> external;
  2506. external.reserve(as_tuple->GetTupleSize() - 1);
  2507. for (size_t i = 1; i < as_tuple->GetTupleSize(); ++i) {
  2508. // TODO(): support named args in GetFunctionArgColumnStatus
  2509. TNodePtr current = as_tuple->GetTupleElement(i);
  2510. if (TAccessNode* as_access = current->GetAccessNode(); as_access) {
  2511. external.push_back(Y("DataType", Q(as_access->GetParts()[1].Name)));
  2512. continue;
  2513. }
  2514. external.push_back(current);
  2515. }
  2516. ExternalTypesTuple = new TCallNodeImpl(Pos, "TupleType", external);
  2517. if (Args.size() == 1) {
  2518. return true;
  2519. }
  2520. if (TStructNode* named_args = Args[1]->GetStructNode(); named_args) {
  2521. for (const auto &arg: named_args->GetExprs()) {
  2522. if (arg->GetLabel() == "TypeConfig") {
  2523. TypeConfig = MakeAtomFromExpression(Pos, ctx, arg);
  2524. } else if (arg->GetLabel() == "RunConfig") {
  2525. RunConfig = arg;
  2526. }
  2527. }
  2528. }
  2529. return true;
  2530. }
  2531. const TNodePtr TUdfNode::GetExternalTypes() const {
  2532. return ExternalTypesTuple;
  2533. }
  2534. const TString& TUdfNode::GetFunction() const {
  2535. return *FunctionName;
  2536. }
  2537. const TString& TUdfNode::GetModule() const {
  2538. return *ModuleName;
  2539. }
  2540. TNodePtr TUdfNode::GetRunConfig() const {
  2541. return RunConfig;
  2542. }
  2543. const TDeferredAtom& TUdfNode::GetTypeConfig() const {
  2544. return TypeConfig;
  2545. }
  2546. TUdfNode* TUdfNode::GetUdfNode() {
  2547. return this;
  2548. }
  2549. const TUdfNode* TUdfNode::GetUdfNode() const {
  2550. return this;
  2551. }
  2552. TAstNode* TUdfNode::Translate(TContext& ctx) const {
  2553. ctx.Error(Pos) << "Abstract Udf Node can't be used as a part of expression.";
  2554. return nullptr;
  2555. }
  2556. TNodePtr TUdfNode::DoClone() const {
  2557. return new TUdfNode(Pos, CloneContainer(Args));
  2558. }
  2559. class TBinaryOpNode final: public TCallNode {
  2560. public:
  2561. TBinaryOpNode(TPosition pos, const TString& opName, TNodePtr a, TNodePtr b);
  2562. TNodePtr DoClone() const final {
  2563. YQL_ENSURE(Args.size() == 2);
  2564. return new TBinaryOpNode(Pos, OpName, Args[0]->Clone(), Args[1]->Clone());
  2565. }
  2566. };
  2567. TBinaryOpNode::TBinaryOpNode(TPosition pos, const TString& opName, TNodePtr a, TNodePtr b)
  2568. : TCallNode(pos, opName, 2, 2, { a, b })
  2569. {
  2570. }
  2571. TNodePtr BuildBinaryOp(TContext& ctx, TPosition pos, const TString& opName, TNodePtr a, TNodePtr b) {
  2572. if (!a || !b) {
  2573. return nullptr;
  2574. }
  2575. static const THashSet<TStringBuf> nullSafeOps = {"IsDistinctFrom", "IsNotDistinctFrom"};
  2576. if (!nullSafeOps.contains(opName)) {
  2577. const bool bothArgNull = a->IsNull() && b->IsNull();
  2578. const bool oneArgNull = a->IsNull() || b->IsNull();
  2579. if (bothArgNull || (oneArgNull && opName != "Or" && opName != "And")) {
  2580. ctx.Warning(pos, TIssuesIds::YQL_OPERATION_WILL_RETURN_NULL) << "Binary operation "
  2581. << opName.substr(0, opName.size() - 7 * opName.EndsWith("MayWarn"))
  2582. << " will return NULL here";
  2583. }
  2584. }
  2585. return new TBinaryOpNode(pos, opName, a, b);
  2586. }
  2587. TNodePtr BuildBinaryOpRaw(TPosition pos, const TString& opName, TNodePtr a, TNodePtr b) {
  2588. if (!a || !b) {
  2589. return nullptr;
  2590. }
  2591. return new TBinaryOpNode(pos, opName, a, b);
  2592. }
  2593. class TCalcOverWindow final: public INode {
  2594. public:
  2595. TCalcOverWindow(TPosition pos, const TString& windowName, TNodePtr node)
  2596. : INode(pos)
  2597. , WindowName(windowName)
  2598. , FuncNode(node)
  2599. {}
  2600. TAstNode* Translate(TContext& ctx) const override {
  2601. return FuncNode->Translate(ctx);
  2602. }
  2603. bool DoInit(TContext& ctx, ISource* src) override {
  2604. YQL_ENSURE(src);
  2605. TSourcePtr overWindowSource = BuildOverWindowSource(ctx.Pos(), WindowName, src);
  2606. if (!FuncNode->Init(ctx, overWindowSource.Get())) {
  2607. return false;
  2608. }
  2609. return true;
  2610. }
  2611. TPtr DoClone() const final {
  2612. return new TCalcOverWindow(Pos, WindowName, SafeClone(FuncNode));
  2613. }
  2614. void DoUpdateState() const override {
  2615. State.Set(ENodeState::Const, FuncNode->IsConstant());
  2616. State.Set(ENodeState::MaybeConst, FuncNode->MaybeConstant());
  2617. State.Set(ENodeState::Aggregated, FuncNode->IsAggregated());
  2618. State.Set(ENodeState::OverWindow, true);
  2619. }
  2620. void DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const final {
  2621. Y_DEBUG_ABORT_UNLESS(FuncNode);
  2622. FuncNode->VisitTree(func, visited);
  2623. }
  2624. void CollectPreaggregateExprs(TContext& ctx, ISource& src, TVector<INode::TPtr>& exprs) override {
  2625. if (ctx.DistinctOverWindow) {
  2626. FuncNode->CollectPreaggregateExprs(ctx, src, exprs);
  2627. } else {
  2628. INode::CollectPreaggregateExprs(ctx, src, exprs);
  2629. }
  2630. }
  2631. protected:
  2632. const TString WindowName;
  2633. TNodePtr FuncNode;
  2634. };
  2635. TNodePtr BuildCalcOverWindow(TPosition pos, const TString& windowName, TNodePtr call) {
  2636. return new TCalcOverWindow(pos, windowName, call);
  2637. }
  2638. template<bool Fast>
  2639. class TYsonOptionsNode final: public INode {
  2640. public:
  2641. TYsonOptionsNode(TPosition pos, bool autoConvert, bool strict)
  2642. : INode(pos)
  2643. , AutoConvert(autoConvert)
  2644. , Strict(strict)
  2645. {
  2646. auto udf = Y("Udf", Q(Fast ? "Yson2.Options" : "Yson.Options"));
  2647. auto autoConvertNode = BuildLiteralBool(pos, autoConvert);
  2648. autoConvertNode->SetLabel("AutoConvert");
  2649. auto strictNode = BuildLiteralBool(pos, strict);
  2650. strictNode->SetLabel("Strict");
  2651. Node = Y("NamedApply", udf, Q(Y()), BuildStructure(pos, { autoConvertNode, strictNode }));
  2652. }
  2653. TAstNode* Translate(TContext& ctx) const override {
  2654. return Node->Translate(ctx);
  2655. }
  2656. bool DoInit(TContext& ctx, ISource* src) override {
  2657. if (!Node->Init(ctx, src)) {
  2658. return false;
  2659. }
  2660. return true;
  2661. }
  2662. TPtr DoClone() const final {
  2663. return new TYsonOptionsNode(Pos, AutoConvert, Strict);
  2664. }
  2665. void DoUpdateState() const override {
  2666. State.Set(ENodeState::Const, true);
  2667. }
  2668. protected:
  2669. TNodePtr Node;
  2670. const bool AutoConvert;
  2671. const bool Strict;
  2672. };
  2673. TNodePtr BuildYsonOptionsNode(TPosition pos, bool autoConvert, bool strict, bool fastYson) {
  2674. if (fastYson)
  2675. return new TYsonOptionsNode<true>(pos, autoConvert, strict);
  2676. else
  2677. return new TYsonOptionsNode<false>(pos, autoConvert, strict);
  2678. }
  2679. class TDoCall final : public INode {
  2680. public:
  2681. TDoCall(TPosition pos, const TNodePtr& node)
  2682. : INode(pos)
  2683. , Node(node)
  2684. {
  2685. FakeSource = BuildFakeSource(pos);
  2686. }
  2687. ISource* GetSource() final {
  2688. return FakeSource.Get();
  2689. }
  2690. bool DoInit(TContext& ctx, ISource* src) final {
  2691. Y_UNUSED(src);
  2692. if (!Node->Init(ctx, FakeSource.Get())) {
  2693. return false;
  2694. }
  2695. return true;
  2696. }
  2697. TAstNode* Translate(TContext& ctx) const final {
  2698. return Node->Translate(ctx);
  2699. }
  2700. TPtr DoClone() const final {
  2701. return new TDoCall(Pos, Node->Clone());
  2702. }
  2703. void DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const final {
  2704. Y_DEBUG_ABORT_UNLESS(Node);
  2705. Node->VisitTree(func, visited);
  2706. }
  2707. private:
  2708. TNodePtr Node;
  2709. TSourcePtr FakeSource;
  2710. };
  2711. TNodePtr BuildDoCall(TPosition pos, const TNodePtr& node) {
  2712. return new TDoCall(pos, node);
  2713. }
  2714. bool Parseui32(TNodePtr from, ui32& to) {
  2715. const TString* val;
  2716. if (!(val = from->GetLiteral("Int32"))) {
  2717. if (!(val = from->GetLiteral("Uint32"))) {
  2718. return false;
  2719. }
  2720. }
  2721. return TryFromString(*val, to);
  2722. }
  2723. TNodePtr GroundWithExpr(const TNodePtr& ground, const TNodePtr& expr) {
  2724. return ground ? expr->Y("block", expr->Q(expr->L(ground, expr->Y("return", expr)))) : expr;
  2725. }
  2726. TSourcePtr TryMakeSourceFromExpression(TPosition pos, TContext& ctx, const TString& currService, const TDeferredAtom& currCluster,
  2727. TNodePtr node, const TString& view) {
  2728. if (currCluster.Empty()) {
  2729. ctx.Error() << "No cluster name given and no default cluster is selected";
  2730. return nullptr;
  2731. }
  2732. if (auto literal = node->GetLiteral("String")) {
  2733. TNodePtr tableKey = BuildTableKey(node->GetPos(), currService, currCluster, TDeferredAtom(node->GetPos(), *literal), {view});
  2734. TTableRef table(ctx.MakeName("table"), currService, currCluster, tableKey);
  2735. table.Options = BuildInputOptions(node->GetPos(), GetContextHints(ctx));
  2736. return BuildTableSource(node->GetPos(), table);
  2737. }
  2738. if (node->GetLambdaNode()) {
  2739. ctx.Error() << "Lambda is not allowed to be used as source. Did you forget to call a subquery template?";
  2740. return nullptr;
  2741. }
  2742. auto wrappedNode = new TAstListNodeImpl(pos, {
  2743. new TAstAtomNodeImpl(pos, "EvaluateAtom", TNodeFlags::Default),
  2744. node
  2745. });
  2746. TNodePtr tableKey = BuildTableKey(node->GetPos(), currService, currCluster, TDeferredAtom(wrappedNode, ctx), {view});
  2747. TTableRef table(ctx.MakeName("table"), currService, currCluster, tableKey);
  2748. table.Options = BuildInputOptions(node->GetPos(), GetContextHints(ctx));
  2749. return BuildTableSource(node->GetPos(), table);
  2750. }
  2751. void MakeTableFromExpression(TPosition pos, TContext& ctx, TNodePtr node, TDeferredAtom& table, const TString& prefix) {
  2752. if (auto literal = node->GetLiteral("String")) {
  2753. table = TDeferredAtom(node->GetPos(), prefix + *literal);
  2754. return;
  2755. }
  2756. if (auto access = node->GetAccessNode()) {
  2757. auto ret = access->TryMakeTable();
  2758. if (ret) {
  2759. table = TDeferredAtom(node->GetPos(), prefix + *ret);
  2760. return;
  2761. }
  2762. }
  2763. if (!prefix.empty()) {
  2764. node = node->Y("Concat", node->Y("String", node->Q(prefix)), node);
  2765. }
  2766. auto wrappedNode = new TAstListNodeImpl(pos, {
  2767. new TAstAtomNodeImpl(pos, "EvaluateAtom", TNodeFlags::Default),
  2768. node
  2769. });
  2770. table = TDeferredAtom(wrappedNode, ctx);
  2771. }
  2772. TDeferredAtom MakeAtomFromExpression(TPosition pos, TContext& ctx, TNodePtr node, const TString& prefix) {
  2773. if (auto literal = node->GetLiteral("String")) {
  2774. return TDeferredAtom(node->GetPos(), prefix + *literal);
  2775. }
  2776. if (!prefix.empty()) {
  2777. node = node->Y("Concat", node->Y("String", node->Q(prefix)), node);
  2778. }
  2779. auto wrappedNode = new TAstListNodeImpl(pos, {
  2780. new TAstAtomNodeImpl(pos, "EvaluateAtom", TNodeFlags::Default),
  2781. node
  2782. });
  2783. return TDeferredAtom(wrappedNode, ctx);
  2784. }
  2785. class TTupleResultNode: public INode {
  2786. public:
  2787. TTupleResultNode(TNodePtr&& tuple, size_t ensureTupleSize)
  2788. : INode(tuple->GetPos())
  2789. , Node(std::move(tuple))
  2790. , EnsureTupleSize(ensureTupleSize)
  2791. {
  2792. }
  2793. bool DoInit(TContext& ctx, ISource* src) override {
  2794. if (!Node->Init(ctx, src)) {
  2795. return false;
  2796. }
  2797. Node = Y("EnsureTupleSize", Node, Q(ToString(EnsureTupleSize)));
  2798. return true;
  2799. }
  2800. TAstNode* Translate(TContext& ctx) const override {
  2801. return Node->Translate(ctx);
  2802. }
  2803. TPtr DoClone() const final {
  2804. return new TTupleResultNode(Node->Clone(), EnsureTupleSize);
  2805. }
  2806. void DoVisitChildren(const TVisitFunc& func, TVisitNodeSet& visited) const final {
  2807. Y_DEBUG_ABORT_UNLESS(Node);
  2808. Node->VisitTree(func, visited);
  2809. }
  2810. protected:
  2811. TNodePtr Node;
  2812. const size_t EnsureTupleSize;
  2813. };
  2814. TNodePtr BuildTupleResult(TNodePtr tuple, size_t ensureTupleSize) {
  2815. return new TTupleResultNode(std::move(tuple), ensureTupleSize);
  2816. }
  2817. class TNamedExprReferenceNode: public IProxyNode {
  2818. public:
  2819. TNamedExprReferenceNode(TNodePtr parent, const TString& name, TMaybe<size_t> tupleIndex)
  2820. : IProxyNode(parent->GetPos(), parent)
  2821. , Name(name)
  2822. , TupleIndex(tupleIndex)
  2823. {
  2824. }
  2825. bool DoInit(TContext& ctx, ISource* src) final {
  2826. Y_UNUSED(src);
  2827. if (!IProxyNode::DoInit(ctx, nullptr) || !IProxyNode::InitReference(ctx)) {
  2828. return false;
  2829. }
  2830. Node = BuildAtom(GetPos(), Name, TNodeFlags::Default);
  2831. if (TupleIndex.Defined()) {
  2832. Node = Y("Nth", Node, Q(ToString(*TupleIndex)));
  2833. }
  2834. return true;
  2835. }
  2836. TAstNode* Translate(TContext& ctx) const override {
  2837. YQL_ENSURE(Node, "Init() should be done before Translate()");
  2838. return Node->Translate(ctx);
  2839. }
  2840. TPtr DoClone() const final {
  2841. // do not clone Inner here
  2842. return new TNamedExprReferenceNode(Inner, Name, TupleIndex);
  2843. }
  2844. private:
  2845. const TString Name;
  2846. const TMaybe<size_t> TupleIndex;
  2847. TNodePtr Node;
  2848. };
  2849. TNodePtr BuildNamedExprReference(TNodePtr parent, const TString& name, TMaybe<size_t> tupleIndex) {
  2850. YQL_ENSURE(parent);
  2851. return new TNamedExprReferenceNode(parent, name, tupleIndex);
  2852. }
  2853. class TNamedExprNode: public IProxyNode {
  2854. public:
  2855. TNamedExprNode(TNodePtr parent)
  2856. : IProxyNode(parent->GetPos(), parent)
  2857. , FakeSource(BuildFakeSource(parent->GetPos()))
  2858. , Referenced(false)
  2859. {
  2860. }
  2861. bool DoInit(TContext& ctx, ISource* src) final {
  2862. YQL_ENSURE(!Referenced, "Refrence is initialized before named expr itself");
  2863. Y_UNUSED(src);
  2864. if (ctx.ValidateUnusedExprs) {
  2865. return IProxyNode::DoInit(ctx, FakeSource.Get());
  2866. }
  2867. // do actual init in InitReference()
  2868. return true;
  2869. }
  2870. bool InitReference(TContext& ctx) final {
  2871. Referenced = true;
  2872. return IProxyNode::DoInit(ctx, FakeSource.Get());
  2873. }
  2874. TAstNode* Translate(TContext& ctx) const override {
  2875. if (ctx.ValidateUnusedExprs || Referenced) {
  2876. return Inner->Translate(ctx);
  2877. }
  2878. auto unused = BuildQuotedAtom(GetPos(), "unused", TNodeFlags::Default);
  2879. return unused->Translate(ctx);
  2880. }
  2881. TPtr DoClone() const final {
  2882. return new TNamedExprNode(Inner->Clone());
  2883. }
  2884. private:
  2885. const TSourcePtr FakeSource;
  2886. bool Referenced;
  2887. };
  2888. TNodePtr BuildNamedExpr(TNodePtr parent) {
  2889. YQL_ENSURE(parent);
  2890. return new TNamedExprNode(parent);
  2891. }
  2892. bool TVectorIndexSettings::Validate(TContext& ctx) const {
  2893. if (!Distance && !Similarity) {
  2894. ctx.Error() << "either distance or similarity should be set";
  2895. return false;
  2896. }
  2897. if (!VectorType) {
  2898. ctx.Error() << "vector_type should be set";
  2899. return false;
  2900. }
  2901. if (!VectorDimension) {
  2902. ctx.Error() << "vector_dimension should be set";
  2903. return false;
  2904. }
  2905. return true;
  2906. }
  2907. } // namespace NSQLTranslationV1